From 8d06aa2d2277529c5619562f965e8db46dbb94ee Mon Sep 17 00:00:00 2001 From: vikram Date: Tue, 7 Mar 2017 15:57:04 +0530 Subject: [PATCH] Fixed python implementation for Apache Apex --- .gitignore | 3 + docs/python/main.md | 38 ++ .../python/resources/hadoop_word_count.txt | 15 + examples/python/resources/stock_data.csv | 96 ++++ examples/python/stock_filter.py | 44 ++ examples/python/word_count.py | 47 ++ library/pom.xml | 2 + .../lib/io/fs/InMemoryDataInputOperator.java | 74 +++ .../lib/window/accumulation/Reduce.java | 26 ++ .../lib/window/accumulation/ReduceFn.java | 3 +- mkdocs.yml | 2 + pom.xml | 26 ++ python/.gitignore | 2 + python/apex-python/.gitignore | 1 + python/apex-python/README | 0 python/apex-python/deps/py4j-0.10.4-src.zip | Bin 0 -> 78717 bytes python/apex-python/setup.py | 46 ++ python/apex-python/src/MANIFEST | 3 + python/apex-python/src/pyapex/__init__.py | 22 + python/apex-python/src/pyapex/apexapp.py | 211 +++++++++ python/apex-python/src/pyapex/commands.py | 63 +++ .../src/pyapex/functions/__init__.py | 21 + .../src/pyapex/functions/functions.py | 110 +++++ .../src/pyapex/functions/transforms.py | 116 +++++ .../src/pyapex/functions/window.py | 133 ++++++ .../src/pyapex/runtime/__init__.py | 20 + .../apex-python/src/pyapex/runtime/worker.py | 79 ++++ python/apex-python/src/pyapex/shellconn.py | 41 ++ python/create_zip.sh | 26 ++ python/pom.xml | 333 ++++++++++++++ python/scripts/log4j.properties | 43 ++ python/scripts/pyshell | 212 +++++++++ .../apache/apex/malhar/PythonConstants.java | 49 ++ .../org/apache/apex/malhar/python/PyApex.java | 120 +++++ .../apache/apex/malhar/python/PythonApp.java | 428 ++++++++++++++++++ .../apex/malhar/python/PythonAppFactory.java | 53 +++ .../apex/malhar/python/PythonAppManager.java | 101 +++++ .../operator/PythonGenericOperator.java | 92 ++++ .../operator/PythonKeyedWindowedOperator.java | 64 +++ .../operator/PythonWindowedOperator.java | 64 +++ .../interfaces/PythonAccumulatorWorker.java | 35 ++ .../interfaces/PythonReduceWorker.java | 25 + .../operator/interfaces/PythonWorker.java | 27 ++ .../proxy/PythonAcummlationWorkerProxy.java | 104 +++++ .../operator/proxy/PythonReduceProxy.java | 109 +++++ .../operator/proxy/PythonWorkerProxy.java | 118 +++++ .../transform/PythonFilterOperator.java | 61 +++ .../transform/PythonFlatMapOperator.java | 61 +++ .../operator/transform/PythonMapOperator.java | 55 +++ .../python/runtime/PythonApexStreamImpl.java | 179 ++++++++ .../malhar/python/runtime/PythonServer.java | 275 +++++++++++ .../python/runtime/PythonWorkerContext.java | 144 ++++++ .../apex/malhar/python/util/LoggerUtils.java | 86 ++++ .../apex/malhar/python/util/NetworkUtils.java | 43 ++ .../malhar/stream/api/PythonApexStream.java | 50 ++ .../stream/api/PythonWindowedApexStream.java | 26 ++ .../operator/runtime/PythonOperatorTest.java | 277 ++++++++++++ .../runtime/PythonWorkerContextTest.java | 64 +++ .../runtime/PythonWorkerProxyTest.java | 93 ++++ python/src/test/resources/log4j.properties | 43 ++ .../apex/malhar/stream/api/ApexStream.java | 4 +- .../malhar/stream/api/WindowedStream.java | 5 +- .../stream/api/impl/ApexStreamImpl.java | 9 +- .../api/impl/ApexWindowedStreamImpl.java | 33 +- .../malhar/stream/api/impl/StreamFactory.java | 25 + .../apex/malhar/stream/sample/MyStream.java | 17 + stream/src/test/resources/log4j.properties | 41 ++ 67 files changed, 4814 insertions(+), 24 deletions(-) create mode 100644 docs/python/main.md create mode 100644 examples/python/resources/hadoop_word_count.txt create mode 100644 examples/python/resources/stock_data.csv create mode 100644 examples/python/stock_filter.py create mode 100644 examples/python/word_count.py create mode 100644 library/src/main/java/com/datatorrent/lib/io/fs/InMemoryDataInputOperator.java create mode 100644 library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/Reduce.java create mode 100644 python/.gitignore create mode 100644 python/apex-python/.gitignore create mode 100644 python/apex-python/README create mode 100644 python/apex-python/deps/py4j-0.10.4-src.zip create mode 100644 python/apex-python/setup.py create mode 100644 python/apex-python/src/MANIFEST create mode 100644 python/apex-python/src/pyapex/__init__.py create mode 100644 python/apex-python/src/pyapex/apexapp.py create mode 100644 python/apex-python/src/pyapex/commands.py create mode 100644 python/apex-python/src/pyapex/functions/__init__.py create mode 100644 python/apex-python/src/pyapex/functions/functions.py create mode 100644 python/apex-python/src/pyapex/functions/transforms.py create mode 100644 python/apex-python/src/pyapex/functions/window.py create mode 100644 python/apex-python/src/pyapex/runtime/__init__.py create mode 100755 python/apex-python/src/pyapex/runtime/worker.py create mode 100644 python/apex-python/src/pyapex/shellconn.py create mode 100755 python/create_zip.sh create mode 100644 python/pom.xml create mode 100644 python/scripts/log4j.properties create mode 100755 python/scripts/pyshell create mode 100644 python/src/main/java/org/apache/apex/malhar/PythonConstants.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/PyApex.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/PythonApp.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/PythonAppFactory.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/PythonAppManager.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/PythonGenericOperator.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/PythonKeyedWindowedOperator.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/PythonWindowedOperator.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonAccumulatorWorker.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonReduceWorker.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonWorker.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonAcummlationWorkerProxy.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonReduceProxy.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonWorkerProxy.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFilterOperator.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFlatMapOperator.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonMapOperator.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/runtime/PythonApexStreamImpl.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/runtime/PythonServer.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/runtime/PythonWorkerContext.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/util/LoggerUtils.java create mode 100644 python/src/main/java/org/apache/apex/malhar/python/util/NetworkUtils.java create mode 100644 python/src/main/java/org/apache/apex/malhar/stream/api/PythonApexStream.java create mode 100644 python/src/main/java/org/apache/apex/malhar/stream/api/PythonWindowedApexStream.java create mode 100644 python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonOperatorTest.java create mode 100644 python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerContextTest.java create mode 100644 python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerProxyTest.java create mode 100644 python/src/test/resources/log4j.properties create mode 100644 stream/src/test/resources/log4j.properties diff --git a/.gitignore b/.gitignore index d69387d9f9..138cc7bfd4 100644 --- a/.gitignore +++ b/.gitignore @@ -12,3 +12,6 @@ nb-configuration.xml hadoop.log site/ .checkstyle +*.pyc +*.out +examples/python/output diff --git a/docs/python/main.md b/docs/python/main.md new file mode 100644 index 0000000000..0e4f379d8b --- /dev/null +++ b/docs/python/main.md @@ -0,0 +1,38 @@ +#Developing Streaming Application in Python# + +Currently we have exposed basic support for Stateless Support. + +##Requirements:## +* Python 2.7 +* py4j + Please install py4j on your machine. + ``` + pip install py4j + ``` + + +Once you have pulled Apache Malhar project, go to python project and follow next steps: + +* Compile all projects under Apache Malhar and make sure you have hadoop installed on local node. +* Once compilation finish, go to python/script directory and launch ./pyshell +* This will launch python shell. Now you can develop your application using python shell. + +You can write simpler application using available apis as well provide custom functions written in python. + +``` + +def filter_function(a): + input_data=a.split(',') + if float(input_data[2])> 100: + return True + return False + +from pyapex import createApp +a=createApp('python_app').from_kafka09('localhost:2181','test_topic') \ + .filter('filter_operator',filter_function) \ + .to_console(name='endConsole') \ + .launch(False) +``` + + +Note: Currently developer need to ensure that required python dependencies are installed on Hadoop cluster. diff --git a/examples/python/resources/hadoop_word_count.txt b/examples/python/resources/hadoop_word_count.txt new file mode 100644 index 0000000000..da9cd3af3e --- /dev/null +++ b/examples/python/resources/hadoop_word_count.txt @@ -0,0 +1,15 @@ +Hadoop is the Elephant King! +A yellow and elegant thing. +He never forgets +Useful data, or lets +An extraneous element cling! +A wonderful king is Hadoop. +The elephant plays well with Sqoop. +But what helps him to thrive +Are Impala, and Hive, +And HDFS in the group. +Hadoop is an elegant fellow. +An elephant gentle and mellow. +He never gets mad, +Or does anything bad, +Because, at his core, he is yellow. diff --git a/examples/python/resources/stock_data.csv b/examples/python/resources/stock_data.csv new file mode 100644 index 0000000000..7b4a77f8f0 --- /dev/null +++ b/examples/python/resources/stock_data.csv @@ -0,0 +1,96 @@ +symbol,timestamp,open +MSFT,2014-10-31T00:00:00-04:00,46.31618 +MSFT,2014-11-03T00:00:00-05:00,46.26685 +MSFT,2014-11-04T00:00:00-05:00,46.6714 +MSFT,2014-11-05T00:00:00-05:00,47.16475 +MSFT,2014-11-06T00:00:00-05:00,47.22396 +MSFT,2014-11-07T00:00:00-05:00,48.26987 +MSFT,2014-11-10T00:00:00-05:00,48.04292 +MSFT,2014-11-11T00:00:00-05:00,48.2008 +MSFT,2014-11-12T00:00:00-05:00,47.91465 +MSFT,2014-11-13T00:00:00-05:00,48.16133 +MSFT,2014-11-14T00:00:00-05:00,49.07897 +MSFT,2014-11-17T00:00:00-05:00,48.75336 +MSFT,2014-11-18T00:00:00-05:00,48.78283 +MSFT,2014-11-19T00:00:00-05:00,48.31615 +MSFT,2014-11-20T00:00:00-05:00,47.66082 +MSFT,2014-11-21T00:00:00-05:00,48.67361 +MSFT,2014-11-24T00:00:00-05:00,47.65089 +MSFT,2014-11-25T00:00:00-05:00,47.32322 +MSFT,2014-11-26T00:00:00-05:00,47.15442 +MSFT,2014-11-28T00:00:00-05:00,47.61117 +MSFT,2014-12-01T00:00:00-05:00,47.54167 +MSFT,2014-12-02T00:00:00-05:00,48.49488 +MSFT,2014-12-03T00:00:00-05:00,48.09771 +MSFT,2014-12-04T00:00:00-05:00,48.04806 +MSFT,2014-12-05T00:00:00-05:00,48.47502 +MSFT,2014-12-08T00:00:00-05:00,47.97855 +MSFT,2014-12-09T00:00:00-05:00,46.77711 +MSFT,2014-12-10T00:00:00-05:00,47.24379 +MSFT,2014-12-11T00:00:00-05:00,46.74732 +MSFT,2014-12-12T00:00:00-05:00,46.35014 +MSFT,2014-12-15T00:00:00-05:00,46.86647 +MSFT,2014-12-16T00:00:00-05:00,45.57566 +MSFT,2014-12-17T00:00:00-05:00,44.73166 +MSFT,2014-12-18T00:00:00-05:00,46.25085 +MSFT,2014-12-19T00:00:00-05:00,47.27357 +MSFT,2014-12-22T00:00:00-05:00,47.44237 +MSFT,2014-12-23T00:00:00-05:00,48.0282 +MSFT,2014-12-24T00:00:00-05:00,48.2963 +MSFT,2014-12-26T00:00:00-05:00,48.06792 +MSFT,2014-12-29T00:00:00-05:00,47.36294 +MSFT,2014-12-30T00:00:00-05:00,47.10477 +MSFT,2014-12-31T00:00:00-05:00,46.39979 +MSFT,2015-01-02T00:00:00-05:00,46.33028 +MSFT,2015-01-05T00:00:00-05:00,46.04234 +MSFT,2015-01-06T00:00:00-05:00,46.05227 +MSFT,2015-01-07T00:00:00-05:00,45.65509 +MSFT,2015-01-08T00:00:00-05:00,46.41965 +MSFT,2015-01-09T00:00:00-05:00,47.27357 +MSFT,2015-01-12T00:00:00-05:00,47.08492 +MSFT,2015-01-13T00:00:00-05:00,46.6381 +MSFT,2015-01-14T00:00:00-05:00,45.63523 +MSFT,2015-01-15T00:00:00-05:00,45.8934 +MSFT,2015-01-16T00:00:00-05:00,44.98983 +MSFT,2015-01-20T00:00:00-05:00,45.97283 +MSFT,2015-01-21T00:00:00-05:00,45.61537 +MSFT,2015-01-22T00:00:00-05:00,46.16149 +MSFT,2015-01-23T00:00:00-05:00,47.02534 +MSFT,2015-01-26T00:00:00-05:00,46.66788 +MSFT,2015-01-27T00:00:00-05:00,42.6465 +MSFT,2015-01-28T00:00:00-05:00,42.43799 +MSFT,2015-01-29T00:00:00-05:00,40.64078 +MSFT,2015-01-30T00:00:00-05:00,41.25639 +MSFT,2015-02-02T00:00:00-05:00,40.30318 +MSFT,2015-02-03T00:00:00-05:00,41.33583 +MSFT,2015-02-04T00:00:00-05:00,41.64364 +MSFT,2015-02-05T00:00:00-05:00,41.92166 +MSFT,2015-02-06T00:00:00-05:00,42.37841 +MSFT,2015-02-09T00:00:00-05:00,41.94152 +MSFT,2015-02-10T00:00:00-05:00,42.43799 +MSFT,2015-02-11T00:00:00-05:00,42.34863 +MSFT,2015-02-12T00:00:00-05:00,42.38834 +MSFT,2015-02-13T00:00:00-05:00,43.07346 +MSFT,2015-02-17T00:00:00-05:00,43.97 +MSFT,2015-02-18T00:00:00-05:00,43.63 +MSFT,2015-02-19T00:00:00-05:00,43.27 +MSFT,2015-02-20T00:00:00-05:00,43.5 +MSFT,2015-02-23T00:00:00-05:00,43.7 +MSFT,2015-02-24T00:00:00-05:00,44.15 +MSFT,2015-02-25T00:00:00-05:00,43.95 +MSFT,2015-02-26T00:00:00-05:00,43.99 +MSFT,2015-02-27T00:00:00-05:00,44.14 +MSFT,2015-03-02T00:00:00-05:00,43.67 +MSFT,2015-03-03T00:00:00-05:00,43.56 +MSFT,2015-03-04T00:00:00-05:00,43.01 +MSFT,2015-03-05T00:00:00-05:00,43.07 +MSFT,2015-03-06T00:00:00-05:00,43 +MSFT,2015-03-09T00:00:00-04:00,42.19 +MSFT,2015-03-10T00:00:00-04:00,42.35 +MSFT,2015-03-11T00:00:00-04:00,42.32 +MSFT,2015-03-12T00:00:00-04:00,41.33 +MSFT,2015-03-13T00:00:00-04:00,40.7 +MSFT,2015-03-16T00:00:00-04:00,41.47 +MSFT,2015-03-17T00:00:00-04:00,41.37 +MSFT,2015-03-18T00:00:00-04:00,41.43 +MSFT,2015-03-19T00:00:00-04:00,42.25 diff --git a/examples/python/stock_filter.py b/examples/python/stock_filter.py new file mode 100644 index 0000000000..6afa947b8d --- /dev/null +++ b/examples/python/stock_filter.py @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +import os +dir_path = os.path.dirname(os.path.realpath(__file__)) +print dir_path +input_data=dir_path+"/"+"/resources/stock_data.csv" +data = [] +with open( input_data, "r") as outfile: + outfile.readline() + for line in outfile: + data.append(line) + +from pyapex import createApp + +def filter_func(a): + input_data=a.split(",") + if float(input_data[2])> 45: + return True + return False + + +from pyapex import createApp +a=createApp('python_app').from_data(data) \ + .filter('filter_operator',filter_func) \ + .to_console(name='endConsole') \ + .launch(False) + diff --git a/examples/python/word_count.py b/examples/python/word_count.py new file mode 100644 index 0000000000..a4a83b144d --- /dev/null +++ b/examples/python/word_count.py @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +import os +dir_path = os.path.dirname(os.path.realpath(__file__)) +print dir_path +input_data=dir_path+"/"+"/resources/hadoop_word_count.txt" +data = [] +with open( input_data, "r") as outfile: + outfile.readline() + for line in outfile: + for d in line.split(' '): + if len(d): + data.append(d) + + +from pyapex import createApp +from pyapex.functions.window import TriggerType,Trigger,TriggerOption + +t=TriggerOption.at_watermark() +t.firingOnlyUpdatedPanes() +t.accumulatingFiredPanes() +t.withEarlyFiringsAtEvery(count=4) + + +from pyapex import createApp +a=createApp('reduce_app2').from_data(data) \ + .window(window='TIME', duration=110, trigger=t,allowed_lateness=100) \ + .countByKey("countByKey") \ + .to_console(name='endConsole') \ + .launch(False) diff --git a/library/pom.xml b/library/pom.xml index 17908ddce2..c334c6e32c 100644 --- a/library/pom.xml +++ b/library/pom.xml @@ -29,6 +29,8 @@ 3.8.0-SNAPSHOT + + malhar-library jar diff --git a/library/src/main/java/com/datatorrent/lib/io/fs/InMemoryDataInputOperator.java b/library/src/main/java/com/datatorrent/lib/io/fs/InMemoryDataInputOperator.java new file mode 100644 index 0000000000..b1900901a8 --- /dev/null +++ b/library/src/main/java/com/datatorrent/lib/io/fs/InMemoryDataInputOperator.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package com.datatorrent.lib.io.fs; + +import java.util.List; + +import com.datatorrent.api.Context; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.InputOperator; + +public class InMemoryDataInputOperator implements InputOperator +{ + private List inputData = null; + private boolean emissionCompleted = false; + public final transient DefaultOutputPort outputPort = new DefaultOutputPort(); + + public InMemoryDataInputOperator() + { + inputData = null; + } + + public InMemoryDataInputOperator(List data) + { + inputData = data; + } + + @Override + public void emitTuples() + { + if (emissionCompleted) { + return; + } + for (T data : inputData) { + outputPort.emit(data); + } + emissionCompleted = true; + } + + @Override + public void beginWindow(long l) + { + } + + @Override + public void endWindow() + { + } + + @Override + public void setup(Context.OperatorContext context) + { + } + + @Override + public void teardown() + { + } +} diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/Reduce.java b/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/Reduce.java new file mode 100644 index 0000000000..b6f51cec17 --- /dev/null +++ b/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/Reduce.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.lib.window.accumulation; + +import org.apache.apex.malhar.lib.window.Accumulation; + +public interface Reduce extends Accumulation +{ + public T reduce(T input1, T input2); +} diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/ReduceFn.java b/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/ReduceFn.java index 2b1b635450..f52674b971 100644 --- a/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/ReduceFn.java +++ b/library/src/main/java/org/apache/apex/malhar/lib/window/accumulation/ReduceFn.java @@ -18,7 +18,6 @@ */ package org.apache.apex.malhar.lib.window.accumulation; -import org.apache.apex.malhar.lib.window.Accumulation; /** * An easy to use reduce Accumulation @@ -26,7 +25,7 @@ * * @since 3.5.0 */ -public abstract class ReduceFn implements Accumulation +public abstract class ReduceFn implements Reduce { @Override public INPUT defaultAccumulatedValue() diff --git a/mkdocs.yml b/mkdocs.yml index 75a862a9aa..c6ecb16841 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -5,6 +5,8 @@ pages: - Apache Apex Malhar: index.md - APIs: - SQL: apis/calcite.md +- Python Support: + - Pyshell: python/main.md - Operators: - Block Reader: operators/block_reader.md - CSV Formatter: operators/csvformatter.md diff --git a/pom.xml b/pom.xml index 298576dab1..6f64a20d14 100644 --- a/pom.xml +++ b/pom.xml @@ -52,6 +52,7 @@ 3.6.0 false -Xmx2048m + 1.4.9 @@ -236,6 +237,31 @@ + + org.apache.hadoop + hadoop-common + ${hadoop.version} + test-jar + test + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + commons-beanutils + commons-beanutils + + + commons-beanutils + commons-beanutils-core + + + diff --git a/python/.gitignore b/python/.gitignore new file mode 100644 index 0000000000..d37a857dba --- /dev/null +++ b/python/.gitignore @@ -0,0 +1,2 @@ +apex-python/deps/*.* +apex-python/deps/pyapex-0.0.4-src.zip diff --git a/python/apex-python/.gitignore b/python/apex-python/.gitignore new file mode 100644 index 0000000000..2723de6f7c --- /dev/null +++ b/python/apex-python/.gitignore @@ -0,0 +1 @@ +jars diff --git a/python/apex-python/README b/python/apex-python/README new file mode 100644 index 0000000000..e69de29bb2 diff --git a/python/apex-python/deps/py4j-0.10.4-src.zip b/python/apex-python/deps/py4j-0.10.4-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..72bb34d0fa02c76b34d26f2665387426a41b2dac GIT binary patch literal 78717 zcmag^V~{RPuq=!|W81dvS!3I_XN_&!)*9QkZQHhO&)j#P`1Zy5b1R}Mx+A(W`d4;T zb}C4NfIUGEbMIQ?LGbv(Aq4p#{cC11*EV4|D4m;C*&oh zV5Dbaq-Ujo{vUkQ|Ii=-CB{Dz!tDe$&0zoSkp92$oh;034XvI27hgajOZ|WH|H7|U zmyO$GN9?{-M;sT`;@rXaq=bE&BbKcf$NrmnW*Q|GfGLC!m^gkE2)9 z|C5|q&Ey!IcX~NE0jqQ&SvDNx>+*X(z|f+~;q`hcwH=XTbUTWL6Z1=msf%Wz`2?C& z!$-ZPy*%2nchbTb{ygmBvOf)0l+k(-O+$eikHtA*eS(72q)29L^|TA=kDvsw)#)^e zy~gErRp6VBC~tS}fwl>>W98`Bq$VdAfoKfg2cinSpHBnujJZ)(mrcj`79JtBV_lv6WDnCDaOboo4f-rSIF zk}ujdA+J$h#A;9Oc+5$4|3=#fS>)Z}v^KzpH=GyEicX_h<3b7rrD{n4%r&tDo6!y{ z3}c!Ad3>HO2T=+$;BVDw!vN0fU(0lSa2ru!GtU}1GpK(9HO$6akR?sVD;6MFw(Es^ z2t;3MnE95-bs3TqxHyG2pv3#ohl6wD8QkS)*U{9X!NNXw7YIjdE)=t%lCIvK`o3b! zE&|hX_ITMiS0^togxd~V$`5upezZSV+}*v`e>5eyf%2Y+r#v}DpC}N*89K977Oq{% z$oP!GzjfX6@Q|giXeyhYv$_|%&EG7*bO5Ij?*->n4;icCcdo-PBzcOIbb8#HFn%iD z&&d$r>GW(bUVy?)TA)5_1|gg0uFDna){`LZm-|Ir>FSg~AcBc3c~Yq|u{64S#}JdT zUh#wS-&|2xjUQ->0y)HC%}Od8iN~SzcwWykiX!80b3Bjf0KwIJ-VO<>m9XbX#e8>a z0i?uh&a=T1&n&^`qf-|;a7NAh9CtH#pnvkj!O$lCz2}ku3P8A;jF|yAI~4OV;(r7$WO*iPq9D5bm{`oXO1k8+OWYn_ z?P6P`pBGm3KwRvzO>;6LvX(g0Y`-K9kBg+F1)BA38;o z|5PZ)0eQou_P~*OaU*Uz@{}DMdU3nr=?I*^X>H&I7nbm*rjXA5i?DqA<;v>7`CUfIsK3Q;rLK@0< zCvQ@}IvGitqrW`DbG2FNj94}x%H-uDI5=y!dMseZ)e%gmB9ESD6utPOVUuBW--D2= zc)>M>#Y}sW2vK4t(QT&Rf@8H71`Wy{rm+R=x-i)ZCXHxSh;@waoe$%g_4?3dFCUU` z-TuVc8}1QN9AKso`_f;WbkeHW)jf0{c*pDrjy86r*>clVoO5;lWm>|Q-BGt zy1704oU_$7Wy|UDPj_8s>nZhh_(b0@^6cc&6-u|GVgiN))R@cr&%8;&ylv!jMI@qI zJk1Z#|A+n4NJS)%s0Aw^5dnZUW&i;3e_t1SM>}UbV>|2rrT_4w2lfBS|BL>cv90Vj zIUhQG!Q?-OciMy&z&l}fOZ*SZsykz^8?*W5=uEUqY??hWJxQO@zi&B&l~Ae1efH@o zGs0Q2ux4OS$<|wJ=w7377f9v5NJ$kNsD#AWt#O8EN&bX^DaJ#*-boz7=^LfXBM5ep zh>ozE_%!_;olK1kHwKy}B)N-A%<`EasHQrgB7e_0fi&H;*@K{+8E2Oqj~%8h-Y*nR zz)y_UM4EgZ4}D{W4N#4%iptt*e3B4gZoVfPdnGM8k4u)DsK~sWsiNs&jr}gC{zvy2h{CD4Yg@pDMZ)uRxlr=0{ zQl_Nu;-EnsKUY6XTP+t|fciSgDkG4TC@S_SG|j4E3q2s0asLOzI^@Q)xU9~73QJbm(|hZ%>|nK#*>eN1L}Y= z*gy3g$^h=9^ZjY3@fFhM5St1-YtWu_{8i178OXEq^ij;M<5`F^G;L$WnJr^r$eQim z`0ji%lWHf6yW1i_`ue);y1r-I z^|WW#&i!qGQ<>CX9d(IVYmRi{efr>TknZyBdQRVUyonVy$j%x4iCaXK3cQIePcggv zxYklrx>(PhR#wc$n7N}Hr`y~h_%b+uZBX^k_73(jL5!-Ajx&bajbhN!%u+k(9T#4Y z-oFmUtsMWAaK>DbH>|~lJFQ&YQah>BVVJF@dfnh{bjAnbwk@BTobCn)lu*b3`Fx9g4 z?mbcN7l^V07j|P5ATPc&@nFjb^b-XJ85{L06!sNDOSpC9n_zYfE{_jyFNJI-!e1wJ zB+foZO(x;P^Eo>N!jbuC6&C0@cz%5i@SPnZp6~_dAX-^cKENzOLk+-st$YW`mr<8JXk~#0O2~P+ZQyz2&1W4;7 zEgl4T9}-T&jkg`rl{nQ5E!O;TBbs2?VlPr5qT+3>(Cm0K;WR%MMVq1f8-gA!VHho! z4~kvbDM>S-rgfp&HxQE#?J!w9+3cCsr_p3=l6XfKmAK^m*F)TH%VBdQ#x(|F^4Q?* zpvc$%3kDq@_TVdVa%-!Bp&VH79KSRR!?s5di>2|&10PAw6J7qQmE~OY+5yo4{ltdQ+T`-|If=fX@sV_%7$l*TK@K8 zodIsj?{V;4RB<+a`+yJ)T^gNRWn`@lO$!v>&sxfvXZ31}@C(5SdH0<4CMa4Skg&C>$m+dpPu@gK0B z+Aji#VFDmoLY2|kzt|f-fvwy@FK+TJl9&%US+%2c4~{me_T*aT<|-)T8U`a!tlbu( z;y0HB{_-MH2_IdO7~iBVSAl^dvQB4$9|BTH=;6myZ@0;UpSZ@#xhFYwjxGiiXb{&1 zek+6YaK4w73rGcf2dC`<(X!ZQfn00ctZ9C zvPxCLYUlD7niJ&J2?6ZRq>_+kidHS9uRsbz`G`;V7LEoM?n9&DC+$k)1Zk-Cn5s5! zjKdHyQ32r=tW@s3*m|nuOthhQ3*h?o=_^9q1bL=6lrVWGR?=^SaO5ipa;}g;U4^at zW@`klow~;I-1r(zLc(x(_5x(HOC6$K7yot)ICaT&3D*L}f-u7v_Jq?j5+&+x8odKT zV(=U(3$FL3b%D^7r*8JuilaZg6j;(`O0k63&AfAja+cC1ZX{OdAFd1euWC#(>qAAL z+j;Ua$vlVw55K%fV?%27zy(5t%*pwM|Fq`g9XgPQv^$XnN|*Q&%acu!Kftr?Y~n1w zRGGk0woTEtmgG~^R=!eipcm4}MM%oYb+b=kYqysEGKUK!vW{sPKM39S!K%POWV(Y(^HNxnfbTS7<7ybRrw6hh}c(2g2epfyq=lw0)(j# zKomve!s!lMirg%nljyY1lk4p|7BT537_=pIiB;r~?o$jTo!2xNdz?cX&TEjNp5-ON z5$`6Eco&J0H6bua8oHKZ6_(5e97B;dC4vUPqBm8zIZA%Lzk3DJ_N+p9iC(PeCGTe= zaVyZ8kLFgmxG)NddS{8Sn} z^B^jq+OZ!O@zErLW(O`eVF}yfr=@1?+W|^blO+C!N{~(b*W!hT*Jl`i1M_EU>iU|e z0#pH^%|=eOn}*Yhe8d&}L_6;nBXtF)V8=N7q_{VM)2d1Jx~nIrR4a{DA0~@UxZH^n zn|**lZ_9XKLrB^G{K{2$(*i5HWXhzLSq zlg%!ouWpJF+a*oOE*WXD8p}!cYY&?Be^e5zwqn4@tg7wx#^>r|kTY~D3L8`qQc?qx z$CDN$oe?Oj$QaaW&WX>Bm)f$rFv!J`Lm6+r0i_d^1L(kc1-8`Z)C`{V>|7?;!VK6! zKn<|`B=ODhtgsBhN-d^TI`(S5T7*w?%1r|RezNeh{K}|G9YKl69Dk`LN3PbFLB6|J zF8O#oYVlwy@Wk6%qut?J_g=M!{`E#i8Ln+rjF|x#YHY3!+_5&QKuTOc9nhI;ZZOJA zIHXvO4D#|W&TY5T+vX0tNh@)vK%x7AXe5n`K3R-AH;h|2!rnMQwLH=8e$8v*S?B2Y z>+__g>nUMSLELp3mYR%*dsQA#Q>O-?K)e0R8jiFk7^vwHl04!TgUUAm_yb<^oe1{b z`{(#^ICVCzmy^@VyqHLg4DVQRPLR6=l71V1K4bG?G&)T-{g>lG=oLWJ@$2^WKAxJJ z(2d%i*iH4yVfy3ip$+<#VYI2Wos<)97&KTD?g^sL4H@UC&s{#o$G<;X!NOLPcJcMx z3HFd}tV%ha-AnPS#4v;d5&-WS#>-UA&s%YRr&le4(okd#!IV&}r*x#i=x*8~!R-iy zu&gXex*Yb;h(bONxLt^*w|KxmD!>|p-0Cdu()<|HX-B2Ilx0)O&N z{98--u(r()nYJ81vte1o*a0s>wMUte5KA=yG64ZF8j)cm&TVkNxP+uh2;EVnS9L*C zg;Z<`v+}A8_X76?M&uxNtBf-7(C@{ZOBlS-V(zm3+b|J#ox9WU?I2X#$!72Q92@O) zF3zW{8@j>ybVjM~aBj-1Un++cJk6w%T;=e%_L<*e|FQR&{lltPXdr}#z~CN1^o_=B zwRr+XF;)$uT(u{mUT)mP#};>v71~iQn(}!@Z0LhCn8+YvrMPTV%Evt=D|#~ZFI}%^ zOj6f~=G!q<3GjzAS#*4$S0#9zqv>NPj=BO1pA?|l0}{-Gyprq7oVBthLjF)m8qcA+ zgU0t!?ZmS7pR$@5WPHvx1W@!xKWq1-A?0Y=3K0Tl4fj@W|0&q`*f0Ql0N{766C zY66RahcMD_O1Tnq-%peJwbE4O=g04{d(W(!^P~%-pSJ)wME*XsQ#12|is@c`xcV%G z#}f530bgeNOqr}F^IHy@tA~fks0jV&(@$29aGd=3-ZdRz+sFEA6C63|AvR%4Bp%eD zl%tLV6lzHN9REk-Y8Cur2GMpX5^GC08J|;xgLeX^oJmwbXAoj9nVr?05ydT#Y8n5$Pe76WIJvK-*IwRrTWr{hE_1LQhR^=DiSl!Fs|6fIcxX&6? zR2~2bQ3C>?{;#57Y3ORGZ)WIh>SpNi|3_8W;<0kt6ieKFry-wG1$$DgyE3`8RV_wf z7Tt6tRisgRIqssjmw=VjAQS+}GqI)q{r+HL@Gq(w>Y3iAqDlgkcy)Mv@Pha*0Sv>{v8!UPtlhl}f3!OEgHKIci0G7HNfhKPbvpBHut$?Tm3#?U0K$ z|0{2cUldKGVxoB{j<2qiZ60epS*%<3q=^!dQ&NqzB%-O8If;^zQP!k)54Fr6%523b zs4~f?NJ-Z)tEZl@&?{3DSk#*!Q=s$oa4l`14k$m+G4UXSW}8lxP8pW?(jzsUaRSCd z0adtTl(kjuI0wXZ3yn(+1J~tLl)u%}pb2=~Yl8Oxh;`{9XR}C#xotQ%*KrLI0t8bk;2o7jMFJ%qXP_$7rL1Gq2*S zR+ZK-diRFc$(-ymsaqzWDFrtGdqVvyQjVD0JyF^^p{qKKor4tW$g7F=u9D)qQ|VpY zP}9cMd7%y99@|+rhpqsVfL$K*k2@1n*U25s`uo0wxX!`mwWW4$Z#>+$*T!A>7lBn`C?mqq#(eI+o~V zh}*WI=_RQDsS1oI^Lu67Fk>^SW#{ic#}L(XKEEukxOqHK+f_VPsZmKT_tN-)h0Ly` z7J5$B#H|(x>j(z9MTb@)kGe65ge5aA!W7Lvk%ZDhIli=P7Bg3P1GaS*Z3l%_tWs$9 zUYRt3huuiQT_}FV-IHd;{N{;#ilMY@)-p#cZqq3}w<2U#70pYbUs}y3DfpwtlK)Q< z%|CLw!^B0jDtCVudag+eYcB#FmNM4LoSQBPbB#Z&*1k68shK-J+L&c(w2v^@0>SzX z$y=Be;;=h4)3tQj)8X;K^p^`e=Ubg1&zI}#vbS08lR@5gMtpw!{M^NlGkEjbZ1$Iz zC-^s~`FgRk{}~dW_jfXm|L4s1huM?2ExM)#;`cp@|9~Q_Y4uV+reJU{!5#QTXJr@% zJ-&O!XcbZ6C$w143Uv1Q_n+u0!e6E-+7;ETT{HdBm@EO%qAd71sG(Nn=^v=9W7B6F zO<~%y7C&NTy6Ds7Y=sMHswlaa0*{1R_4#XF89i{WArJ+A8lv%K3!WJOmM=C;ruaXj zibZo;bV4@$zNQnZeNa85Rr3UcmSV;P2*b0hVjX#mv{1NNy=>jswkm+RgdN!*yg5)^ z>flh~51GI57-8wb$v@oTiJHrS7I_6&^5UGpj_>1LEj zPGAf+>EkX?fmGzM?#rL?G6LO~T|?iwpebTc7v8q^kC^nMFS)~o2YRYo?NsW(U`trk>PrLzQLeTTht#d(n3nm?#E0^L%GgqvGj( zlu8m%4U8{HA_#Y&*!|)D9uGFWC;zLsuG?h&2HGrFPTjsTaOu3trVwcN}H&+3rJ5Qm;^#0b5XmB zNYOi}_sk))bfCL4vR0|WSfzwd4t_5vLSEBXbEFq!b`9<1pOS%I{GZUDkp$2ma&S0Me&`JCGE?7C+p%Q6 zUcS%j@7IkdB2p%4&Fj75#z8Xg;jVo1sjVKM#-HbRJ0T|wzKn*9-lXG;Feep(IQikc=Vt#GrsciGaR!hPYFR3L5ohUoFB|1hWm32koXvYUB@Hl*? zAGfJRJKu7=GFjnvv%y$R^2Fs2Sg~GkOzzvHz+S)kVd1C10=|2wiw+GLzfUV}u z-5(2>MyL`q86$vNzF({02x5Lj_UOv-j_mJ(efn(wwsW=rRR<0@%DH&p;wCB~EZIjo zn~lXlXx|O6k3pB{!|a3$)^$iXCII_=zF(TNsLmGrisG9f)*83mKc*`r?yn*%1@iPE zQW4BiqwE<9MFbRBl>S+F*)BeGpMu!&Ge`;f^H%{OUtB=r4w`7o3TN;cn^xvgNvHya ztslMzQv5L#Ingt|1n5+JcMw@+Gl+2Pz~zlep4khD0t`Dfs`Lbil!FS2hIoq62BJU_ zlkfO9LSpH;w2a%;R~X>%42IlN!rH_m5(u^-3XN)&$1S0tTak@N{3*6TiP4}SGC(i5 z^w$|xepM6*+FPMaWn5{Qs8_(UJ)lx(I&oH@?`ItfX62x0b`J@v(CInL8Aq{GJ z&#l(%bRC87{(5o+vRub(F~mU|=p*V-*f{_!P@#+>)w0$;h0u#8Zr5)O`t1#af_i{{ z__XMtE^b;x1C>ow994M0L595-W`&?EG7utCLuhY0(XVWbhaVs9avPJfiK+Sxt$bG* zu(Yz`3PTM+c!0wBc{89`Vja3mDgk(?xb@ehI0Ogh-LuC(x>%`> z95S2gA2JQ}4lvd3;{>frM<6FB$nsn^ZC9l-MpFt&EMrHS4Lk-kDR&yxQ<%xM#wI$2 z@;IQc=b#~ZtWwsheojv>M)J}%P1Z~Dz+GSiPO`q0OK5zepl>)0N})mMq9I)gVr#ok zegI{u#EGVPY&B0*0kVyaMABY)6gWDY`%I~rg7X7d1&neAzWc<$MDbUfsm znF>b_!brL5#=e&V6~r2S+t_pf9qymaK9yM{kqcd-YKXRpl6OOml9rjIEbA7y81+~{ z{D=|KrAUI6U0L>xUP-@~H{0S_%&$mE%%Cya+US$(6{%|B+xTBF^g3|2!d7o-?Q1Ow zv%(fkvFa5#KA{GY*&PKz%e#DhBR_wyz@@y02-`>?XbyC1bq2xKc1!K?U~ObR5C0;d z=QT|gF!w%-jKWtF;6$ZX{FOa#_CTW#Qq{?tv^;R`%#4Y|jk>1s0IMvi?mpSb&b4qO zoFQ^z`1~QGNf^@AzrXT?)!6(Vrm zdhCBH@EB%%F8Uo5P{f{D%y~DRcqo~_Z)k<9*QV*zy*AdF-Fna55!gnC@wbTniJIdP%J+qAj)O_1YRq82&4(yAn2C}^U zF^)9L1&>KQ=;UKS$)vJ3297skwEN)Zsj9eAv(h}aN^+{?QUCoQ;jq9ZF{3X3E$*@= z(q^ETC~=}c-CB0k* z(9Oe#*-%5;oHamXx5LTim3BBv$Y7&UNf!snW6%)fkgC4b`^BandAIRg*aX_d{yMEvh|x-cyR zNXyb|npM)TM2OHg*?1;%`&BkLb+b5WSN^xdyhEf9emYZm`v2DdMo=79`EnFF$PC;S zg-dQ>6qP-8b)aT~yhJP#ZsA1fSl@l=w&@Ks^0+VQmE?dD6mUksS2L0qa*7fCH^jBy zu367bHg2oBKMYZfQSVUweUqlPr((B9I;+|dQQRUMG2NdF{p>5r{_-hYq0u}g9&nDX zrxj2W%8oVi6|>PT{SxLvXiphTX^(OmAsd0rmbc3lQe{7-WXt2~EX@z{q1iygM=QZ| z>dMx16*BsB#$(_wA*t*0f>bYThDT&A9!GK%NoHs!fUA4o(|;5uzW=CgjzsX(#C@%R zSi%IsIWyE5Xe+uF>+DlvrHTesG*c|+j|Pr~i4;|j!-xKXS#ofpXpH;GeN{r0&kfq> zGGT@?;;4HpGkp)MthhbKT*GEIDtHcGXmd5z_Rgtae~RpkFq+)O8iH_sW+t$Ku!058 zA~lTus6T0+p-sv#`*B*@$szGJU&(hJceRc0z!Qwrnrwwb7Q4XT#-NnBrVDzJ^e>lp8nRN zWzXK`X5~Sf7Y}^IxgN<*uka_gF9Ozv8p1&A&o6`wEOWEzJ_X^?VAI-a2XUAQaaQuD zGiYhY9K(VVWNAiR5J^lGVd+Mg@aM8B#|dzka-NiQdUYpAObdErF1xqcMp44;Frpq0 z&fCL&#FGJ+glSgii7^&kuAeH7pD%4W?l;OOTNE3O`1T{}o{iv29(nF+{uAoZ-=La; zkuI?HLxG-^pf3VvLVahf-637;PH5bE6$2AE{r=Y%F%u|IWeTr~=n)oBt=OrH{fBu$ z%s8{Sa*q>&2jc(+(o&-rM0BbQ5heGNcy?ug-NZDSH`4H0eM!tBHQlYem4xO2i8Z8R zsl;g+Q<7>}hSUx0FHzi}xwnFR+a{m*YzG6-Tx6i+c*wv^xH1xgjVxME6$S!jiXv(q z%Ai5=EHJP6Dvye)7lm;{8hg4FDlJ|}GV9D5ysAMnN%qnQ)k7!g3Y}_LOAly9I7ugx zK}!N`3&bi65O_wY&!NQvNi_e~^g*hFh`(hEQtk@#@ikmSQjIT5RhYmtvHIbv>?IzH zn`M*^LopuN4~}57x?KF8aokZhS$o{*!aHFGV}{QJen`~12PT#|o-EeYR62G|3wdwL zAo0J;h6jv7@@G1r*307Xm9)FVZRVmJ_%^2R15M72swFu50I3O<=@SF=C9I0`&Rb!; zfPQ+Cwvuz{ZNJVWL+EBx3DTrdXWTY$9#2Y0;4Q+FaClqDWd;*T(kpP+HnQp#3YX4` z%My#G8_Xh?B<$tIE=ra5iN>^-ZNFE`$?iczx!?pit65d zD^n2>aVhttjAE0u~RN z=b`p|t(O@N)NcluEgTZ(04A)~PHg`9yz{bFo6A!e_*fI=wI^k00D%|O1kCYO^JAZo zi@(!*xwoV#=A{w7y|2GYS_eVg`3wbf&O%#HCMlv(v=QN`8<_mrP>APQo2n4;1F;X5 z13w;zM{rPF3Ej05km*CKDdSos=Xr*v`sx^>$SCprgkJ0h4K6LwXB%5Y@>9GeqhU+R ztZYu6a(U-a5oV?V%U!sWGOH9?&Li^HaZ#+8bKHmp+HCfbVG73q-)`b@cJ&i<`r;w0 zBQ#P=8PR5W$*5J{dzF{39dH$?M#DJa00OrC&gV#}%b{aougpa>6g3bkh(MmJN$-y3r(zNdQ zvNLaP4s@BmSzq%9V~{J?VMRf#J3eeg5`rR!#7Aj*+pw)+;`v@9s=5aKry}!}3V!gv zFe_;qywjwvBxmD4;B5{U_;Txi%s8BrhppDGiQVOAN*_E$%Wiy|k=AoADrp^5w)@dt zT5TG6Im9gGoXpYg_A+^)%f5^bUXt5a%z069*&RS~heXGP?{1<>?}(xsyPYJwYxc zNeJjw34maiHH4ohI}KPOGU4$hv#$7{Cs^8xlIE%TD%zHys1kBFV=cRuV9z8RZ3f=N z=UQwOLf26`nq%I>d2vR#gv*Hry86hnE#`*Hxbws_KUohe73=QV=F~f2Mb$Pq z>XlABn*ktziY2L_m*B97p&D<3G*vH=V|XV1dl97s=yWXnQ`cTkeRqlUk8Z>>d>&*+ zA#>QQ1Y~}Xu6{pOcGp@U64T4Sl zto;(Mesu1_BX#TTFxhoB)E3WgFps_EZPpjMe1jJJup_rIbGjFV*0wKqz5nAH z`0mwKorkvibA!%dt>M$U+gJ0;#)W{8^mskKw*I!(AX1CHvpIXQ{x7sTfarecx=f=u zp0J>GK+-yFP<6t6YP`g;#4w6MZtmU15~qezeDDRe*UKsM_*G|iA&TY1?0_EBw5;uO zT2n)r{C9h&mA%!06w3Ft_AG#Xz#k*980vNy-0n;_qrfzlOi6s-q1heBMqk-q2L*e+wqYxmPA-4*7?5E#7M_hT zYt(!8X>mwd3a{=)|>9fXDH5D%%;OIh5JPn(EIXK!BJ;0cvI7A!BNMAIWs+PC(!6m zs_`}`o)JjVMK{c{NI|rtAk4U9q(=8RO4L>Z@7FYMXA>_BF=Iohyu%N+-e3W53$YxL zjYp&}yM)!DddFmBvZLo&3&GyQ5bWQ$4K9QUf(hJ_VGSY*M$846e1`q3(8E0994d;U znF;{ocSZBmAQX63lK)3A)L8c?RNmo=L0T&6L zr@>*Md1sAuG3BhzTt4~4p7I@qnUrg}ixac5ga0|JJvip}-)*D|Wx^TeIYft|w<5mN=1Af?}Jph*)P4Eyu7m)6zB+j>(dZYx1 z)KuA+r{T`kU0&U5Q$i;>o@2D^zdYmoNX#|omv~zf-#z!qKC9QKe8o|M@wmIw-a=hy zc_5%fD;23$Clr9|r(pmquqc@WI!MtB(Vo78wT+2y>>|Jh-+F@hp~#baq1<~ zdCngn2s%%>d;gTMMTY0^68Z4_8cxfow0&8a=SW2U)BvNdInTdAdgv|DFHE3`XtHAF zhbd<$&Ldn`>f%oSvE~`*B5qjAoY~zPj3c}i1ImYP)fVJ2Y+i5y6z_Ap1jd>btnM&z zI>TYu1z;8{Msd}+d3vzH3JLkQDh&HZ`>xM}5-}$%sw$j$Fq$|vR>d9{n}T$}r0UvE``SP2QfG07@gU9CKhaRV}XNO!SPqN;EQDheg>fnKt=z{Ld|}| zJ5q0q(L0Ng)ZmP3(G&ahAkTF~ari7AG*>+_K1^O#)Y)^8M)UHkM>8EFE2%;UbK-45 zrpvY}UDcp*l2!O+nZy+}88(GmygLvgIGQML(n`q|*D@xu*j8`Xl(pg)s zf{+ea4$5^=amAg5MLslkzGm-YEg2xoTh0kq%#rwDcj3ubrFQQR4~8v+fEau$bfqt|hhEvu3mwJ67CaM_Xf8;%sjot z5P*2ah!QODstuqCQV0BbRHaO(@(b{}ji!4{o78{9xr_gq>xIX`Dgj5B0!Jn#-ztd0 zR(>Mtml~Ne7?S*`sz|ML@N?38;72Td?|d+S5qM!m5pnoNOTR6bBvK^*SDByH>Djg% z!|4cw`!J5s5R7aNnF`+qGS>s%Kog3Xzt|>XE!6vu&AjMg1w~G%`Eht0bpaB!2V456D~|L-X*eGLHbL!7JTw4$TGpdofBZ z4_gO((m4u2`0@qDSusjxZfhX-^g(6!$4&=oMg*C}34FMLI+3r9+TpIXN%Nj&ezOeo zfr3l~<=`XX2Py%yZZE)(Tc&=iLLsFXHHKzwQzzT(F617+VD zC=}Y5mMnrOEHfYkq={#1_=i3{3lBP2AaH3~@6aZ##q4r((~I`tu+btx?ho}harFdK zXj=UN)-F`g=cjxl9;fhM?xlUi@H5B7J>e;Fqvf5ws1d>5lhnzw%-R#@(5}-<$3N?J zK;J@NxLc}_&E51s_IV+DcG-<*zC-BEH5t4td$pb<()A@o*kL^`NVr9|CQd@z2D$5X&uG~ z)9|47tZ&$4wjMe+7!YxFTXj$a9^lSSC=#N;SgVVnDG?rZv(Pv^C^6ZH;u3R)@1`bN zx^wJacmP)ZChxVmNG6TJ;7sDE3vwC1??@h-e^3h2AElkQ4Tdb-L0K@ccB#v*LhKGn z!*JU$9*5BMA&ZZ?ZAR%8mK)(dPdkUkXf_keZjuTbg;We4t- z+M`Qz`-n24KGX4LF0B0XMq>!s;wwBKQDy2&&1%B8d9WIld2^~;Ps==Xtbx*b<9oHU~J+^7Yj^m+t)$bXV{&~}^`BQM7qr)hTW zZT8XEZ9V^9{5M_^By?5a^@P9KVJ;$}VS;~Rwo6{^smAwq+-2P>_WKm?j(Q_3%c_n( z#{6EBC)fHmpo264q;Qn5eLOQrlk)RV=RRI$>nU$+3sg8Mw6c}UT$-X(gRw)eL0ngT z!7Gg~9x~^SJc$ScQ)udl<>FQ&u6k%d47~6)|LK*qqh&RH#ubLgD4RI(1Y)Pu%I)4h zTrzotfP!Xe+w6Mub3c@!2&L35y@nF0!^tbDRt_f1gd9D(Ft4#krbKRCScfIFg5-vO zmrP@8&-+J1;uT?5wWF54>Qh1ef?`8lftqY0P?cpVAl2CE1=BWrg^<)%)c(7*BQ?mp z*NbW4%)s_G%0H`sooA>hKA99r3fD%retmbp>{+Mlb1n?*+tPCvpUx^^?Q z>xP_^6AWD;LV+H@DI*FU3^26?40FqXvcBh<__!vtqbP|jzA`utPhelG@eHiIVe!_Q z8|2&-PY7N#ZEcQ%^h0z63a(<>l0c7CDF`T2i!+^n(N#X(gF|#(B25hV?td$4pa(DXp zFS|V-dn*C&Jn0PlfXyVg50J$yE+J2KaU(INfl5J-PEok3g-ZqyBK55D zZ(mWjo}+O?!Bf?}2;>RNc&n)6e8mUVdQ*J+C{-ilC|2CHq5BmS)5h-?)A)C_{dzfVdVToRz| zxrf%FZ7k!Z*nIUyV0>LYsQ=lLPr{P1qb$QL2^H405z9>jqOthEDjFTz*y|w9rWOzm zhP;?`;5H98j!`TU>s*|1n^;V~Fo|t04*WiMM0KWwUhUo_n0Va1k*OFcz1pXvwtkfE zJwYqa_NDfcd?L?sZQ%7wEbt5#9E374=D3Vw3%R(td7Xxc1uocUGVeP=0!$6Cj%}9| z;ZE>sMYxOL|ELs8Fco8+{{jRRy_#;T-P}EydHA_JPQ;w`y7?OW^11pnxSH|6_Dy8M z$AlE9CDo*iP7s*owTDTY(hUQGq!|m<)zt|>a4xK0E;jBV>HTDemMe%5$~@yr&w~f4 z;4X=yVOV-usi6~dd^X3^VmO&5SVS3kWIV-Ql%qF@R3_I(#v0hFHS%TVF<-3QA;Szt zSWvoZIt=z%@B%M0V2d~9uv6krHM0wQt`ljzH{j3`^t8Y5tN61#PsoDHSt;hD(m}K_ zB|XYDU9ckwc2_1-_9>!L$w-ZB!EKNrsQTH<`@TS|h^sM_q+p;rp4}d?+hkFfwiMbd zY(7T$^IR*tuz_>I!)4{m4Q&DVJ*u8(Js>cW!qdON-Ka$AA;Q1TSUpldAmB4mjfFv} zIf<-H^0TzLh^qxW^f(6A*z#V!N1>Z#jyPL%lVGQ55xeqW* z&v9;udM@fIcGD^T&Vf*qVGL!dt*J>3ZQ3t`24g&g&*R>-qBm*!B~QgVmmo={W$mua=80DhPeed^76hrpXcYZ zyqT(;tw>aO@rUX*P$ust$qOafh7Dd+d{y!bLQ+A62c;lO3I}Yr^+1W|j$2sg1o5g$Zo0sB(3vBmB?Em$z;H&^3T3S4DolDws$Y zJV+!|H$ZuCoW&LP{w$;Aj#pk6PUiP^bjy^}a!a^N6QJ!Hf-936Q+DY)NV3U!X-Ru3 zLFb1R(VsANLd4CAQIB&P6orVdlHP7#ZgAK=eq%fGrSA4l)~)_9y?FZGuk4p^8hhxxVi{&UZ2aR6Rmgj)RI;JwPFrsL3kFefAV0wEaZ3 zVwx%-8a&vK%o9l3!_WQcunm{mD^a!xsIFYgr=dZqbi~O@e;q_3c+4KT zKl6x~677wh-ATpy4H|liB>qx6%MFt*eFlcGmsOMjj78&;SzWBwvNVGxD>qxyiI=x^oaX?tK<)+Y@+N!8$0nac%UG)ja~;Bzg(yJQmm8|nId%(lbzXlgqh8zcTXeJZodF*Ue+Z%IsBK<6RQ|FpWAV!ms--*y@lkgdg%bjI%W!_1opM4_ACpNteLgWIu~_aSGB8Z*IUm6i-%;ZTcv|m+Poa-E#q<$yGcq)*iF?7p9xMd z?bARNr>|GEC$7h&g*r3pKj3w8dD~NidK#%~^mRrYblfB0B z2&Qn)=Y{W&9Q~vnh1t?ZkivGCU`VK%vOt7B;zCBySUWFpCzH~zpbX~JvV`UUG9G_T zcC*w6B~j^M1)D*WBoC~=WcPfU%e)S6dCx+ZPW+;+MN+_J2HM4#UM|@0T#fLUA{AqO}h6n$qlzbTgFyhycoJufF zNFzy2leXi-2ffTGV7`?!VFsKOETm;FCM^s`pfgUxkk0>IJ4=R`gbQ@n@eecjB|il_ z^N}m$XjAGNz65sy{N}|wtcau=3_~>d2rG{Ahmb&!PxCLz;TROP_`k+UY zg`sUhbwkOhZ+*hR)P|h{%Wp{u623Z9w!R}1Bl5`sION7oPTtdDP85S4@!Tf!=hW~T zRsuD^`R-ruK!p_pPQJ6`yv&fVN&J?;^n=P4looU@`0{-6DMIX%k_*sa|5htrG*tbHl{Y^C@XE6K)9u+{XbzQdkp)&rGO+W!DA{Z91_dAPhZ~eFVqzHFvfoYO*t( z3XhwxK5uG;A8Epin20IV5XCf>z?M9mHno~Izc6;8A}3FgdZ-m1C}N|}8N6Co12Fwi z8V)=Xw$SNF2UfkLpbDT6)RllP-9R&yLxr@J%ixhY7s-G;Cy4^X(?A{l5Y*bCXC&Un z1bw(n8|mKv89uo>+C%6oE&B}R*wlBc*)StItSg8_-)>A|sb$FI<>s?Qn?5|yGJblE zHVOVLjAA>tE<7aTe0xRNdN$dAGNYKEA>i<4LjjZxLgj=(hndfNFfrYBc%b~7VWpDZ zN^{JI_mEI`GF2QyKikGNVl#J_z4S#|-sVL4gOP&VNdDTcYot#KNi$1v^cCCY$LoEx z4%T|r(U&sXm1J}P?_K9T3eY_(DaJ6QPD-M!!d&Pw8_S*&x``m441bea9`7QoYiZ%i z#6(egjNbFPKc=Vch6@K87~)|s;r4YXi>V|o20D{>R-xNZu`>L#u}XHF{X5|B@~qxL zFQS;BTEs7_kpU%+L`131@YNp?d&&1=V}WRmSFyS-6)ZTiBCo<&m=|D6DQlOPvPU-~ zb6CRWz6_*=8;~$?v0Am^s0Xk1;FyH*Wix3JfBGWf1<4BN@8hCoaNu2UMYo|tr70## zv50gEfmCS%>cvHyln9gn-%bLl_fJwZrc#>V%;9rjA&Cw4hg{)m$-e;^X%xaUSL)9P zk^nEt$6jkr1srm%zVy|D2A!8zv9n<-mSa9-QRl%hW|Ha-i8`R%1-6LscXi^q zn=4Dlw||JTwc%?t&q(~WmW}O4H1!BH=j_Cv+k-GkLV;PTBrwbDqOB`B;A{F6)e^m0 zuTz9Z?_`G~frM2zXSTrd5DYrF+K1Q!5Dr;>pOM60-Ssc=O1fVol1jfp#s8huAJNR! z$JN0lu$k33k*?)x^A$?LmQ)EzM_@P-E7=g{xqk7uk$olx6~zM)#oNb8feLlgvrkGS z>CuTzT9j5;tM~kZNKjh7gW#{RWHFfmP@>^LJX&Dq;CCE5P`^UCxI7+377j)pEGE*u z8vcp8yxl?U;l=FX;Bdb_|FmIm@HlO8#@oh2lHOsv6Y%x8Z(Jl*zsW~ev!;vnA;pl%TlMr)`wQf?W!O^} z_!92cEk}vgbZ)^9*Hd5Dz@71>1U$umwgZV*{rUV}ZXuawbNsYDUc zGx)uN`RbwF8(P>)m1?+zl^`n(Z?xip!?3jHs}5M&Q}Zc{_x*b|{x-Z02*oJ93J_B& z`dzox`Mw(@|*J9x}|8}(jPnXHYf^QzOE&i{SYBz2t8@7oV$dkqbCFF4I} zXi+u=9uTmji3YCKcJ2+mOF7pa0p z*YkT?9^x0Yyz6Ypk*Zhoi(aIsrz|z&Oz-=d?<~wpAg;II^A^ptJ{|ad9U=;Tu>aji zU^g5f8&`s_)VI5HYPKo-Rb5GL{c0Jw0?J2|0GZnm2ef6y_u}2HVne|(;&@1W?FVBV zDOogF1gGNirH)Hi5rMF5xCRKxORKwSBUS$*|fl3t|o7 zHFO1-U*gTOX0>zEkQW+N+8a~0PLK^`r^MqBUCnG{d>bAa3!ndkaxukct&Hz?#ut}t zE+hoi4v#}(sQ@^G-UU|=}sA-|rb-}C#W(akh@y=CKRSFuM_YjG(^dH9tCAI{@4RKm>E z4XH^%stJ|E@(J4M!+C3)I>=MUXGoZBFAW;b8f~Wz9X2dlB7>Ox$zeif>2{o{BcpRf zM&fsnIoDTs3#sg(E6c?=kW65=CE_?Wxt5D`yeH`nwGlAW^0a;Bb&3LRHeNO^em=;= zUu#GcBo4<9p@+i#EM4z<#uyXz>i#zFTuCXncumh1A2}cxcro^$Il?A&%@DlEpXJ6( z3r%G!8KBW}wXqUsoxg?{RgTum6d)Y|S-hcM#OjOGs+?CajmrA9r6F{KV3Y%prl-WV zpFJDBc{NacHAl$g^I=|w%LYtf<>DPzE`nlD_Nu4LC=^emU6ue{Cr!U#z^_mfm0ed zEMZTN(arqd^fIMt=9hjISYmYJ%e+zL4_Dmh?4G+?SFr>sHxn5|Rdb|{p9I6M^^ybf zb)ON#D`9U&>P)4G&y^R?*4LZY8t0zH1*0ru6-|7s&P5lX+|$ap-X|Q z)Ck)xSrNi+vK8@v%5avzM-sfrf}0qQj{$?OHP_Trz{7thS{c}+H7g{Rjx&dkh-?SOR0kmf5MflTAkCvmc0g5r!#S#=x(URt@LkU1mdz{F(i2CJ#K@zR3U zRUHGnfR6C*tTD-jPpD)ns;5M5^qW7R-uq(}h2#&gEGn%4p{}Pc9jFNF^1gDt6=jHo z47y3lCpRBwlXzS@{zmK^Wu+&fZojt(OF1Ew2WR8T4O?Byh0;`7W=;oz7^|K)FMn6~ zER3B&Y$wTy8-^a{+P%ugMPIMnGWl@C0R<>R5{lTqWT5d5GGE8xJmrV5MAe7KK;#05 z@?|?^XGX*+HebeviPVWJdvFwSVj9oI+k`oc8$SXaVO@v?AC^*zQ&h8xsTz)~IKb&? z+{gTu+?3)p4R7gE_%9uXZntrB{dMKU1}P8}j^&!Br`@c8#1q}EvPP@C4y91$w{UR6 z?a)iZ>+l>SXk7C~Vwl}{hCvqh#6AYX%-sa)iFyxxDf}^6)TP^wDoA3u4H;fYZ;1e+zcx3>8Qd*DXK@$1jswr3}byP>Mnc~vT zIC^ea4BTVW4a?u}-k12|u<=htEqXms!L<@+6ZqIFP2)c;I7(@>$&iXd2kuzgY z&c2g#PU?5WoR)7r7h4yOB)c8wOH}_-S%-*f+bDu^w;)=k)200u!nE*>qhW|jpG|NM zF9|@Mww>Kj8)~l^ei>bwTkwRvUEv{XyO=EN9`=_c^> zV)NmHnE_-xNz6lGtEX>+SVB{^-F0P|Zv-8SvfIEnh&kA&Zp@h(nz#mM;H-6d>eFOs zV-DtZ!k?gj>~#C3MXJ4VU=qxT8VoH~Bpkf%&od+U_Rhvz#mz7(`o>EzhC`3QTdc!| zVH3AZD9_0&4a?{kZIdR*Q=6(t?c9vpI1L4>KGr$1RC+o1xmY~A9>Q+=9OYy z^0s2Tbp_zBQ42g0{6>-^Xo3P zE>C)JRcE%HdzJi!?DkSZric&Ol0~9(4_{?Ut)cmYv(z9Juuj?oKKT&e4XCN*ai zYEj7q~EGnbwIo0}2)atZ_4=kcO1?5G^Cgg<2B1U2fv2aMvd2@b05d>8i~kKRON|F*5;}+rWR9SZx0HsyagNU<9FB|x+ie}z zrbL)tWf*%THg0zZ+7NaTINkj|cU^s!w=J5#DE;c~!Ky#W5v(2eh=Mj1TSNBpomw5Q zEKzs$%X|i4qz|N9&(jUCzj_@;(|N`p1rmnHWn*)UfbBWPi$l#hXSoZD{tdIKf-6ID zfoceZ5h%Wbt9|QV*HqAD^hR#S(|yRcZ4<_b`ASInBNb;p&{=@6>1#mtY|S5H^D#8} zx%#EQ=+A>O(*+JZ|HNA?LX&kn$E;UwRDDt+F3g6ke!k*9A*kJ^ev{ku$VrG8{g&y! zbNHa>MHCo*?VrX1@@zd68D>E%kUM{_ z@cf5RXG*g9jFF}e$rw#lh5WXL;&3NBcu+&^!#3ss-L*d8WDt23ZRlG-qO#wOFR2%t z{(^ctpI1(H1ViRK2~!>|_6PTrzVv<)Y0GX|6VIGx2Z>lny$i8IQXY=0h?VTgpK~y! zf`aZw@*Bbf!a_oi2iHeT9uCZ+MBO1y#Qa>Syk%~1f_blnEI9ziBMUDx{`>u z<}?PH+S%h$T?@YI{BWO9*GUePP){KVMpf>dLs3d6WL{8S(XAhd?B_H~Xk-{Df+J0Z zmi4WDhfN-Pmp-9c<`zy6>ekv|CL3BUAUbiQ70l4Gas^fED9ta3>-bR35v}#a>NBshdg_q6d*Z>Eh0h z`;V0P#DwW^YFAEoW^ZzMoFj-<<;5oUsOAEo@~w-p;fS|(se6=*NpPeTk57wEm{5Wt zQGCs_lv&y;i;_YnB@VaGl(; zchvo6OyHyRWBCHHND!Rj9*uOR*FUM zXw#Secqts0MNNqkED&&L;b+|_+?$og(E0NuNwH1l`aO1xf}oKn8}07gWV7A#9RFTt zm;&w^`b{`GBmur&I!$=lhB(J~)eQ-a5-hyl9uBZT&=Hah*DoIya1E{9t)%N_s?3DJ z@;D*jZUJyB2uIZowF4E!t6DCjfZ)|cj!M-n+D+L7Q>xm(i$T&$*oW*j;cM&LdN*Db zn^xTBx@oHNViBS)I`HD`Xe7(@$+CJosQ7@yjx9vlH}0H0o~>-|_#sVWRGR)g;rzU1 zEiK#evC#Th7qXLjnXaX^+ILG0>71gAwHbM^QNds3EdR7(PrS&q#=$J8?2hgNo6(T} zxxkp@WONz4IP&V2V2Wq3qX>XvVA?g@#|;p$7frYeS7ze$Qi&?)D)o)Lr-WZx&5h+q z0onG%dsnp7^3YQq-cC;6+aBX@_~q0aj0HqV(+1D6#sZ@W7LxXdlW&(1NHd$H!D7x2 z22g;vFqH-o-VWJ@cIYlB8+b8CTfwC3B$En8{J3Az0S;Q!ByNFcUlHZU6OIy<<#>$3 z_rdLD8Nv$=bIjr9GRC;`Rx7bS=NQEUqsEpwZ5F%dos%>fj!e1Uh%Y6s1C0^z0Am^D zpMMLJd#}o5ah>VECsKRGLRU%DqY!19DiIa;@#2nrZhZA4Q{JB2!`7 zRfUnJTFF}bCm8V!g(~sQTEPqQ1H|Rm&=6ITtzeY?vOjN@jWP+izOf=x-V}kCFx(XX zYRFJ5N`aIg;LFcOo$nwR$Le{QaDK_|${QVBkfXB-*NCVUl#I(8 zWH4RDAn^F&0%vf)4b(_I@LupT^GD>|3Ybg8=U4DAixvHWsEsvy7^L6q5=?tBpf?w0 zEWXwLcL+W=;$E&2kPEDn^#^yktd2Im8wvcg-h24)?msWxS;*lASsd?ZqS$}$^yt3_ z2}cX}Z+C|y84wV9>1Xuu@bG=zy(l%QLBvNG6~V*a{PSUUnoSM212YyAGj>MNL=g+s z8L1VrS8hnS>7pc$BjleE>`~;ZErvHctKyj8R8?J8Ifv&GA|fqp(Ta)m;}FyoSAbu{msh?yfgB{(E=C>z*~{H04A4H=Yo{oe-o!F=B#tw0z#;>0%D< z0e?UP$0k%TSmS`Nc4Ow}bhWZg0>b9&kcKS~uW6c?Zv*RaOHO8iqA) z4TW4>BS=#{2Y&9!gwN4OMiy`yslJ(GY)s~~ACFReUP{sfc#Tbprs)LAR`woQphH=a z|1$JcN;O(;P~MigF##}XmWT6(bXg$D7rXe`N{j=u`~=<@^YaRVUg6yJUgHRm|9YJy z!emDc$Ioxn<1|pxDh`G%)W*}7>P;9+fxFs1WgIxb z%r$5OgqhgW$$j9(w}6TQ^H(Pl9AHc(G-`BjQJt+;< zN1SDti9ygiw+2rY3ZxUpZ^RLHsb0iK(L3-*rX}CHWFC<5&*jyt#+ag5igf*{)c;)m znKaXbTqADR{H`-a+@nx{q!<+K%We(0L>6`(1AV6qyg~N#?KBMt zhw7xPf!uh4<>w!ry#H=U$cjPgm{Yh6==UXwuEiMkxFg`JaoF^B%-jv7SA`)9b#@fx zIh}0_@@sbq=bq8QuHVy+)10iO=$EidBtzQ$eL>(Gpe4bYrHYz33V%U?dX0cyuudU< zo1>XbZJ{U*WFhifm32s7i;j}YYMM?||Mt`3(1piJkU>%?8|=a;k7UVqL!S}C4uZJm zE&qb7RebvK(R90txyB&}P@dybw-xTfWjLeN7)aCs}%mp{U1?@cCD81tmgGw*7%t zO2DB>Zfsu|<*Tq1va)9y^I!bg*ZH%u@k$WM%l>{u_rh2OZV=ytVzmeo2z!!66Q#+h zV-E8SgroFdL9v+VHh{1GFaK{zZsLt}LA5`hz76YQ_%cc}75dWy{DVBxY|`Ck-L>I0-)O)d#~k`mggE%TY54d| zMmcVBVZMRq=nd&Q>Co@lYIwc}Jl3M6uROni9*E@#{VN z3X6ppcc1K^Gc>6D8oG)I3a_T9Y_szLIf)G3mAD>$Ab#5VIygqQ*GNxk)msL3P7ki; znEGp68GtNMj%?JdPwc6Zvw{-DLBC@@eANK`WQ4EQ9*}68da$5;ho0g3&`yhhd<$yS zR0555&zD;fR}<}JNXDyg4lW}ID3apyv4>XVWTIM1b^aA1|DJJdL?~=-(6c6sNr^C3 zBM<7^F(*XivY)jJB*jJGlu=D85}d&9LD@mBFh;a9*O9T4iRKe;8A z9gjMj>HHrI8{?zK)reX)^pLCK9vD|ECR;V_oy|GBN_f z;9T#9zwjC|iRRJuda`>5Svln)D}=9ypw}4^?v$WH1T2%94oA{ zYy^d8GpN+m=fEihUYC|qUEz6Y1dj54@a*V)=3q@7H`|3VacBr3puJxJ1>mBaB8T%( z-%F}{j(WUw&9=n6^haV`k`^T96!kmcBZ&c*7BadCw`aOXc#CE%B-9lA3-n_`=@Pu> zW$XdFtW;Kx{%lKLq7J_V!+QAGJrDOkkThmsD3DlOG?O8dtve{mcBYW`mZ0^n`=o1* z?=UPoxj5ZTuaDmrzpc~$y9SP(k&0NAOmXc@ZO3}fla)Hn_u}`b+s;^)jsRSRo$2@y+6J|E17}a6x;;f0F`7GS61X@RWf4;mb^H5%p z?uSFu`H{MI6zu5LevNqjZPqJDPh>%i%K}Fb8Vr2j+y&5OMx!P4T5OX2{8ymNUD{(2=v;glvCYeBLjlzZ!8cKz7V37R>b%4n{a{cjI+Qcj^OS9y^Sw z$B4gdtg8A|H`<%lP#FbtAOK2x?I+=LqWDxBHH^-+9^3a|A`A-n7wHE}t#g`nLgWVO zEr%*`Cyn^V8V)JnOVeHL_U`gyeYpI_-Z|>EAJD*40WhxPI|SO4@wttR;Tc{9PG%kj z-mUrmJ?+UO{S<0w`g!{FnLm|;CgG|_yT-p3D^3>;G1R6G3BZPYVEV3C|YBPVcAZ1LL%$?wHnuJhl?9_1_wJeiVA6*4TXY8bzd z0d9Sd?l{m*%bY>rXecJ|f#ih&=M=1U;5Qdi$>ohPW9e|fkZVUdIr4ng84v+|Z8nsX z28jwrUL&fUYTuI}2O*4#3q{PhjDO|J__o6y^ToB5 z^Cgi8wqSK)>1g4?_ONJfP^+jhuL@c>OueCHkRem#&+jG#8=%;Qq>Rg^%JifBB2#ue zZy~k)L0(3`!kvu@d$hSxv|mxM{eK6M8a3IEz+bcy1j2X;r_a_|m}N#%*0}b*o|*m3 zHlBY)h%8WCcMQT@u>{%UhR|TD3)f^^YeOTDE4ainiGf5LC=)}vWMd4nGf<%Rre6JS z^i`_SCutY501Kp^9LU@A(kVoZlXamTr&wCT)ij@9$y`-bOP$*ooEcl?^Om1rt^n#J z_-ps?rTdOQ&(4(- zONYAIdW+;nH5??HZa3HTaRm=j6PhZWPHbREB-*YA4lC^VdHFJFe>VtJkYQXKnHK)63Jt z+dpo1UoIEGfe7v0E*I3L6 z!ti;nPbh=O@yQOp?%9`bBUTf5MRX`gYv=oyI4i!GMxj_!(=9FuZ$9vl4?q74@_$pG z>Y)k`1Oy@m1cdoNkbM6j|5hf(PXAOg|JU{M|A75p=~(}R{iA$e>-RhMN@%Z2bbRa~ zS)q@aZgoB;u!e1cqi+cnTXU(=Nb@Pm?bAH>?o8c^7U|(q$ZgUpfQni>m~v!c@nsPe z^B6Puxb>5-v&z*XtZmL$V*CuH$hB@>Wtl9h5^K;OGE1BAgxoaJLhe6m9fC8~t$t<< z-jE_E63-~HOSUBmH4qzPTo}dBwKNuTe^{}`l4(=!-O_0r0p!=*RO?6&l$lHp*fJxq z^vOhf7te4{0x;%a?)4s$y1mwo8D5@WxUxZe0u=8-pJ`?z+a-pKrhN|7iI3`wsHX0% z`V0QFP^-0v5u-DtXzJCeVh<*XCJhCTxf6Rv#hRp4!5~vb#O%{Hr-rbdGQMV2!yr6! z988j}<&TE!_tM16iOfE(L8N+7!jRGp#GsR}D#bSG{5;n4kE2WLy^`dI+}Z+jmF%Ox zp6J{cjP*m*Cu{CJ)7j9VWmadm-lcFuJhR%VO-?uA>Q*W8u$Ih z;K>$~zAdsC2j`%Ru5oVw={(-2PBDn2luM-{EhPklmrT2wRtsLm6M9wHjlZWjy(XRv zbZFTFhW0Z6Hj^5W-6bcCjf#E5oYN8QTt(;)pDJuOAkcXuFvusBbOEIS?r0LBscKT? zGT{uOuELZH;Dd{_Di0Ev@-_gu8nihkCKFT}08=@Q?wjyWi(PCB zD-ekIyd@L0T^{{dlLmQPj)UY9cn^6NHRi>_xOjbg>^>L*_OjhJ`cA zctDyHKJEb$$)DNZ1O%+Za=^pHCyhgLz>KlVm_u@9QLrYujOFo+2`A-i7C~Jkv7UhW zRD}{i1zlVhTYhT>;DmsRc?HhoPkZK2jW>LZ(?Z6GzfS zD+jhmG)bPQIs-D_d;U_@^OabiHV)AoPK6P9skOw~gIf092h1sq^bFtkDm%XiN)sSgRuI8b@j`zzQ1o@Fg!-*kF=)PQ_5!db*UF|i{@bqj3=fM(i#Imr zGJ1?q1A(peFubxq3d7Onv=lr$W2HoS6XltiSpDuxhc>gap1-QhH+uh8!)I+a`L~t{ zx>VNu9$o9NA#?{US%=%H;;?7CXw;;#V9 zN4Bo*^=~4t1mX9kdo)4Vf|-fb?pjl#-@(F%rd;?zgX_dqmNMtS$OLjz^Zf!(CN=gq z@#cvHr**D6X3~V9(M5Aitm=HkoKNHo#U3dy9Pzq`jz!N>4%SG}%RqKCIcz$W0%wfk z-J>Z_3h*}&Rp{plL!6I zW0IE|mJL$oVWpV_a1~33sO72x(Iwx16;4hT@)sNc%*@13dPPsf%|M?&<>f6?ul4<) z23vb=idwO3k`(_61c4D@Hh!3GRF~g<*0}_roSrQ?``9Gm_sxj~M;1=T7ax{)A-A&y z5u|!k89>p|&i$LNwhjsRKe;>1gtI{-xl`%{f8qLRsj44NPC%4f4?_jMzpER>Q(K9EH&agD=CR>_ z8tjuiGL=+_UGCZI)<7a|cbE(3AzLA0`z=BIHf;v78_Y%u`K2D`@bafM0BkdbyjhlD z8i;H68Bg>UewMO;vdWYQx*TJ#zoJ!a>=&=|H&|y2i{d@~k*d7Fu}iU72RDl0i1 zX(=*0oHj2x3CthDT7h73sH}FDg8lM%9m|A>)W=JS5&p3K_q>)iG@Z|wEEoohU$1BP zAaLy}W^hU)>gMUx!eYYhDy`|LdEo5^;Q@`qDp;$`?W1&M2C+-kepYyg5)T&UOK<}5 zBLuGVH2H`?MvpGnoY_#m-yQx9@BMYto0WQ2Oel9{@-d#KDCFNV?2WTDX>~p=z zw^sWXjxdtzb4grJaPGXeqqU-I62_K#2-Q@Ep(mT4-O(4K&TzvdXkBrZt~b3JSwrap z7H9dF_ZJpb4us}B%$sPngaJfy%)n*?tE^C+=<#2Mvnq9j>S*(*&WVAE&02!M-5FDN z3238xi{@3=5qT8@SBpt!Qt}lEW-N>hie~O!Xr(+vhQHfveVGIR^DkT(9}^Vvi1b~$ zvNEeL3!Wznn@?DJKEaus?u(W8 zJS}t?`N;~z(yRCVxT}xyEardT&y082wxM$C{6s_#MThnH0h_S)=(1`v?R$B;&~SEh z`1V20COHJyvTzV2v!Ftf)dRx94>mX6g##gty1FP!MYY)Z6}NzDX@Xk5qe(JTo|kMW z>hh#DD{Zo#J-Gp~yP7_YExFCOJM$GwjuLpC=gIu!?T$}v+~ZYZ`dn)92|hQgo`8T&bFX+7uD5H|{@XiKCAf>jpUX ze1DmOa4&-hD!)n5Bp`h~mQ%F+PuGTJBGkU>c(9>$+;xO@Rf6v4^>bD?@7D5sGrVm1 z60W}d(r_m2d8dXVdz+hhX%qcOBE>;LsOJcxfo=mkzkO-MMo1~)6Vbre#N<1&9D4rt7LS#&ra(a+gSo{UK;QpzBro3u4mIF zUe}tv%t_tpS2s5^iV)kxBPAXKn`&T3&R1_c&cEeKUrLT!w@z;?>=H-Rqv|dvT?T*v z8}YMapc}fgr;esM@vP% zMGKHNy67QlFjdmP0uH+Ar6wkhQFUDuvDH$oeQUQqo9TnMmepR+4`$m{R(=}2uh5My0HA{B7)FG_7I}cL=eg>Yon#lHp40nm*YHwi8_WWMz z{h}XR`jfdW>s|};QVAOW?UMVL;HhTLGzIemNdLXkBB8~VDXXw5&O1F{fG3~vs3VhN zY!RTBm-b|xTCvh03N{IA@}Qrh2mE?WDV7QX{D)KtlXsO>&A?jhSl01kxTan5CIlTW zD|K!fw|-WfSM;srY&Kx?h7ww9mO<8%w+QB?a=BcPL*Qc;NZ3p60i?qG&eh#E)C z%ROXi{s;XagG2y%jYhVEj};FTYX8_e_elTF>oVRUGy_m6e z@T0eNwmqFbPUzXoZ+-upoK6|tB%%CX7_|6L0=n?e$ARi8<8sDQ$gHMjR|8txq;WrvPP9XvG??z^j7^ZSr=e0W&&UJR{Tpbo83 z!C;&W+rVy+4E2-5Vho%Gf;B@-a&Wasi(2*v@K5l~!BSx3{%;`O0*^m1p?==nxJt-7Vqe#h2cDVso=zo2dV z1xaHc+b>hSV`y`t3G%2AqHYJscY*vX*otg1EizQi2Ii;g*<}!Ux!pOhQ6QF%nv)F* z%93Lw#1HKd5Hrm z+1-p!Y{q#TrsQLd+}Zv%=xEJCC(S>@mKqjiCvh3u`yC9DpRQEk9w5U1*>^6sJMb0}t`pqXvuim!e&K&;AqZ2VR81*BEp1o_jDC(UEIaWR z`Vb8r?~fT_anUW5Dm-rgz|$n40tYI=v+R}^wv(e ztnBH)?b)7DwuQA{dIEtkRg`L;kd1$*K^f3{ve1%I*Rioa*p>RCIFSC3jnkuFZAmrQM1(^O;(HHED7ve=t zZBm?`Vmbh5!5|_}T-j|E8iCLXN+o)_3v*-08;Fd%T9aBq^vA}ne@yPGN-h>_cWJvp z2AUgSzbp^l1SM@d1V$<*OGRv9ifED945Q}P+gx79S%8+3h})mVG{dt-NMF>1Z9<{& zx9$I_5hj@aysWu(Q202V7MblLkN!~rhyFGe7N=a*oyb~%8?|D8X6J8-fR3NHx=u+y z7-?D&aexep2z}^xbpMXe;)xs2)7OtZ{5h3~`M8KeoHc&DC{3AoDMr0Nn_A;68N5ue{~RZq79zV-TMc!fSVAp78_p z-__y2OpSLRF>a9ms)G?U5D>zDs)MnuwVk2Ue@esuqjl4)wq>)(_^W4D8P^$brb4_# z6h!U8M6?S6?JuwoA{(EWRkf|e-Iz`1&ZOsuaj-UHyXXuTuD3Mf+VgO5P|L|1sA8(4 zW1GkP776G#s)yPh_a29nck7o0bth9AES`bAQ9EC&qTkk0+5O55v*q1tTS>)%e z0*Z>IBjWrE3`Y&Qg56Hj0G?p^M&F1cvU@UtrahEnhDMnx%6RufiP{)dK@?NjMx|c-En} zE1=bG5A^rS?53z2XLn3iu?aPKO}vNthpc{g zGg^XH&(q{k9Aoqb<@HoyD0F$ZtC3TudDIA@as-Ps zu!`@QCltFvXuP=@U!U zsLo2gF<}kr{doiU8C3EFWidU7#q0&^jgJA`E{ik28QlAwCfj z?Pv{ni1cEhj}+oN^@)?GBsDZJk0|z!+RM3YMUqqDbyx5u@hy+_MJ z&8*q`{QDMdTM^wFJga07i@gk&1U(HICZ!nyi@0FoLR+O9om=N#`|Im2Er{}rkTy?+(MFn7Hn5_p#RpC7+aZ} z*f=?wIQ;Ww|9=+Mm5#RUf!M#aL2a2Le^7=Z?6@feCe$s6p7z$6rapEH_c|eW9#sP= zONj)$WV1H++E34klITiw!{znkr7)P2k)mp8#mrNsa_Os7*Y|;Cyqcz7jp_d2RpzSi zFok&G{As~NOYpBN?}Z4jZmL7A6l)Tc_@l|_2N05GTNNyPGA{IO{;7c|@X#RY%CP7S!7+Y)&$E=cw@vhOb&mX25}L)=I6+ zgE1B@1l#3LNajT&s2m{I5^17}0<{)&?QDEm3YFKDZd0&_z;M)y)xeq@vidXo7^i*7H2G(zMSZn_6`K>t1e88{x1Cx69?h@SvzisUM#RM zxcV$^-)Yf&(vT|?1eJ%uR9Zp3OQFg_=PZMXROQ`oDeapVP2l@G_Fh=EL%}wNfWGa- zJ1y>FOr8f>Hlb8vt(R=uDw!=iwEj6g9{RBPdU$#1;p4c-@8;|5J*nGC!N2ou=Zw|c z+w)$V`*zava(4FlW$pEGZRGO=td*0SHRIdH#h`F4j6QC&tnmu!UCie?$-Y7>fAxnTQa-Q`c< z=JEOq&PqG=ZnS7^;%Y(DHbA)(66gyb?FU2-w*)Wc9D$8CfK z8^!6PlnWSrDc~di0tlj6QZJxCz}~8Z?_9jjnlTN&3BnP$hY;_yeoEZ{+aTSwijVfW z^DG-b>^payEKO1G*cz8*^RYC37-foLvQLxAjd+O`E)gG%tP8I~{qDjl}dYdI@8<#jR_0(bHor!3?pa`?pc_I?cOE zmv)41^A`l!66>jM4_= zA)GyK(m+Q!^|`6v^YiWLTcx*YikW^bG2&V##Awj`idGguDod6*?5%69{xU>miF;K@ zQ0V}MTI0%bR#1W8f+IMDgWP}Vs%v1B!1*V;Y{G~FojsCqB%gKORpEyz{5__KZJw0d zs15>2HKyBzwQ7piVG~{pJGvG1h4TOmng}gSWs+VG_80l^U+4R|);rUcFc4J{dcir$ zXyBTJWM=i{E=j-aGVJ((Hv=cVOStuoK_d7ZLU5Hiumu$Z6Qx3rW!O4ZLako{DyG89 zhCc@KA;rI0`~Yv47Rfa7Rv09Q$|T|IWJVpQ1A0d~sj z1cK`(-BlL1d{nT8M2o*+D`(MUgv@EEq^|Ox#Cqf4LdLBZ4eZdp25Xwda&rM-mH;^`2H#X#$LAmXjCrCf zgs30^RkoUhKR)&hlm3Il-t-sl^^TI--f_zOAIjb-%9d!`63yIc+qP|Mr)}G|?VWb+ zv~AnAZQFKMp4;kH)j6+j-Sa+TwfKs*Vh+q0y^q-kxO+lk47|Ety?;nBW@_!{YIvDd zcodPq`7hC@(n{U_wxRCb#3zreyr`wrfHujm-(peL?O;P8 z#Sj3xj5}m@*SLBS0LJq!Gfy*}DPL|aZhHWR!yR4Jv|oP-$7J}Kkm?ZJ?a-Ks*?M8f z$t~N@>Cw$o_eCh+ej$l)o``Y)2AJgdGsh)oO0WyAy(A2!u1Lu zJ#;w6mevz*z@$PqMu{6&GUKw;Q?4iqtRLee7W{}=+hSTW74C%fL#+wK9iN%nitW`4 zavxK(kDS*a|2Eg<9dnhRfx6F2jKKtc$lZEM854wqel(~Z#P?U!@teTK{#&PG$Vd;7 zFS#y`3k|jM>K0N;QnEk?4Ck@Bho53iGQE2_BgLX}#}6-!?jfjUJedyf9an)bcn2St z3i`mUe*N*4_m|labbbiEr8QZ6w0AKNxL?1*8fk^AHw#^qs!wl%Qd17zX$oXg0O7f^NX5MI+>QT$1t?!3xVdt+hg26fjU+F=Z3$ zgh(TTe7XsV!5!Wv1Q-U+8K&pcvUFYIQ4u?Z-iyc%+gPq2(xFom;Obi<^HoIqL3i?; z0DE-Z%}=WzR$BX`n6-r=&YVyyzi1dv9aE=zf2-gMQx4^+PF#gq<(5o&Xt@n~_zQm! z$bliDD~hfeD60fo+?K+pFZCScLTO(U=W-RQx~))$A0k{d7zhivyF3ZTm*L*qNUd_ z|H=U}n3r+viX#GjJQo zZ^A$zo^h2x>iShD>6Neq9-#8GrvI7nzywun)e=E*v8oU7*kaX}xx{ zQeFf-MTzZ_e#t5NksyzQscM)PH{YUC;$Uvy5*zdy)Ff|LX)*$0Ol-$ z`SdrWM=W$=|Iw%&>%E-j7!zLOu&9vIRMtr3EP9w-x_*yByJE}Q zS}E5k>E!w&t&FlYq!%vR|JG~8&FV~(awMPIyK`PUd|KW0ZMO`;vmyGvQS1O0iAD-u zY*hEW>XK2qS!7cz&N-fY;g?C6Oy!kIr=>oog66>8Y99W7dvE^@?2 z*kKgxydJ`^7XaEEnDNMPQ7Kq(ZpFh2tC8u<%+ok-ra>rYh1^N6**FMn$PXICDxTX8 zeO}2)+lY{6ueL^}0_mKLRe&WoLq_3xjyVFtJ_=3eZd{J{o!eDj!^`82)ZL!3ZBf9| z*~k+TuW_7>gwJoQS8TB+j%(+TFg5vI- z`Gw@2n^PY5w6Wp#XO`0zNmwU4Z2w`1fN+f{g?M>1Q0pi-QsmJ9`Q#vlIO_}y6DB2g zQG*2RxGiJOq~89JnDL0sCoeA0z+3<4Jb|h3%W)1Kqo%yGnXQnl{?+GuR4fI|i^G>; z^*nKwE7Sb4%L7k#k9l8IfzU-&sl4N9GPuj;33<}ps5eAo@cky{90tk=i61H7QwB83 z2x6xcif09215J&N3T<<6nF#z2BuVCwvy+lP5=;@5+erxEgSF#oZx@?IGJA z4Hmx45A19mZLJKgtiiuzX!jI;CzL?<+fC!kzBs`jpwk26cW~%;RQ4*TTHAPUBKQ+t z&fDRX(#>je?LaoG!_Z!q=4qpYE+|oL(Hr?i2IM!21EOVl_$HK%xY7YWs zWYkO$`$N>XQ7tbFR~hH|oH+E}ObWMbtEZ|4#K+MHPMt!+(|i0%Idql>(qkJeZ->rL zRtR}6KOv_KNWWe$zoUzsC~~77h==-Jdg&GV>643|+YJ%hQY_`hwT*|$J*Rf#G(gVa zPe|6!b zGXB6>#@D$MJ6Y*4Gi$@{77^0*zWJtheqCJ~#styqB9WG`_D2V~H`L1RC%pikmob3P zc!T6eE8V3Ui3~3_JIVFf0AG_N30ARP((=pFNF`xnI;#y;!X8xnPK7B~E?{uP+UC`= z^M<_a+}p&^vtYgD1GGi@Kl-z8$4vt|D#iW|bg%pa;?2|jY3GfhJ7H@hvo$pggtU8r zGyihfzPjD1f(S3wQXTy?T;^FL-}u5ZRo~AXtb9l=7CL3W=r$~oBr7{Lt?b{1k1K_CW+-dw%e{!VHtn9_=IQW>hi|0U z|N8pz@T4vUF>X-fP8w0NzV_Ys0h#jrcW>6t2^B};)y@M_0(TwKmz*oK0gG=;RGf7R z?oh})Z{t&7D7 z==7cZ)14rn!wGxrXgVxLmxx@Wmkphu`ndSsm2YY?Q53AQ78TGg>2GgsvJ>PaSC6?) zCL1e#h$5RSJT;E#HUC$im5ifZZM09Xpc~uA0j|I#A^)0Bc7N_cCaAc-jLEPxBUf?G zk5(0vviape00bx-U{S%|qQ(xG_-cX+2JcN87$c1?+Nb@b0i-kcvUvvFC{BYdMaEij zStax2Dz~9rToHp)^uUENWflnN&s;sXNLTX?0MZs>DG}hz)f0_e$@xITE`d#*;M?z4 zoR$YY)AecDg&?3W+(cQLA#zqKqtcvUkx*c`ZkSNq@2*h74|nhrs60HbvTO7E)IZ1L zd6}m8nWi}zfA0+FcYJVvv?Z8zAt$-vMpN93{WhTFCZN#Y2p{r7z;Q{;mQRhFo;~tr zE8D;ZXq^plAPvLv?3Yaso@E-4@IXamjaOe-p-62ckzMsWiuEw1DqyGw29&yc86%+y z?LDkijvja;dQ+aX+*?gW%Qz|9(;wO$yWT?}h40?twC-&CTyd#QS}w zlA#9Q6Orzu=e=H$avLOB&own(HP}fw=Rb{`q$IBvbs88OEj?fxkP2klfvc(zF$x}k+t^q zNDEMI&hXReu`c0>u9LLEq0_zS?na?>ACGghi`$Wob|ENk6U|3Be-k_?ymL~BU;B7J z&a5cWYMk)Rgw3sYF>$A$`?V>419|CzRw!1wcAlhF7&s5<*R9h7|V z!KY<;B=Fr_8u(`{o)ZS4UC$}x0O95GUyf?fhdXFqU!gL?xCfn{JLU5~rCn=Iitp#; z?Q_Lj{7*V-UNWB#xhkwLlFPmqe|grG2W-}QtBM$_boq)4R-sBvl#|&ALYdy69b$D& z^qV$I<_SF|vUX?x&nC=&uzO2mm@!=-KiEAc008`d;`avn zhL(1=4*v_m|1ZeBMK#HobyoPUcUAaNed8X4aO^b$d9)KM;Tb~mSP&2JU%^yS%M%g> zf(q^BzS+A%jKcCwY@5I(G9k<#N1u~3)8Z7AL$t2e!|g&10$+3~ga&ix!70D3RB|t7)$8FOb2m#9p^ocZZ&JoADQiAzs~G zS_obhyMsJ4nh;!jv$w}%J~WnP$P_yHx(n#rdch63eQSb}@8_~42<1G0>QWRXkQaC&R@|UB9 zU-=n@-y$CJuOC$2FM_M+!$U!{&I`w3WV0p;LRf=4B#)B396anMkfS^O;( za7ElaX7a=Xj1T}bi{VK62nT!3022K=#DotX<`Vc7iA@#saCxd){+vk0h`BA^t|DG zS$hK-AP_t6kWVKJwJ~#s-jLCBl*Hnj_3NI$6172=Fi}-1DI_0zjysZ9?xgj*F&y&_ z?LfZ}hG?90Qf zEOmE2-a#HasDDJp(`niQ|8Zl10LFPMoK})W?+qYQmoSJ!t4_&XKce1$KyD^j}s8jN= zl*bKOol7s8t&)+B(2pHy!@-d>q-j0Ud=M1I-uG|asG1=Za0}kl5E|r@C}~588t#44 zQOBVTh8TsQZ}?AoBuw0==eW|eu%>w!-cmwNZJOAk31 zt+O*h1;wGuF=4Flh@8hpPX^KKe+|+Ek13sWrCG}uT7{Us6G3DgO~*V#`0bMVewA*D z%^W0MAZo>1!B_^7HN^(fq)w4F+Wzg%(LF)MLK(YRV;C_)`*YajYo@UYmp&$Jf$%4c zsgW{cIE|OZF&fgPruM8`+T4AT2tk;IQL4^%J0|dmWc-8lhhSp&-&-G4{9B9BEGJ}m zZL?9T<;SSCCwyG5J7TX=Faf)a{ruUXDLm-nGlC0vY!`FBJfoB>ums`%0icq>Jct9) zs_nd-yaq=uS%gYfe_|e~KG7S_UU0RqN)rs6ar$hiOrLvoW9g9&C&?E*y#l;H$CkX4 zde}RWq~`xQPZGX>G2d^IQOXOS%|(xKlc?GRE;u+&nyCwp>RJlFF>I>>9-3I_!Wq`t|CT+c}doX@R$nE%S!qJ_p*@Bxhk6tjBj&v zWj)C1^d!EKQZ#%y$KPyE;j@=22PC{P{pB74QL zV?H`VHcMq1(VaJAMe55`@jGTUWA#%xkG zS{apz>S_iLB)Vu`B=Xx$>Iv9Y&$?x=ebMxP6Houx2Z;56R-^#{0BnD*|FsY3>YCe_ zJL&5FC+YNG>;sA_;3dqr3gBW=Q!-MrhZ0m{)5?EhWT|K8#vb(Qj7xsMf&c(|<6Tn&;#*Vum$sAt{ciu@sC^KbyPAFh0JuQ|0AT*}LmbUb ze;QdFb^r0m|9s2;g0JpW8MRrbhwnV60-X|eK_RO1CWn5BC54@_+2>CGAr1M6Klz0kb`LAo%*|ajqjX(43;f&T5N}0LqSK-2OsHUpE1%Aa{`trHh zzCou}YYPM4x6U};FrB1l7LY3WIEMKGU8xc(k6o@B4$Qbj>&Z7NtOQDJkCu5#b$0jz zBdl`BVS{d3#dm3t-J#M6m&wfYkyu$dsT3t(%;=Tc#(xP!5wUX^*=VVWKBC(CLWh0f zx>z$=dR!@V`9TJLI}MtC8*@g|943?FiC>B0Jm)1Z!nzj99pRzeB8`cLbV9@CHkujlzvT zMcqtgk$<`drQs2#pyt2joQ|n&;=dJ#^0@kOZ{WIrfc$eLxJFT}G5vIE2>cwo!T6U* zFt#%JM>H54(b^eX|3`5BSAmec3R^A~A9C#uxjj$fLSM*1LlUw2Joyqm=m!X75?XA# zOyN99;k+Hi3@&i+oS+Gx8mFb8)59H8?Ub60?wK3u! zX!TBQ-guk~-mOc1Cs>cB(K53DB1#ABLNEd)@ag4ayC*n<_Ulu;Yv@t1#uqu)_ENX) z(@)yLtnzqIiwV zNBv;89Xcqg-my*<7Jk&E1e5EXb|nmXwIEJ0}La7{TKD zKr4&(8fT+%P~I3FQ2zKyuP*vaNU0;L@Qzg}Jm1ME8>=zi6sHxR744^pO8a^9m|bLE ztegW5OK?rurXHn4Hez003aEMAyi_fD#e`_VpI|PhG#{Y!Ago!l9V#AI`9#YdC+c4^$Q7tCQQOzb(18C1vDH~Ja6iPAk_*ONXuuq zS}a0Gn%hMVNRzcpTkfnKnDn)`webg(g^;T6$*t#|qaOxo zH}B1W_NOR(GGRSd2gP47e3aCk&-gW4NF~^H&gW!sfxMR+wK@_46v7Q z1Erq4JDtP$Bk!=dNlp66E}Jz#Th}ctTrU|o%!uBG=zc}%xF@KsjNUc4cU!z527;M| zXN_hRsgZbRh#?U6j~7~QG{m9VfsSu*=CBJ5nF^M3CWe1xI*mc26L(R-lq#`$H@pq!7XjX1H*D zrdNW9;?JI+ghrb6=ccp>Nku6~FUgP&>xs=^XgBRKy=TFmqj4v=L3LOhYsYe136_+< zt{6a`48Kl|4J@t69-Sk8(w|AZR(PQ$YOQ@y?kT7pes)#HRAn|ar{j#|_rD3{=9CU464vH_;re6=?x#WUJ_uOXNBrC|UiQ#6p z5*rXb2V+NyEOcR;avN$$i1UgU5aMVcTvs zsrhvUkl0*3->#0gZ^j?GQ_8IfUq*rLEsK%a%9YBNBJH~NC#6Pi_9shv^p>@eTCraBHpv=96-etx?V8`ua<0lrH z1@iIQd3;&=o};p*o9=!FotL%H%on+sOHAZ?8n_kQ%#e{k#O*k0GO-tx#r~e3t%rOP zf1Guz5Z7Jn2!*4o&$^nNL2c;JBXh(AGT-)*(#N5(?I z`)owm!v5Btg0qEz$8cXW$Qqp%J2t1!yL9Nt)xI4lOV5s&o|g1wL(s%=sA6KyDl!?? zdYw6s|HPxqG{@ZW{KC_?z;8S2W~}|By~sPN zS(IwAZ|B90C7b11+gG|{k9730LN*cL_MT%-NJHt7BM-=f2MxZQLposuA;xM#GvwtL zLy-9^G{-bNT3#G2ZY0pw&3~W-Kd{}V?mmhct&yD;)pqVoW?#i)&&z!_Sy2RKnt=}VJ%)?FY7gPd-{eFy2D_!&ICUK3@HZrG7 zZcYN7XdWTxAc$;Q=yj@wuKaG)OZxbzb5#hrRu_j14E*zmq%MLIvf+=C*uKE~%lZDF z@zV#sPPo5K9-c39&f`K5+1`-J4Bkfy)}*ULu0=#P)wrW9oc37x%2>|?$S8J4=4Xpd zK1K$qY@q|dv7Ly^fAxMVGGPbSGpGN-ou;ZP)3cJC9Y(P6w~F3_)DbArFw9eoz|nnhR#=fHU$i&A~hB zR`u`C61a7T3u2eEDlX;|f=2N!(J+q!hkz-WX}N)8`??N(J`=oY-V(BW;h|;lePP=B zNQ1~d=P{%aVaxu;!)yEu*zz;v_7h3$MudH0g3`O zZp}l7DK-f113}iTGdo?sJ+gj!{-<%6(UG*iyqW)`SJsQ;7_n(uDXW==YmXdyE_9X zpXo%_u*YD|*g_!tSCA5EeiiTo%E;9SJmqf%9YQgeMxK%KNB?T%P-73i7z}%$;?~qd zm6`j1kP0T zN)t5(w@_Y)en?|}Vp=S|J505}ged}=J8vDn^lETXcYixB(7GXIqRF8X`N88`)i>>s zy+Hry%>EI4++HC6of>)sObGI;aRS5N2A&S-#5?6js5^XVm7|ruTs#MrC2Y>5aPenQ zqj$jRDR6lSsbq6faFbr=FZu?$k2J+u88Vwaoyh@)e)Cmr(x|JWSAqU&B|FKf+wDb2 zo^4xMNi_FV6`--v`6nAid8Bb3Erqb=mx~^YG6k@e5jikC1YHTOA-R@Q{7R0Tz9F)@tj#!Ea#caf9Dy zqATb}-5hg#08}_k5@5rB^Mif+kJ1gV=2HUWXk+$rohYq>^Lirh<7$QDQO9L#zhJL|pb}gcPIAHwM)0~ji2X1A(aFdOW1KoMy(2Wv%*YOm}Oq#O=-EJSjyvTkB$PlBLNvZ zT%6A_$F?eS!W6Y5BvaVeZf09wkoW^Z74$3#uK&`&8WFX$ujJoDv;J1_U~qr$<)Ag# zOp_XR){KbRG;#Q9#28E67q}kld44D~zZz{h|AZih;1zjg>63L7*jIp1V#z9UZb_M? z7G7)yWSML>W2pY;T${TC-*WM@HYZ>J05t!XTx+OrW%UobL)XFB(AmM!+{IY;-&?Xl zC27lzA2{f`8lHEx=8ws|q8=PiK|>)70c>Z(pj7An1$CVfaV3`G+cwo`ba!2PfgEca zy5#*<`n8rH!vzpzt^yRQ;iF?{t28a@L?#R>^|)C4WZg;*d7G#22i>#)c>mJHdW}roZE9`W!RCp@fO;z#*7KiIdk6kBA?4hS0Z|4ZvHVTT(L7|c z{-!Mf6O{MQBiR9{Ldlw2C2D~C*eqGO{vG0^q#ndV7=Kaadzax`+>gp#JfV{~$>F@g zqTUW^vFvrEqqC&Gu3oQdm_<>d?1sg7XQ@=CjOUe;QHq!^nN$%jcWes9H4#Hik7oo6-R~V&GIaTDJn}zlM*hc%SD(Hc1^W1Jg0yfW%AJy(}*sNFNc3 z_t}8j9CP)soSVqn6+Y3Fk9vtxapiyURYPNs!QyPl9;#%H6@f5}biFm`8dW%>6_g3v zJEm*v3XC2W?543f@wgtY7jK>I24A86*`;=YO<$ycT&m^=Ql$QWxs;W;qm!|XvBSUV zQO@$?7X9?_9q&|4oE-2@XQewryv zPW@VQn+=M?eWWYN zL|OZLz(W9hrH2LjY-ArSR^iS}n{z2YEB&Vng{89?NZ~BLBPS6wEkmC3iuP>MDx#|y za|Dw2?mA%{R~+;}EIzn?IHl}&@LIdu1{#LJux?gSH1FXwGTtjKWE|lb(wL9!tUa`ZLN{lxH8}d+FYwejz(jlQKktmY{l~u?t00D{jX(Rzk}*GKBmRGx?0?2SqpJog&w4 z@Qu`%s)oLkBYbd7P$$X=`n+a1f=lal;4vpQ*;S)E3#dY-n;+A87WQ)7WvrcBVh!yc zLBg1g{Mgxap^^D&?J_uym-2J^OAI6Cx@$Ex&|?NmnXTavAiVdTU{F;cf8peSv)^f?1Pj)2->OV#}M66VN}qexu>fSka#~EctPLqJLu@|C?n@l9QI~=l@3z zMR$;32x3)+4EAdT>kWkTKvHKN9k5Be2w9c3{OU}X^^}|8=wiD+>O6MGT}Oh9z%T=8 zPs3jWhNn@L9I_EMgiC9R1+N36uC+)#hIN=jS-5mYK;0vL32o<%Ks8dfth?E%c?X2l zqZ?yHIK2fq4SJv!Q!_4Kkd~+JKj@OZpV~U z2V&@O_l9_(;9Lj_)Ro@iGMqv_9xFV=-b3e}q9Q;!-pXNdM%7(9oVJ~-Wwm+Ra^pcm z`(7&PR(m-1VBRe_wgD4iQQkwvVeo;|hP807EUY7dNIE@}p%l189>^X5^-tT#;~V?X zzDGIr%}Yq+U}l{^p87`nXKRhF7n+*?Sjy>#Pr>eJBASXlYPEaQxGxRWcPL)3YknBG1R!)aRj{N zF^}jq(J8S!avT=WhJcK+@U_7F*EueZ)?+oe`?w{W>QUNMi@js4T$NZ*Kwn8bqsl@R zYY#d~p~}!~8)E91zxZyCow;Y?bVYn45GtXVTpI`Kgz)1GWbq`xYLPvx6;`EczHnNp zg5&ZKmJScUsY@>&hbKRrWS52EmE8p^v@@Z%PCL@{ha6Qn#NaSz!}w0vUyWb;8cfzo zpr|nF5IIPKVx+B&PT_0q)d@b>*u=|GyrI8N!5ezjtOQVb+Gv|#Z)D{Pxa>zwmbwco z^X;j~(jSVd0u0h?R)hD*_oOEevxU6y^JV zi&0o*5OalL0^c%$1|~oY`rpF?!ytm|(gN?mRl>?@2-h9I{sv4Nn1Lnmu=+EY9s&a&EI zZCmKxw1dLwLES*>jL9u6LRjsEMZvQo$n-gJZK>S%OMkX_jdUXq{+vt96I}HlD9v1^ zeSVwr@(gccsW?v)MZK;wY4jBSJN5x!Xy3Tan%SsHV{QYmzU9X@>RRW>B?yXFGu7Ds zSD)Vcrjd$UwoNXN^N1!Rcv=o1D;W9-#|HLuRKyE-3}0uv`ELx6s%Nb5iTgW#^b77# zSM?WK5g&T~gzoCc?D)DD;Rl@Hpyao%9d{9eAqg6ap?s|MTLzwRuk640Z|ziM4-^2v zp*sKo?!Wl2v9+y(yRMb7zU9Be-TlafIj*%Ker4(A66j9wBvLS&&rcW~9VyHe(Geh> zn0t6|;SQ7UEUaLQ%Z$A~nv%Q!DeBpOxlNv#>eY%XY7NF)~>xq zsp1zg3uv*#YRxqL8_n;Vf!rDUdD0ccH;H`hhZxaMRT?J;50{Ux2fD9Yq3`~kz4})Z zdN(#`kg_K0tBZ@sTb8^D!U-kkh)S~1VKdxtRzy5YM<#UL0c{_amPss{1vdwd_pa^J zsR2_@*l)%onKJP9lFFt&_)P;5I5D(3u3~1){I)W?d}Bmx-@yASmhoU`Ps3g~&)`Jm+W3$U zL-9O2eZG_64^EOeBip$}Mddvha(000^9yWzxpzbCc^b%Mozci<-{e1GuDYs)Z5i?ffl|VTMh%sD&tPAvvu{qH`g1s zqZUeNoMMh1%kgYUB2App&oh&|R1?SoXK~an@`4-o)w=)#h@MP5+r6b$fW{nz9_Z7D z0w^Y$h22fMY*;bTrc25TlAK6XoNa*gZ*W*yY;8;(%>Nc0KO9m)wiY6YTU-Zp+<0Oj zSlZ)h$YC~bwMGZLh>8lCLGZo{fXdZpmx>;q&>V-kLBNYZrGqoqX-&fEh`fvkbyp)Ja!Waz*8%|F zV7-Gxg|VtI8{*9EGFR@0l~6roPj0xrdD%fNpa;-3;)KZ$X_6I2imKMH3tB@LBvW$z zk(5_Q-`JW-yO%YB^2w)(c>0E=Uf!A37#Yk$=+mojt!LGEga1 z&!24p=MG;%2J#K`1vVVWzzEUg?Zbru_sZ6Q?idno61wPV1cx4EAzi6wsQVKsx=zMV zpVIDU%JhxO%@9vF9}JfPG4u;zJY8=;b&nXS-TJ+1oQFjmm!LeBo+Y3P6M@U>-j*lA zVSR{qtVUSp)m4X;QCScW7*>Mf3mKEjm5){PXU9aQY?rv}v++mp z?z)s`OXM|4E)9-byx zhvR}(2&NI~0-`qZgGbqs|}j`tmuMSqXv!bZ}`so3Z{3E zq&E3X*&M8>j-+iV0%e2|Wi^YZDm|51=ZYWi{;FoI=GB;^xLBiByTN z+FK%Hk1&{{60zf6l1bvNzd=S8N7oV2Ws=Z(BO!7QsV_2Co<|!etMhsl*E`7^ZY?nk z&|qM?m%=vkxRPcx%{o)K7j4A1MuZF((JWJFkf~!u zZR*MxQh{j|Z2@$kNJX;+tjhlgL5v!m7nD#`&tz%{_2C= z6W8`wN=DO-5i{A01FQJEA92g6Af-0*uQTI#R-91(J;w{klZZQe3=(Mp-RQI% zstfS&fdi0QAbcg$G}h-ASD7qZ-p*)P#=6sZT6fw);A%cNWCe9t=Ij3EoC;vZ(J5Ss zH2?Z$_@{{%wHXM?=g^tDaf8+)OHV_!e$+qGjEeWp&IL>iD0+Sr9MPHSp*6v92nWXN z&^XDKLiS;ao zMfj-!A;&)vJXpNMA8hrvG9Ztk0{vO$@rs^o$v^Up}<&ai$}?X*uO*F;%mf zLBQQwoJIFN3U^fq>v1k~T?*2444z%4JdwDdKUyFAV5@?@_#*ohN)yS_F{Qx?3g=eN zLUJS!%1m5M(Zr6 z&b6t!FUAo)lJE8ka^jgaxjy6HA$Tzlv8li}jc_hqCwyvNji-!nkpvA`lO9Ie$XRcHjSeKm;Co)L&Q@69#c7}5@E%Pr-_@qvGZ;?Mdz{uc@4)q; zNZ7Vc(k+F$hFfbFkr@eg?OBBN`gjX#%W~$~L%2=z((~fqYQ&rJt=BDh4C=6=O39#7 z7vBhBxJE4t%?JJUnX@xHvi=iJjGhydeYy14xLE|?@NY(o0+7Ebc*Dgf&TzNcO3NjE zmJ%e%zh@(qmvxh~Pq#1LBmv2ULcS|ghxTrkApJ4`G}PFnr{x&gkukI{7nDOM!7MrR zRDc{|CRpgV@lu+fFB=TvYtE=FSlmlSg`{zPsP61y3@up_Ry0WEZwH^!BtC4E@^n3wD;ty z1wQ(u_T`uGEsErsqOj**{-v%3QI}B200sb{|LNz$`scd9Lf=JS*XiGD=qpswv|VRK z_F1mRUkDvZDIyVQ{{_NK?Oe|yu;f--uWtq!P!w&&iCh}5gnhkyEr%xBlu_9KVGz#! zG39y%-A|8>r3cHHhw=(?bihnnJWR$1+B;q0AZfi7CmXPx!1uRL06x#l#z}yDl1ddL zC-G0-!cd$9L={sG;vsYbE2))DV+Hn%2%(t`YLZhPb6Q}dAWhqOf(e-!pEV1et1J9A zM~w}YzH#${>@sVUvZ9Snbv%lvo|FGl43NJnxU=R5%I)|3V{SyxdW%&ewO0-?_(+Wb-7YP!?IRXkHC z2iA9Vq;ZEF7xh-DY7`#{KL=#s?H*E285x=F94o(o0ZtR_U{RGs#FB*MwY3Sehbji! z^V5NS;p}tmZz!I9deYtJj)88T09k{^sLEgI8Eg@!4`&tBg*y`vm3?&6^X84NUJ~&x zqwu*GzqH;mAo7m{6|zMqRWFNOtN|32;t-V`M=MB7)k8wgUm;s87u{u2yJ#6qd1glf zCJYG;>MTG!_yfE|RRlWMcV4MI;t{iS0LRpscki<_F=5t%bOZPgc;%5Adh#SFm8wQ% zDZ*+Vwwg3_Uq57K!VD|1xecnwO`TNisg-g^i~qHxZ#nLGGeQI~;&<{Z*Y~_@{!bA(Ul1C+=P%gn=bzRdS@@x9?uCP4+K1BRkVo!_~ zx!rL%VbQFWizOjJ-X&i_*j3P*A?M7vTUtNOdMEnkXl>2Xj;XFDPy4iE^Z@J+RXxDc z&{MR!9Xe)X@^JimzZ{%=4gFhnPoif`*XX>Z_d6vOQm2iWW!$V>?X2j@Y9RGldX2+# z`lBfMEBjR_G6WKgTC2+leNM(xNkyYJep)&f3XpW@`wRxjSx!_5w8JAQ#MuyX< z#j=;GuWmwbz85gKvQ;mdF042P1t3Eibp$gSI+l~x$IdyomWy3Ga66Yf1y-o4^66nD zRNCVrU!U+922;crUNx1hEB#Ay97pG*$aoDePN+?YX?}NU#_t#?o!7SkuLLUN9w3-R z@j?PLjXTA{2aeYUD-VDRh3Ey|enlNJO5i>rj=d9%xJCeybZ~nIk546EiXwHqk>UbQ z<&358woq=R%N0j?I%WT0^45O(xNDX}?+9q>T&UB=G>7dM*xuqp_bi|{kn++dt8G=_ znunuZJKOW>OoT(72l<(+IAN*=v7#w`?Y64>RW_KXDw0ZaP91p%lk3@(H!y{BJv7{k ze{fWz<5#1>-AvR95a%X&qzIVQJB(eQ@rrK+6UuP!Yo!Py-~Q~B$frvmfE`P2#Q;>ga;?Ur*N&p0fknAT=7 zbLW||SM1$g*N!31+RLftf|3j)zj$*)P8Nal%5TQ(bqvkf6t;Ub7hHre4n122Sx{2Y zhYv@Rk~zR@DXmMz!NnX%8iVV^k{LJ|xv zYa&}-nT5Wy?u#~c+Daj@cWmSiWwZpG0;`Tn(FK=CS>sLv`a~LY2l(h!F(Q^E%4z(_ zGUbD+5>FI0)NY`7P5eo`W}jjdGTf0iKa@h%oO78{?`-p?Zih zrUGWDof_jGa9XW-^U>ufInXff@A^=hg<8Ui$cQ6%P@AYgoxWAagTHyy=p$hMGyl=g8OC9D!%SX;m0yJQ%-~n_g}D zlE_~Y%lb!yJ`rZ8_tc>ZQRgk?GB*t65vvo~og2(ov<(gnVqxw0U_RVDD1x*^8_9-i zc`n?~FSPtmI^1U8s%;4M$e<2CK@EO2NerOSXW}G=SHBY8$ZHwVo2|_Bq>O_nkujB9#To908v% zBu!FBlkxoQ_Yz(E&!wU(y0I`2j+g?pYvWy~{BS}V8J+&l1F^q_5a#Tw&5&{|1;*VW z!R2g;jy_~%(my)-e>sibyY6$0N7(Lz>OQ~0@MH&23dM-y2W?Ir+(Rg;qd)z(R-V)W zvccuQ6O8c>KK|cMu(iJ3e{zC{Rd#I`>EL{3b@c}%DU{_i2PB1w=SKL%#60~XJVjMALTTAiZoG>Vh(JOY z_wSdMDT8W}4JW5a@&QE{fMTsyj~}<>&qDA)7m!Wp58L$LX_Oh!IqNXVtw4WbN4ian z2%^xQu}X?IX)e(GAoIrUHz0&v30fWnTix(w+Jca-A7^jWLi*2rHtEAnxhCT&RqCAK zEt*!WUD;oa<0y+FT7L&+27^j~@%H;pr=Ko5^YhAo3Y&6`5eFFdZ>B-Bn-QdpR0{^*N^7=vOd&pczM8N9rZj5l=s4_QLKf|@G`xJc%>v9^`!nw0I&4bCZzV9;> z74mgBDP-u(f}C~cNp?zn;04N+dE}$wd%ky1&lpa7)#Z3SJp4^`gp!^K&CY`UL=*3K zr(?C6Jvn~FSm!@x?o9Hl$rlz*$KRCbwtodNhF*#XGEOWhH?^-m7NDx|G!?!unXXfQ zVW!H^G~F1pHgfl5eyh(cP^S}oM9|eC$cl}H40@!(I>o7qZ9xLZ zkY2Onqix?NcOuL%0Xl`|wtGqqCW~QDt}SjtkezV@CP#+GfKv#YX;=dQz5co`rWH8t%3+72uoNQPYB zNag^-t)251uMaf|fg?`Xy_^rEw`#?btfB&C3+Es=%HCLD?LZnsFl>?OhjUG(wvs=d&P|^uFsxpb*Rpue+nY4i#LTd&@J6X`RkVgX{f&hs)?BEDY z;uhW32~o;JJ-I4}th*mQi8_*oQSZ0N2JF6}D(T66UUGhk2c}im zwT48m!s7PEd#F@_a$9z&iJqAzmI`48@@A2iDU@fmg(QiL+CfWk2lip)N#Q!BH*LM0bIFlF| z;HHM^OqNC&1i5o1q?^U^0I_U}u;?7#`|2l)7R7l^NR7Ah5tzRh_Ae+0i^J9eu9{Nw$I_!qKu3%&C{kF%;k~01h5XlY2`s+3@i1tQRD}`>1Lb?UbFZMN%>u!os~pYmh(7Hiw>Q?V6m}bUyWFmFFnR;~LT42YH>>Gsp zCB-jpWG{{PP4%5ws_qYlbA#382veb^+cGeg$aBBNx9vz-*1qrrri`b!+XSeLa+1`)Z9k(4{J>Vv70eX+*nkWE;irB@!EwV8P z^b$Q=CACl#9|52RGdqi^jK{{DIN6y8)e1F+ohP@fV#;w zz~u=JF~`_(r6A&3R(x@gFYlIq9*>;qM>6sBul2|Pe^Q&)fIP9;FNB`Y8hFUUXYFWl9JFtQwBJVoKOy7 z`y+2=jHOg8rYxcbAqrNU4K!1okW9EkCQ>+0kaqE5lg!pILqe%VI6FTNW{H#HGPBW4 zK8FHH9bUMig_)etydx_D`PZR`sJuO+mdx4efewh5Wq>+1{rcrK;{}QZuqm$BHiTa@ zjX&q;$Y=?sLk1sFMY76NmdE|Va$e&>Jvz4=MG3t8iE{4a8;7(2 zB>EikWC_56vF@<$aF*ku7Oz!leHuMuv%5raR+bhbwgo(v|E_=Tfu9#V0b4m#wH!-U zULq?rh$l6T(RsI|o>MMZUrsH+mL5NULewycosF9Kde60*+$O~Dtnd*#p+taB?e2Y$ z=67tAW6-M{m!(ML^wuw77pju9bPkLK6Ls})-^r}>1ZY3jc{R|3Q?2ZiYprzg+U^r_ zAJGX@7F8sZNq+11p1M5%_f0_W`w&u?IQz~fpSfu-keI1&DI-cLO@8GD`xmbo8?iVu zi~l3%J0i88eIs!hi{?x6z~|KK>G}P^%aD5%=CjnRdvtX*%&nOwK_`}ETZTt_Z+=kq zTNVRp&g|p2()DldKJ@?12x#`BhZN%i0Em?R50?J_^a4%wos3=e-T#x?=!)CiaZ{vW z_nC5Ju2N$z9}XeOO7!{30y(|mco|vVrNOve4v)P^?@-T9%;abOC+6F5lbaF1#16h7 z=FRCw-CdaRbOIZAxiS2xCWyWi!md9|*)az_v^DDm zf+rTJ4T55%pw(SUVxRLlkqZ#7M5fI7Ex7dV2mLoj=M0-Sd5&9)?X3A&eu>k8TT5m@ z*NHCc(CRzU_tE-2v+X{y%@x&voptu&UQ(1*Kyn{J;x_5y?B(vW+4i0P`m+`Z! zJ=~SE2ZJ`>1!Ow&NiuX=B`x}A6vk0xui;g^lk%lUV$V20P8|re4MdI>QeA~U;j_1T z;a(Fd|GdLAk#a87$?Mq>qB|59k-8rKJz29)#|>f`CY_Nb7(vf)T@*<)C11bU0=BD! ztKxNHo4wi2s%u(sX*1Cay_2_=%P)Sx@-FC5~eI= z$;|L#RGtj^k!u6}PzZPg#hiEEHTSj%oJd^wu~8r4mZI@g7ITGc0F?5_Ex3sWdUR`TSTQumq$eZqysf+Hr>KMf^1@Y*-;=F(1YuBe^OlAeK|EsW zvb}*A?Qi^lQh`nk)U#o1e{6mSc`UgCR&dcF%`<2H4=;y#*it8Or|EpAegyYz^2dC! zNMks^%-<$BJ(Xq*I6)l8i^x9i zuP`ro1z5)>Rt#U6XGTU|4x?p3qi3gy3n0rYw}|GDG~e*LO`oQ*LBRMB+LLIUURZ(G zF40e9Y8xlZLgs!l2oLS3NaEQ5di7(q-<+h!*Z<-MF@=-Wmnx$D{5{J>cv2LJ2>y4;%JAd3S-yNo{JNe2S#7uwIHLamTamX2l|l@28c}`!~!C?>1&p4)>Z?u z-&iJjzi!5%PX1m+h*L9E)VLC*c%!V~f?mXavN zT#u$>x2u-Zd$AJPa3?5S(FqA%@JHXb-Yz>l`o5BE_wU1?)M^hn#IGn92EK7>Rvj2v z*6?(sZNyFCVu>8!Py&7S+S#=&c{=E^H7c!ml87-?*Y@$z5^@cuyo23Vx|=0tR{kvY22(hlA+b(6)ln!uMw84 z6|XVUu0Dv@>psyS5u&MA&wbQ9LqNrB8j=3rUDmUoA&s4vwn_vDGf(r9fwmRou9=iHhSC+^Ps^10^3v;V3D{Iq4~ zYTF(rkGo!;x5cW4GqR1OR!M8f3pu)dq8zZ+o*WgQ01{J?n(MKcLpskzVqZk8?`lGO zic4@y^1kS#@2W@{Fj5icu^+%_6?&s(&cyXy$V7Ch5P{s>z-oc*_pkR<%`c*{4Adqb zL7n`c;tv&7F?j4XajZPRfcH_)RDFSB9!~WeubRSiNUECLy(_Dro&S8JtK94I!Oar#gc` ztpJ5nMNp4E6NZN)F5h7*%q{xpIcY7{BY`MJx?w-dQK?+r+dVETQA;wS~qV5D1G$$LJ0lXI2ilg{T_Q~3B#q=VU_V(O)QUU|LYxIJs3StH zLdw^0S7}WS2X_%Smz!IcoC7Q0D888gYR4TeC#%r&7k9KBOi~bK%bdzyD8mbK2oO>r z)fI|pjc>h1)a3W^Ir|WO90t1cqt-y5Ye8`6E0!B|8F`DZYZ03On_?{Z4Q5_YL@St< z%}N-GAqd$k5^2SM9Zkg6eqp^$g#PlqS%^j)i}A1R^%vK^b_V@nrib9c1nvWlrK`V6 zsG6!N5=2)gqBzrj2YOGw?fV@K`<;ko^O^5Y#wEyuC4qedy@se!@scX9T*Q!W&qKV} zN}>{tkWg;Kejblbs2ESzf==@1L8R2bUPm~pK=%?*Urlgpd6u|rG9L|UQkiu2JQ@@l z!mH-12d4nCp@y*r8W?&;EO@NPb!t%fi(oBPrmgi0Alc9_C=-p6#0H17Y6RpovE^>i z|1P_gOIJyH{rH9>c<|QP$s`R)`tGbjn)(=Mky_KYpQ|DaAo;fLfiC$9NwiRl{PU5t zlcMb+qQ$P4Fg5Oj9OD+paWDe`LI)(#=<(*g3R%#Eye2qlA{LaxpW*SRI#rs!$+CXi zFB35knf+ltp3I}UVR;JrN^O-dx@j4j@?w@5Vh%wgi*AnjFP&kk2T@v`fJMb|Trt7% z--jvpu@e&2lax5-F52S#E(tUAMwano<|ve}(G`5P;jsu@75oO3w}kuv34N*jpL}(e zz(*p$wo6k7F7E&*Rb%EvJsCkHNdqckU*F;L=J1=}aG}g!?|HnUd4fB_sW(cj-)tb! zhzhY7^`bu(v2bc2nGu_dCOd&_jVT=*vssF}M`@YZbs7w%Xg?3gH9RXh-oxt2x3f(b zP7ebv1Q_QFFnFskBZ{7_#vRXU6ch>pz({V9;U#1|&`aMc$7E5^kAGpo0YSuRwL|^G za_X?bb?gM~dkJi~_4<7H*6JkF@TmWu--aVj45xGOD&SkBeAxfVm+OhR@!sj(qC|*z zjEh3-`KBc;3f?g06ja4XKP+O^+Lpeu-&i@pAQuM}_i0!XH zDN)MlI`Xb+{8GLMa!&~W*?rU=D!xZUB@Ub9ZG#=O`?nJ+%HYiG$B`j(9}K2E9dILr zv1&uv?O1o%YbM5fiNac7=$F9>HGUZIR*ZJ$WJSutH&6+((`4{B%;Z}Qr3IEgWnM>I z8M4)5R%~Y{NpcXFWk~J-=F_JZS})gjeC5;IecE8kI>$e)o1$7DW3G%GtDp!Q8G2zH zZbzFpm#Z8nWz`9O0UU>{Z4dx?_#173=I#6!m*c3u`+mXg_Q_&&EPP=bqJy+$>2&il z^5=rd%|VbONY24llEQ%^wqf8Jd*i0bjM4<#bAcZ8U&1`&O#6~-(wZt+zqY~HV{bM5 zfMScFdzPTg0g2$&=-+O0@2?0?ce$Pd!g{Hk$kjTBbJSg)5=40 z$*F$5P92zdol*qz$>#?^cbJ282AJcWDIUy_P6Y62j)l2m%M$n8>m}w21=XUzr$frl zZi0q7@=8fQ1mB@vII6;f#5UD+!4A5diV>M>ZU_}peJscC*Ufo5C&!0kS}q36YL{c+ z$~bR1B$W{olnqTg_ul0u<_YdF{o4mycpn`)jFmbB{6W(0k&QSAZE11N1ScJfn7y(B=Z-ubc4XR+g zgCRSSxJHWl()5t&xK4|WCCfX)K`9UveYtl(>DLF~4I|N9&}dVfdU`(T;_~OYlg*B_ zLMG4z##pfhg-p8SQ3PV$-*6HaW|}$OW;QP?F*UMQ#sf}?SU(OMamwSR)XwP6s&E^` z3vqSxWhv;8dloQqeL5Lg))z7nmBColjIkOuIDTPZ_>7otTJTj;%o7Ovr>3{8b6>k8 zI~oOm!q;Pp@j7rcvTHipu)t#)HJt}r4fOW??;7jyr)6y9A47;Z=Zo({aG}SOk5QlY zRamuMCA=4UupR$_cmY&Q0bq=J0hAHVa?qb_njg40k+NttnEeig$$fAUapzpmU zmy)_TOHcYW9x5QeYEtt6!IIwH68esB&XtkQq1F^^sg}|9=J74YqbC(PfCC8JEa}4f zdz35=EpSBZ6*wU@UQld#33AbV4#p#R85}ION^>n$LHr_PsRZp;I?d2%9yP1 z`KuwOoqVI4b4WXT50F#n$~$Xs+Fb1naUR}z9Dq}o7&jkyZTkIE5)cFI>!b8y_nL{A z%mT8f-dpk>YYn<_W!t;u!|Yc7UE!TsT{SOzjai?p*xnkF0_CE)9l~w4xePH!v30e+ z7en+-#H_eM+Be^BK0Jg5;t7moFMMjXYWBd3j zj00=m>Sa?YEb`8+_fy5L&;^6=w-apWBJ5$d!{uo94(qDHyjXI(BKKw4-EYU&t?tH~ zLpKtH*w0+kHP2L%M9fK-XyWCTCQVzTM)EUy%IQ;;v5_RX_gS$%2ew)a&r<2GHQDle#iYH$X z3_-{o>&=EqO!3ZbXT0}!IEB`|RdZL;=6054opRWLQk7u0TLk3_t zRn3XXSXUFCM3$ODGnSgX)7hz~_1@oPUgOl~=XVe8GTf-4KbxSEVISDYBUOI%nF3Hp zihf0t4_Z5thC%{%ieRjneSvUJ#lQ8h;2L&EZ|c!f^D{; zlkyDuLrB>wm()<1R;A-<_<s(6r)S;?l7MuhzZR-q@jfpLOd-OV-h>Jj@wfas2E#Sf{}6Hwl`dtEZ8xb(vdmm|9F8%+0#Dw=h6A=R8X5ok?z$cyA~!6*^ZOXqv53 z?o)`46c$YYZI=}%jVjS3JSs)I^Q{u-Lr&S?T%9j7xO2!zch{!MjNr>&<-#Eq&%iqY z)V&l2D0uBzjq7{uNSk<45+ndkRR+=_CK9Mdz8{#EJT~kT;YLlo1sjo;{_%2=uX-|I zv6W%o%;kaS$XQdQtTn4ioQ9g{@{K@+*{VQMpGw0=27Ogv z@fLtr+khunh^x>)b|@Xuy9+xOi6lHCUByKJ$Y)c`^T{DAUJ>=iV8Fm%=D}IV@yLay zHz&X%PfZ~pp2w!^Ks?qsf6_ju*OW8T8b0{Lde!jM;V6|0BVfQf1&kSvz9{odLq7+e zY$UwC?h(Q5NiBrF*HlUtS?$f{=$Wl!W}VTIBKFEqeCPmjc_T248<*XT$tAL`wJAS` zAUQ@k&(@gT+|-c7*REcDE*dB?=)ehESC>pzyY(%u33x~_bd*JwHA5fW$*bh+e0%bY zwnAP6W`_q%u<4Rqih^i1l^?iGovHe@-2u9Z6zMfhw(`ESrk!|>@olGPXoROd`ZuC$ ze`+~AJ;-4evHf?g93ebc{6 z_>0@HxaJDXWotvK4iPp{dQI0IBv{!75-kKh9OcFl3ptHD=+b9SZGxsBy)!x(sjnWW zuprKsBFQ#|{TNbdStE<4K@A~loNPi3yF3)C<`s^=g#EcnfWpf6`)q@!AW`v6sWW7c zb?4I6n0(Mqjan2UJ({b{l)dL43&o9)b{BiO@^Vd>3?}$EADYXIM;mj&Zk`;#!}ek% z*P;KS4g9W6Yb*Ls4HTgZx*p#qAop0QYR*!JyZzn_j=y4@7xO7+gn}`bk99d#wZWI2 z^KClc3+eZKhjtGH=MDVKBIWJM(Nm1Vb_N|Lz`Yy?TG{B5G`z$GZR+xCz|09{jH60! zjYO!P17eh2649VI;zH7W%R(OjL^|51hj^c*!WbE-)9qWj7sH>BHmO*(8qhT}3w`6$ z`Xp8AKzTdnHM2Fwxs!2etBqrE+G(6zXXr|AF98NHP2ECGrPAs+vy&Vioh^OT`F{DV zx#Qj67s3np@FQSe(Qw2=B(dZ*>RzxLOePVlNXoQ{%>=?OT;%sl0;{M#Slhf(Y`ZE< z)CFZ$^;C&&5i4R5o&UKUispP)AOraeB4RNgDz}B{inCX1UdD^5O7u85{wH(?CH(Q6 znukUqR8=xCts2v+z+Li<>Ycz|x7Vz5hF&s+AN?I@rr@V0TpJ!xb>~mK-xH%Bv9T&l z4tK91At@aUKC<~*tO5flc~;%w6u(xRwG%PemAnur;SYvo<^57_iuj+nL!)sBLGoZB zbr>M6c=Q}Ftoe*gQ?x(kq(BX0Z44KJyyVsM$=g%=>**teYZ2i*LZ%o1Ivi~0pG;L; zKO0svU6#uOoALNu$@UFHwgDw6**(}exVh=WKAW%Uj+R+piyUm6*xEO_IufsL-NiQt zp3~~_tn#q6P;}1xpjFUs`@JK1Onsy$vCGjQJsCbk%+?man+!=wz7{oma$Vt#mo;=`O`4JDOe7A!N?+C-EQT9%R}=9 zSwAzIMk=oHdD;=umkIgEzMjGU>IZ*3L0O9#D?=74_z*22AL8Vf)r;SXdSH6u4q&SR zYMz(lsG;-J93CK4lTk{E?qV~kJL>*kI7z_GM5QWWKUl@G8ADMBT9r}ikKirP7PCTF z9vtOsk{C38=)t7QEm{d+Ua%Rjql2=%$nJHhZlNwLjH*I39GnVL*uuC6t@&jJOvwC6 zjkij@CI*Hl++sY(ZpF_cgM0Z*uKjrG{Kok@7qF*pTqS2_o|LY~w#@$7l&>sfNa+8L zi{kn^rw$*y(hl!V;v85v{9Q7ju_v1sQ8*Y1MHT~t^)IUZa?2OepMK<@e)PG2@{TBZ zxX*iQLqk(2>Q7asGl5J43>4Q6sB1CgyeC~{va)}+@l349FfNX-KDd6T=YAQKAHdag z6zWyA{~g*ELZ!N4w8#Yw>5jc3wb8Zm)Oa@OU^&v7D@PQ_>VlF%Y;_Va$5x(DfV^RD z@#O8kx+<9UgxoEBLS5%8fW+vKM1TM~Q-b%|E<&5>s<*V%)?J;;h-j^^gv#Ex75Z}T zxSt+7NDU%62JiW8a;5H$-sF4R(ZweV-6*4E15XI_M>=a+GoalDqTayjpi5ELYK#F2`GlOFQ>>j@#yH@cHq7(#CW2++h0(4&zjl2BEq>oEZkL$%ni%1bJH_--MbjNx_M z`;%L{<5wHe#THe|1rR;qxT7R!C>WuS#?c;`A{@%F3Xx% zS8xT`L`yP8etCW#&L_BvJS%vS(F30gST67+*1`ojkj-5(@f%+A_m~o zgLIc$f^~Bda!w0PX18o8;JMPebuT4c9) z1(z=2Vx2kwpddF6Lmer7lRtQ?i^CCr#0+E|dLuKcsCFnKK@f&K~AB%iGY4G*& zYzZ+b!0Dy0X~Z&SC+T@lifb5LU;6v zvGAs#dV^kk$PN1N(Dgn-;(H}KMK4z0E7&fMI6 z)9r3FBM?5&Q|m;xc9`o+XDvOGT3uhV&Um7o!$*vXD8dKM`T&cCjg4m_Fx2cJ>Fiky3(ey6x~ys0o*nx_a|2yu@scxNcg z>cE9Ontv1W&n2{8JBakY<4j>=WV>-NW`hi%c#6>T5NdsePi_iwIFC(9n%2}QwUp$1 z9&}|@I*oftW+R;vd(F&&=DLJLV~#Q29SkN_Li7fK8Y@y`geO@VX`@>*70@Pefe*8x znkXntWvwWjPS--})$uz|I~HXkQPj|C8siXu`mG6GrJi2CBIP^P@hguEE6S^)c9h!i zX~4op(GeMYVMi5t*X?HLXzKmLIQY(j!!S^p!1;19qzKn@F;J31$`@q{8(y-;hXovr zx<_wKc?@E|EVjxV8)bHN5fzai2V{bh{mEj>0f@^Iet;NvkG($s`(HAz9(dyPI7BD$ zGT4*0C6ZONZljHpd{Wuo@<*lx`MY^s$OEVhcw(?n^mr~2vwxH)5i@$$)H;Ur3kfCf1$nIWY`Q?J5Zscb@g z79x#+0~yvuP8?a7OleA39E$@0e!YFJ~YUiow$mH~B z(f!PHf#duh)D##2-Ze$z>QqT*GqG3CjN({(@aHM;iNgu=FmDBDgML4&(G}LZ_6+dX zOe{knF-k{cbJua(rY6g26P3WnN@gjiO#Cymx@PLpv4zo+Ka-Xw60MShy$6}WOoZZw zH;ZKws^gfM{(+^v-ZFNtSWgFRj&m+}g{L{TH>!U=u`Pph)OXEa8$M-#eVk=IdwtmX zTF+E4CW`k9GnKQpIl8qcX&cbiQfdf8yp%TLNmm^akrwU)d1r5mpe5-NbI8$Db2F^W zClsh}ixjCSDevV&`Bn;IoraLF4J4vd_8p8g0TIJyy3l!^z#trSoPqGZ-y&%B@tl_h z%nF;Cn95M-lTCe-P_gXvF=Lmbo3hNPUaG1>=;W~(UX!JJXSBYVIV$ku4o@DkEw0`m z&1)C+t!WrjCP+5YU&~O|XVex8d4NzRjIHB-;JTAtV2H`_&qTn&#R+lO7|& z|Be+KnLGR^lmBZqP1!4!e^_yC-GF34jmkDaH&*zc2)^!YM~dvlPzd}pzl;E;k&zX=i;=x*)>}B)1-zgG>=XD|mcDlWeDst0dWn3}XJU6p zmIFL6N5%xGyo^xSdV=Ey{h#4(B^WHe_h1aidGLGrD8z^s^PPrsaXW!U~JLQjBz92;t5 z(%d?%KzfO2)uC3fhG_1jqqHPkv>mpGA9E^Xax#s&=ry-#01`YiGZ%H?bw^|Pu*{C< z0nHKBSX&URT4Lmjhdr6dvb#m|0Vb__kdtAk{t!KcT<*NsK+OUSicZqrLl(8>`8w2F zc_gP8ejO_AvZtSJAE!2p9`dI2@LV0l)?4P=uC&4q244@oAP8a5A!hh??ViD8mGI--edgFv+ zE3?{ecV%snzctSB8{{2U-wQKQ}4^7#RKx&YF`<5dcmXdLc=tc zYCf2ZpKZM;(R-m%TjQnk9}vFzZ7c5f7&0acS|&lkfVy*!8M6B8VzqVe*+$8^BDK&k z*b{8hG1>BL6z4%|L2WPg2dXTa7}_dI?^5t9E0JaY(@P1{#|~F@mfc)!oRUzzTZ|$+ za9)IEvL(;5GmbK8p6d@_vjF+3ufo#AF5lsgjx%)5qe|I=^V|4Y4%#JAH_Wv1zLLyf zyyoL#Jw~kPZ?}sS3Sv^k)Pb8 ziu?;um+yaLsNscfecS%!4;twJ0Py}7hWh_i!TkqDoNEnh$4wT5pBi2Nu~+|$3Jh_Z z8pd<;l7=dxnc;d;h0QG@92g-16srV1{)Lu}bKl-!NO8X+ovStJT6SrYl(FH1^=(sB z$hd2lPUd}((jieQfy1=IQ*EXz*pKR!ST;%3s9rwNv{FihIJ@YWd*X09X$@<%wj(N` zn$D?(D8yA^rYLk-c@+8B_aSIA2{xDnnt{lmf5}KK;&}x^Zj|C98t5|u-16EWwgB_C z1c*kwVtP{?aRdzQlY-L=3CUlnhTs(W!UbfsW6TFfcmCa@HA2WM?Ne4mnZhMvSDi{G zd};9n!syjq?Po5O5w_`2J1lL4!yDwD{u|=Adz4y7^wQwHN3feN1=fx;FKC)*fUoV3 ztopo261Kk}N$C5mLAi!8fA-!Ew-?h@L@DI~pJF#Biy;@Tp+kp0W?@Zj$iCJJOKmp6c; z7jL0MW`pVe77&4+`03iC{h!#^5F@76UlcgTjcG<1~n)zm6Wn zWUQzWzE@jNYjA5Eh>l7H=F`;qm)o){`f*yi&QUrj(`|@3zg;#cp>+g;R1Go+n~O~@ z78RnR&G#Ils&Q{3s3HNWlvrF%0^yOO>7g|Tkz%qL>87mPSr^YC=*Ed=hL*9Jx}1G| zR~dxqP0|m|wM|3aBjTscf5tc5Ema7kuHn<#B%*pvuh2U%5eP6wou}!zD6mXa<%rIg z7msVx(owE~KL$ICKm*VPyLw4?))nOV@?rDZ`Ten@BfCNHdoc*DVo=E-&_Dx@yGv2) zK)5>_g*I;YmS;OaHvIAa)?2HMmw$rp@Zge5q!L$mYYO=x>(v}BDM=B)*aYh7OmpmF zfzvRj=7H?PBPz~25&cnTfcd7|5@KRlT8Ah~z$pR__YxKN{YRgdgi%lA?S#ufNgw0d zl~y;L*=LI|2WZ1S)f@{A4&O40!QYh4Y~EE|Dwx%3t9B+y(yOv8X(I`$TD7S_->3^p zd>yFD!hooi;Aje*fdJl*_Vm~6KV+G1IG{gB&Cm7W5r_ykO`Kb;t{l*6pF}d_=k}lq z+DdUUmm`3UohpiTu{})c;+Fg3rzmLTicmjj$YO`QQT2tcu4Gdv!^yyv!=_}}kFl&Z zOf0uknofn|c`(**+CNw;%W9?X2_>};Jet!e+xN!k7@r!|se$S+C@RW~jGepA=fUR2 zm?T3LQFdu<@m^`E>8n%}g(+x|Cp;L`GjQpd^;GN{7CkYy#osB(Ezf7HIhWeqNu88F zNq2|!&v5H$N?Lw4!@VPY54i2M<}VPD|7p0ORXye+Rk#BosgzCJd$x2WX^-l6Y0zgc zoSsyVG3g1|ypyf)T-oV1TPO5+L;BwNJ$&uQy~c549g1@WD17WYD3M~R&~PRtD(lKz zgU+_Do>FSLKd@|{J{g*H6R;KAsaebX_#&L2Q8?3@{E7pyL$rs7q9Tq$nt-JdIWhki zJmo9cAY!G7p%BTOq05q4Rmk_wLq@AXLx+n#YAZgsute3_Rd=!qIP35JavPND24;fK z#*$EFOEw{lwYPJ`s4A`>TCHzcg-Q-z)MdnC8>E2{J3>1-#W;f!syUwh;(D$mvk|`kWkJ5Y$Ik$m7DYY==B% z5r27-I0;uFK#`;(ull^CrC1pp_;dc@^nIoHB-p8$SR>7`WvMd#xh2U??*}ypT4K50 z;1s2#sKqwC3POxdN^yLYllnuVVv{>gVggTh+i|kGpUCFYHaRl*XDtm`DLEpZIkHejhDD=%4&Qd{D06EaZ{ zbV2OXI9olZez7$_R~E5EDpd&hGyivlv41V6l$jVUx7>>*iN&bP?X^p`S&z%gMb|Cd zrdG4)mUbZWAyCOr%8x>`<+t z2m-o*vbSpA-`HSC3_xNdY7;o87^T(uOJpTINv7Pa&fX~5la&X$#=Io9$p>=}yIPEn zY9Ahv>n$DD*V^>X%VOgW{^Mw|kBS+!!xofGgIc1^Zwgh4$qiNPL~QBl?vBJ?P8Y;= zOpL(q@u~y6*%>`c)jKlm{t}x#pAhv0&nFPiC*WqCJP(dKQi&&8rYfQ_`biYPdes_> zi6xnisqzqCkxiNtO#=+I^0w=(#tXcA-QfpCpiM5{fdZoAbW)iBSEXgiLm|wl7MM0k zauKlQ2RyO5XZjNQFTR}2e7OL|{ymm=Tm5Sgt0s1IKUP16L|S{A2R#DW{{(Ap{i)UoQ48}hr3DtW-o0XyKRL9s909+NoE|_@lcQhbc^bTbURR7%Gf~5 z8mkM}f4kOaZGO!seK_09^0H<&>%YB@e`Kj2R^vuS5+e&g1To9}4l4?u1(WL^_e4qF zk3>g~hKEZl_(OdoF_b0))hG1tyHs&)<~7djW2BZ~J*E4sx-99rN$5#)+```=MOC9~ zgPfHt-+L>_x`%tKO$TUyFVj^wWfATxaxm$xVvh5|7}>6%@rPRV-+&W#KiX^#DXKn>#yUj(1&GbgQs22colI zce+?{R^EDUsgVQvsV9j)0V9Z6;U!HT(@(tv7O)(gXL$k{{vf4guR*+!;V=K}s^MeYALg_fsNCb9?xakBknTPD88$(%8s5A_ppD_Cb zv+j91xCSX!{$$Vetcb!aFgU}`Udu~DaEahrll>j_jY9@vM*^i)>fx^zoPA^gYBGtH z=s#zr4_uM8>ZRs4bkhyDvL9v2;1$l3zseYt$nF^d^-Qxt48mR`iEZYi3Q+gLd=cIl zg0)d2aQ%i3&^Q)lxA70jZ%{&@MnjEx&S%n`%w;)xS}4hws8dIr19NGX$8bIn&XGX8 zo&Op-RD^cQ@L$eEY@Dy_(-kRp&~?SYOXPXTWD-<6_RGPgczkLFfu|xMkppusyaT!$ z0VL5+^h7$RMSdo!kkDLUnG4ax+3L#`SQK zh0*-{Iqac9T7J5CPc!X+ll?qvy-8egn?A2xpuPWmYg;E(YpX6b>^Ny5UQ+)86b3rx ztyQa7E120idI9w(8@%IpK!21{YgKi~>^+3{JU-qfg(kXb-Eei7`0d*B?a>MwCTEC( zR(I$vaZRQ89B}m+|7*fYYktR0o8z4*m`G-1fjfd~ReA)FydjIfkCq_xUeeN(a`C?6 z0lh06~Ex+Qw9e+?iyzeUOCVnJnITt<}1t zkQWAl;j?w$--&G&FFEVex!~s5QuaLO7xw=SM^IVS^nLzoj|?FH4>`EGjiZx;v$e5} z(|;{|a{vF^_q44h`R}a**Yl{Vw*{H3?|w1P$V1e!Kp{;fdw{h72}Y30V4a``Fc$IS z%?CgP2$zs7A@1ni$L_VwXGEfjCkqu(mfPC7 zi?f&_&wnf~#HMgqcIH;zT}6=(kF~~Fmt%;9|K6^`nY%Q@Y>h$9V!wj;6vKtsosid= z?Uon4ldjHyLkz8h2&mccXcSkW5q~v)^3q8n6^H(`HT@JbkCL`V9N~{bzS?trLsmT^ z$f8GRq}A9Dz86rs4QCnx$2i6kl{)LUk6FG6;jtv5wO&+|g2MlAF*HOcD#7yweE(3__x%@Lq~-;6^%FMDR>OYBotT z@5eOev(i69F~F|Df`CdV0sO z2N<#8$CF<-qbJRLVCQO_MJ6UA5slLd+Y}U!K?LBlTrX96$)ZxPA5Cf*;WS36Z4w3D zt*qHs79zTIxxWIf3Xj@cj$&R}5xDV4mC9E7TG+t>u+|zT+Og}--Q#_8`O5w0Mffi6 z1h^x~3S?LqTsh|rlre``wB9eIilUEsDe2ly8lxeE22pf+QUKQ`*>6WAL1>}1$!f@$ zZOiIU@%2dL1mL1hOL))1!uxf1m5uA<@HMwik2`B`&uC<3cm77C^QB)V&de3xI7bDV zTWY#$qf9N>x?THwKGV&cvL2U%6co|^R0b$Wu@)T3fw=}05`QPG{PlD+11c)&Zb z>>IFtT;Z~f7?TYS+-1*B0!i!8ebcEO3@MCk+~PJ`M9(~RVzQ%f{xb(hC4`Q)ODU;N z>5t}pt_(i8zlkhf5div(G?@773XlTsRQ0(bW1Q9rgTj`8!4(&zwnBv7ud|Vrqty#2 z-k+Q)Qjwjc!*)U1wVI**V?{xGSxYCC;42#n7mdorzRfBh9Tej#LDq5 z_^Rhr+>m{%M})DC^`va65f^D97aWrM(yMxMHkbK(HZJDltCg=S8yme(LE9ML>1KY~ z4E&T^1L9dX+ahJC_NV%rb8BVBOH+_gE$3TrTdV?)au`h#hl`c*=GFJ%ldF8YHhAS1 z=T14wjo+4EQkCYTuPB(M@)T-| zsKWn`%$|w)zcq^H9{*hD|3UL)S#|fHPWYdPL>GSxH4_vub07~+?Vp{FpLMOHR6fWm zSF^MsS>pfc?5yLeT)I6@r$|VLbV+whBi+*74bt5q(hULv(jXw+Ahk(Bx*O?E>AN{~ z*{|om$8$HI&1drmzwgX?X3g{LnKf%23tdbepx?*uR7*6{mk^>!xHMev^UU`;Yfq|6 z>ftJqNZJpQx$%<@^QQ3(!m4ap63dAXDLCg0Y^&=n>zy;t_B55-B$k)ZU*%xuqAmeM@2Mwqj-Eq*;lNiO8}ZCKAvKn>dh=W3u?s z!vBuNeOUk*bHVgH%!@5cdnv|ynW(lOTF);JL6o0CSLp*1>V|wLk&D4cBWS()cZ!uw zP+TURZ<$0+Oc5IhsP+H^aI5bAh-G_RehO;L(2#n#Jb+i_)ZMRnY{(aaK7p-TUswk= z!Ka$-rd6P{v7Hf22nj`sQ`@MQJv-wb7VmaOJ=k6)o(=F4qzL3v8y0BQMzmfWZng7b zaD;aajZb8WOMR)CEix9wxZBj?MX@cd#<_h`gf4!&Rb$Wu{%lUQXw}p8s9YSEt*#o<>`b8!jUWwpdomF!%vy-VNO^b15$$jR;Q4NJ>YA?CJP~I9xkAP5$ zz$uyJ&JyjHS8Ir5ndosYWWj;25#IKQk8rX^6<1#xx

Wq0VzPFE0)PHo@LdH5baxUcj|-*p+#uEnlPZVu{vKzAK&6_DorA--DV0 z@L}$&GkSEhE~4%$x5#n%>`<54hHv;yCA4pt&b|237IcMBI&7BY!Rr2l#OsvV<71r4 z*3#t6j`jNxs|4GRf-PM&u3t=k_-N?aywn)Lg3C4X15nW&S_UZAnzUF3n_%GF|3{kj zIrUnFCZLm5DgU%mX=r5*yl3k=7~4DlnzQX`-P_IrV(;gDx?W!T#_glG+iT1bAKchQ zu96K(8IY>XqFWQok4er1u2szCV$3h!U8JUHd#lHyQFqHO61+e!6x|;j92oR8F=^dQ z=DSRyq>V-dp4=%X%wx#7;d#lWk;m-cUB0gCNVP645-ROvh*}2E7hv?dJQ^jsa%D(dDUFj&9!sM zLD1G_>uYlJ(ZLNK{~2kiTM=N_q9S~-tWn`l7_Pr75!cr)hHTVFntbTA_ds89ddAUb88%u2T+4yD3=iDGqdQ1WSffQ zRk0wydVVJ)cf=x)1#8Ieddzfne7nZ_u{8kdPBpG}SEzl;FWv=lAjnhF+2aFQ7M$l8 zV2LA1sv4%yL_>>?eR8c??45IRgcs8 z2r>r4tAiu+6mQ`!i`8VWn;$XFbtVU8`MclGLR6EK*qcp!THUV`Wps6!eM7_gXEmMA zVq0VxZ1cZe9VgB#-r2o*<96u}7fK$MYcmLavMTyLqNBG`C^j(t`zCLoGfn9b_q?S@ zvm2F0L=)`Zf^1Awjw^DJwzJt=^5@mh-`hfIpC534p6;feRL8A^lHTOEY(thQ8cjw3 zxTwF*FU;)ZqjR~?E9t<7cWR&;Z^Yo$@~!%Dq1d@pkdPF5A%UPQ-EmKSIGgpUC8peV z`TD?a(*oujf;&O}AEzDliOj8_h^!ZB6dnITHr9DDFar3@Rmu>3=A2|FPFjnV;)T z1<~TV&ronCe`Lq?!u%-=B;03pGXd@{itt#9xEn?~>X@^4PtUooq6}UllCMop`TG@3 z+Ao)Hy&u)AiPh4Vq^lfvA6Um(pKQvEvrp(zdy{uN)RYlhyD;c%pa!L%v1t}lxB z0DUVIui)%yQ6F7M@q6$o35*w#mVhWt&L?kjAfn}(Mi1n~z8V;qMd0w`fqPGf4rAb% zsp;SKR<5ulc^r&tuhnn5nBUb2RN_bC2JYW53NzOk%z1~_xWhU3w3VOB{9M@Z zO#XWZfM&$jrUBo{f>A_9KscT`iS7-4MNoei}bCaITWa+*|~aM ziRth2A@1z;tZuN+n0{!!;4oCCsRKt>>$|g7(EA=%w<0`-WhzsSRD<@O4h8W^I}|?JZSor$Z8^5Vkz{Ng{PeqF9VDI=n!S zT1r9D4vHg9H@lLY{D{b)zj>7Q9XqxI1F5St!A#<8npWRcD&9JmrTiF zm^R7$gqu*b1V^=)+oZTfu~$4PTwZ!6k|Iqvd?y{tMgv06RWEUzLhTxag9ppCm{+-{ zG3k=mE;_GUByN#aJ4QcPaGGpvdOL}nvd|G%z&p-ckAiD*dNFux9hu8S@pX_vdvKW5 z(+<{RpL>dzG`9u74D$+m!!x0w=D6fp9V%)>Q#-lPkf2^sL+}uhQiS&5^sIG4hnfb< zvz3>Im`Lork2lyyR53mz;^j!U?#yed_V1s8+C<2ypT})$rVHG~{_I3c$vfI_0)s^7 zr7Z%T%1kTo<83a$j#RKC9t!OO*^@0A-5mirLT><>J)Z(1bzYY^mT%+7Qw)|gbl z1aHi#ys4x0ud0rx?eoQ=l>t=0#_iOxDlj@su_VzerZT2TCTbP6BtOO)L;29=Db1#b z8Nr;;JyI{jZ^zi0ScHxq{Ca@CR_qm>zp7?C%Aq($t~48>|8e3q6|ePdIJoQEVt<`Y zQ>G?%+7vN2XpJk_32QX87O?pwl~)n>TuMf+moVcp{jtM_w8(qm^2h`k&Wa}kYpf8X z$<+*$)4bjzmAn=a+|hkI`X$n|&hdKz77&X%YTrNIXi72KmN`6+OaZvUQeNcq7@EBi z)3f#V(O0ip->2{N^wO|R>Mk?*s!E~1Un?FtfY;4JZ#-Xy$u!}Lu9tU63nne&#+YeP zfShU`SuFfK&ADbR^;($HJ9EFA04wfoYD11MM-_wbJ#HPF#k7)AzlvHu{wIxL_Nyn0 z{n#WZcRP~S4crs6yH7};nsHCbF(Y9GsG!9y;yPuZwWSg+j^|(Ma8CQIT+6oXJW+6n z*!E=$aj$_My8~EoPBWA=jH4KSynpt>G8q2{%lsf3wZUFVo%aw2k7SmiP%ak+|1SJ> zU@yYHG)B4ZAitj+r`JA3-H@LnFkhyeps>J&t18bn&q}y{&p8WmRpDNj5HM&RcJpb% z1voRL&-3FA&94jt9c=}~PO*aP6jju9B|qNfyulrDR24akPr&9e_`JD*fcgR}5%V*E zN4B~8lm8I|JS!MJ_jUE_A6ADddAtTB3hJGh%z?b3S<-sshHJ9=F6n|6%_yPr;OptU z=nc73Rsu3RZ@FIWZ*j)`%uF`r?o(bDi^H5~B)eXpu1@eeKM;rBa|sdXAp z)@zn8jHH06%_FBzue#VgHhhBpDW$kptDY?7SEZReFAPllB885GRm+6>{W@*V|CF|2$pM&1%G_%1@4 zAMZRq@Cx^s`T$uZ=cRdDq`RZ8z5E`-5{@Ii9IMOnpq@q2WkI(t%<2j&6h+>t-CL1Y z1dg~Bje+-9mb_eUwO+Ni{bfhGuwOefnKsSK6s*|~wu?AD5)770YIR7Ar=%$*ku&Z6 z5x9?Z87WZ`Wr8sz3)t54s(O%jIPsu^<%dTopm%W*PjBi={1I=#KYv zl~efij%E{fTg7ikKZqKX6CqGJbyoyC(2=MjO|r{8k38&m5PUUxiH06c?xqOr!EcZ+OkX69Glw_BcCl(H@bm=L#g z#E0c-J}kpeceDQ#L!4dN2B#F?`dx$p=bab@d6f43^{AN}YQEstborn>QPZNRguI&S~ zJ_DDMolI06vtc{A*0*0J-3d=^(n`9wFPxE5P9f?!uxvDTfRzJnYmXg|;Uf2*FmgHi zo>%UE6)9^Anl1N6`u5?iiy;A^D$c!trx4I!axKw(+i9-LLf6ICkH$>xLobw78!x%` zhFI6S9~EHL8X9Cz!|U!@?b=n%u4J;xH>q^p*Hu3AK7ZC{WWKe=&**f`k2Y^{EQ{k) z`Q(VpNaaD!V6D8Hhh75}!b#9liW|fd^wQ|q?GaO>pQJKG!SMdK{U!o1$bOZ*kPK?R zr1;9jZghGDII(Bp1(Wa7CgLVw-lvBZUq2u6=JBo}7;N3i|gq-dO8w|1MNgLOcJ z1xb20V)nDQD@6A~RCGiVj2`z|ONES;Qh~2@2htJMNd?t&SNj{uUT~143cZf6idqdQ zkeMl9i`Ybu5HIIhu1!|>zS}|UY%KGx8!&W=M+Uz9HKT-GWgR%<4H5xFp|yVT$!l+Qz4uc}ZpY1JZG5#V^q&u)Wo z*~pa+iTvVyb_byc9PunflsLp^YlFl^Le?}tPd6RBWwyL4-Yo-|bH7lTMxuJFL`Byb zgm4H>s)+WK6b*9Qp2n=T9B9=!OeZ6AbLQSqX?HX(3K(K%UoKn*5vkC3$wHH51)?ka zWJ&3Q-0!#XEx6|!aZ$SW8a4rMZAju;1eEX^v8rLD#~d<_nUQREN*J*w@dkGg(2vqt zb@I-Gk*WhGZ?m#TGtaw=&IMg|4JTmc4SlVnZ^s1Ay&zg&*YhX2%uU4RTSbfwpz`vz zk9>3gK9?PK7=_wI^XxGDFoN41&4#a`T%&8)LzaI)=g8?p|(U6rFuJe$fbk@$X6s=H-vjPG3T2sCCn}=f}8rdUl~RA zD{d0MRy2RHe~9I@+vY`vm)eN3x#hSuaKjm*K8k4cL~>%io6_^V#{FQ*rhN4*uJnn+ z+UD1Czur_-tLtl)Y<4^5(hvkFwF(cvRA7z?5mqz|{BuQ4Y(%&CysFxS?ZiWBMx#utCcG zk*SR6^CuX!y)Z@a7T;)##HXudXq<*sEb>wnjJfRKMF4r{Z8}dNs?CESJ`TnW6B1A#2cY$&vlJ1dd-1~!o{1%i zY?^Hye|G1Ll{?oJpg}RSf1H4;QaQ?~?8%r`$KN4s|%8je-K4V-j zXl^lTIJM)fFGWA!{+87@)MX>K4`BOr&*y|4vBYH+HUfTc1XjJ80@h0!B<{&oV{ zSuwfosYA!xF|!EjBQ(B}9IWgQ$b8NSEn^cUX=UPA*Tpr!nGwFKnu_IQDNw%B(MV1e;5C1UGd^E*oSxPEsIJz?%=7VSZ zGu@+QwXXSkPRe~?VffS*K8ZQH6wWxwlSmmzvz!4VXC6%m*q9W2Tcj?ONk(rxV z>WtX~8@IT>j#zEtWZmVsaf(mzmoBM%l-cOV%#eN-qctnw~Y}?~{HZ;J~%QEl}4Yt6>jZROG^P7b)_{sK!DsGB&qTr`ExHnJG#QZSu zmJ^=m+jT_YMb*wE(t{rrT%zg2;yV?071Jlch>BU!g8OpmVvW1L=}3W~3FbmO-42JM zw5do|m6*E0;RvHCtU)BnuIE;-Vkj z#kqE%Uq6WsA;8P%t1G>GS%RJa`39>9^=bN7^gj9=UE9Pget_9nxshBCr09z{^a65j zN<8*rDHOd~dtuTXsESw6qW&-9;UY-#gh)sYRhB}f^HXZpmLoe#pdI13)y-{!4d+9T-SRql3YX|O;uy|qew{e=1`v&%}s5W^lNFgOfS@ zjTQOTX)UTkX#r^!eHaY4vq!$}5hf-dtGwGYq>-Yq$l7Yv)+y;^33w-dNFj=F2NQqV zrseBy?8y?n29@^!*N!hXPA&7YkRAO|IIE}$&|73O=4dHJ9}KS>1*SdNq{q|AwUf2L zR9j|;Haws!S|_dJ+-dsi4;yS!LmHxS=G=}sLn&A>?fHn|)z{O@;qUqwn^{(&yf9*4 zeZUjL2+ETT@WTrM=P8HX?aG6*JxMhO9F@l#d9}qm_k|t?EG>;-yCW{KG6)vy8sL>gG#1gGyuh%t;BPOyB}_$DD#$%rH)6zPchq>T z6%oK50jav`l?u3K*O8IdT-#d-a4v^jkQMjxL|3r`U%M=x`fgXfmi~nmHfhD8O-r~> zxK)^cX4a*3K?lM*zvXyTZLD1*u}i!5TC#tZ+0(UIG8bF%mhL^@+JXt**H4@8X0?D3 zWwo83`*U!@$v3c2v3*mER;`cBZT2Xf=gC}_53GF4&a3fELhiTKn#QjOv4V6?nhp*Z zt(tiFTAF4&7#hwQD$Lwx@s6Y_?ct~;V8qm5f%mZR_NA=SV7q5`(25R@-OHkU zMFD}MWs@eAWw_yu8>Evf9;3VCD_3WB6=uhD+-euNK)1jE-gd!+JXG09=NhMwn&EG} zDlI)_!mhntBJ_lfCPoe>2Y>=t7U4n{o`y?HKjk%+)?mx_)q(t&=tdc~^#tx1|^1uq@`liMR@>1Xs&mn)O!25ejKG6Pr zK?CRM-(M(Tw}KC!9tylOyjF$=gS>h+$>$IJ{t)s1> zt<|5ZZ`Y3hNtNV_6_ltP7+8=BIPjwVZpz>Le}MUi>f?l=|3dkoJjwkv5*QeS*u$;# zv9iC(V`X4_t&9yFffoMD8voF4s)#CarK{|~QQ$=#0a^qVM4j645B0w`-~Vj%+TF{4 zGRM(%q$?~?)&O`Q;wgyiE7w0{4Q;LM^d0}y{D-DKTsZf0{A^O-b#zgE@QcTWNc8wq z^SABG-*+#6QTf3L6MeW*Ex>X16!$-e1Z?yl$_79eva_}StG)kVg9lYIJ;*5;;5fAf zzCi~Xa^fGVAifOriT_je?|l~W4=X(A-wNjasrnEj;qVWh`@jC+jmFwi1B_C11*S^I z1u^RmsNf&2_P^f%fDQhI%jOb6Gs6NL27$WJ^N^wU@u={~29ba8Y~BB7AH-3+4hMjjQ55LXpf>8x{~7W}mG@s9 zfDc1EYYC-p7pT1n3kF69(uoaV;^E)sq~C4yTPOZHNn~$q=meZoosD(>J#rb!^oI-( z`5AcWX+eJ!UkvfG~c$ zg@M$+`!xV)g{jAYPaQu4KBo?+FFYGGcK TriggerType.LATE: + raise Exception("Incorrect Trigger Type") + return True + +class AccumulationMode: + DISCARDING = 1 + ACCUMULATING = 2 + ACCUMULATING_AND_RETRACTING = 3 + @staticmethod + def isValid(type): + if type < TriggerType.DISCARDING: + raise Exception("Incorrect Accumulation Mode") + if type > TriggerType.ACCUMULATING_AND_RETRACTING: + raise Exception("Incorrect Accumulation Mode") + return True + +class TriggerOption(object): + triggers = [] + accumulation_mode = None + firingOnlyUpdatedPanes = False + @staticmethod + def at_watermark(): + triggerOption = TriggerOption() + trigger = Trigger(TriggerType.ON_TIME) + triggerOption.triggers.append(trigger) + return triggerOption + + def withEarlyFiringsAtEvery(self,*args,**kwargs): + trigger = None + if 'count' in kwargs: + trigger = CountTrigger(TriggerType.EARLY, kwargs['count']) + if 'duration' in kwargs: + trigger = TimeTrigger(TriggerType.EARLY, kwargs['duration']) + if trigger is None: + raise Exception("Unsufficent for trigger") + self.triggers.append(trigger) + return self + + def withLateFiringsAtEvery( self, *args, **kwargs ): + trigger = None + if 'count' in kwargs: + trigger = CountTrigger(TriggerType.LATE, kwargs['count']) + if 'duration' in kwargs: + trigger = TimeTrigger(TriggerType.LATE, kwargs['duration']) + if trigger is None: + raise Exception("Unsufficent for trigger") + self.triggers.append(trigger) + return self + + def discardingFiredPanes(self): + self.accumulation_mode = AccumulationMode.DISCARDING + return self + + def accumulatingFiredPanes(self): + self.accumulation_mode = AccumulationMode.ACCUMULATING + return self + + def accumulatingAndRetractingFiredPanes(self): + self.accumulation_mode = AccumulationMode.ACCUMULATING_AND_RETRACTING + return self + + def firingOnlyUpdatedPanes(self): + self.firingOnlyUpdatedPanes = True + return self + + @staticmethod + def get_java_trigger_options(trigger_option, gateway): + _jtrigger_option = None + for trigger in trigger_option.triggers: + if trigger.trigger_type == TriggerType.ON_TIME: + _jtrigger_option = gateway.jvm.TriggerOption.AtWatermark() + elif trigger.trigger_type == TriggerType.EARLY: + if isinstance(trigger, TimeTrigger): + _jduration = gateway.jvm.Duration(trigger.duration) + _jtrigger_option = _jtrigger_option.withEarlyFiringsAtEvery(_jduration) + else: + _jcount = gateway.jvm.Duration(trigger.count) + _jtrigger_option = _jtrigger_option.withEarlyFiringsAtEvery(_jcount) + elif trigger.trigger_type == TriggerType.LATE: + if isinstance(trigger, TimeTrigger): + _jduration = gateway.jvm.Duration(trigger.duration) + _jtrigger_option = _jtrigger_option.withLateFiringsAtEvery(_jduration) + else: + _jcount = gateway.jvm.Duration(trigger.count) + _jtrigger_option = _jtrigger_option.withLateFiringsAtEvery(_jcount) + return _jtrigger_option + +class Trigger(object): + trigger_type = None + + def __init__(self,trigger_type): + self.trigger_type = trigger_type + +class TimeTrigger(Trigger): + duration = None + + def __init__(self, trigger_type, duration): + super(TimeTrigger,self).__init__(trigger_type) + self.duration = duration + +class CountTrigger(Trigger): + count = None + + def __init__(self, trigger_type, count): + super(CountTrigger, self).__init__(trigger_type) + self.count = count diff --git a/python/apex-python/src/pyapex/runtime/__init__.py b/python/apex-python/src/pyapex/runtime/__init__.py new file mode 100644 index 0000000000..d321c94c9b --- /dev/null +++ b/python/apex-python/src/pyapex/runtime/__init__.py @@ -0,0 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# \ No newline at end of file diff --git a/python/apex-python/src/pyapex/runtime/worker.py b/python/apex-python/src/pyapex/runtime/worker.py new file mode 100755 index 0000000000..72f3788250 --- /dev/null +++ b/python/apex-python/src/pyapex/runtime/worker.py @@ -0,0 +1,79 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +''' +Worker.py file is responsible for instantiating specific workers such as MapWorkerImpl, FlatMapWorkerImpl, FilterWorkerImpl. + +Worker.py is ran using python and then we register back WorkerImpl with Java Process for each calls. +''' +import sys +import site +from py4j.java_gateway import JavaGateway, CallbackServerParameters, GatewayParameters, java_import + +# TODO this may cause race condition +def find_free_port(): + import socket + s = socket.socket() + s.listen(0) + addr, found_port = s.getsockname() # Return the port number assigned. + s.shutdown(socket.SHUT_RDWR) + s.close() + return found_port + + +def main(argv): + import os, getpass + + PYTHON_PATH = os.environ['PYTHONPATH'] if 'PYTHONPATH' in os.environ else None + os.environ['PYTHONPATH'] = PYTHON_PATH + ':' + site.getusersitepackages().replace('/home/.local/', + '/home/' + getpass.getuser() + '/.local/') + '/' + sys.path.extend(os.environ['PYTHONPATH'].split(':')) + print "PYTHONPATH " + str(os.environ['PYTHONPATH']) + gateway_params = GatewayParameters(address='127.0.0.1', port=int(argv[0]), auto_convert=True) + callback_params = CallbackServerParameters(daemonize=False, eager_load=True, port=0) + gateway = JavaGateway(gateway_parameters=gateway_params, callback_server_parameters=callback_params) + + # Retrieve the port on which the python callback server was bound to. + python_port = gateway.get_callback_server().get_listening_port() + + # Register python callback server with java gateway server + # Note that we use the java_gateway_server attribute that + # retrieves the GatewayServer instance. + gateway.java_gateway_server.resetCallbackClient( + gateway.java_gateway_server.getCallbackClient().getAddress(), + python_port) + + # Instantiate WorkerImpl for PythonWorker java interface and regsiter with PythonWorkerProxy in Java. + from pyapex.functions import WorkerImpl + print "Registering Python Worker " + workerImpl = WorkerImpl.factory(gateway, argv[1]) + if argv[1] in ['REDUCE','REDUCE_BY_KEY']: + serialized_object = gateway.entry_point.getSerializedData() + import dill + workerImpl=dill.loads(serialized_object) + print type(workerImpl) + gateway.entry_point.register(workerImpl) + else: + gateway.entry_point.register(workerImpl) + + print "Python process started with type: " + argv[1] + + +if __name__ == "__main__": + main(sys.argv[1:]) diff --git a/python/apex-python/src/pyapex/shellconn.py b/python/apex-python/src/pyapex/shellconn.py new file mode 100644 index 0000000000..47dfa61513 --- /dev/null +++ b/python/apex-python/src/pyapex/shellconn.py @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +from py4j.java_gateway import JavaGateway,java_import + + +class ShellConnector(object): + gateway = None + entry_point = None + + def __init__(self): + self.gateway = JavaGateway() + java_import(self.gateway.jvm, 'org.joda.time.*') + java_import(self.gateway.jvm, 'org.apache.apex.malhar.lib.window.*') + + def __new__(cls): + if not hasattr(cls, 'instance'): + cls.instance = super(ShellConnector, cls).__new__(cls) + return cls.instance + + def get_jvm_gateway(self): + return self.gateway + + def get_entry_point(self): + return self.gateway.entry_point diff --git a/python/create_zip.sh b/python/create_zip.sh new file mode 100755 index 0000000000..517b329072 --- /dev/null +++ b/python/create_zip.sh @@ -0,0 +1,26 @@ +#!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +VERSION=$1 +echo $1 +cd apex-python/src +zip -r pyapex-$VERSION-src.zip pyapex +mv pyapex-$VERSION-src.zip ../deps +cd ../../ diff --git a/python/pom.xml b/python/pom.xml new file mode 100644 index 0000000000..eb24625044 --- /dev/null +++ b/python/pom.xml @@ -0,0 +1,333 @@ + + + 4.0.0 + + + org.apache.apex + malhar + 3.8.0-SNAPSHOT + + + malhar-python + Apache Apex Malhar Python Support + jar + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + test-jar + + package + + + + + maven-assembly-plugin + + + + org.apache.apex.malhar.python.PyApex + + + + jar-with-dependencies + + + + + + maven-dependency-plugin + + + create-client-mvn-generated-classpath + generate-resources + + build-classpath + + + + ${project.build.directory}/mvn-generated-runtime-classpath + + + + + + create-client-mvn-generated-classpath-no-hadoop + generate-resources + + build-classpath + + + + ${project.build.directory}/mvn-generated-runtime-classpath-no-hadoop + + org.apache.hadoop + + + + create-mvn-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mvn-generated-classpath + + + + + + maven-dependency-plugin + 2.8 + + + copy-dependencies-for-launch-time + prepare-package + + copy-dependencies + + + ${project.build.directory}/libs + + + + + + maven-dependency-plugin + 2.8 + + + copy-dependencies-for-run-time + prepare-package + + copy-dependencies + + + ${project.build.directory}/runtime_libs + provided + org.apache.hadoop + + + + + + + + + + + + + + + + + + + + + + + + + + + + exec-maven-plugin + org.codehaus.mojo + + + Python Zip Build + generate-sources + + exec + + + create_zip.sh + + 0.0.4 + + + + + + + + + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + provided + + + joda-time + joda-time + 2.3 + provided + + + org.apache.apex + apex-engine + ${apex.core.version} + + + org.apache.apex + apex-api + ${apex.core.version} + + + org.apache.apex + apex-common + ${apex.core.version} + + + + net.sf.py4j + py4j + 0.10.4 + + + + org.apache.apex + malhar-stream + ${project.version} + provided + + + org.apache.apex + malhar-kafka + ${project.version} + provided + + + + org.apache.kafka + kafka-clients + 0.9.0.1 + provided + + + org.apache.kafka + kafka_2.10 + 0.9.0.1 + true + + + + + org.apache.apex + malhar-library + ${project.version} + provided + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-mockito + ${powermock.version} + test + + + + diff --git a/python/scripts/log4j.properties b/python/scripts/log4j.properties new file mode 100644 index 0000000000..a57fbc06e6 --- /dev/null +++ b/python/scripts/log4j.properties @@ -0,0 +1,43 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +log4j.rootLogger=TRACE,CONSOLE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.CONSOLE.threshold=${test.log.console.threshold} +test.log.console.threshold=TRACE + +log4j.appender.RFA=org.apache.log4j.RollingFileAppender +log4j.appender.RFA.layout=org.apache.log4j.PatternLayout +log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.RFA.File=/tmp/app.log + +# to enable, add SYSLOG to rootLogger +log4j.appender.SYSLOG=org.apache.log4j.net.SyslogAppender +log4j.appender.SYSLOG.syslogHost=127.0.0.1 +log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout +log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n +log4j.appender.SYSLOG.Facility=LOCAL1 + +log4j.logger.org=trace +#log4j.logger.org.apache.commons.beanutils=warn +log4j.logger.com.datatorrent=trace +log4j.logger.org.apache.apex=trace diff --git a/python/scripts/pyshell b/python/scripts/pyshell new file mode 100755 index 0000000000..1e49d6c8ab --- /dev/null +++ b/python/scripts/pyshell @@ -0,0 +1,212 @@ +#!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + + +# Support functions + +# TODO : +# ACCEPTED ARGUMENTS: +# shell +# launch input_python.py +# kill app-id +# shutdown app-id + +usage() { echo "Usage: $0 [-sh] [-l ] [-m HADOOP|LOCAL ] [-k ] [-t ]" 1>&2; exit 1; } +COMMAND_MODE='SHELL_MODE' +RUNNING_MODE='HADOOP' +EXECUTABLE_FILE=false +while getopts ":s:l:k:t:m:" o; do + case "${o}" in + s) + s=${OPTARG} + COMMAND_MODE="SHELL_MODE" + ;; + l) + EXECUTABLE_FILE=${OPTARG} + COMMAND_MODE="LAUNCH_MODE" + ;; + k) + APP_ID=${OPTARG} + COMMAND_MODE="KILL_MODE" + ;; + m) + RUNNING_MODE=${OPTARG} + ;; + t) + APP_ID=${OPTARG} + COMMAND_MODE="SHUTDOWN_MODE" + ;; + *) + usage + ;; + esac +done +echoerr() { echo "$@" 1>&2; } +if [ -e "pyapex" ] +then + rm pyapex +fi +if [ -z $APEX_HOME ] +then + echo "APEX_HOME is not set correctly; Please set it to apex-core path" + exit +fi +real_dir() { + SOURCE="${1:-${BASH_SOURCE[0]}}" + while [ -h "$SOURCE" ]; do # resolve $SOURCE until the file is no longer a symlink + SOURCE_DIR="$( cd -P "$( dirname "$SOURCE" )" && pwd )" + SOURCE="$(readlink "$SOURCE")" + [[ $SOURCE != /* ]] && SOURCE="$SOURCE_DIR/$SOURCE" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located + done + SOURCE_DIR="$( cd -P "$( dirname "$SOURCE" )" && pwd )" + echo $SOURCE_DIR +} +script_dir=$(real_dir "${BASH_SOURCE[0]}") +# Create missing clirc file for current user +if [ ! -f "${HOME}/.dt/clirc" ]; then + mkdir -p "${HOME}/.dt" + cat >${HOME}/.dt/clirc </dev/null` +fi + +if [ "$DT_CLIENT_OPTS" = "" ]; then +# DT_CLIENT_OPTS="-Xmx1024m -XX:MaxPermSize=512m -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled" + DT_CLIENT_OPTS="-Xmx1024m -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled" +fi + +export HADOOP_CLIENT_OPTS="$DT_CLIENT_OPTS" + +PYTHON_BUILD="$( dirname "$0" )/../dist" +BUILD_DIR="$( dirname "$0" )/../target" +PYAPEX_HOME="`pwd`/../apex-python" + +if [[ (-z "$DT_HADOOP" ) || ( "$RUNNING_MODE" == "LOCAL" ) ]]; then + echo "Development Mode without Hadoop Installation: Using jars from mvn generated path" + MVN_GENERATED_PATH="$BUILD_DIR/mvn-generated-runtime-classpath" +else + echo "Development Mode with Hadoop Installation: Using jars to be provided externally " + MVN_GENERATED_PATH="$BUILD_DIR/mvn-generated-runtime-classpath-no-hadoop" +fi +if [ -f "$MVN_GENERATED_PATH" ]; then + # development launch mode + DT_CORE_JAR="$BUILD_DIR/malhar-python-3.8.0-SNAPSHOT.jar" + if [ ! -f "$DT_CORE_JAR" ]; then + echoerr "Error: Cannot find $DT_CORE_JAR"; + exit 1; + fi + DT_CLASSPATH="$DT_CLASSPATH:$DT_CORE_JAR" + DT_CLASSPATH=$BASEDIR/libs'/*'":${DT_CLASSPATH}" + DT_CLASSPATH="$DT_CLASSPATH:`cat $MVN_GENERATED_PATH`" +else + # running from installation + if [ -z "$DT_HADOOP" ]; then + echoerr "Hadoop installation not found. Please include hadoop in PATH." + exit 1; + fi + BASEDIR=$( cd ${script_dir}/..; pwd -P ) +fi + +if [ -n "$DT_CLASSPATH" ]; then + if [ -z "$HADOOP_CLASSPATH" ]; then + export HADOOP_CLASSPATH="$DT_CLASSPATH" + else + export HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$DT_CLASSPATH" + fi +fi +APEXRUNERPID=0 +APEXRUNNERLOG="$( dirname "$0")/ApexRunner.out" +rm -f $APEXRUNNERLOG +export PYAPEX_HOME=$PYAPEX_HOME +cp $BUILD_DIR/runtime_libs/* $PYAPEX_HOME/deps +cp $DT_CORE_JAR $PYAPEX_HOME/deps +echo "PYAPEX_HOME $PYAPEX_HOME" + +echo "Intiating PYSHELL now " + +#ln -sf $PYAPEX_HOME "$script_dir/pyapex" +if [ -z "$PYTHONPATH" ] +then + export PYTHONPATH="$PYAPEX_HOME/deps/pyapex-0.0.4-src.zip" + +else + export PYTHONPATH="$PYTHONPATH:$PYAPEX_HOME/deps/pyapex-0.0.4-src.zip" +fi +if [ "$COMMAND_MODE" = "KILL_MODE" ] +then + echo "Killing APP $APP_ID" + echo "kill-app $APP_ID"| $APEX_HOME/engine/src/main/scripts/apex + exit 0 +elif [ "$COMMAND_MODE" = "SHUTDOWN_MODE" ] +then + echo "Shutting down APP $APP_ID" + echo "shutdown-app $APP_ID"| $APEX_HOME/engine/src/main/scripts/apex + exit 0 +fi + +echo "Launching python shell now" +if [[ ( -z "$DT_HADOOP" ) || ( "$RUNNING_MODE" == "LOCAL" ) ]]; then + echo "Warning: hadoop executable not found. Running standalone with ${DT_JAVA:-java}." + echoerr "Warning: hadoop executable not found. Running standalone with ${DT_JAVA:-java}." + echo "Starting Apex Runner without hadoop" + export CLASSPATH=$DT_CLASSPATH + "${DT_JAVA:-java}" $DT_CLIENT_OPTS org.apache.apex.malhar.python.PyApex "$@" >$APEXRUNNERLOG 2>&1 & + APEXRUNNERPID="$!" + echo $APEXRUNNERPID +else + echo "Warning: hadoop found. Running with $DT_HADOOP" + export HADOOP_USER_CLASSPATH_FIRST=1 + # remove hadoop and duplicate slf4j binding (bash replace is too slow) + export HADOOP_CLASSPATH=$(echo -n "$HADOOP_CLASSPATH" | tr ":" "\n" | sed "/slf4j-log4j/d" | sed "/org\/apache\/hadoop/d" | tr "\n" ":") + echo "Starting Apex Runner with hadoop $@" + echo $DT_CLASSPATH + "$DT_HADOOP" org.apache.apex.malhar.python.PyApex "$@" >$APEXRUNNERLOG 2>&1 & + APEXRUNNERPID="$!" +fi +echo "Apex Runner is started as process id: " $APEXRUNNERPID + +if [ "$COMMAND_MODE" = "SHELL_MODE" ] +then + PYTHONPATH=$PYTHONPATH:"python" ipython "$@" +elif [ "$COMMAND_MODE" = "LAUNCH_MODE" ] +then + PYTHONPATH=$PYTHONPATH:"python" ipython "$EXECUTABLE_FILE" +fi +if [[ "$RUNNING_MODE" == "LOCAL" ]]; then + sleep 60 +fi +if ! kill $APEXRUNNERPID > /dev/null 2>&1; then + echo "Could not send SIGTERM to process $PID. Force killing" >/dev/null >&2 + kill -9 $APEXRUNNERPID >/dev/null 2>&1 +fi +if [ -e pyapex ] +then + rm pyapex +fi diff --git a/python/src/main/java/org/apache/apex/malhar/PythonConstants.java b/python/src/main/java/org/apache/apex/malhar/PythonConstants.java new file mode 100644 index 0000000000..8aad2af8ec --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/PythonConstants.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar; + +public class PythonConstants +{ + + public enum OpType + { + MAP("MAP"), + FLAT_MAP("FLAT_MAP"), + FILTER("FILTER"), + REDUCE("REDUCE"), + REDUCE_BY_KEY("REDUCE_BY_KEY"); + + private String operationName = null; + + OpType(String name) + { + this.operationName = name; + } + + public String getType() + { + return operationName; + } + } + + public static String PY4J_SRC_ZIP_FILE_NAME = "py4j-0.10.4-src.zip"; + public static String PYTHON_WORKER_FILE_NAME = "worker.py"; + public static String PYTHON_APEX_ZIP_NAME = "pyapex-0.0.4-src.zip"; + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/PyApex.java b/python/src/main/java/org/apache/apex/malhar/python/PyApex.java new file mode 100644 index 0000000000..4bc920001e --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/PyApex.java @@ -0,0 +1,120 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python; + +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.client.api.YarnClient; + +import py4j.GatewayServer; +import py4j.Py4JException; + +public class PyApex +{ + + private PythonApp streamApp = null; + private static final Logger LOG = LoggerFactory.getLogger(PyApex.class); + private PythonAppManager.LaunchMode mode = null; + + public PyApex(PythonAppManager.LaunchMode mode) + { + this.mode = mode; + } + + public PythonApp createApp(String name) + { + if (streamApp == null) { + streamApp = new PythonApp(name); + if (this.mode != null) { + streamApp.setMode(this.mode); + } + } + return streamApp; + } + + public PythonApp getAppByName(String name) + { + if (streamApp == null) { + try { + + YarnClient client = YarnClient.createYarnClient(); + List apps = client.getApplications(); + for (ApplicationReport appReport : apps) { + if (appReport.getName().equals(name)) { + LOG.debug("Application Name: {} Application ID: {} Application State: {}", appReport.getName(), appReport.getApplicationId().toString(), appReport.getYarnApplicationState()); + return new PythonApp(name, appReport.getApplicationId()); + } + } + } catch (Exception e) { + throw new Py4JException("Error getting application list from resource manager", e); + } + streamApp = new PythonApp(name); + } + return streamApp; + } + + public static void main(String[] args) + { + + LOG.info("Starting PYAPEX with {}", StringUtils.join(args, ' ')); + Options options = new Options(); + + Option input = new Option("m", "mode", true, "Launch Mode"); + input.setRequired(false); + options.addOption(input); + + Option pyfile = new Option("l", "launch-file", true, "Launch file"); + pyfile.setRequired(false); + options.addOption(pyfile); + + CommandLineParser parser = new BasicParser(); + HelpFormatter formatter = new HelpFormatter(); + CommandLine cmd; + + try { + cmd = parser.parse(options, args); + } catch (ParseException e) { + LOG.error("Parsing Exception while parsing arguments",e); + formatter.printHelp("utility-name", options); + System.exit(1); + return; + } + + String launchModeValue = cmd.getOptionValue("mode"); + PythonAppManager.LaunchMode mode = launchModeValue != null ? PythonAppManager.LaunchMode.valueOf(launchModeValue) : null; + LOG.info("Starting PYAPEX with {}", mode); + PyApex pythonEntryPoint = new PyApex(mode); + GatewayServer gatewayServer = new GatewayServer(pythonEntryPoint); + gatewayServer.start(); + LOG.debug("Gateway Server Started"); + } + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/PythonApp.java b/python/src/main/java/org/apache/apex/malhar/python/PythonApp.java new file mode 100644 index 0000000000..0d442baef8 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/PythonApp.java @@ -0,0 +1,428 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python; + +import java.io.File; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.kafka.PartitionStrategy; +import org.apache.apex.malhar.lib.fs.GenericFileOutputOperator; +import org.apache.apex.malhar.lib.function.Function; +import org.apache.apex.malhar.lib.window.TriggerOption; +import org.apache.apex.malhar.lib.window.Tuple; +import org.apache.apex.malhar.lib.window.WindowOption; + +import org.apache.apex.malhar.python.operator.PythonGenericOperator; +import org.apache.apex.malhar.python.operator.proxy.PythonReduceProxy; +import org.apache.apex.malhar.python.runtime.PythonApexStreamImpl; +import org.apache.apex.malhar.python.runtime.PythonWorkerContext; +import org.apache.apex.malhar.stream.api.ApexStream; +import org.apache.apex.malhar.stream.api.Option; +import org.apache.apex.malhar.stream.api.PythonApexStream; +import org.apache.apex.malhar.stream.api.impl.ApexStreamImpl; +import org.apache.apex.malhar.stream.api.impl.StreamFactory; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; +import com.datatorrent.api.StreamingApplication; +import com.datatorrent.contrib.kafka.KafkaSinglePortOutputOperator; +import com.datatorrent.lib.util.KeyValPair; +import com.datatorrent.stram.client.StramAppLauncher; + +public class PythonApp implements StreamingApplication +{ + + private PythonApexStream apexStream = null; + private StreamFactory streamFactory; + private ApplicationId appId = null; + private static final Logger LOG = LoggerFactory.getLogger(PythonApp.class); + + private PythonAppManager manager = null; + private String name; + private Configuration conf; + + private String apexDirectoryPath = null; + private PythonAppManager.LaunchMode mode = PythonAppManager.LaunchMode.HADOOP; + + public PythonApp() + { + this(null, null); + + } + + public PythonApp(String name) + { + + this(name, null); + } + + public PythonApp(String name, ApplicationId appId) + { + this.appId = appId; + this.name = name; + this.conf = new Configuration(true); + this.apexDirectoryPath = System.getenv("PYAPEX_HOME"); + this.conf.set("dt.loggers.level", "com.datatorrent.*:INFO,org.apache.*:DEBUG"); + } + + public String getApexDirectoryPath() + { + return apexDirectoryPath; + } + + public String getName() + { + return name; + } + + public void populateDAG(DAG dag, Configuration conf) + { + + LOG.trace("Populating DAG in python app"); + this.apexStream.populateDag(dag); + + } + + public void setRequiredJARFiles() + { + + LOG.debug("PYAPEX_HOME: {}" + getApexDirectoryPath()); + File dir = new File(this.getApexDirectoryPath() + "/deps/"); + File[] files = dir.listFiles(); + ArrayList jarFiles = new ArrayList(); + for (File jarFile : files) { + LOG.info("FOUND FILES {}" + jarFile.getAbsolutePath()); + jarFiles.add(jarFile.getAbsolutePath()); + + } + jarFiles.add(this.getApexDirectoryPath() + "/deps/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + jarFiles.add(this.getApexDirectoryPath() + "/src/pyapex/runtime/" + PythonConstants.PYTHON_WORKER_FILE_NAME); + jarFiles.add(this.getApexDirectoryPath() + "/deps/" + PythonConstants.PYTHON_APEX_ZIP_NAME); + + extendExistingConfig(StramAppLauncher.LIBJARS_CONF_KEY_NAME, jarFiles); +// this.getClassPaths(); + } + + public List getClassPaths() + { + LOG.info("PROCESSING CLASSPATH"); + List paths = new ArrayList<>(); + ClassLoader cl = ClassLoader.getSystemClassLoader(); + URL[] urls = ((URLClassLoader)cl).getURLs(); + for (URL url : urls) { + LOG.info("FOUND FILE PATH {}" + url.getFile()); + paths.add(url.getFile()); + } + return paths; + } + + public void setRequiredRuntimeFiles() + { + + ArrayList files = new ArrayList(); + files.add(this.getApexDirectoryPath() + "/deps/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + files.add(this.getApexDirectoryPath() + "/src/pyapex/runtime/" + PythonConstants.PYTHON_WORKER_FILE_NAME); + files.add(this.getApexDirectoryPath() + "/deps/" + PythonConstants.PYTHON_APEX_ZIP_NAME); + + extendExistingConfig(StramAppLauncher.FILES_CONF_KEY_NAME, files); + + } + + public void extendExistingConfig(String fileVariable, ArrayList fileList) + { + Configuration configuration = this.getConf(); + String fileCSV = configuration.get(fileVariable); + String filesCSVToAppend = StringUtils.join(fileList, ","); + + if (StringUtils.isEmpty(fileCSV)) { + fileCSV = filesCSVToAppend; + + } else { + fileCSV = fileCSV + "," + filesCSVToAppend; + } + + configuration.set(fileVariable, fileCSV); + } + + public Configuration getConf() + { + return conf; + } + + public void setConf(Configuration conf) + { + this.conf = conf; + } + + public StreamFactory getStreamFactory() + { + if (streamFactory == null) { + streamFactory = new StreamFactory(); + } + return streamFactory; + } + + public String launch(boolean local) throws Exception + { + + LOG.debug("Already set Launching mode option : {} Local {} ", mode, local); + + if (local) { + mode = PythonAppManager.LaunchMode.LOCAL; + } + + mode = mode != PythonAppManager.LaunchMode.LOCAL ? PythonAppManager.LaunchMode.HADOOP : mode; + + LOG.debug("Launching mode: {} ApexDirectoryPath: {}", mode, this.getApexDirectoryPath()); + this.setRequiredJARFiles(); + this.setRequiredRuntimeFiles(); + this.manager = new PythonAppManager(this, mode); + + DAG dag = this.apexStream.createDag(); + + Map pythonOperatorEnv = new HashMap<>(); + if (mode == PythonAppManager.LaunchMode.LOCAL) { + + pythonOperatorEnv.put(PythonWorkerContext.PYTHON_WORKER_PATH, this.getApexDirectoryPath() + "/src/pyapex/runtime/" + PythonConstants.PYTHON_WORKER_FILE_NAME); + pythonOperatorEnv.put(PythonWorkerContext.PY4J_DEPENDENCY_PATH, this.getApexDirectoryPath() + "/deps/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + pythonOperatorEnv.put(PythonWorkerContext.PYTHON_APEX_PATH, this.getApexDirectoryPath() + "/deps/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + + } + + Collection operators = dag.getAllOperatorsMeta(); + for (DAG.OperatorMeta operatorMeta : operators) { + if (operatorMeta.getOperator() instanceof PythonGenericOperator) { + LOG.debug("Updating python operator: {}" + operatorMeta.getName()); + PythonGenericOperator operator = ((PythonGenericOperator)operatorMeta.getOperator()); + operator.getServer().setPythonOperatorEnv(pythonOperatorEnv); + } + } + return manager.launch(); + } + + public LocalMode.Controller runLocal() + { + return this.apexStream.runEmbedded(true, 0, null); + } + + public ApexStream getApexStream() + { + return apexStream; + } + + public void setApexStream(PythonApexStreamImpl apexStream) + { + this.apexStream = apexStream; + } + + public PythonApp fromFolder(String directoryPath) + { + ApexStream currentStream = StreamFactory.fromFolder(directoryPath); + if (currentStream instanceof ApexStreamImpl) { + apexStream = new PythonApexStreamImpl((ApexStreamImpl)currentStream); + } + return this; + } + + public PythonApp fromKafka08(String zookeepers, String topic) + { + ApexStream currentStream = StreamFactory.fromKafka08(zookeepers, topic); + if (currentStream instanceof ApexStreamImpl) { + apexStream = new PythonApexStreamImpl((ApexStreamImpl)currentStream); + } + return this; + } + + public PythonApp fromData(List inputs) + { + + ApexStream currentStream = StreamFactory.fromData(inputs); + if (currentStream instanceof ApexStreamImpl) { + apexStream = new PythonApexStreamImpl((ApexStreamImpl)currentStream); + } + return this; + } + + public PythonApp fromKafka09(String brokers, String topic) + { + ApexStream currentStream = StreamFactory.fromKafka09(brokers, topic, PartitionStrategy.ONE_TO_ONE,1); + + if (currentStream instanceof ApexStreamImpl) { + apexStream = new PythonApexStreamImpl((ApexStreamImpl)currentStream); + } + return this; + } + + public PythonApp map(String name, byte[] searializedFunction) + { + apexStream = (PythonApexStream)apexStream.map(searializedFunction, Option.Options.name(name)); + + return this; + } + + public PythonApp flatMap(String name, byte[] searializedFunction) + { + apexStream = (PythonApexStream)apexStream.flatMap(searializedFunction, Option.Options.name(name)); + return this; + } + + public PythonApp filter(String name, byte[] searializedFunction) + { + apexStream = (PythonApexStream)apexStream.filter(searializedFunction, Option.Options.name(name)); + return this; + } + + public PythonApp window(WindowOption windowOption, TriggerOption triggerOption, Duration allowedLateness) + { + + apexStream = (PythonApexStream)apexStream.window(windowOption, triggerOption, allowedLateness); + return this; + } + + public PythonApp countByKey(String name) + { + + Function.ToKeyValue toKeyValueFunction = new Function.ToKeyValue() + { + @Override + public Tuple> f(String input) + { + String[] data = input.split(","); + + return new Tuple.PlainTuple>(new KeyValPair(data[data.length - 1], 1L)); + } + }; + if (apexStream instanceof PythonApexStreamImpl) { + + apexStream = (PythonApexStream)((PythonApexStreamImpl)apexStream).countByKey(toKeyValueFunction, Option.Options.name(name)); + } + return this; + } + + public PythonApp count(String name) + { + if (apexStream instanceof PythonApexStreamImpl) { + apexStream = (PythonApexStream)((PythonApexStreamImpl)apexStream).count(); + } + return this; + } + + public PythonApp reduce(String name, byte[] serializedObject) + { + + if (apexStream instanceof PythonApexStreamImpl) { + PythonReduceProxy reduceProxy = new PythonReduceProxy(PythonConstants.OpType.REDUCE, serializedObject, String.class); + + apexStream = (PythonApexStream)((PythonApexStreamImpl)apexStream).reduce(reduceProxy, Option.Options.name(name)); + } + return this; + } + + public PythonApp reduceByKey(String name, byte[] serializedObject) + { + Function.ToKeyValue toKeyValueFunction = new Function.ToKeyValue() + { + @Override + public Tuple> f(String input) + { + String[] data = input.split(","); + return new Tuple.PlainTuple>(new KeyValPair(data[0], input)); + } + }; + + if (apexStream instanceof PythonApexStreamImpl) { + PythonReduceProxy reduceProxy = new PythonReduceProxy(PythonConstants.OpType.REDUCE_BY_KEY, serializedObject, String.class); + + apexStream = (PythonApexStream)((PythonApexStreamImpl)apexStream).reduceByKey(reduceProxy, toKeyValueFunction, Option.Options.name(name)); + } + return this; + } + + public PythonApp toConsole(String name) + { + apexStream = (PythonApexStream)apexStream.print(Option.Options.name(name)); + return this; + } + + public PythonApp toKafka08(String name, String topic, Map properties) + { + KafkaSinglePortOutputOperator kafkaOutputOperator = new KafkaSinglePortOutputOperator(); + kafkaOutputOperator.setTopic(topic); + List propertyList = new ArrayList(); + for (String key : properties.keySet()) { + propertyList.add(key + "=" + properties.get(key)); + } + + String producerConfigs = StringUtils.join(propertyList, ","); + LOG.debug("PropertyList for kafka producer {}" + producerConfigs); + kafkaOutputOperator.setProducerProperties(producerConfigs); + apexStream = (PythonApexStream)apexStream.endWith(kafkaOutputOperator, kafkaOutputOperator.inputPort, Option.Options.name(name)); + return this; + } + + public PythonApp toFolder(String name, String fileName, String directoryName) + { + + GenericFileOutputOperator outputOperator = new GenericFileOutputOperator<>(); + outputOperator.setFilePath(directoryName); + outputOperator.setOutputFileName(fileName); + outputOperator.setConverter(new GenericFileOutputOperator.StringToBytesConverter()); + apexStream = (PythonApexStream)apexStream.endWith(outputOperator, outputOperator.input, Option.Options.name(name)); + return this; + } + + public PythonApp setConfig(String key, String value) + { + getConf().set(key, value); + return this; + } + + public void kill() throws Exception + { + if (manager == null) { + throw new Exception("Application is not running yet"); + + } + manager.shutdown(); + } + + public PythonAppManager.LaunchMode getMode() + { + return mode; + } + + public void setMode(PythonAppManager.LaunchMode mode) + { + this.mode = mode; + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/PythonAppFactory.java b/python/src/main/java/org/apache/apex/malhar/python/PythonAppFactory.java new file mode 100644 index 0000000000..944ca2288f --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/PythonAppFactory.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python; + +import com.datatorrent.stram.client.StramAppLauncher; +import com.datatorrent.stram.plan.logical.LogicalPlan; +import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration; + + +public class PythonAppFactory implements StramAppLauncher.AppFactory +{ + private PythonApp streamingApp; + private String name; + + public PythonAppFactory(String name, PythonApp app) + { + this.name = name; + this.streamingApp = app; + } + + public LogicalPlan createApp(LogicalPlanConfiguration logicalPlanConfiguration) + { + LogicalPlan logicalPlan = new LogicalPlan(); + logicalPlanConfiguration.prepareDAG(logicalPlan, streamingApp, getName()); + return logicalPlan; + } + + public String getName() + { + return name; + } + + public String getDisplayName() + { + return name; + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/PythonAppManager.java b/python/src/main/java/org/apache/apex/malhar/python/PythonAppManager.java new file mode 100644 index 0000000000..9d731806ec --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/PythonAppManager.java @@ -0,0 +1,101 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.exceptions.YarnException; + +import com.datatorrent.api.LocalMode; +import com.datatorrent.stram.client.StramAppLauncher; + +public class PythonAppManager +{ + private LaunchMode mode; + private Object appIdentifier; + private PythonApp app = null; + private static final Logger LOG = LoggerFactory.getLogger(PythonApp.class); + + public enum LaunchMode + { + LOCAL, HADOOP; + } + + public PythonAppManager(PythonApp app, LaunchMode mode) + { + this.app = app; + this.mode = mode; + } + + public String launch() throws Exception + { + + LOG.info("Launching app in python app"); + + try { + if (mode == LaunchMode.LOCAL) { + appIdentifier = app.runLocal(); + return "LocalMode"; + } else { + StramAppLauncher appLauncher = null; + appLauncher = new StramAppLauncher(app.getName(), app.getConf()); + appLauncher.loadDependencies(); + PythonAppFactory appFactory = new PythonAppFactory(app.getName(), app); + this.appIdentifier = appLauncher.launchApp(appFactory); + return this.appIdentifier.toString(); + } + + } catch (Exception e) { + e.printStackTrace(); + LOG.error("Failed TO Launch PYTHON Streaming Application"); + LOG.error("Encountered Exception " + e.getMessage()); + throw e; + } + + } + + public void shutdown() + { + if (mode == LaunchMode.LOCAL) { + ((LocalMode.Controller)this.appIdentifier).shutdown(); + } else { + try { + + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(app.getConf()); + yarnClient.start(); + yarnClient.killApplication((ApplicationId)this.appIdentifier); + yarnClient.stop(); + } catch (YarnException e) { + e.printStackTrace(); + LOG.error("FAILED TO SHUTDOWN PYTHON STREAMING APPLICATION "); + LOG.error("Encountered Exception {}" + e.getMessage()); + } catch (IOException e) { + LOG.error("FAILED TO SHUTDOWN PYTHON STREAMING APPLICATION "); + LOG.error("Encountered Exception {} " + e.getMessage()); + } + + } + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/PythonGenericOperator.java b/python/src/main/java/org/apache/apex/malhar/python/operator/PythonGenericOperator.java new file mode 100644 index 0000000000..6d22083c8b --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/PythonGenericOperator.java @@ -0,0 +1,92 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.runtime.PythonServer; + +import com.datatorrent.api.Context.DAGContext; +import com.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.common.util.BaseOperator; + +public abstract class PythonGenericOperator extends BaseOperator +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonGenericOperator.class); + protected byte[] serializedFunction = null; + private PythonServer server = null; + protected transient PythonConstants.OpType operationType = null; + + public final transient DefaultInputPort in = new DefaultInputPort() + { + @Override + public void process(T tuple) + { + processTuple(tuple); + } + + }; + public final transient DefaultOutputPort out = new DefaultOutputPort(); + + public PythonGenericOperator() + { + this(null, null); + + } + + public PythonGenericOperator(PythonConstants.OpType operationType, byte[] serializedFunc) + { + this.serializedFunction = serializedFunc; + this.operationType = operationType; + this.server = new PythonServer(this.operationType, serializedFunc); + + } + + public void setup(OperatorContext context) + { + LOG.debug("Application path from Python Operator: {} ", (String)context.getValue(DAGContext.APPLICATION_PATH)); + // Setting up context path explicitly for handling local as well Hadoop Based Application Development + server.setup(); + + } + + public void teardown() + { + if (server != null) { + server.shutdown(); + } + } + + public PythonServer getServer() + { + return server; + } + + public void setServer(PythonServer server) + { + this.server = server; + } + + protected abstract void processTuple(T tuple); + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/PythonKeyedWindowedOperator.java b/python/src/main/java/org/apache/apex/malhar/python/operator/PythonKeyedWindowedOperator.java new file mode 100644 index 0000000000..17d70b6a3e --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/PythonKeyedWindowedOperator.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl; +import org.apache.apex.malhar.python.operator.proxy.PythonWorkerProxy; +import org.apache.apex.malhar.python.runtime.PythonServer; + +import com.datatorrent.api.Context; + +public class PythonKeyedWindowedOperator extends KeyedWindowedOperatorImpl +{ + + private static final Logger LOG = LoggerFactory.getLogger(PythonWindowedOperator.class); + private transient PythonServer server = null; + protected byte[] serializedFunction = null; + protected transient PythonConstants.OpType operationType = null; + + public PythonKeyedWindowedOperator() + { + this.serializedFunction = null; + } + + public PythonKeyedWindowedOperator(byte[] serializedFunc) + { + this.serializedFunction = serializedFunc; + this.server = new PythonServer(this.operationType, serializedFunc); + } + + public void setup(Context.OperatorContext context) + { + super.setup(context); + server.setOperationType(((PythonWorkerProxy)this.accumulation).getOperationType()); + server.setProxy((PythonWorkerProxy)this.accumulation); + server.setup(); + } + + public void teardown() + { + if (server != null) { + server.shutdown(); + } + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/PythonWindowedOperator.java b/python/src/main/java/org/apache/apex/malhar/python/operator/PythonWindowedOperator.java new file mode 100644 index 0000000000..970fd9f12b --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/PythonWindowedOperator.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl; +import org.apache.apex.malhar.python.operator.proxy.PythonWorkerProxy; +import org.apache.apex.malhar.python.runtime.PythonServer; + +import com.datatorrent.api.Context; + +public class PythonWindowedOperator extends WindowedOperatorImpl +{ + + private static final Logger LOG = LoggerFactory.getLogger(PythonWindowedOperator.class); + private PythonServer server = null; + protected byte[] serializedFunction = null; + protected transient PythonConstants.OpType operationType = null; + public PythonWindowedOperator() + { + this.serializedFunction = null; + } + + public PythonWindowedOperator(byte[] serializedFunc) + { + this.serializedFunction = serializedFunc; + this.server = new PythonServer(this.operationType, serializedFunc); + } + + public void setup(Context.OperatorContext context) + { + super.setup(context); + server.setOperationType(((PythonWorkerProxy)this.accumulation).getOperationType()); + server.setProxy((PythonWorkerProxy)this.accumulation); + server.setup(); + } + + public void teardown() + { + if (server != null) { + server.shutdown(); + } + } + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonAccumulatorWorker.java b/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonAccumulatorWorker.java new file mode 100644 index 0000000000..a2d22fdcd1 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonAccumulatorWorker.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.interfaces; + +public interface PythonAccumulatorWorker extends PythonWorker +{ + public T setObject(byte[] obj, String opType); + + public T defaultAccumulatedValue(); + + public T accumulate(T accumulatedValue, T input); + + public T merge(T accumulatedValue1, T accumulatedValue2); + + public T getOutput(T accumulatedValue); + + public T getRetraction(T output); + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonReduceWorker.java b/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonReduceWorker.java new file mode 100644 index 0000000000..02c9cb9ce1 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonReduceWorker.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.interfaces; + +import org.apache.apex.malhar.lib.window.accumulation.Reduce; + +public interface PythonReduceWorker extends PythonAccumulatorWorker, Reduce +{ +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonWorker.java b/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonWorker.java new file mode 100644 index 0000000000..42bc299128 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/interfaces/PythonWorker.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.interfaces; + +public interface PythonWorker +{ + public Object setFunction(byte[] func, String opType); + + public Object execute(T tuple); + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonAcummlationWorkerProxy.java b/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonAcummlationWorkerProxy.java new file mode 100644 index 0000000000..2a1f360938 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonAcummlationWorkerProxy.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.proxy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.lib.window.Accumulation; +import org.apache.apex.malhar.python.operator.interfaces.PythonAccumulatorWorker; + +import py4j.Py4JException; + +public class PythonAcummlationWorkerProxy extends PythonWorkerProxy implements Accumulation +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonAcummlationWorkerProxy.class); + + public PythonAcummlationWorkerProxy() + { + super(); + + } + + public PythonAcummlationWorkerProxy(byte[] serializedFunc) + { + super(serializedFunc); + this.serializedData = serializedFunc; + } + + @Override + public T defaultAccumulatedValue() + { + return null; + } + + @Override + public T accumulate(T accumulatedValue, T input) + { + if (getWorker() != null) { + + T result = null; + LOG.trace("Processing accumulation: {}", input); + try { + result = (T)((PythonAccumulatorWorker)getWorker()).accumulate(accumulatedValue, input); + LOG.trace("Processed accumulation: {}", result); + return result; + } catch (Py4JException ex) { + LOG.error("Exception encountered while executing operation for tuple: {} Message: {}", input, ex.getMessage()); + } finally { + return null; + } + } + return null; + + } + + @Override + public T merge(T accumulatedValue1, T accumulatedValue2) + { + if (getWorker() != null) { + + T result = null; + LOG.trace("Processing accumulation: {} {}", accumulatedValue1, accumulatedValue2); + try { + result = (T)((PythonAccumulatorWorker)getWorker()).merge(accumulatedValue1, accumulatedValue2); + LOG.trace("Processed accumulation: {}", result); + return result; + } catch (Py4JException ex) { + LOG.error("Exception encountered while executing operation for accumulation: {} {} Message: {}", accumulatedValue1, accumulatedValue2, ex.getMessage()); + } finally { + return null; + } + } + return null; + + } + + @Override + public T getOutput(T accumulatedValue) + { + return null; + } + + @Override + public T getRetraction(T value) + { + return null; + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonReduceProxy.java b/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonReduceProxy.java new file mode 100644 index 0000000000..cb3a61ff04 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonReduceProxy.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.proxy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.lib.window.accumulation.Reduce; +import org.apache.apex.malhar.python.operator.interfaces.PythonReduceWorker; + +public class PythonReduceProxy extends PythonAcummlationWorkerProxy implements Reduce +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonReduceProxy.class); + private Class clazz; + + public PythonReduceProxy() + { + super(); + } + + public PythonReduceProxy(PythonConstants.OpType operationType, byte[] serializedFunc, Class clazz) + { + super(serializedFunc); + this.operationType = operationType; + this.clazz = clazz; + } + + @Override + public T defaultAccumulatedValue() + { + T generatedDefaultValue = getInstance(); + LOG.debug("defaultAccumulatedValue received {} " + generatedDefaultValue); + return generatedDefaultValue; + } + + @Override + public T getOutput(T accumulatedValue) + { + LOG.debug("getOutput received {}", accumulatedValue); + if (accumulatedValue == null) { + accumulatedValue = getInstance(); + } + return accumulatedValue; + } + + @Override + public T getRetraction(T value) + { + LOG.debug("Retraction received {}", value); + return null; + } + + @Override + public T accumulate(T accumulatedValue, T input) + { + LOG.debug("Accumulate call received {}", input); + + if (accumulatedValue == null) { + return input; + } + return reduce(accumulatedValue, input); + } + + @Override + public T merge(T accumulatedValue1, T accumulatedValue2) + { + LOG.debug("Merge received {} {} ", accumulatedValue1, accumulatedValue1); + return reduce(accumulatedValue1, accumulatedValue2); + } + + @Override + public T reduce(T input1, T input2) + { + LOG.debug("Reduce Input received {} {} ", input1, input2); + + T result = (T)((PythonReduceWorker)getWorker()).reduce(input1, input2); + LOG.debug("Reduce Output generated {}", result); + return result; + } + + protected T getInstance() + { + try { + return clazz.newInstance(); + } catch (InstantiationException e) { + LOG.error("Failed to instantiate class {} " + clazz.getName()); + } catch (IllegalAccessException e) { + LOG.error("Failed to instantiate class {} " + clazz.getName()); + } + return null; + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonWorkerProxy.java b/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonWorkerProxy.java new file mode 100644 index 0000000000..744c36b7c4 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/proxy/PythonWorkerProxy.java @@ -0,0 +1,118 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.proxy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.operator.interfaces.PythonAccumulatorWorker; +import org.apache.apex.malhar.python.operator.interfaces.PythonWorker; + +import py4j.Py4JException; + +public class PythonWorkerProxy +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonWorkerProxy.class); + + protected PythonWorker worker = null; + protected PythonConstants.OpType operationType; + protected boolean functionEnabled = false; + protected byte[] serializedData = null; + protected boolean workerRegistered = false; + + public PythonWorkerProxy() + { + this.serializedData = null; + } + + public PythonWorkerProxy(byte[] serializedFunc) + { + this.serializedData = serializedFunc; + } + + public Object execute(T tuple) + { + if (worker != null) { + + Object result = null; + LOG.trace("Processing tuple: {}", tuple); + try { + result = worker.execute(tuple); + LOG.trace("Processed tuple: {}", result); + return result; + } catch (Py4JException ex) { + LOG.error("Exception encountered while executing operation for tuple: {} Message: {}", tuple, ex.getMessage()); + } + } + return null; + } + + public void register(PythonWorker pythonWorker) + { + if (pythonWorker == null) { + throw new RuntimeException("Null Python Worker"); + } + LOG.debug("Current proxy instance {}", this); + LOG.debug("Registering python worker now {} {}", (pythonWorker != null), (pythonWorker instanceof PythonAccumulatorWorker)); + this.worker = pythonWorker; + this.workerRegistered = true; + LOG.debug("Python worker registered"); + } + + public void setSerializedData(String opType) + { + if (this.isWorkerRegistered() && !isFunctionEnabled()) { + LOG.debug("Setting Serialized function"); + this.worker.setFunction(this.serializedData, opType); + this.functionEnabled = true; + LOG.debug("Set Serialized function"); + } + } + + public byte[] getSerializedData() + { + return serializedData; + } + + public PythonWorker getWorker() + { + return worker; + } + + public PythonConstants.OpType getOperationType() + { + return operationType; + } + + public void setOperationType(PythonConstants.OpType operationType) + { + this.operationType = operationType; + } + + public boolean isWorkerRegistered() + { + return this.workerRegistered; + } + + public boolean isFunctionEnabled() + { + return this.functionEnabled; + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFilterOperator.java b/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFilterOperator.java new file mode 100644 index 0000000000..e290c433bd --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFilterOperator.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.transform; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.operator.PythonGenericOperator; + +import com.datatorrent.api.DefaultOutputPort; + +public class PythonFilterOperator extends PythonGenericOperator +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonFilterOperator.class); + + DefaultOutputPort falsePort = new DefaultOutputPort<>(); + DefaultOutputPort truePort = new DefaultOutputPort(); + + public PythonFilterOperator() + { + this(null); + } + + public PythonFilterOperator(byte[] serializedFunc) + { + + super(PythonConstants.OpType.FILTER, serializedFunc); + } + + @Override + protected void processTuple(T tuple) + { + LOG.trace("Received Tuple: {}", tuple); + Object result = getServer().getProxy().execute(tuple); + if (result instanceof Boolean) { + Boolean b = (Boolean)result; + LOG.trace("Filter response received: {}", b); + if (b.booleanValue()) { + out.emit(tuple); + } + } + + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFlatMapOperator.java b/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFlatMapOperator.java new file mode 100644 index 0000000000..590d389669 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonFlatMapOperator.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.transform; + +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.operator.PythonGenericOperator; + +public class PythonFlatMapOperator extends PythonGenericOperator +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonFlatMapOperator.class); + + public PythonFlatMapOperator() + { + this(null); + } + + public PythonFlatMapOperator(byte[] serializedFunc) + { + + super(PythonConstants.OpType.FLAT_MAP, serializedFunc); + } + + @Override + protected void processTuple(T tuple) + { + LOG.trace("Received Tuple: {}" + tuple); + List result = (List)getServer().getProxy().execute(tuple); + if (result != null) { + LOG.trace("List response received: {}" + result); + if (result instanceof List) { + for (T item : result) { + out.emit(item); + } + } else { + LOG.warn("Returned response is not list: {}" + result); + } + } + } + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonMapOperator.java b/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonMapOperator.java new file mode 100644 index 0000000000..7436c4773a --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/operator/transform/PythonMapOperator.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.transform; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.operator.PythonGenericOperator; + +public class PythonMapOperator extends PythonGenericOperator +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonMapOperator.class); + + + public PythonMapOperator() + { + + this(null); + } + + public PythonMapOperator(byte[] serializedFunc) + { + super(PythonConstants.OpType.MAP, serializedFunc); + + } + + @Override + protected void processTuple(T tuple) + { + + LOG.trace("Received Tuple: {} ", tuple); + Object result = getServer().getProxy().execute(tuple); + if (result != null) { + LOG.trace("Response received: {} ", result); + out.emit((T)result); + } + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonApexStreamImpl.java b/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonApexStreamImpl.java new file mode 100644 index 0000000000..3c9bd76f46 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonApexStreamImpl.java @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.runtime; + +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.lib.window.Accumulation; +import org.apache.apex.malhar.lib.window.TriggerOption; +import org.apache.apex.malhar.lib.window.WindowOption; +import org.apache.apex.malhar.lib.window.WindowState; +import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedKeyedStorage; +import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedStorage; +import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl; +import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl; +import org.apache.apex.malhar.python.operator.PythonGenericOperator; +import org.apache.apex.malhar.python.operator.PythonKeyedWindowedOperator; +import org.apache.apex.malhar.python.operator.PythonWindowedOperator; +import org.apache.apex.malhar.python.operator.proxy.PythonWorkerProxy; +import org.apache.apex.malhar.python.operator.transform.PythonFilterOperator; +import org.apache.apex.malhar.python.operator.transform.PythonFlatMapOperator; +import org.apache.apex.malhar.python.operator.transform.PythonMapOperator; +import org.apache.apex.malhar.stream.api.ApexStream; +import org.apache.apex.malhar.stream.api.Option; +import org.apache.apex.malhar.stream.api.PythonApexStream; +import org.apache.apex.malhar.stream.api.WindowedStream; +import org.apache.apex.malhar.stream.api.impl.ApexStreamImpl; +import org.apache.apex.malhar.stream.api.impl.ApexWindowedStreamImpl; +import org.apache.apex.malhar.stream.api.impl.DagMeta; +import org.apache.hadoop.classification.InterfaceStability; + +import com.datatorrent.api.Operator; + +@InterfaceStability.Evolving +public class PythonApexStreamImpl extends ApexWindowedStreamImpl implements PythonApexStream +{ + private static final Logger LOG = LoggerFactory.getLogger(PythonApexStreamImpl.class); + + public PythonApexStreamImpl() + { + super(); + } + + public PythonApexStreamImpl(ApexStreamImpl apexStream) + { + super(apexStream); +// this.lastBrick = apexStream.getLastBrick(); +// this.graph = apexStream.getGraph(); + + } + + @Override + public PythonApexStream map(byte[] serializedFunction, Option... opts) + { + LOG.debug("Adding Python map operator"); + PythonGenericOperator operator = new PythonMapOperator(serializedFunction); + return addOperator(operator, (Operator.InputPort)operator.in, (Operator.OutputPort)operator.out, opts); + + } + + @Override + public PythonApexStream flatMap(byte[] serializedFunction, Option... opts) + { + LOG.debug("Adding Python flatmap operator"); + PythonGenericOperator operator = new PythonFlatMapOperator(serializedFunction); + return addOperator(operator, (Operator.InputPort)operator.in, (Operator.OutputPort)operator.out, opts); + + } + + @Override + public PythonApexStream filter(byte[] serializedFunction, Option... opts) + { + LOG.debug("Adding Python filter operator"); + PythonFilterOperator operator = new PythonFilterOperator<>(serializedFunction); + return addOperator(operator, (Operator.InputPort)operator.in, (Operator.OutputPort)operator.out, opts); + + } + + @Override + protected ApexStream newStream(DagMeta graph, Brick newBrick) + { + PythonApexStreamImpl newstream = new PythonApexStreamImpl<>(); + newstream.graph = graph; + newstream.lastBrick = newBrick; + newstream.windowOption = this.windowOption; + newstream.triggerOption = this.triggerOption; + newstream.allowedLateness = this.allowedLateness; + return newstream; + } + + @Override + public WindowedStream window(WindowOption windowOption, TriggerOption triggerOption, Duration allowLateness) + { + PythonApexStreamImpl windowedStream = new PythonApexStreamImpl<>(); + windowedStream.lastBrick = lastBrick; + windowedStream.graph = graph; + windowedStream.windowOption = windowOption; + windowedStream.triggerOption = triggerOption; + windowedStream.allowedLateness = allowLateness; + return windowedStream; + } + + /** + * Create the windowed operator for windowed transformation + * + * @param accumulationFn + * @param + * @param + * @param + * @return + */ + @Override + protected WindowedOperatorImpl createWindowedOperator(Accumulation accumulationFn) + { + + if (!(accumulationFn instanceof PythonWorkerProxy)) { + return super.createWindowedOperator(accumulationFn); + } + WindowedOperatorImpl windowedOperator = windowedOperator = new PythonWindowedOperator(((PythonWorkerProxy)accumulationFn).getSerializedData()); + //TODO use other default setting in the future + windowedOperator.setDataStorage(new InMemoryWindowedStorage()); + windowedOperator.setRetractionStorage(new InMemoryWindowedStorage()); + windowedOperator.setWindowStateStorage(new InMemoryWindowedStorage()); + if (windowOption != null) { + windowedOperator.setWindowOption(windowOption); + } + if (triggerOption != null) { + windowedOperator.setTriggerOption(triggerOption); + } + if (allowedLateness != null) { + windowedOperator.setAllowedLateness(allowedLateness); + } + windowedOperator.setAccumulation(accumulationFn); + return windowedOperator; + } + + protected KeyedWindowedOperatorImpl createKeyedWindowedOperator(Accumulation accumulationFn) + { + if (!(accumulationFn instanceof PythonWorkerProxy)) { + return super.createKeyedWindowedOperator(accumulationFn); + } + + KeyedWindowedOperatorImpl keyedWindowedOperator = new PythonKeyedWindowedOperator(((PythonWorkerProxy)accumulationFn).getSerializedData()); + //TODO use other default setting in the future + keyedWindowedOperator.setDataStorage(new InMemoryWindowedKeyedStorage()); + keyedWindowedOperator.setRetractionStorage(new InMemoryWindowedKeyedStorage()); + keyedWindowedOperator.setWindowStateStorage(new InMemoryWindowedStorage()); + if (windowOption != null) { + keyedWindowedOperator.setWindowOption(windowOption); + } + if (triggerOption != null) { + keyedWindowedOperator.setTriggerOption(triggerOption); + } + if (allowedLateness != null) { + keyedWindowedOperator.setAllowedLateness(allowedLateness); + } + + keyedWindowedOperator.setAccumulation(accumulationFn); + return keyedWindowedOperator; + } + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonServer.java b/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonServer.java new file mode 100644 index 0000000000..a7311eb1da --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonServer.java @@ -0,0 +1,275 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.runtime; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.operator.proxy.PythonAcummlationWorkerProxy; +import org.apache.apex.malhar.python.operator.proxy.PythonWorkerProxy; +import org.apache.apex.malhar.python.util.LoggerUtils; +import org.apache.apex.malhar.python.util.NetworkUtils; + +import py4j.GatewayServer; +import py4j.GatewayServerListener; +import py4j.Py4JServerConnection; + +public class PythonServer +{ + + private static final Logger LOG = LoggerFactory.getLogger(PythonServer.class); + + private byte[] serializedFunction = null; + + private PythonWorkerContext pythonWorkerContext = null; + + private PythonWorkerProxy proxy = null; + protected Map environementData = new HashMap(); + protected transient GatewayServer gatewayServer = null; + protected transient PythonGatewayServerListenser py4jListener = null; + private PythonConstants.OpType operationType = null; + + public PythonServer() + { + + } + + public PythonServer(PythonConstants.OpType operationType, byte[] serializedFunction) + { + + this.serializedFunction = serializedFunction; + this.operationType = operationType; + this.pythonWorkerContext = new PythonWorkerContext(this.operationType, serializedFunction, environementData); + + } + + public boolean setup() + { + + LOG.debug("LAUNCHING GATEWAY SERVER...{}", this.pythonWorkerContext); + // Setting up context path explicitly for handling local as well Hadoop Based Application Development + this.pythonWorkerContext.setup(); + + if (proxy == null) { + proxy = new PythonWorkerProxy<>(this.serializedFunction); + } + // Instantiating Py4j Gateway Server for Python Worker Process connect back + boolean gatewayServerLaunchSuccess = false; + int serverStartAttempts = 5; + while (!gatewayServerLaunchSuccess && serverStartAttempts > 0) { + try { + this.gatewayServer = new GatewayServer(proxy, NetworkUtils.findAvaliablePort()); + this.py4jListener = new PythonGatewayServerListenser(this.gatewayServer, this.pythonWorkerContext); + this.py4jListener.setOperationType(this.operationType); + this.gatewayServer.addListener(this.py4jListener); + this.gatewayServer.start(true); + gatewayServerLaunchSuccess = true; + --serverStartAttempts; + } catch (Exception ex) { + LOG.error("Gateway server failed to launch to due: {}" + ex.getMessage()); + gatewayServerLaunchSuccess = false; + } + } + + LOG.debug("LAUNCHING GATEWAY SERVER..."); + if (!gatewayServerLaunchSuccess) { + throw new RuntimeException("Failed to launch Gateway Server"); + } + + serverStartAttempts = 5; + + while (!this.py4jListener.isPythonServerStarted() && !proxy.isFunctionEnabled() && serverStartAttempts > 0) { + try { + Thread.sleep(500L); + LOG.debug("Waiting for Python Worker Registration"); + --serverStartAttempts; + } catch (InterruptedException ex) { + LOG.error("Python Callback server failed to launch to due: {}" + ex.getMessage()); + } + } + if (!proxy.isWorkerRegistered()) { + this.gatewayServer.shutdown(); + throw new RuntimeException("Failed to launch Call Back Server"); + } + + // Transferring serialized function to Python Worker. + LOG.debug("Checking if worker is registered {} {} ", proxy.isWorkerRegistered(), this.operationType); + if (proxy.isWorkerRegistered() && !(proxy instanceof PythonAcummlationWorkerProxy)) { + LOG.debug("Setting serialized function back "); + proxy.setSerializedData(this.operationType.getType()); + } + return true; + } + + public void shutdown() + { + gatewayServer.shutdown(); + } + + public void setPythonOperatorEnv(Map environementData) + { + this.environementData = environementData; + if (pythonWorkerContext == null) { + this.pythonWorkerContext = new PythonWorkerContext(this.operationType, serializedFunction, environementData); + } else { + this.pythonWorkerContext.setEnvironmentData(environementData); + } + } + + public static class PythonGatewayServerListenser implements GatewayServerListener + { + + private GatewayServer server = null; + private Process pyProcess = null; + private boolean pythonServerStarted = false; + + private PythonConstants.OpType operationType = null; + private static final Logger LOG = LoggerFactory.getLogger(PythonGatewayServerListenser.class); + private PythonWorkerContext context = null; + + public boolean isPythonServerStarted() + { + return this.pythonServerStarted; + } + + public PythonGatewayServerListenser(GatewayServer startedServer, PythonWorkerContext context) + { + this.server = startedServer; + this.context = context; + } + + public void connectionError(Exception e) + { + LOG.debug("Python Connection error : {}", e.getMessage()); + + } + + @Override + public void connectionStarted(Py4JServerConnection py4JServerConnection) + { + LOG.debug("Python Connection started: {}", py4JServerConnection.getSocket().getPort()); + + } + + @Override + public void connectionStopped(Py4JServerConnection py4JServerConnection) + { + LOG.debug("Python Connection stopped: {}", py4JServerConnection.getSocket().getPort()); + if (this.pyProcess != null) { + this.pyProcess.destroy(); + LOG.debug("Destroyed python worker process"); + } + LOG.debug("Restarting python worker process"); + this.startPythonWorker(this.server.getPort()); + } + + public void serverError(Exception e) + { + LOG.debug("Gateway Server error: {}", e.getMessage()); + } + + public void serverPostShutdown() + { + + LOG.debug("Gateway server shut down"); + } + + public void serverPreShutdown() + { + LOG.debug("Gateway server shutting down"); + + if (this.pyProcess != null) { + this.pyProcess.destroy(); + LOG.debug("Destroyed python worker process"); + } + } + + public void serverStarted() + { + LOG.debug("Gateway server started: {}", this.server.getPort()); + this.startPythonWorker(this.server.getPort()); + } + + public void serverStopped() + { + LOG.debug("Gateway server stopped"); + if (this.pyProcess != null) { + this.pyProcess.destroy(); + LOG.debug("Destroyed python worker process"); + } + + } + + private void startPythonWorker(int gatewayServerPort) + { + ProcessBuilder pb = new ProcessBuilder(new java.lang.String[0]); + try { + LOG.info("Starting python worker process using context: {}", this.context); + LOG.info("Worker File Path: {}", this.context.getWorkerFilePath()); + LOG.info("Python Environment Path: {}", this.context.getPythonEnvPath()); + Map processEnvironment = pb.environment(); + processEnvironment.put("PYTHONPATH", this.context.getPythonEnvPath()); + this.pyProcess = pb.command(new java.lang.String[]{"/usr/bin/python", "-u", this.context.getWorkerFilePath(), "" + gatewayServerPort, operationType.getType()}).start(); + LoggerUtils.captureProcessStreams(this.pyProcess); + this.pythonServerStarted = true; + LOG.info("Python worker started: {} {} ", this.pyProcess, this.operationType); + } catch (IOException exception) { + + LOG.error("Failed to start python server: {}" + exception.getMessage()); + } + } + + public PythonConstants.OpType getOperationType() + { + return operationType; + } + + public void setOperationType(PythonConstants.OpType operationType) + { + this.operationType = operationType; + } + + } + + public PythonWorkerProxy getProxy() + { + return proxy; + } + + public void setProxy(PythonWorkerProxy proxy) + { + this.proxy = proxy; + } + + public PythonConstants.OpType getOperationType() + { + return operationType; + } + + public void setOperationType(PythonConstants.OpType operationType) + { + this.operationType = operationType; + } + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonWorkerContext.java b/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonWorkerContext.java new file mode 100644 index 0000000000..b279e8b130 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/runtime/PythonWorkerContext.java @@ -0,0 +1,144 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.runtime; + +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.commons.lang3.StringUtils; + +import static org.apache.apex.malhar.PythonConstants.PY4J_SRC_ZIP_FILE_NAME; +import static org.apache.apex.malhar.PythonConstants.PYTHON_APEX_ZIP_NAME; +import static org.apache.apex.malhar.PythonConstants.PYTHON_WORKER_FILE_NAME; + +public class PythonWorkerContext implements Serializable +{ + public static String PY4J_DEPENDENCY_PATH = "PY4J_DEPENDENCY_PATH"; + public static String PYTHON_WORKER_PATH = "PYTHON_WORKER_PATH"; + public static String PYTHON_APEX_PATH = "PYTHON_APEX_PATH"; + public static String ENV_VAR_PYTHONPATH = "PYTHONPATH"; + + private static final Logger LOG = LoggerFactory.getLogger(PythonWorkerContext.class); + + private String py4jDependencyPath = null; + private String apexSourcePath = null; + private String workerFilePath = null; + private String pythonEnvPath = null; + private byte[] serializedFunction = null; + private PythonConstants.OpType opType = null; + + // environment data is set explicitly with local paths for managing local mode execution + private Map environmentData = new HashMap(); + + public PythonWorkerContext() + { + + } + + public PythonWorkerContext(PythonConstants.OpType operationType, byte[] serializedFunction, Map environmentData) + { + this(); + this.opType = operationType; + this.serializedFunction = serializedFunction; + this.environmentData = environmentData; + } + + public void setup() + { + LOG.info("Setting up worker context: {}", this); + LOG.debug("Final python environment path with Py4j depenency path: {}", pythonEnvPath); + + if ((this.apexSourcePath = environmentData.get(PYTHON_APEX_PATH)) == null) { + this.apexSourcePath = new File("./" + PYTHON_APEX_ZIP_NAME).getAbsolutePath(); + } + + if ((this.py4jDependencyPath = environmentData.get(PY4J_DEPENDENCY_PATH)) == null) { + this.py4jDependencyPath = new File("./" + PY4J_SRC_ZIP_FILE_NAME).getAbsolutePath(); + } + + LOG.info("FINAL WORKER PATH: {}", environmentData.get(PYTHON_WORKER_PATH)); + if ((this.workerFilePath = environmentData.get(PYTHON_WORKER_PATH)) == null) { + File pythonWorkerFile = new File("./" + PYTHON_WORKER_FILE_NAME); + this.workerFilePath = pythonWorkerFile.getAbsolutePath(); + } + + List dependencyFilePaths = new ArrayList(); + dependencyFilePaths.add(this.apexSourcePath); + dependencyFilePaths.add(this.py4jDependencyPath); + + pythonEnvPath = System.getenv(ENV_VAR_PYTHONPATH); + LOG.info("Found python environment path: {}", pythonEnvPath); + if (pythonEnvPath != null) { + dependencyFilePaths.add(pythonEnvPath); + pythonEnvPath = StringUtils.join(dependencyFilePaths, ":"); + } else { + pythonEnvPath = StringUtils.join(dependencyFilePaths, ":"); + } + LOG.info("Python dependency Path {} worker Path {}", this.py4jDependencyPath, this.workerFilePath); + } + + public synchronized String getPy4jDependencyPath() + { + return this.py4jDependencyPath; + } + + public synchronized String getWorkerFilePath() + { + return this.workerFilePath; + } + + public synchronized String getPythonEnvPath() + { + return this.pythonEnvPath; + } + + public synchronized byte[] getSerializedFunction() + { + return this.serializedFunction; + } + + public synchronized Map getEnvironmentData() + { + return this.environmentData; + } + + public synchronized void setEnvironmentData(Map environmentData) + { + this.environmentData = environmentData; + } + + public String getApexSourcePath() + { + return apexSourcePath; + } + + public void setApexSourcePath(String apexSourcePath) + { + this.apexSourcePath = apexSourcePath; + } + +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/util/LoggerUtils.java b/python/src/main/java/org/apache/apex/malhar/python/util/LoggerUtils.java new file mode 100644 index 0000000000..f9a9b232bb --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/util/LoggerUtils.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.util; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.python.operator.PythonGenericOperator; + +public class LoggerUtils +{ + + public static class InputStreamConsumer extends Thread + { + private static final Logger LOG = LoggerFactory.getLogger(PythonGenericOperator.class); + private InputStream is; + private String name; + private StreamType streamType; + private String processId; + + public enum StreamType + { + ERROR, OUTPUT + } + + public InputStreamConsumer(String name, InputStream is, StreamType streamType) + { + this.is = is; + this.name = name; + this.streamType = streamType; + } + + @Override + public void run() + { + LOG.info("Starting Stream Gobbler " + this.name); + try { + + InputStreamReader isr = new InputStreamReader(this.is); + BufferedReader br = new BufferedReader(isr); + String line; + while ((line = br.readLine()) != null) { + if (this.streamType == StreamType.ERROR) { + LOG.error(" From other process :" + line); + } else { + LOG.info(" From other process :" + line); + + } + } + } catch (IOException exp) { + exp.printStackTrace(); + } + + LOG.info("Exiting Stream Gobbler " + this.name); + } + } + + public static void captureProcessStreams(Process process) + { + InputStreamConsumer stdoutConsumer = new InputStreamConsumer("outputStream", process.getInputStream(), InputStreamConsumer.StreamType.OUTPUT); + InputStreamConsumer erroConsumer = new InputStreamConsumer("errorStream", process.getErrorStream(), InputStreamConsumer.StreamType.ERROR); + erroConsumer.start(); + stdoutConsumer.start(); + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/python/util/NetworkUtils.java b/python/src/main/java/org/apache/apex/malhar/python/util/NetworkUtils.java new file mode 100644 index 0000000000..c730583ae2 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/python/util/NetworkUtils.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.util; + +import java.io.IOException; +import java.net.ServerSocket; + +import com.google.common.base.Throwables; + +public class NetworkUtils +{ + + public static int findAvaliablePort() + { + // Find port + int foundPort = -1; + try { + ServerSocket serverSocket = new ServerSocket(0); + foundPort = serverSocket.getLocalPort(); + serverSocket.close(); + return foundPort; + } catch (IOException e) { + throw Throwables.propagate(e); + } + + } +} diff --git a/python/src/main/java/org/apache/apex/malhar/stream/api/PythonApexStream.java b/python/src/main/java/org/apache/apex/malhar/stream/api/PythonApexStream.java new file mode 100644 index 0000000000..aa9d6be492 --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/stream/api/PythonApexStream.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.stream.api; + + + +/** + * Created by vikram on 7/6/17. + */ +public interface PythonApexStream extends ApexStream +{ + + /** + * Add custom serialized Python Function along with options + * @param serializedFunction stores Serialized Function data + * @return new stream of type T + */ + > STREAM map(byte[] serializedFunction, Option... opts); + + /** + * Add custom serialized Python Function along with options + * @param serializedFunction stores Serialized Function data + * @return new stream of type T + */ + > STREAM flatMap(byte[] serializedFunction, Option... opts); + + /** + * Add custom serialized Python Function along with options + * @param serializedFunction stores Serialized Function data + * @return new stream of type T + */ + > STREAM filter(byte[] serializedFunction, Option... opts); + +} diff --git a/python/src/main/java/org/apache/apex/malhar/stream/api/PythonWindowedApexStream.java b/python/src/main/java/org/apache/apex/malhar/stream/api/PythonWindowedApexStream.java new file mode 100644 index 0000000000..196cc57ecc --- /dev/null +++ b/python/src/main/java/org/apache/apex/malhar/stream/api/PythonWindowedApexStream.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.stream.api; + +/** + * Created by vikram on 20/6/17. + */ +public class PythonWindowedApexStream +{ +} diff --git a/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonOperatorTest.java b/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonOperatorTest.java new file mode 100644 index 0000000000..0d603e2622 --- /dev/null +++ b/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonOperatorTest.java @@ -0,0 +1,277 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.runtime; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; + +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.operator.transform.PythonFilterOperator; +import org.apache.apex.malhar.python.operator.transform.PythonFlatMapOperator; +import org.apache.apex.malhar.python.operator.transform.PythonMapOperator; +import org.apache.apex.malhar.python.runtime.PythonWorkerContext; +import org.apache.commons.codec.binary.Base64; + +import com.datatorrent.api.Context; +import com.datatorrent.api.DAG; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.InputOperator; +import com.datatorrent.api.LocalMode; +import com.datatorrent.common.util.BaseOperator; +import com.datatorrent.stram.StramLocalCluster; + +public class PythonOperatorTest +{ + private static int TupleCount; + private static List lengthList = new ArrayList<>(); + private static int tupleSumInResultCollector = 0; + + private static final int SourcedNumTuples = 10; + private static int ExpectedNumTuples = 0; + private static final Logger LOG = LoggerFactory.getLogger(PythonOperatorTest.class); + + public static class NumberGenerator extends BaseOperator implements InputOperator + { + private int num; + + public final transient DefaultOutputPort output = new DefaultOutputPort(); + + @Override + public void setup(Context.OperatorContext context) + { + num = 0; + } + + @Override + public void emitTuples() + { + + if (num < SourcedNumTuples) { + output.emit(new Integer(num)); + num++; + } + } + } + + public static class ResultCollector extends BaseOperator + { + + public final transient DefaultInputPort input = new DefaultInputPort() + { + + @Override + public void process(Integer in) + { + + LOG.debug("Input data " + in); + TupleCount++; + lengthList.add(in); + tupleSumInResultCollector += in; + } + }; + + @Override + public void setup(Context.OperatorContext context) + { + TupleCount = 0; + tupleSumInResultCollector = 0; + + lengthList = new ArrayList<>(); + } + + } + + @Test + public void testPythonMapOperator() + { + + LocalMode lma = LocalMode.newInstance(); + DAG dag = lma.getDAG(); + + ExpectedNumTuples = SourcedNumTuples; + + NumberGenerator numGen = dag.addOperator("numGen", new NumberGenerator()); + + ResultCollector collector = dag.addOperator("collector", new ResultCollector()); +// Base64 string for map function written and serialized using pickle +// Function : +// lambda(a): int(a)*2 + String funcAsBase64String = "gAJjY2xvdWRwaWNrbGUuY2xvdWRwaWNrbGUKX2ZpbGxfZnVuY3Rpb24KcQAoY2Nsb3VkcGlja2xlLmNsb3VkcGlja2xlCl9tYWtlX3NrZWxfZnVuYwpxAWNjbG91ZHBpY2tsZS5jbG91ZHBpY2tsZQpfYnVpbHRpbl90eXBlCnECVQhDb2RlVHlwZXEDhXEEUnEFKEsBSwFLAktDVQ50AAB8AACDAQBkAQAUU3EGTksChnEHVQNpbnRxCIVxCVUBZnEKhXELVR48aXB5dGhvbi1pbnB1dC00LTU4NDRlZWUwNzQ4Zj5xDFUIPGxhbWJkYT5xDUsBVQBxDikpdHEPUnEQXXERfXESh3ETUnEUfXEVTn1xFnRSLg=="; + + byte[] decoded = Base64.decodeBase64(funcAsBase64String); + + Map environmentData = new HashMap<>(); + + final String cwd = System.getProperty("user.dir"); + String pythonRuntimeDirectory = cwd + "/../python/apex-python/src/pyapex/runtime"; + String pythonDepsDirectory = cwd + "/../python/apex-python/deps"; + + LOG.debug("Current working directory:" + pythonRuntimeDirectory); + environmentData.put(PythonWorkerContext.PYTHON_WORKER_PATH, pythonRuntimeDirectory + "/" + PythonConstants.PYTHON_WORKER_FILE_NAME); + environmentData.put(PythonWorkerContext.PY4J_DEPENDENCY_PATH, pythonDepsDirectory + "/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + environmentData.put(PythonWorkerContext.PYTHON_APEX_PATH, pythonDepsDirectory + "/" + PythonConstants.PYTHON_APEX_ZIP_NAME); + PythonMapOperator mapOperator = new PythonMapOperator(decoded); + mapOperator.getServer().setPythonOperatorEnv(environmentData); + + dag.addOperator("mapOperator", mapOperator); + + dag.addStream("raw numbers", numGen.output, mapOperator.in); + dag.addStream("mapped results", mapOperator.out, collector.input); + + // Create local cluster + LocalMode.Controller lc = lma.getController(); + lc.setHeartbeatMonitoringEnabled(false); + + ((StramLocalCluster)lc).setExitCondition(new Callable() + { + @Override + public Boolean call() throws Exception + { + + return TupleCount == ExpectedNumTuples; + } + }); + + lc.run(100000); + + Assert.assertEquals(ExpectedNumTuples, TupleCount); + Assert.assertEquals(90, tupleSumInResultCollector); + } + + @Test + public void testFilterOperator() + { + + LocalMode lma = LocalMode.newInstance(); + DAG dag = lma.getDAG(); + + NumberGenerator numGen = dag.addOperator("numGen", new NumberGenerator()); + + ResultCollector collector = dag.addOperator("collector", new ResultCollector()); + ExpectedNumTuples = SourcedNumTuples - 2; +// def f(a): +// if a< 2: +// return False +// return True + + String funcAsBase64String = "gAJjZGlsbC5kaWxsCl9jcmVhdGVfZnVuY3Rpb24KcQAoY2RpbGwuZGlsbApfbG9hZF90eXBlCnEBVQhDb2RlVHlwZXEChXEDUnEEKEsBSwFLAktDVRR8AABkAQBrAAByEAB0AABTdAEAU3EFTksChnEGVQVGYWxzZXEHVQRUcnVlcQiGcQlVAWFxCoVxC1UePGlweXRob24taW5wdXQtMS05NDcxMmNkN2IyY2I+cQxVAWZxDUsBVQYAAQwBBAFxDikpdHEPUnEQY19fYnVpbHRpbl9fCl9fbWFpbl9fCmgNTk59cRF0cRJScRMu"; + + byte[] decoded = Base64.decodeBase64(funcAsBase64String); + + Map environmentData = new HashMap<>(); + + final String cwd = System.getProperty("user.dir"); + String pythonRuntimeDirectory = cwd + "/../python/apex-python/src/pyapex/runtime"; + String pythonDepsDirectory = cwd + "/../python/apex-python/deps"; + + LOG.debug("Current working directory:" + pythonRuntimeDirectory); + environmentData.put(PythonWorkerContext.PYTHON_WORKER_PATH, pythonRuntimeDirectory + "/" + PythonConstants.PYTHON_WORKER_FILE_NAME); + environmentData.put(PythonWorkerContext.PY4J_DEPENDENCY_PATH, pythonDepsDirectory + "/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + environmentData.put(PythonWorkerContext.PYTHON_APEX_PATH, pythonDepsDirectory + "/" + PythonConstants.PYTHON_APEX_ZIP_NAME); + PythonFilterOperator mapOperator = new PythonFilterOperator(decoded); + mapOperator.getServer().setPythonOperatorEnv(environmentData); + + dag.addOperator("mapOperator", mapOperator); + + dag.addStream("raw numbers", numGen.output, mapOperator.in); + dag.addStream("mapped results", mapOperator.out, collector.input); + + // Create local cluster + LocalMode.Controller lc = lma.getController(); + lc.setHeartbeatMonitoringEnabled(false); + + ((StramLocalCluster)lc).setExitCondition(new Callable() + { + @Override + public Boolean call() throws Exception + { + return TupleCount == ExpectedNumTuples; + } + }); + + lc.run(100000); + + Assert.assertEquals(ExpectedNumTuples, TupleCount); + Assert.assertEquals(44, tupleSumInResultCollector); + } + + @Test + public void testFlatMapOperator() + { + + LocalMode lma = LocalMode.newInstance(); + DAG dag = lma.getDAG(); + ExpectedNumTuples = SourcedNumTuples * 2; + NumberGenerator numGen = dag.addOperator("numGen", new NumberGenerator()); + + ResultCollector collector = dag.addOperator("collector", new ResultCollector()); +// def f(a): +// return [int(a)*2, int(a)*3] + + String funcAsBase64String = "gAJjZGlsbC5kaWxsCl9jcmVhdGVfZnVuY3Rpb24KcQAoY2RpbGwuZGlsbApfbG9hZF90eXBlCnEBVQhDb2RlVHlwZXEChXEDUnEEKEsBSwFLA0tDVR50AAB8AACDAQBkAQAUdAAAfAAAgwEAZAIAFGcCAFNxBU5LAksDh3EGVQNpbnRxB4VxCFUBYXEJhXEKVR48aXB5dGhvbi1pbnB1dC0xLWFjNjk0MzQ3NzhlYT5xC1UBZnEMSwFVAgABcQ0pKXRxDlJxD2NfX2J1aWx0aW5fXwpfX21haW5fXwpoDE5OfXEQdHERUnESLg=="; + + byte[] decoded = Base64.decodeBase64(funcAsBase64String); + + Map environmentData = new HashMap<>(); + + final String cwd = System.getProperty("user.dir"); + String pythonRuntimeDirectory = cwd + "/../python/apex-python/src/pyapex/runtime"; + String pythonDepsDirectory = cwd + "/../python/apex-python/deps"; + + LOG.debug("Current working directory:" + pythonRuntimeDirectory); + environmentData.put(PythonWorkerContext.PYTHON_WORKER_PATH, pythonRuntimeDirectory + "/" + PythonConstants.PYTHON_WORKER_FILE_NAME); + environmentData.put(PythonWorkerContext.PY4J_DEPENDENCY_PATH, pythonDepsDirectory + "/" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + environmentData.put(PythonWorkerContext.PYTHON_APEX_PATH, pythonDepsDirectory + "/" + PythonConstants.PYTHON_APEX_ZIP_NAME); + PythonFlatMapOperator mapOperator = new PythonFlatMapOperator(decoded); + mapOperator.getServer().setPythonOperatorEnv(environmentData); + + dag.addOperator("mapOperator", mapOperator); + + dag.addStream("raw numbers", numGen.output, mapOperator.in); + dag.addStream("mapped results", mapOperator.out, collector.input); + + // Create local cluster + LocalMode.Controller lc = lma.getController(); + lc.setHeartbeatMonitoringEnabled(false); + + ((StramLocalCluster)lc).setExitCondition(new Callable() + { + @Override + public Boolean call() throws Exception + { + return TupleCount == ExpectedNumTuples; + } + }); + + lc.run(100000); + + Assert.assertEquals(ExpectedNumTuples, TupleCount); + Assert.assertEquals(225, tupleSumInResultCollector); + } + +} diff --git a/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerContextTest.java b/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerContextTest.java new file mode 100644 index 0000000000..aa6e6f1e30 --- /dev/null +++ b/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerContextTest.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.runtime; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.apex.malhar.PythonConstants; +import org.apache.apex.malhar.python.runtime.PythonWorkerContext; + +//import org.apache.apex.malhar.python.operator.PythonGenericOperator; + +public class PythonWorkerContextTest +{ + + @Test + public void testPythonWorkerContextTest() + { + PythonWorkerContext context = new PythonWorkerContext(); + String currentWorkingDirectory = "/home/data"; + Map environmentData = new HashMap<>(); + environmentData.put(PythonWorkerContext.PY4J_DEPENDENCY_PATH, currentWorkingDirectory + "/./" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME); + environmentData.put(PythonWorkerContext.PYTHON_WORKER_PATH, currentWorkingDirectory + "/./" + PythonConstants.PYTHON_WORKER_FILE_NAME); + + context.setEnvironmentData(environmentData); + context.setup(); + Assert.assertEquals(currentWorkingDirectory + "/./" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME, context.getPy4jDependencyPath()); + Assert.assertEquals(currentWorkingDirectory + "/./" + PythonConstants.PYTHON_WORKER_FILE_NAME, context.getWorkerFilePath()); + + } + + @Test + public void testPythonWorkerContextWithDeafaultTest() + { + + PythonWorkerContext context = new PythonWorkerContext(); + String currentWorkingDirectory = System.getProperty("user.dir"); + context.setup(); + Assert.assertEquals(currentWorkingDirectory + "/./" + PythonConstants.PY4J_SRC_ZIP_FILE_NAME, context.getPy4jDependencyPath()); + Assert.assertEquals(currentWorkingDirectory + "/./" + PythonConstants.PYTHON_WORKER_FILE_NAME, context.getWorkerFilePath()); + + + } + +} diff --git a/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerProxyTest.java b/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerProxyTest.java new file mode 100644 index 0000000000..eb895a4b20 --- /dev/null +++ b/python/src/test/java/org/apache/apex/malhar/python/operator/runtime/PythonWorkerProxyTest.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.apex.malhar.python.operator.runtime; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.python.operator.interfaces.PythonWorker; +import org.apache.apex.malhar.python.operator.proxy.PythonWorkerProxy; + +import py4j.Py4JException; + +import static org.mockito.Matchers.any; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({PythonWorkerProxy.class, LoggerFactory.class}) +public class PythonWorkerProxyTest +{ + + public static class PythonTestWorker implements PythonWorker + { + + @Override + public Object setFunction(byte[] func, String opType) + { + return opType; + } + + @Override + public Object execute(Object tuple) + { + return tuple; + } + + } + + @Test + public void testPythonWorkerRegisterAndExecute() + { + mockStatic(LoggerFactory.class); + Logger logger = mock(Logger.class); + when(LoggerFactory.getLogger(PythonWorkerProxy.class)).thenReturn(logger); + + String functionData = new String("TestFunction"); + PythonWorkerProxy workerProxy = new PythonWorkerProxy(functionData.getBytes()); + PythonTestWorker worker = new PythonTestWorker(); + workerProxy.register(worker); + workerProxy.setSerializedData("DUMMY_OPERATION"); + Assert.assertEquals("TUPLE", worker.execute("TUPLE")); + } + + @Test() + public void testPythonFailureWhileProcessingTuple() + { + mockStatic(LoggerFactory.class); + Logger logger = mock(Logger.class); + when(LoggerFactory.getLogger(any(Class.class))).thenReturn(logger); + + String exceptionString = "DUMMY EXCEPTION"; + String functionData = new String("TestFunction"); + PythonWorker failingMockWorker = mock(PythonWorker.class); + when(failingMockWorker.execute("TUPLE")).thenThrow(new Py4JException(exceptionString)); + + PythonWorkerProxy workerProxy = new PythonWorkerProxy(functionData.getBytes()); + workerProxy.register(failingMockWorker); + String tupleValue = "TUPLE"; + Assert.assertEquals(null, workerProxy.execute(tupleValue)); + } +} diff --git a/python/src/test/resources/log4j.properties b/python/src/test/resources/log4j.properties new file mode 100644 index 0000000000..dcbece51eb --- /dev/null +++ b/python/src/test/resources/log4j.properties @@ -0,0 +1,43 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +log4j.rootLogger=DEBUG,CONSOLE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.CONSOLE.threshold=${test.log.console.threshold} +test.log.console.threshold=DEBUG + +log4j.appender.RFA=org.apache.log4j.RollingFileAppender +log4j.appender.RFA.layout=org.apache.log4j.PatternLayout +log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.RFA.File=/tmp/app.log + +# to enable, add SYSLOG to rootLogger +log4j.appender.SYSLOG=org.apache.log4j.net.SyslogAppender +log4j.appender.SYSLOG.syslogHost=127.0.0.1 +log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout +log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n +log4j.appender.SYSLOG.Facility=LOCAL1 + +log4j.logger.org=debug +#log4j.logger.org.apache.commons.beanutils=warn +log4j.logger.com.datatorrent=debug +log4j.logger.org.apache.apex=debug diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java index c09efa5d7d..3e9e08b6d6 100644 --- a/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java +++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java @@ -32,6 +32,7 @@ import com.datatorrent.api.Context.OperatorContext; import com.datatorrent.api.Context.PortContext; import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; import com.datatorrent.api.Operator; /** @@ -178,8 +179,7 @@ public interface ApexStream * @param async true if run in Async mode * false if run in sync mode */ - void runEmbedded(boolean async, long duration, Callable exitCondition); - + LocalMode.Controller runEmbedded(boolean async, long duration, Callable exitCondition); /** * Submit the application to cluster diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java index 554f5d6337..78f5d64051 100644 --- a/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java +++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java @@ -27,6 +27,7 @@ import org.apache.apex.malhar.lib.window.TriggerOption; import org.apache.apex.malhar.lib.window.Tuple; import org.apache.apex.malhar.lib.window.accumulation.FoldFn; +import org.apache.apex.malhar.lib.window.accumulation.Reduce; import org.apache.apex.malhar.lib.window.accumulation.ReduceFn; import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl; import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl; @@ -134,7 +135,7 @@ return type * @return new stream of same type */ - >> STREAM reduce(ReduceFn reduce, Option... opts); + >> STREAM reduce(Reduce reduce, Option... opts); /** * Add {@link KeyedWindowedOperatorImpl} with specified {@link ReduceFn}
@@ -147,7 +148,7 @@ return type * @return new stream of key value pair */ - >>> STREAM reduceByKey(ReduceFn reduce, Function.ToKeyValue convertToKeyVal, Option... opts); + >>> STREAM reduceByKey(Reduce reduce, Function.ToKeyValue convertToKeyVal, Option... opts); /** diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java index bb0f78169b..446f75166a 100644 --- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java +++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java @@ -146,6 +146,10 @@ public Pair getLastStream() } } + public DagMeta getGraph() + { + return graph; + } /** * Graph behind the stream @@ -247,7 +251,7 @@ public > STREAM addOperator(Operator op, Operat newBrick.lastStream = Pair.of(lastBrick.lastOutput, inputPort); } - if (this.getClass() == ApexStreamImpl.class || this.getClass() == ApexWindowedStreamImpl.class) { + if (this instanceof ApexStream) { return (STREAM)newStream(this.graph, newBrick); } else { try { @@ -422,7 +426,7 @@ public void populateDag(DAG dag) } @Override - public void runEmbedded(boolean async, long duration, Callable exitCondition) + public LocalMode.Controller runEmbedded(boolean async, long duration, Callable exitCondition) { LocalMode lma = LocalMode.newInstance(); populateDag(lma.getDAG()); @@ -440,6 +444,7 @@ public void runEmbedded(boolean async, long duration, Callable exitCond lc.run(); } } + return lc; } diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java index feaf968963..4adeebb66b 100644 --- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java +++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java @@ -28,20 +28,17 @@ import org.apache.apex.malhar.lib.window.Tuple; import org.apache.apex.malhar.lib.window.WindowOption; import org.apache.apex.malhar.lib.window.WindowState; - import org.apache.apex.malhar.lib.window.accumulation.FoldFn; -import org.apache.apex.malhar.lib.window.accumulation.ReduceFn; +import org.apache.apex.malhar.lib.window.accumulation.Reduce; import org.apache.apex.malhar.lib.window.accumulation.SumLong; import org.apache.apex.malhar.lib.window.accumulation.TopN; import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedKeyedStorage; import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedStorage; import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl; import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl; - import org.apache.apex.malhar.stream.api.ApexStream; import org.apache.apex.malhar.stream.api.Option; import org.apache.apex.malhar.stream.api.WindowedStream; - import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.classification.InterfaceStability; @@ -63,6 +60,17 @@ public class ApexWindowedStreamImpl extends ApexStreamImpl implements Wind protected Duration allowedLateness; + public ApexWindowedStreamImpl() + { + } + + public ApexWindowedStreamImpl(ApexStreamImpl apexStream) + { + + super(apexStream); + + } + private static class ConvertFn implements Function.MapFunction> { @@ -77,11 +85,6 @@ public Tuple f(T input) } } - - public ApexWindowedStreamImpl() - { - } - @Override public >> STREAM count(Option... opts) { @@ -132,7 +135,6 @@ public >>> STREAM top( return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts); } - @Override public >>> STREAM accumulateByKey(Accumulation accumulation, Function.ToKeyValue convertToKeyVal, Option... opts) @@ -142,7 +144,6 @@ public >> STREAM accumulate(Accumulation accumulation, Option... opts) { @@ -151,17 +152,17 @@ public >> STREAM a return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts); } - @Override - public >> STREAM reduce(ReduceFn reduce, Option... opts) + public >> STREAM reduce(Reduce reduce, Option... opts) { WindowedStream> innerstream = map(new ConvertFn()); WindowedOperatorImpl windowedOperator = createWindowedOperator(reduce); + return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts); } @Override - public >>> STREAM reduceByKey(ReduceFn reduce, Function.ToKeyValue convertToKeyVal, Option... opts) + public >>> STREAM reduceByKey(Reduce reduce, Function.ToKeyValue convertToKeyVal, Option... opts) { WindowedStream>> kvstream = map(convertToKeyVal); KeyedWindowedOperatorImpl keyedWindowedOperator = createKeyedWindowedOperator(reduce); @@ -231,7 +232,7 @@ protected ApexStream newStream(DagMeta graph, Brick newBrick) * @param * @return */ - private WindowedOperatorImpl createWindowedOperator(Accumulation accumulationFn) + protected WindowedOperatorImpl createWindowedOperator(Accumulation accumulationFn) { WindowedOperatorImpl windowedOperator = new WindowedOperatorImpl<>(); //TODO use other default setting in the future @@ -251,7 +252,7 @@ private WindowedOperatorImpl createWindowedOperat return windowedOperator; } - private KeyedWindowedOperatorImpl createKeyedWindowedOperator(Accumulation accumulationFn) + protected KeyedWindowedOperatorImpl createKeyedWindowedOperator(Accumulation accumulationFn) { KeyedWindowedOperatorImpl keyedWindowedOperator = new KeyedWindowedOperatorImpl<>(); diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java index d6201ad5d3..6ea872cb5e 100644 --- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java +++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java @@ -18,6 +18,12 @@ */ package org.apache.apex.malhar.stream.api.impl; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.kafka.AbstractKafkaInputOperator; import org.apache.apex.malhar.kafka.KafkaSinglePortInputOperator; import org.apache.apex.malhar.kafka.PartitionStrategy; import org.apache.apex.malhar.lib.fs.LineByLineFileInputOperator; @@ -28,6 +34,7 @@ import com.datatorrent.api.InputOperator; import com.datatorrent.api.Operator; import com.datatorrent.contrib.kafka.KafkaSinglePortStringInputOperator; +import com.datatorrent.lib.io.fs.InMemoryDataInputOperator; import static org.apache.apex.malhar.stream.api.Option.Options.name; @@ -39,6 +46,8 @@ @InterfaceStability.Evolving public class StreamFactory { + + private static final Logger LOG = LoggerFactory.getLogger(StreamFactory.class); /** * Create a stream of string tuples from reading files in hdfs folder line by line * @param folderName @@ -53,6 +62,19 @@ public static ApexStream fromFolder(String folderName, Option... opts) return newStream.addOperator(fileLineInputOperator, null, fileLineInputOperator.output, opts); } + /** + * Allow you to provide data as in-memory list and various options to configure in-memory data input operator + * @param input + * @param opts + * @return + */ + public static ApexStream fromData(List input, Option... opts) + { + InMemoryDataInputOperator inMemoryDataInputOperator = new InMemoryDataInputOperator(input); + ApexStreamImpl newStream = new ApexStreamImpl<>(); + return newStream.addOperator(inMemoryDataInputOperator, null, inMemoryDataInputOperator.outputPort, opts); + } + public static ApexStream fromFolder(String folderName) { return fromFolder(folderName, name("FolderScanner")); @@ -114,11 +136,14 @@ public static ApexStream fromKafka09(String brokers, String topic, Optio */ public static ApexStream fromKafka09(String brokers, String topic, PartitionStrategy strategy, int partitionNumber, Option... opts) { + KafkaSinglePortInputOperator kafkaInput = new KafkaSinglePortInputOperator(); + LOG.debug(" Stategy Name {} , topic {}", strategy.name(), topic); kafkaInput.setClusters(brokers); kafkaInput.setTopics(topic); kafkaInput.setStrategy(strategy.name()); kafkaInput.setInitialPartitionCount(partitionNumber); + kafkaInput.setInitialOffset(AbstractKafkaInputOperator.InitialOffset.EARLIEST.name()); ApexStreamImpl newStream = new ApexStreamImpl<>(); return newStream.addOperator(kafkaInput, null, kafkaInput.outputPort, opts); } diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java index ef6a88e60a..329250cd2e 100644 --- a/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java +++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java @@ -19,7 +19,9 @@ package org.apache.apex.malhar.stream.sample; import org.apache.apex.malhar.lib.function.Function; +import org.apache.apex.malhar.stream.api.ApexStream; import org.apache.apex.malhar.stream.api.impl.ApexStreamImpl; +import org.apache.apex.malhar.stream.api.impl.DagMeta; import com.datatorrent.api.DAG; @@ -29,6 +31,11 @@ public class MyStream extends ApexStreamImpl { + public MyStream() + { + super(); + } + public MyStream(ApexStreamImpl apexStream) { super(apexStream); @@ -39,4 +46,14 @@ MyStream myFilterAndMap(Function.MapFunction map, Function.FilterFu return filter(filterFunction).map(map).with(DAG.Locality.THREAD_LOCAL); } + + @Override + protected ApexStream newStream(DagMeta graph, Brick newBrick) + { + MyStream newstream = new MyStream<>(); + newstream.graph = graph; + newstream.lastBrick = newBrick; + return newstream; + } + } diff --git a/stream/src/test/resources/log4j.properties b/stream/src/test/resources/log4j.properties new file mode 100644 index 0000000000..1c9776b5d8 --- /dev/null +++ b/stream/src/test/resources/log4j.properties @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +log4j.rootLogger=DEBUG,CONSOLE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n + +log4j.appender.RFA=org.apache.log4j.RollingFileAppender +log4j.appender.RFA.layout=org.apache.log4j.PatternLayout +log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n +log4j.appender.RFA.File=/tmp/app.log + +# to enable, add SYSLOG to rootLogger +log4j.appender.SYSLOG=org.apache.log4j.net.SyslogAppender +log4j.appender.SYSLOG.syslogHost=127.0.0.1 +log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout +log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n +log4j.appender.SYSLOG.Facility=LOCAL1 + +#log4j.logger.org.apache.commons.beanutils=warn +log4j.logger.com.datatorrent=debug +log4j.logger.org.apache.apex=debug +log4j.logger.org=info