From 760d1a6bb75eb9519a4b93eb3cf34ad1605621da Mon Sep 17 00:00:00 2001 From: yew1eb Date: Tue, 7 Nov 2017 09:06:45 +0800 Subject: [PATCH 001/367] [hotfix][docs] Add type for numLateRecordsDropped metric in docs --- docs/monitoring/metrics.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index bb15cec3364c8..4c423bedcd3dc 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -999,6 +999,7 @@ Thus, in order to infer the metric identifier: numLateRecordsDropped The number of records this operator/task has dropped due to arriving late. + Counter Operator From 07830e7897a42b5d12f0b33c42933c6ca78e70d3 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 7 Nov 2017 12:16:04 +0100 Subject: [PATCH 002/367] [hotfix][rat] Add missing rat exclusions Another set of RAT exclusions to prevent errors on Windows. --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index d537be9407473..5a13012e12793 100644 --- a/pom.xml +++ b/pom.xml @@ -1039,6 +1039,8 @@ under the License. **/src/test/resources/*-snapshot **/src/test/resources/*-savepoint + flink-core/src/test/resources/serialized-kryo-serializer-1.3 + flink-core/src/test/resources/type-without-avro-serialized-using-kryo flink-formats/flink-avro/src/test/resources/testdata.avro flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/*.java From aab36f934548a5697c5c461b2a79c7cf3fd0d756 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Mon, 6 Nov 2017 12:43:18 +0100 Subject: [PATCH 003/367] [FLINK-7823][QS] Update Queryable State configuration parameters. --- .../configuration/QueryableStateOptions.java | 25 +++---- .../network/AbstractServerBase.java | 8 +-- .../HAAbstractQueryableStateTestBase.java | 2 +- .../NonHAAbstractQueryableStateTestBase.java | 2 +- .../network/AbstractServerTest.java | 2 +- .../io/network/NetworkEnvironment.java | 2 +- .../QueryableStateConfiguration.java | 71 ++++++++++++------- .../taskexecutor/TaskManagerServices.java | 56 ++++++++------- .../TaskManagerServicesConfiguration.java | 38 +++++----- .../minicluster/LocalFlinkMiniCluster.scala | 7 -- 10 files changed, 114 insertions(+), 99 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java index adba93842c3b7..ac88bedaf5fa6 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java @@ -32,11 +32,6 @@ public class QueryableStateOptions { // Server Options // ------------------------------------------------------------------------ - /** Flag to indicate whether to start the queryable state server. */ - public static final ConfigOption SERVER_ENABLE = - key("query.server.enable") - .defaultValue(true); - /** * The config parameter defining the server port range of the queryable state proxy. * @@ -59,6 +54,16 @@ public class QueryableStateOptions { key("query.proxy.ports") .defaultValue("9069"); + /** Number of network (event loop) threads for the client proxy (0 => #slots). */ + public static final ConfigOption PROXY_NETWORK_THREADS = + key("query.proxy.network-threads") + .defaultValue(0); + + /** Number of async query threads for the client proxy (0 => #slots). */ + public static final ConfigOption PROXY_ASYNC_QUERY_THREADS = + key("query.proxy.query-threads") + .defaultValue(0); + /** * The config parameter defining the server port range of the queryable state server. * @@ -100,16 +105,6 @@ public class QueryableStateOptions { key("query.client.network-threads") .defaultValue(0); - /** Number of retries on location lookup failures. */ - public static final ConfigOption CLIENT_LOOKUP_RETRIES = - key("query.client.lookup.num-retries") - .defaultValue(3); - - /** Retry delay on location lookup failures (millis). */ - public static final ConfigOption CLIENT_LOOKUP_RETRY_DELAY = - key("query.client.lookup.retry-delay") - .defaultValue(1000); - // ------------------------------------------------------------------------ /** Not intended to be instantiated. */ diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java index 9c88774cfa99e..07ca26ddce60a 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java @@ -180,16 +180,16 @@ public InetSocketAddress getServerAddress() { */ public void start() throws Throwable { Preconditions.checkState(serverAddress == null, - "Server " + serverName + " already running @ " + serverAddress + '.'); + "The " + serverName + " already running @ " + serverAddress + '.'); Iterator portIterator = bindPortRange.iterator(); while (portIterator.hasNext() && !attemptToBind(portIterator.next())) {} if (serverAddress != null) { - LOG.info("Started server {} @ {}.", serverName, serverAddress); + LOG.info("Started the {} @ {}.", serverName, serverAddress); } else { - LOG.info("Unable to start server {}. All ports in provided range are occupied.", serverName); - throw new FlinkRuntimeException("Unable to start server " + serverName + ". All ports in provided range are occupied."); + LOG.info("Unable to start the {}. All ports in provided range are occupied.", serverName); + throw new FlinkRuntimeException("Unable to start the " + serverName + ". All ports in provided range are occupied."); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java index ab75cf462046d..fc4b2bc103be4 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java @@ -55,8 +55,8 @@ public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); - config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true); config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 2); + config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 2); config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 2); config.setString(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS)); config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS)); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java index 3f1a1fb5e1a68..6945cca6360aa 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java @@ -46,7 +46,7 @@ public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1); - config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true); + config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 1); config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1); config.setString(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS)); config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS)); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java index 0b2727c2f1feb..2775cd42c790d 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java @@ -58,7 +58,7 @@ public void testServerInitializationFailure() throws Throwable { // the expected exception along with the adequate message expectedEx.expect(FlinkRuntimeException.class); - expectedEx.expectMessage("Unable to start server Test Server 2. All ports in provided range are occupied."); + expectedEx.expectMessage("Unable to start the Test Server 2. All ports in provided range are occupied."); TestServer server1 = null; TestServer server2 = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java index cb43fbfc2cc7f..4fffacd42449b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java @@ -310,7 +310,7 @@ public void start() throws IOException { if (kvStateServer != null) { try { kvStateServer.start(); - LOG.info("Started Queryable State Data Server @ {}", kvStateServer.getServerAddress()); + LOG.info("Started the Queryable State Data Server @ {}", kvStateServer.getServerAddress()); } catch (Throwable ie) { kvStateServer.shutdown(); kvStateServer = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java index 5e6b7c5d2acb4..7823a1a889b12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.configuration.QueryableStateOptions; +import org.apache.flink.util.NetUtils; + import java.util.Iterator; import static org.apache.flink.util.Preconditions.checkArgument; @@ -27,44 +30,43 @@ */ public class QueryableStateConfiguration { - private final boolean enabled; - private final Iterator proxyPortRange; private final Iterator qserverPortRange; + private final int numProxyThreads; + + private final int numPQueryThreads; + private final int numServerThreads; - private final int numQueryThreads; + private final int numSQueryThreads; public QueryableStateConfiguration( - boolean enabled, Iterator proxyPortRange, Iterator qserverPortRange, + int numProxyThreads, + int numPQueryThreads, int numServerThreads, - int numQueryThreads) { + int numSQueryThreads) { - checkArgument(!enabled || (proxyPortRange != null && proxyPortRange.hasNext())); - checkArgument(!enabled || (qserverPortRange != null && qserverPortRange.hasNext())); + checkArgument(proxyPortRange != null && proxyPortRange.hasNext()); + checkArgument(qserverPortRange != null && qserverPortRange.hasNext()); + checkArgument(numProxyThreads >= 0, "queryable state number of server threads must be zero or larger"); + checkArgument(numPQueryThreads >= 0, "queryable state number of query threads must be zero or larger"); checkArgument(numServerThreads >= 0, "queryable state number of server threads must be zero or larger"); - checkArgument(numQueryThreads >= 0, "queryable state number of query threads must be zero or larger"); + checkArgument(numSQueryThreads >= 0, "queryable state number of query threads must be zero or larger"); - this.enabled = enabled; this.proxyPortRange = proxyPortRange; this.qserverPortRange = qserverPortRange; + this.numProxyThreads = numProxyThreads; + this.numPQueryThreads = numPQueryThreads; this.numServerThreads = numServerThreads; - this.numQueryThreads = numQueryThreads; + this.numSQueryThreads = numSQueryThreads; } // ------------------------------------------------------------------------ - /** - * Returns whether queryable state is enabled. - */ - public boolean isEnabled() { - return enabled; - } - /** * Returns the port range where the queryable state client proxy can listen. * See {@link org.apache.flink.configuration.QueryableStateOptions#PROXY_PORT_RANGE QueryableStateOptions.PROXY_PORT_RANGE}. @@ -85,7 +87,23 @@ public Iterator getStateServerPortRange() { * Returns the number of threads for the query server NIO event loop. * These threads only process network events and dispatch query requests to the query threads. */ - public int numServerThreads() { + public int numProxyServerThreads() { + return numProxyThreads; + } + + /** + * Returns the number of threads for the thread pool that performs the actual state lookup. + * These threads perform the actual state lookup. + */ + public int numProxyQueryThreads() { + return numPQueryThreads; + } + + /** + * Returns the number of threads for the query server NIO event loop. + * These threads only process network events and dispatch query requests to the query threads. + */ + public int numStateServerThreads() { return numServerThreads; } @@ -93,18 +111,19 @@ public int numServerThreads() { * Returns the number of threads for the thread pool that performs the actual state lookup. * These threads perform the actual state lookup. */ - public int numQueryThreads() { - return numQueryThreads; + public int numStateQueryThreads() { + return numSQueryThreads; } // ------------------------------------------------------------------------ @Override public String toString() { - return "QueryableStateConfiguration {" + - "enabled=" + enabled + - ", numServerThreads=" + numServerThreads + - ", numQueryThreads=" + numQueryThreads + + return "QueryableStateConfiguration{" + + "numProxyServerThreads=" + numProxyThreads + + ", numProxyQueryThreads=" + numPQueryThreads + + ", numStateServerThreads=" + numServerThreads + + ", numStateQueryThreads=" + numSQueryThreads + '}'; } @@ -114,6 +133,8 @@ public String toString() { * Gets the configuration describing the queryable state as deactivated. */ public static QueryableStateConfiguration disabled() { - return new QueryableStateConfiguration(false, null, null, 0, 0); + final Iterator proxyPorts = NetUtils.getPortRangeFromString(QueryableStateOptions.PROXY_PORT_RANGE.defaultValue()); + final Iterator serverPorts = NetUtils.getPortRangeFromString(QueryableStateOptions.SERVER_PORT_RANGE.defaultValue()); + return new QueryableStateConfiguration(proxyPorts, serverPorts, 0, 0, 0, 0); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index aed03f621cf8c..4daff05d05373 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -327,33 +327,35 @@ private static NetworkEnvironment createNetworkEnvironment( TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); KvStateRegistry kvStateRegistry = new KvStateRegistry(); - KvStateClientProxy kvClientProxy = null; - KvStateServer kvStateServer = null; - - if (taskManagerServicesConfiguration.getQueryableStateConfig().isEnabled()) { - QueryableStateConfiguration qsConfig = taskManagerServicesConfiguration.getQueryableStateConfig(); - - int numNetworkThreads = qsConfig.numServerThreads() == 0 ? - taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numServerThreads(); - - int numQueryThreads = qsConfig.numQueryThreads() == 0 ? - taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numQueryThreads(); - - kvClientProxy = QueryableStateUtils.createKvStateClientProxy( - taskManagerServicesConfiguration.getTaskManagerAddress(), - qsConfig.getProxyPortRange(), - numNetworkThreads, - numQueryThreads, - new DisabledKvStateRequestStats()); - - kvStateServer = QueryableStateUtils.createKvStateServer( - taskManagerServicesConfiguration.getTaskManagerAddress(), - qsConfig.getStateServerPortRange(), - numNetworkThreads, - numQueryThreads, - kvStateRegistry, - new DisabledKvStateRequestStats()); - } + + QueryableStateConfiguration qsConfig = taskManagerServicesConfiguration.getQueryableStateConfig(); + + int numProxyServerNetworkThreads = qsConfig.numProxyServerThreads() == 0 ? + taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numProxyServerThreads(); + + int numProxyServerQueryThreads = qsConfig.numProxyQueryThreads() == 0 ? + taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numProxyQueryThreads(); + + final KvStateClientProxy kvClientProxy = QueryableStateUtils.createKvStateClientProxy( + taskManagerServicesConfiguration.getTaskManagerAddress(), + qsConfig.getProxyPortRange(), + numProxyServerNetworkThreads, + numProxyServerQueryThreads, + new DisabledKvStateRequestStats()); + + int numStateServerNetworkThreads = qsConfig.numStateServerThreads() == 0 ? + taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numStateServerThreads(); + + int numStateServerQueryThreads = qsConfig.numStateQueryThreads() == 0 ? + taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numStateQueryThreads(); + + final KvStateServer kvStateServer = QueryableStateUtils.createKvStateServer( + taskManagerServicesConfiguration.getTaskManagerAddress(), + qsConfig.getStateServerPortRange(), + numStateServerNetworkThreads, + numStateServerQueryThreads, + kvStateRegistry, + new DisabledKvStateRequestStats()); // we start the network first, to make sure it can allocate its buffers first return new NetworkEnvironment( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 990fb2259829e..bae683b6ab350 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -400,23 +400,27 @@ public static boolean hasNewNetworkBufConf(final Configuration config) { * Creates the {@link QueryableStateConfiguration} from the given Configuration. */ private static QueryableStateConfiguration parseQueryableStateConfiguration(Configuration config) { - final boolean enabled = config.getBoolean(QueryableStateOptions.SERVER_ENABLE); - - if (enabled) { - final Iterator proxyPorts = NetUtils.getPortRangeFromString( - config.getString(QueryableStateOptions.PROXY_PORT_RANGE, - QueryableStateOptions.PROXY_PORT_RANGE.defaultValue())); - final Iterator serverPorts = NetUtils.getPortRangeFromString( - config.getString(QueryableStateOptions.SERVER_PORT_RANGE, - QueryableStateOptions.SERVER_PORT_RANGE.defaultValue())); - - final int numNetworkThreads = config.getInteger(QueryableStateOptions.SERVER_NETWORK_THREADS); - final int numQueryThreads = config.getInteger(QueryableStateOptions.SERVER_ASYNC_QUERY_THREADS); - return new QueryableStateConfiguration(true, proxyPorts, serverPorts, numNetworkThreads, numQueryThreads); - } - else { - return QueryableStateConfiguration.disabled(); - } + + final Iterator proxyPorts = NetUtils.getPortRangeFromString( + config.getString(QueryableStateOptions.PROXY_PORT_RANGE, + QueryableStateOptions.PROXY_PORT_RANGE.defaultValue())); + final Iterator serverPorts = NetUtils.getPortRangeFromString( + config.getString(QueryableStateOptions.SERVER_PORT_RANGE, + QueryableStateOptions.SERVER_PORT_RANGE.defaultValue())); + + final int numProxyServerNetworkThreads = config.getInteger(QueryableStateOptions.PROXY_NETWORK_THREADS); + final int numProxyServerQueryThreads = config.getInteger(QueryableStateOptions.PROXY_ASYNC_QUERY_THREADS); + + final int numStateServerNetworkThreads = config.getInteger(QueryableStateOptions.SERVER_NETWORK_THREADS); + final int numStateServerQueryThreads = config.getInteger(QueryableStateOptions.SERVER_ASYNC_QUERY_THREADS); + + return new QueryableStateConfiguration( + proxyPorts, + serverPorts, + numProxyServerNetworkThreads, + numProxyServerQueryThreads, + numStateServerNetworkThreads, + numStateServerQueryThreads); } /** diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index 89197e293d326..8ef2e364293f6 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -94,13 +94,6 @@ class LocalFlinkMiniCluster( config.addAll(userConfiguration) setMemory(config) initializeIOFormatClasses(config) - - // Disable queryable state server if nothing else is configured explicitly - if (!config.containsKey(QueryableStateOptions.SERVER_ENABLE.key())) { - LOG.info("Disabled queryable state server") - config.setBoolean(QueryableStateOptions.SERVER_ENABLE, false) - } - config } From 819995454611be6a85e2933318d053b2c25a18f7 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Mon, 6 Nov 2017 17:21:45 +0100 Subject: [PATCH 004/367] [FLINK-7822][QS][doc] Update Queryable State docs. --- docs/dev/stream/state/queryable_state.md | 265 ++++++++++++----------- 1 file changed, 144 insertions(+), 121 deletions(-) diff --git a/docs/dev/stream/state/queryable_state.md b/docs/dev/stream/state/queryable_state.md index 8012f67aaadac..af646dfa7a49b 100644 --- a/docs/dev/stream/state/queryable_state.md +++ b/docs/dev/stream/state/queryable_state.md @@ -32,38 +32,68 @@ under the License. likely that there will be breaking API changes on the client side in the upcoming Flink versions. -In a nutshell, this feature allows users to query Flink's managed partitioned state -(see [Working with State]({{ site.baseurl }}/dev/stream/state/state.html)) from outside of -Flink. For some scenarios, queryable state thus eliminates the need for distributed -operations/transactions with external systems such as key-value stores which are often the -bottleneck in practice. +In a nutshell, this feature exposes Flink's managed keyed (partitioned) state +(see [Working with State]({{ site.baseurl }}/dev/stream/state/state.html)) to the outside world and +allows the user to query a job's state from outside Flink. For some scenarios, queryable state +eliminates the need for distributed operations/transactions with external systems such as key-value +stores which are often the bottleneck in practice. In addition, this feature may be particularly +useful for debugging purposes.
- Attention: Queryable state accesses keyed state from a concurrent thread rather - than synchronizing with the operator and potentially blocking its operation. Since any state - backend using Java heap space, e.g. MemoryStateBackend or - FsStateBackend, does not work with copies when retrieving values but instead directly - references the stored values, read-modify-write patterns are unsafe and may cause the - queryable state server to fail due to concurrent modifications. - The RocksDBStateBackend is safe from these issues. + Attention: When querying a state object, that object is accessed from a concurrent + thread without any synchronization or copying. This is a design choice, as any of the above would lead + to increased job latency, which we wanted to avoid. Since any state backend using Java heap space, + e.g. MemoryStateBackend or FsStateBackend, does not work + with copies when retrieving values but instead directly references the stored values, read-modify-write + patterns are unsafe and may cause the queryable state server to fail due to concurrent modifications. + The RocksDBStateBackend is safe from these issues.
+## Architecture + +Before showing how to use the Queryable State, it is useful to briefly describe the entities that compose it. +The Queryable State feature consists of three main entities: + + 1. the `QueryableStateClient`, which (potentially) runs outside the Flink cluster and submits the user queries, + 2. the `QueryableStateClientProxy`, which runs on each `TaskManager` (*i.e.* inside the Flink cluster) and is responsible + for receiving the client's queries, fetching the requested state from the responsible Task Manager on his behalf, and + returning it to the client, and + 3. the `QueryableStateServer` which runs on each `TaskManager` and is responsible for serving the locally stored state. + +In a nutshell, the client will connect to one of the proxies and send a request for the state associated with a specific +key, `k`. As stated in [Working with State]({{ site.baseurl }}/dev/stream/state/state.html), keyed state is organized in +*Key Groups*, and each `TaskManager` is assigned a number of these key groups. To discover which `TaskManager` is +responsible for the key group holding `k`, the proxy will ask the `JobManager`. Based on the answer, the proxy will +then query the `QueryableStateServer` running on that `TaskManager` for the state associated with `k`, and forward the +response back to the client. + +## Activating Queryable State + +To enable queryable state on your Flink cluster, you just have to copy the +`flink-queryable-state-runtime{{ site.scala_version_suffix }}-{{site.version }}.jar` +from the `opt/` folder of your [Flink distribution](https://flink.apache.org/downloads.html "Apache Flink: Downloads"), +to the `lib/` folder. Otherwise, the queryable state feature is not enabled. + +To verify that your cluster is running with queryable state enabled, check the logs of any +task manager for the line: `"Started the Queryable State Proxy Server @ ..."`. + ## Making State Queryable -In order to make state queryable, the queryable state server first needs to be enabled globally -by setting the `query.server.enable` configuration parameter to `true` (current default). -Then appropriate state needs to be made queryable by using either +Now that you have activated queryable state on your cluster, it is time to see how to use it. In order for a state to +be visible to the outside world, it needs to be explicitly made queryable by using: -* a `QueryableStateStream`, a convenience object which behaves like a sink and offers incoming values as -queryable state, or -* `StateDescriptor#setQueryable(String queryableStateName)`, which makes the keyed state of an -operator queryable. +* either a `QueryableStateStream`, a convenience object which acts as a sink and offers its incoming values as queryable +state, or +* the `stateDescriptor.setQueryable(String queryableStateName)` method, which makes the keyed state represented by the + state descriptor, queryable. The following sections explain the use of these two approaches. ### Queryable State Stream -A `KeyedStream` may offer its values as queryable state by using the following methods: +Calling `.asQueryableState(stateName, stateDescriptor)` on a `KeyedStream` returns a `QueryableStateStream` which offers +its values as queryable state. Depending on the type of state, there are the following variants of the `asQueryableState()` +method: {% highlight java %} // ValueState @@ -91,18 +121,16 @@ QueryableStateStream asQueryableState( list which may not be cleaned up and thus will eventually consume too much memory. -A call to these methods returns a `QueryableStateStream`, which cannot be further transformed and -currently only holds the name as well as the value and key serializer for the queryable state -stream. It is comparable to a sink, and cannot be followed by further transformations. +The returned `QueryableStateStream` can be seen as a sink and **cannot** be further transformed. Internally, a +`QueryableStateStream` gets translated to an operator which uses all incoming records to update the queryable state +instance. The updating logic is implied by the type of the `StateDescriptor` provided in the `asQueryableState` call. +In a program like the following, all records of the keyed stream will be used to update the state instance via the +`ValueState.update(value)`: -Internally a `QueryableStateStream` gets translated to an operator which uses all incoming -records to update the queryable state instance. -In a program like the following, all records of the keyed stream will be used to update the state -instance, either via `ValueState#update(value)` or `AppendingState#add(value)`, depending on -the chosen state variant: {% highlight java %} stream.keyBy(0).asQueryableState("query-name") {% endhighlight %} + This acts like the Scala API's `flatMapWithState`. ### Managed Keyed State @@ -110,7 +138,7 @@ This acts like the Scala API's `flatMapWithState`. Managed keyed state of an operator (see [Using Managed Keyed State]({{ site.baseurl }}/dev/stream/state/state.html#using-managed-keyed-state)) can be made queryable by making the appropriate state descriptor queryable via -`StateDescriptor#setQueryable(String queryableStateName)`, as in the example below: +`StateDescriptor.setQueryable(String queryableStateName)`, as in the example below: {% highlight java %} ValueStateDescriptor> descriptor = new ValueStateDescriptor<>( @@ -119,61 +147,79 @@ ValueStateDescriptor> descriptor = Tuple2.of(0L, 0L)); // default value of the state, if nothing was set descriptor.setQueryable("query-name"); // queryable state name {% endhighlight %} +
- Note: The `queryableStateName` parameter may be chosen arbitrarily and is only + Note: The queryableStateName parameter may be chosen arbitrarily and is only used for queries. It does not have to be identical to the state's own name.
+This variant has no limitations as to which type of state can be made queryable. This means that this can be used for +any `ValueState`, `ReduceState`, `ListState`, `MapState`, `AggregatingState`, and the currently deprecated `FoldingState`. ## Querying State -The `QueryableStateClient` helper class may be used for queries against the `KvState` instances that -serve the state internally. It needs to be set up with a valid `JobManager` address and port and is -created as follows: +So far, you have set up your cluster to run with queryable state and you have declared (some of) your state as +queryable. Now it is time to see how to query this state. -{% highlight java %} -final Configuration config = new Configuration(); -config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, queryAddress); -config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, queryPort); +For this you can use the `QueryableStateClient` helper class. This is available in the `flink-queryable-state-client` +jar which you have to explicitly include as a dependency in the `pom.xml` of your project, as shown below: -final HighAvailabilityServices highAvailabilityServices = - HighAvailabilityServicesUtils.createHighAvailabilityServices( - config, - Executors.newSingleThreadScheduledExecutor(), - HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION); +
+{% highlight xml %} + + org.apache.flink + flink-queryable-state-client-java_{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} +
+ +For more on this, you can check how to [set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink.html). + +The `QueryableStateClient` will submit your query to the internal proxy, which will then process your query and return +the final result. The only requirement to initialize the client is to provide a valid `TaskManager` hostname (remember +that there is a queryable state proxy running on each task manager) and the port where the proxy listens. More on how +to configure the proxy and state server port(s) in the [Configuration Section](#Configuration). -QueryableStateClient client = new QueryableStateClient(config, highAvailabilityServices); +{% highlight java %} +QueryableStateClient client = new QueryableStateClient(tmHostname, proxyPort); {% endhighlight %} -The query method is this: +With the client ready, to query a state of type `V`, associated with a key of type `K`, you can use the method: {% highlight java %} -Future getKvState( - JobID jobID, - String queryableStateName, - int keyHashCode, - byte[] serializedKeyAndNamespace) +CompletableFuture getKvState( + final JobID jobId, + final String queryableStateName, + final K key, + final TypeInformation keyTypeInfo, + final StateDescriptor stateDescriptor) {% endhighlight %} -A call to this method returns a `Future` eventually holding the serialized state value for the -queryable state instance identified by `queryableStateName` of the job with ID `jobID`. The -`keyHashCode` is the hash code of the key as returned by `Object.hashCode()` and the -`serializedKeyAndNamespace` is the serialized key and namespace. +The above returns a `CompletableFuture` eventually holding the state value for the queryable state instance identified +by `queryableStateName` of the job with ID `jobID`. The `key` is the key whose state you are interested in and the +`keyTypeInfo` will tell Flink how to serialize/deserialize it. Finally, the `stateDescriptor` contains the necessary +information about the requested state, namely its type (`Value`, `Reduce`, etc) and the necessary information on how +to serialize/deserialize it. + +The careful reader will notice that the returned future contains a value of type `S`, *i.e.* a `State` object containing +the actual value. This can be any of the state types supported by Flink: `ValueState`, `ReduceState`, `ListState`, `MapState`, +`AggregatingState`, and the currently deprecated `FoldingState`. + +
+ Note:These state objects do not allow modifications to the contained state. You can use them to get + the actual value of the state, e.g. using valueState.get(), or iterate over + the contained entries, e.g. using the mapState.entries(), but you cannot + modify them. As an example, calling the add() method on a returned list state will throw an + UnsupportedOperationException. +
+
Note: The client is asynchronous and can be shared by multiple threads. It needs - to be shutdown via QueryableStateClient#shutdown() when unused in order to free + to be shutdown via QueryableStateClient.shutdown() when unused in order to free resources.
-The current implementation is still pretty low-level in the sense that it only works with -serialized data both for providing the key/namespace and the returned results. It is the -responsibility of the user (or some follow-up utilities) to set up the serializers for this. The -nice thing about this is that the query services don't have to get into the business of worrying -about any class loading issues etc. - -There are some serialization utils for key/namespace and value serialization included in -`KvStateRequestSerializer`. - ### Example The following example extends the `CountWindowAverage` example @@ -183,7 +229,7 @@ by making it queryable and showing how to query this value: {% highlight java %} public class CountWindowAverage extends RichFlatMapFunction, Tuple2> { - private transient ValueState> sum; + private transient ValueState> sum; // a tuple containing the count and the sum @Override public void flatMap(Tuple2 input, Collector> out) throws Exception { @@ -214,74 +260,51 @@ public class CountWindowAverage extends RichFlatMapFunction, Once used in a job, you can retrieve the job ID and then query any key's current state from this operator: {% highlight java %} -final Configuration config = new Configuration(); -config.setString(JobManagerOptions.ADDRESS, queryAddress); -config.setInteger(JobManagerOptions.PORT, queryPort); - -final HighAvailabilityServices highAvailabilityServices = - HighAvailabilityServicesUtils.createHighAvailabilityServices( - config, - Executors.newSingleThreadScheduledExecutor(), - HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION); - -QueryableStateClient client = new QueryableStateClient(config, highAvailabilityServices); - -final TypeSerializer keySerializer = - TypeInformation.of(new TypeHint() {}).createSerializer(new ExecutionConfig()); -final TypeSerializer> valueSerializer = - TypeInformation.of(new TypeHint>() {}).createSerializer(new ExecutionConfig()); - -final byte[] serializedKey = - KvStateRequestSerializer.serializeKeyAndNamespace( - key, keySerializer, - VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE); - -Future serializedResult = - client.getKvState(jobId, "query-name", key.hashCode(), serializedKey); - -// now wait for the result and return it -final FiniteDuration duration = new FiniteDuration(1, TimeUnit.SECONDS); -byte[] serializedValue = Await.result(serializedResult, duration); -Tuple2 value = - KvStateRequestSerializer.deserializeValue(serializedValue, valueSerializer); -{% endhighlight %} - -### Note for Scala Users - -Please use the available Scala extensions when creating the `TypeSerializer` instances. Add the following import: +QueryableStateClient client = new QueryableStateClient(tmHostname, proxyPort); -```scala -import org.apache.flink.streaming.api.scala._ -``` - -Now you can create the type serializers as follows: - -```scala -val keySerializer = createTypeInformation[Long] - .createSerializer(new ExecutionConfig) -``` - -If you don't do this, you can run into mismatches between the serializers used in the Flink job and in your client code, because types like `scala.Long` cannot be captured at runtime. +// the state descriptor of the state to be fetched. +ValueStateDescriptor> descriptor = + new ValueStateDescriptor<>( + "average", + TypeInformation.of(new TypeHint>() {}), + Tuple2.of(0L, 0L)); + +CompletableFuture>> resultFuture = + client.getKvState(jobId, "query-name", key, BasicTypeInfo.LONG_TYPE_INFO, descriptor); + +// now handle the returned value +resultFuture.thenAccept(response -> { + try { + Tuple2 res = response.get(); + } catch (Exception e) { + e.printStackTrace(); + } +}); +{% endhighlight %} ## Configuration The following configuration parameters influence the behaviour of the queryable state server and client. They are defined in `QueryableStateOptions`. -### Server -* `query.server.enable`: flag to indicate whether to start the queryable state server -* `query.server.port`: port to bind to the internal `KvStateServer` (0 => pick random available port) -* `query.server.network-threads`: number of network (event loop) threads for the `KvStateServer` (0 => #slots) -* `query.server.query-threads`: number of asynchronous query threads for the `KvStateServerHandler` (0 => #slots). +### State Server +* `query.server.ports`: the server port range of the queryable state server. This is useful to avoid port clashes if more + than 1 task managers run on the same machine. The specified range can be: a port: "9123", a range of ports: "50100-50200", + or a list of ranges and or points: "50100-50200,50300-50400,51234". The default port is 9067. +* `query.server.network-threads`: number of network (event loop) threads receiving incoming requests for the state server (0 => #slots) +* `query.server.query-threads`: number of threads handling/serving incoming requests for the state server (0 => #slots). + -### Client (`QueryableStateClient`) -* `query.client.network-threads`: number of network (event loop) threads for the `KvStateClient` (0 => number of available cores) -* `query.client.lookup.num-retries`: number of retries on location lookup failures -* `query.client.lookup.retry-delay`: retry delay on location lookup failures (millis) +### Proxy +* `query.proxy.ports`: the server port range of the queryable state proxy. This is useful to avoid port clashes if more + than 1 task managers run on the same machine. The specified range can be: a port: "9123", a range of ports: "50100-50200", + or a list of ranges and or points: "50100-50200,50300-50400,51234". The default port is 9069. +* `query.proxy.network-threads`: number of network (event loop) threads receiving incoming requests for the client proxy (0 => #slots) +* `query.proxy.query-threads`: number of threads handling/serving incoming requests for the client proxy (0 => #slots). ## Limitations -* The queryable state life-cycle is bound to the life-cycle of the job, e.g. tasks register +* The queryable state life-cycle is bound to the life-cycle of the job, *e.g.* tasks register queryable state on startup and unregister it on disposal. In future versions, it is desirable to decouple this in order to allow queries after a task finishes, and to speed up recovery via state replication. From 564c9934fd3aaba462a7415788b3d55486146f9b Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 7 Nov 2017 18:27:16 +0100 Subject: [PATCH 005/367] [hotfix] Use correct commit id in GenericWriteAheadSink.notifyCheckpoint --- .../streaming/runtime/operators/GenericWriteAheadSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericWriteAheadSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericWriteAheadSink.java index 3dc8857f7728c..370d021dc32fd 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericWriteAheadSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericWriteAheadSink.java @@ -230,7 +230,7 @@ public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception { in), serializer), serializer), - checkpointId, + pastCheckpointId, timestamp); if (success) { // in case the checkpoint was successfully committed, From 3cbf467ebdf639df4d7d4da78b7bc2929aa4b5d9 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 6 Nov 2017 14:03:16 +0100 Subject: [PATCH 006/367] [hotfix][kafka] Extract TransactionalIdsGenerator class from FlinkKafkaProducer011 This is pure refactor without any functional changes. --- .../kafka/FlinkKafkaProducer011.java | 53 +++++------- .../internal/TransactionalIdsGenerator.java | 81 +++++++++++++++++++ 2 files changed, 99 insertions(+), 35 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 873ef08f71205..031001983810b 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -37,6 +37,7 @@ import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer; +import org.apache.flink.streaming.connectors.kafka.internal.TransactionalIdsGenerator; import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; @@ -59,7 +60,6 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.serialization.ByteArraySerializer; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,8 +81,6 @@ import java.util.concurrent.BlockingDeque; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.LongStream; -import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -182,6 +180,11 @@ public enum Semantic { */ private transient ListState nextTransactionalIdHintState; + /** + * Generator for Transactional IDs. + */ + private transient TransactionalIdsGenerator transactionalIdsGenerator; + /** * Hint for picking next transactional id. */ @@ -785,6 +788,11 @@ public void initializeState(FunctionInitializationContext context) throws Except nextTransactionalIdHintState = context.getOperatorStateStore().getUnionListState( NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR); + transactionalIdsGenerator = new TransactionalIdsGenerator( + getRuntimeContext().getTaskName(), + getRuntimeContext().getIndexOfThisSubtask(), + kafkaProducersPoolSize, + SAFE_SCALE_DOWN_FACTOR); if (semantic != Semantic.EXACTLY_ONCE) { nextTransactionalIdHint = null; @@ -803,15 +811,8 @@ public void initializeState(FunctionInitializationContext context) throws Except // (1) the first execution of this application // (2) previous execution has failed before first checkpoint completed // - // in case of (2) we have to abort all previous transactions, but we don't know was the parallelism used - // then, so we must guess using current configured pool size, current parallelism and - // SAFE_SCALE_DOWN_FACTOR - long abortTransactionalIdStart = getRuntimeContext().getIndexOfThisSubtask(); - long abortTransactionalIdEnd = abortTransactionalIdStart + 1; - - abortTransactionalIdStart *= kafkaProducersPoolSize * SAFE_SCALE_DOWN_FACTOR; - abortTransactionalIdEnd *= kafkaProducersPoolSize * SAFE_SCALE_DOWN_FACTOR; - abortTransactions(LongStream.range(abortTransactionalIdStart, abortTransactionalIdEnd)); + // in case of (2) we have to abort all previous transactions + abortTransactions(transactionalIdsGenerator.generateIdsToAbort()); } else { nextTransactionalIdHint = transactionalIdHints.get(0); } @@ -834,16 +835,7 @@ protected Optional initializeUserContext() { private Set generateNewTransactionalIds() { checkState(nextTransactionalIdHint != null, "nextTransactionalIdHint must be present for EXACTLY_ONCE"); - // range of available transactional ids is: - // [nextFreeTransactionalId, nextFreeTransactionalId + parallelism * kafkaProducersPoolSize) - // loop below picks in a deterministic way a subrange of those available transactional ids based on index of - // this subtask - int subtaskId = getRuntimeContext().getIndexOfThisSubtask(); - Set transactionalIds = new HashSet<>(); - for (int i = 0; i < kafkaProducersPoolSize; i++) { - long transactionalId = nextTransactionalIdHint.nextFreeTransactionalId + subtaskId * kafkaProducersPoolSize + i; - transactionalIds.add(generateTransactionalId(transactionalId)); - } + Set transactionalIds = transactionalIdsGenerator.generateIdsToUse(nextTransactionalIdHint.nextFreeTransactionalId); LOG.info("Generated new transactionalIds {}", transactionalIds); return transactionalIds; } @@ -862,7 +854,7 @@ private void cleanUpUserContext() { if (!getUserContext().isPresent()) { return; } - abortTransactions(getUserContext().get().transactionalIds.stream()); + abortTransactions(getUserContext().get().transactionalIds); } private void resetAvailableTransactionalIdsPool(Collection transactionalIds) { @@ -874,22 +866,13 @@ private void resetAvailableTransactionalIdsPool(Collection transactional // ----------------------------------- Utilities -------------------------- - private void abortTransactions(LongStream transactionalIds) { - abortTransactions(transactionalIds.mapToObj(this::generateTransactionalId)); - } - - private void abortTransactions(Stream transactionalIds) { - transactionalIds.forEach(transactionalId -> { + private void abortTransactions(Set transactionalIds) { + for (String transactionalId : transactionalIds) { try (FlinkKafkaProducer kafkaProducer = initTransactionalProducer(transactionalId, false)) { kafkaProducer.initTransactions(); } - }); - } - - private String generateTransactionalId(long transactionalId) { - String transactionalIdFormat = getRuntimeContext().getTaskName() + "-%d"; - return String.format(transactionalIdFormat, transactionalId); + } } int getTransactionCoordinatorId() { diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java new file mode 100644 index 0000000000000..eabdad9bc0261 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java @@ -0,0 +1,81 @@ +/* + * 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.flink.streaming.connectors.kafka.internal; + +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Class responsible for generating transactional ids to use when communicating with Kafka. + */ +public class TransactionalIdsGenerator { + private final String prefix; + private final int subtaskIndex; + private final int poolSize; + private final int safeScaleDownFactor; + + public TransactionalIdsGenerator( + String prefix, + int subtaskIndex, + int poolSize, + int safeScaleDownFactor) { + this.prefix = checkNotNull(prefix); + this.subtaskIndex = subtaskIndex; + this.poolSize = poolSize; + this.safeScaleDownFactor = safeScaleDownFactor; + } + + /** + * Range of available transactional ids to use is: + * [nextFreeTransactionalId, nextFreeTransactionalId + parallelism * kafkaProducersPoolSize) + * loop below picks in a deterministic way a subrange of those available transactional ids based on index of + * this subtask. + */ + public Set generateIdsToUse(long nextFreeTransactionalId) { + Set transactionalIds = new HashSet<>(); + for (int i = 0; i < poolSize; i++) { + long transactionalId = nextFreeTransactionalId + subtaskIndex * poolSize + i; + transactionalIds.add(generateTransactionalId(transactionalId)); + } + return transactionalIds; + } + + /** + * If we have to abort previous transactional id in case of restart after a failure BEFORE first checkpoint + * completed, we don't know what was the parallelism used in previous attempt. In that case we must guess the ids + * range to abort based on current configured pool size, current parallelism and safeScaleDownFactor. + */ + public Set generateIdsToAbort() { + long abortTransactionalIdStart = subtaskIndex; + long abortTransactionalIdEnd = abortTransactionalIdStart + 1; + + abortTransactionalIdStart *= poolSize * safeScaleDownFactor; + abortTransactionalIdEnd *= poolSize * safeScaleDownFactor; + return LongStream.range(abortTransactionalIdStart, abortTransactionalIdEnd) + .mapToObj(this::generateTransactionalId) + .collect(Collectors.toSet()); + } + + private String generateTransactionalId(long transactionalId) { + return String.format(prefix + "-%d", transactionalId); + } +} From 460e27aeb5e246aff0f8137448441c315123608c Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 6 Nov 2017 14:14:01 +0100 Subject: [PATCH 007/367] [FLINK-7978][kafka] Ensure that transactional ids will never clash Previously transactional ids to use and to abort could clash between subtasks. This could lead to a race condition between initialization and writting the data, where one subtask is still initializing/aborting some transactional id while different subtask is already trying to write the data using the same transactional id. --- .../kafka/FlinkKafkaProducer011.java | 1 + .../internal/TransactionalIdsGenerator.java | 32 +++++--- .../TransactionalIdsGeneratorTest.java | 79 +++++++++++++++++++ 3 files changed, 102 insertions(+), 10 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGeneratorTest.java diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 031001983810b..08599d82e3914 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -791,6 +791,7 @@ public void initializeState(FunctionInitializationContext context) throws Except transactionalIdsGenerator = new TransactionalIdsGenerator( getRuntimeContext().getTaskName(), getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getNumberOfParallelSubtasks(), kafkaProducersPoolSize, SAFE_SCALE_DOWN_FACTOR); diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java index eabdad9bc0261..2c4e6c9028cec 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java @@ -19,27 +19,42 @@ import java.util.HashSet; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.LongStream; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** * Class responsible for generating transactional ids to use when communicating with Kafka. + * + *

It guarantees that: + *

    + *
  • generated ids to use will never clash with ids to use from different subtasks + *
  • generated ids to abort will never clash with ids to abort from different subtasks + *
  • generated ids to use will never clash with ids to abort from different subtasks + *
+ * In other words, any particular generated id will always be assigned to one and only one subtask. */ public class TransactionalIdsGenerator { private final String prefix; private final int subtaskIndex; + private final int totalNumberOfSubtasks; private final int poolSize; private final int safeScaleDownFactor; public TransactionalIdsGenerator( String prefix, int subtaskIndex, + int totalNumberOfSubtasks, int poolSize, int safeScaleDownFactor) { + checkArgument(subtaskIndex < totalNumberOfSubtasks); + checkArgument(poolSize > 0); + checkArgument(safeScaleDownFactor > 0); + checkArgument(subtaskIndex >= 0); + this.prefix = checkNotNull(prefix); this.subtaskIndex = subtaskIndex; + this.totalNumberOfSubtasks = totalNumberOfSubtasks; this.poolSize = poolSize; this.safeScaleDownFactor = safeScaleDownFactor; } @@ -65,14 +80,11 @@ public Set generateIdsToUse(long nextFreeTransactionalId) { * range to abort based on current configured pool size, current parallelism and safeScaleDownFactor. */ public Set generateIdsToAbort() { - long abortTransactionalIdStart = subtaskIndex; - long abortTransactionalIdEnd = abortTransactionalIdStart + 1; - - abortTransactionalIdStart *= poolSize * safeScaleDownFactor; - abortTransactionalIdEnd *= poolSize * safeScaleDownFactor; - return LongStream.range(abortTransactionalIdStart, abortTransactionalIdEnd) - .mapToObj(this::generateTransactionalId) - .collect(Collectors.toSet()); + Set idsToAbort = new HashSet<>(); + for (int i = 0; i < safeScaleDownFactor; i++) { + idsToAbort.addAll(generateIdsToUse(i * poolSize * totalNumberOfSubtasks)); + } + return idsToAbort; } private String generateTransactionalId(long transactionalId) { diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGeneratorTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGeneratorTest.java new file mode 100644 index 0000000000000..7ad377986eeaa --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGeneratorTest.java @@ -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. + */ + +package org.apache.flink.streaming.connectors.kafka.internal; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link TransactionalIdsGenerator}. + */ +public class TransactionalIdsGeneratorTest { + private static final int POOL_SIZE = 3; + private static final int SAFE_SCALE_DOWN_FACTOR = 3; + private static final int SUBTASKS_COUNT = 5; + + @Test + public void testGenerateIdsToUse() { + TransactionalIdsGenerator generator = new TransactionalIdsGenerator("test", 2, SUBTASKS_COUNT, POOL_SIZE, SAFE_SCALE_DOWN_FACTOR); + + assertEquals( + new HashSet<>(Arrays.asList("test-42", "test-43", "test-44")), + generator.generateIdsToUse(36)); + } + + /** + * Ids to abort and to use should never clash between subtasks. + */ + @Test + public void testGeneratedIdsDoNotClash() { + List> idsToAbort = new ArrayList<>(); + List> idsToUse = new ArrayList<>(); + + for (int subtask = 0; subtask < SUBTASKS_COUNT; subtask++) { + TransactionalIdsGenerator generator = new TransactionalIdsGenerator("test", subtask, SUBTASKS_COUNT, POOL_SIZE, SAFE_SCALE_DOWN_FACTOR); + idsToUse.add(generator.generateIdsToUse(0)); + idsToAbort.add(generator.generateIdsToAbort()); + } + + for (int subtask1 = 0; subtask1 < SUBTASKS_COUNT; subtask1++) { + for (int subtask2 = 0; subtask2 < SUBTASKS_COUNT; subtask2++) { + if (subtask2 == subtask1) { + continue; + } + assertDisjoint(idsToAbort.get(subtask2), idsToAbort.get(subtask1)); + assertDisjoint(idsToUse.get(subtask2), idsToUse.get(subtask1)); + assertDisjoint(idsToAbort.get(subtask2), idsToUse.get(subtask1)); + } + } + } + + private void assertDisjoint(Set first, Set second) { + HashSet actual = new HashSet<>(first); + actual.retainAll(second); + assertEquals(Collections.emptySet(), actual); + } +} From b677c8d69b81fb3594798ba2761fdb7e2edea5db Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Tue, 7 Nov 2017 23:43:45 +0100 Subject: [PATCH 008/367] [hotfix] [docs] Improve Supported Types section of Table API & SQL docs. --- docs/dev/table/sql.md | 63 +++++++++++------------ docs/dev/table/tableApi.md | 102 +++++++++++++++++-------------------- 2 files changed, 77 insertions(+), 88 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 22eca0d31dd66..8525ecfb720d9 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -800,7 +800,7 @@ val result4 = tableEnv.sqlQuery( Data Types ---------- -The SQL runtime is built on top of Flink's DataSet and DataStream APIs. Internally, it also uses Flink's `TypeInformation` to distinguish between types. The SQL support does not include all Flink types so far. All supported simple types are listed in `org.apache.flink.table.api.Types`. The following table summarizes the relation between SQL Types, Table API types, and the resulting Java class. +The SQL runtime is built on top of Flink's DataSet and DataStream APIs. Internally, it also uses Flink's `TypeInformation` to define data types. Fully supported types are listed in `org.apache.flink.table.api.Types`. The following table summarizes the relation between SQL Types, Table API types, and the resulting Java class. | Table API | SQL | Java type | | :--------------------- | :-------------------------- | :--------------------- | @@ -823,21 +823,14 @@ The SQL runtime is built on top of Flink's DataSet and DataStream APIs. Internal | `Types.MAP` | `MAP` | `java.util.HashMap` | | `Types.MULTISET` | `MULTISET` | e.g. `java.util.HashMap` for a multiset of `String` | - -Advanced types such as generic types, composite types (e.g. POJOs or Tuples), and array types (object or primitive arrays) can be fields of a row. - -Generic types are treated as a black box within Table API and SQL yet. - -Composite types, however, are fully supported types where fields of a composite type can be accessed using the `.get()` operator in Table API and dot operator (e.g. `MyTable.pojoColumn.myField`) in SQL. Composite types can also be flattened using `.flatten()` in Table API or `MyTable.pojoColumn.*` in SQL. - -Array types can be accessed using the `myArray.at(1)` operator in Table API and `myArray[1]` operator in SQL. Array literals can be created using `array(1, 2, 3)` in Table API and `ARRAY[1, 2, 3]` in SQL. +Generic types and composite types (e.g., POJOs or Tuples) can be fields of a row as well. Generic types are treated as a black box and can be passed on or processed by [user-defined functions](udfs.html). Composite types can be accessed with [built-in functions](#built-in-functions) (see *Value access functions* section). {% top %} Built-In Functions ------------------ -Both the Table API and SQL come with a set of built-in functions for data transformations. This section gives a brief overview of the available functions so far. +Flink's SQL support comes with a set of built-in functions for data transformations. This section gives a brief overview of the available functions. - - - - {% highlight text %} -array ‘[’ index ‘]’ -{% endhighlight %} - - -

Returns the element at a particular position in an array. The index starts at 1.

- - - - - - {% highlight text %} -ARRAY ‘[’ value [, value ]* ‘]’ -{% endhighlight %} - - -

Creates an array from a list of values.

- - +--> @@ -2283,6 +2255,18 @@ tableName.compositeType.* + + + + + + + + + + + - - - - - - - - - - + + + + + + + + + + - - - - - - - - - - + + + + + + + + + + From 5af710080eb72d23d8d2f6a77d1825f3d8a009ae Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 7 Nov 2017 11:40:15 +0100 Subject: [PATCH 010/367] [FLINK-8004][metrics][docs] Fix usage examples --- docs/monitoring/metrics.md | 54 +++++++++++++++++++++++++++++--------- 1 file changed, 41 insertions(+), 13 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 4c423bedcd3dc..0bcab55704bcf 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -43,7 +43,7 @@ You can create and register a `Counter` by calling `counter(String name)` on a ` {% highlight java %} -public class MyMapper extends RichMapFunction { +public class MyMapper extends RichMapFunction { private Counter counter; @Override @@ -53,8 +53,10 @@ public class MyMapper extends RichMapFunction { .counter("myCounter"); } - @public Integer map(String value) throws Exception { + @Override + public String map(String value) throws Exception { this.counter.inc(); + return value; } } @@ -64,7 +66,7 @@ Alternatively you can also use your own `Counter` implementation: {% highlight java %} -public class MyMapper extends RichMapFunction { +public class MyMapper extends RichMapFunction { private Counter counter; @Override @@ -73,6 +75,12 @@ public class MyMapper extends RichMapFunction { .getMetricGroup() .counter("myCustomCounter", new CustomCounter()); } + + @Override + public String map(String value) throws Exception { + this.counter.inc(); + return value; + } } {% endhighlight %} @@ -87,8 +95,8 @@ You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `Metr
{% highlight java %} -public class MyMapper extends RichMapFunction { - private int valueToExpose; +public class MyMapper extends RichMapFunction { + private int valueToExpose = 0; @Override public void open(Configuration config) { @@ -101,6 +109,12 @@ public class MyMapper extends RichMapFunction { } }); } + + @Override + public String map(String value) throws Exception { + valueToExpose++; + return value; + } } {% endhighlight %} @@ -109,15 +123,19 @@ public class MyMapper extends RichMapFunction {
{% highlight scala %} -public class MyMapper extends RichMapFunction[String,Int] { - val valueToExpose = 5 +public class MyMapper extends RichMapFunction[String,String] { + val valueToExpose = 0 override def open(parameters: Configuration): Unit = { getRuntimeContext() .getMetricGroup() .gauge("MyGauge", ScalaGauge[Int]( () => valueToExpose ) ) } - ... + + override def map(value: String): String = { + valueToExpose += 1 + value + } } {% endhighlight %} @@ -133,7 +151,7 @@ A `Histogram` measures the distribution of long values. You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`. {% highlight java %} -public class MyMapper extends RichMapFunction { +public class MyMapper extends RichMapFunction { private Histogram histogram; @Override @@ -143,8 +161,10 @@ public class MyMapper extends RichMapFunction { .histogram("myHistogram", new MyHistogram()); } - @public Integer map(Long value) throws Exception { + @Override + public Long map(Long value) throws Exception { this.histogram.update(value); + return value; } } {% endhighlight %} @@ -183,7 +203,7 @@ A `Meter` measures an average throughput. An occurrence of an event can be regis You can register a meter by calling `meter(String name, Meter meter)` on a `MetricGroup`. {% highlight java %} -public class MyMapper extends RichMapFunction { +public class MyMapper extends RichMapFunction { private Meter meter; @Override @@ -193,8 +213,10 @@ public class MyMapper extends RichMapFunction { .meter("myMeter", new MyMeter()); } - @public Integer map(Long value) throws Exception { + @Override + public Long map(Long value) throws Exception { this.meter.markEvent(); + return value; } } {% endhighlight %} @@ -212,7 +234,7 @@ To use this wrapper add the following dependency in your `pom.xml`: You can then register a Codahale/DropWizard meter like this: {% highlight java %} -public class MyMapper extends RichMapFunction { +public class MyMapper extends RichMapFunction { private Meter meter; @Override @@ -223,6 +245,12 @@ public class MyMapper extends RichMapFunction { .getMetricGroup() .meter("myMeter", new DropwizardMeterWrapper(meter)); } + + @Override + public Long map(Long value) throws Exception { + this.meter.markEvent(); + return value; + } } {% endhighlight %} From 49dc380697627189f6ac2e8bf5a084ac85c21ed5 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 7 Nov 2017 15:36:49 +0100 Subject: [PATCH 011/367] [FLINK-8010][build] Bump remaining flink-shaded versions --- pom.xml | 45 +++++++++++++++++++++++---------------------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/pom.xml b/pom.xml index 5a13012e12793..c466626d2ebf2 100644 --- a/pom.xml +++ b/pom.xml @@ -98,6 +98,7 @@ under the License. ${flink.forkCount} true log4j-test.properties + 2.0 18.0 2.4.20 1.8 @@ -228,16 +229,36 @@ under the License. --> + + org.apache.flink + flink-shaded-asm + 5.0.4-${flink.shaded.version} + + org.apache.flink flink-shaded-guava - 18.0-1.0 + 18.0-${flink.shaded.version} org.apache.flink flink-shaded-jackson - ${jackson.version}-2.0 + ${jackson.version}-${flink.shaded.version} + + + + org.apache.flink + flink-shaded-netty + + 4.0.27.Final-${flink.shaded.version} @@ -271,12 +292,6 @@ under the License. 3.3.2 - - org.apache.flink - flink-shaded-asm - 5.0.4-1.0 - - org.xerial.snappy snappy-java @@ -486,20 +501,6 @@ under the License. - - org.apache.flink - flink-shaded-netty - - 4.0.27.Final-1.0 - - + + com.typesafe.akka + akka-stream_${scala.binary.version} + + + + + com.typesafe.akka + akka-protobuf_${scala.binary.version} + + com.typesafe.akka akka-slf4j_${scala.binary.version} @@ -429,8 +443,6 @@ under the License. shade - - true org.apache.flink:flink-shaded-curator diff --git a/pom.xml b/pom.xml index c466626d2ebf2..d9d0a686c37c2 100644 --- a/pom.xml +++ b/pom.xml @@ -437,6 +437,22 @@ under the License. + + + com.typesafe.akka + akka-stream_${scala.binary.version} + ${akka.version} + + + + + com.typesafe.akka + akka-protobuf_${scala.binary.version} + ${akka.version} + + com.typesafe.akka akka-slf4j_${scala.binary.version} From a126bd3e7d9614749f61692fbb53c5b284f17091 Mon Sep 17 00:00:00 2001 From: Xpray Date: Fri, 3 Nov 2017 15:19:42 +0800 Subject: [PATCH 014/367] [FLINK-7971] [table] Fix potential NPE in non-windowed aggregation. This closes #4941. --- .../table/runtime/aggregate/GroupAggProcessFunction.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index 91c379f393a6f..397032003ec81 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -97,11 +97,14 @@ class GroupAggProcessFunction( if (null == accumulators) { firstRow = true accumulators = function.createAccumulators() - inputCnt = 0L } else { firstRow = false } + if (null == inputCnt) { + inputCnt = 0L + } + // Set group keys value to the final output function.setForwardedFields(input, newRow.row) function.setForwardedFields(input, prevRow.row) From 8c60f97a43defc57bb1bfaabdd6081b329db53b8 Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Tue, 31 Oct 2017 11:05:38 -0700 Subject: [PATCH 015/367] [FLINK-7922] [table] Fix FlinkTypeFactory.leastRestrictive for composite types. This closes #4929. --- .../table/calcite/FlinkTypeFactory.scala | 39 +++++---- .../api/batch/sql/SetOperatorsTest.scala | 55 ++++++++++++ .../api/batch/table/SetOperatorsTest.scala | 59 +++++++++++++ .../table/api/stream/sql/UnionTest.scala | 83 +++++++++++++++++++ .../expressions/ScalarOperatorsTest.scala | 5 +- .../utils/ScalarOperatorsTestBase.scala | 9 +- .../stream/table/SetOperatorsITCase.scala | 20 ++++- .../table/runtime/utils/CommonTestData.scala | 6 ++ .../flink/table/utils/TableTestBase.scala | 26 ++++++ 9 files changed, 280 insertions(+), 22 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala index 2874e6116606e..04fab76ba181e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala @@ -268,30 +268,37 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp override def leastRestrictive(types: util.List[RelDataType]): RelDataType = { val type0 = types.get(0) if (type0.getSqlTypeName != null) { - val resultType = resolveAny(types) - if (resultType != null) { - return resultType + val resultType = resolveAllIdenticalTypes(types) + if (resultType.isDefined) { + // result type for identical types + return resultType.get } } + // fall back to super super.leastRestrictive(types) } - private def resolveAny(types: util.List[RelDataType]): RelDataType = { + private def resolveAllIdenticalTypes(types: util.List[RelDataType]): Option[RelDataType] = { val allTypes = types.asScala - val hasAny = allTypes.exists(_.getSqlTypeName == SqlTypeName.ANY) - if (hasAny) { - val head = allTypes.head - // only allow ANY with exactly the same GenericRelDataType for all types - if (allTypes.forall(_ == head)) { - val nullable = allTypes.exists( - sqlType => sqlType.isNullable || sqlType.getSqlTypeName == SqlTypeName.NULL - ) - createTypeWithNullability(head, nullable) - } else { + + val head = allTypes.head + // check if all types are the same + if (allTypes.forall(_ == head)) { + // types are the same, check nullability + val nullable = allTypes + .exists(sqlType => sqlType.isNullable || sqlType.getSqlTypeName == SqlTypeName.NULL) + // return type with nullability + Some(createTypeWithNullability(head, nullable)) + } else { + // types are not all the same + if (allTypes.exists(_.getSqlTypeName == SqlTypeName.ANY)) { + // one of the type was ANY. + // we cannot generate a common type if it differs from other types. throw TableException("Generic ANY types must have a common type information.") + } else { + // cannot resolve a common type for different input types + None } - } else { - null } } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala index bc9b453501d3b..bff0b78505c88 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala @@ -18,8 +18,11 @@ package org.apache.flink.table.api.batch.sql +import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo} import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase import org.junit.{Ignore, Test} @@ -178,4 +181,56 @@ class SetOperatorsTest extends TableTestBase { expected ) } + + @Test + def testUnionNullableTypes(): Unit = { + val util = batchTestUtil() + util.addTable[((Int, String), (Int, String), Int)]("A", 'a, 'b, 'c) + + val expected = binaryNode( + "DataSetUnion", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a") + ), + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "CASE(>(c, 0), b, null) AS EXPR$0") + ), + term("union", "a") + ) + + util.verifySql( + "SELECT a FROM A UNION ALL SELECT CASE WHEN c > 0 THEN b ELSE NULL END FROM A", + expected + ) + } + + @Test + def testUnionAnyType(): Unit = { + val util = batchTestUtil() + val typeInfo = Types.ROW( + new GenericTypeInfo(classOf[NonPojo]), + new GenericTypeInfo(classOf[NonPojo])) + util.addJavaTable(typeInfo, "A", "a, b") + + val expected = binaryNode( + "DataSetUnion", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a") + ), + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "b") + ), + term("union", "a") + ) + + util.verifyJavaSql("SELECT a FROM A UNION ALL SELECT b FROM A", expected) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala index 686973e1b2d26..2d4e2056518d9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala @@ -20,8 +20,12 @@ package org.apache.flink.table.api.batch.table import java.sql.Timestamp +import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.Null +import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ import org.junit.Test @@ -76,4 +80,59 @@ class SetOperatorsTest extends TableTestBase { util.verifyTable(in, expected) } + + @Test + def testUnionNullableTypes(): Unit = { + val util = batchTestUtil() + val t = util.addTable[((Int, String), (Int, String), Int)]("A", 'a, 'b, 'c) + + val in = t.select('a) + .unionAll( + t.select(('c > 0) ? ('b, Null(createTypeInformation[(Int, String)])))) + + val expected = binaryNode( + "DataSetUnion", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a") + ), + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "CASE(>(c, 0), b, null) AS _c0") + ), + term("union", "a") + ) + + util.verifyTable(in, expected) + } + + @Test + def testUnionAnyType(): Unit = { + val util = batchTestUtil() + val typeInfo = Types.ROW( + new GenericTypeInfo(classOf[NonPojo]), + new GenericTypeInfo(classOf[NonPojo])) + val t = util.addJavaTable(typeInfo, "A", "a, b") + + val in = t.select('a).unionAll(t.select('b)) + + val expected = binaryNode( + "DataSetUnion", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a") + ), + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "b") + ), + term("union", "a") + ) + + util.verifyJavaTable(in, expected) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala new file mode 100644 index 0000000000000..7e807f6d73a0c --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala @@ -0,0 +1,83 @@ +/* + * 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.flink.table.api.stream.sql + +import org.apache.flink.api.java.typeutils.GenericTypeInfo +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Types +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo +import org.apache.flink.table.utils.TableTestUtil._ +import org.apache.flink.table.utils.TableTestBase +import org.junit.Test + +class UnionTest extends TableTestBase { + + @Test + def testUnionAllNullableCompositeType() = { + val streamUtil = streamTestUtil() + streamUtil.addTable[((Int, String), (Int, String), Int)]("A", 'a, 'b, 'c) + + val expected = binaryNode( + "DataStreamUnion", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a") + ), + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "CASE(>(c, 0), b, null) AS EXPR$0") + ), + term("union all", "a") + ) + + streamUtil.verifySql( + "SELECT a FROM A UNION ALL SELECT CASE WHEN c > 0 THEN b ELSE NULL END FROM A", + expected + ) + } + + @Test + def testUnionAnyType(): Unit = { + val streamUtil = streamTestUtil() + val typeInfo = Types.ROW( + new GenericTypeInfo(classOf[NonPojo]), + new GenericTypeInfo(classOf[NonPojo])) + streamUtil.addJavaTable(typeInfo, "A", "a, b") + + val expected = binaryNode( + "DataStreamUnion", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a") + ), + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "b") + ), + term("union all", "a") + ) + + streamUtil.verifyJavaSql("SELECT a FROM A UNION ALL SELECT b FROM A", expected) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala index cbdce8b87d2de..6dd2afcca9036 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala @@ -211,7 +211,6 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { @Test def testOtherExpressions(): Unit = { - // nested field null type testSqlApi("CASE WHEN f13.f1 IS NULL THEN 'a' ELSE 'b' END", "a") testSqlApi("CASE WHEN f13.f1 IS NOT NULL THEN 'a' ELSE 'b' END", "b") @@ -222,6 +221,10 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { testAllApis('f13.get("f1").isNull, "f13.get('f1').isNull", "f13.f1 IS NULL", "true") testAllApis('f13.get("f1").isNotNull, "f13.get('f1').isNotNull", "f13.f1 IS NOT NULL", "false") + // array element access test + testSqlApi("CASE WHEN f18 IS NOT NULL THEN f18[1] ELSE NULL END", "1") + testSqlApi("CASE WHEN f19 IS NOT NULL THEN f19[1] ELSE NULL END", "(1,a)") + // boolean literals testAllApis( true, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala index 149d8c1dc876e..3eeb215955484 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala @@ -22,6 +22,7 @@ import java.sql.Date import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.scala.createTypeInformation import org.apache.flink.table.api.Types import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.types.Row @@ -29,7 +30,7 @@ import org.apache.flink.types.Row class ScalarOperatorsTestBase extends ExpressionTestBase { def testData: Row = { - val testData = new Row(18) + val testData = new Row(20) testData.setField(0, 1: Byte) testData.setField(1, 1: Short) testData.setField(2, 1) @@ -48,6 +49,8 @@ class ScalarOperatorsTestBase extends ExpressionTestBase { testData.setField(15, Date.valueOf("1996-11-10")) testData.setField(16, BigDecimal("0.00000000").bigDecimal) testData.setField(17, BigDecimal("10.0").bigDecimal) + testData.setField(18, Array[Integer](1,2)) + testData.setField(19, Array[(Int, String)]((1,"a"), (2, "b"))) testData } @@ -70,7 +73,9 @@ class ScalarOperatorsTestBase extends ExpressionTestBase { Types.STRING, Types.SQL_DATE, Types.DECIMAL, - Types.DECIMAL + Types.DECIMAL, + Types.OBJECT_ARRAY(Types.INT), + Types.OBJECT_ARRAY(createTypeInformation[(Int, String)]) ).asInstanceOf[TypeInformation[Any]] } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/SetOperatorsITCase.scala index cf195a5c02e69..5e15e14f8489a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/SetOperatorsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/SetOperatorsITCase.scala @@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData} import org.apache.flink.types.Row import org.junit.Assert._ @@ -88,9 +89,22 @@ class SetOperatorsITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } - class NonPojo { - val x = new java.util.HashMap[String, String]() + @Test + def testUnionWithCompositeType(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + StreamITCase.testResults = mutable.MutableList() + val s1 = env.fromElements((1, (1, "a")), (2, (2, "b"))) + .toTable(tEnv, 'a, 'b) + val s2 = env.fromElements(((3, "c"), 3), ((4, "d"), 4)) + .toTable(tEnv, 'a, 'b) - override def toString: String = x.toString + val result = s1.unionAll(s2.select('b, 'a)).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = mutable.MutableList("1,(1,a)", "2,(2,b)", "3,(3,c)", "4,(4,d)") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala index e8568cab638b2..9223887de8704 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala @@ -176,4 +176,10 @@ object CommonTestData { this(null, null) } } + + class NonPojo { + val x = new java.util.HashMap[String, String]() + + override def toString: String = x.toString + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index 0a0d12e932fd2..4042f504cb232 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -190,6 +190,19 @@ case class BatchTableTestUtil() extends TableTestUtil { actual.split("\n").map(_.trim).mkString("\n")) } + def verifyJavaSql(query: String, expected: String): Unit = { + verifyJavaTable(javaTableEnv.sqlQuery(query), expected) + } + + def verifyJavaTable(resultTable: Table, expected: String): Unit = { + val relNode = resultTable.getRelNode + val optimized = javaTableEnv.optimize(relNode) + val actual = RelOptUtil.toString(optimized) + assertEquals( + expected.split("\n").map(_.trim).mkString("\n"), + actual.split("\n").map(_.trim).mkString("\n")) + } + def printTable(resultTable: Table): Unit = { val relNode = resultTable.getRelNode val optimized = tableEnv.optimize(relNode) @@ -268,6 +281,19 @@ case class StreamTableTestUtil() extends TableTestUtil { actual.split("\n").map(_.trim).mkString("\n")) } + def verifyJavaSql(query: String, expected: String): Unit = { + verifyJavaTable(javaTableEnv.sqlQuery(query), expected) + } + + def verifyJavaTable(resultTable: Table, expected: String): Unit = { + val relNode = resultTable.getRelNode + val optimized = javaTableEnv.optimize(relNode, updatesAsRetraction = false) + val actual = RelOptUtil.toString(optimized) + assertEquals( + expected.split("\n").map(_.trim).mkString("\n"), + actual.split("\n").map(_.trim).mkString("\n")) + } + // the print methods are for debugging purposes only def printTable(resultTable: Table): Unit = { val relNode = resultTable.getRelNode From 1b20f70dea3fddfaeaf00ceae44e4dc0fcb4f47b Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Wed, 8 Nov 2017 01:17:57 +0800 Subject: [PATCH 016/367] [FLINK-7996] [table] Add support for (left.time = right.time) predicates to window join. This closes #4977. --- docs/dev/table/sql.md | 17 ++-- docs/dev/table/tableApi.md | 26 ++--- .../table/runtime/join/WindowJoinUtil.scala | 95 ++++++++++++------- .../flink/table/api/stream/sql/JoinTest.scala | 66 +++++++++++++ .../sql/validation/JoinValidationTest.scala | 29 ++++++ .../table/api/stream/table/JoinTest.scala | 34 +++++++ .../table/runtime/stream/sql/JoinITCase.scala | 47 +++++++++ 7 files changed, 259 insertions(+), 55 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 23182714cf6db..3097d9edea769 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -400,14 +400,15 @@ FROM Orders LEFT JOIN Product ON Orders.productId = Product.id
@@ -765,7 +765,7 @@ pattern.oneOrMore().greedy() Pattern.begin("start").where(_.getName().equals("c")) .followedBy("middle").where(_.getName().equals("a")) .oneOrMore().consecutive() - .followedBy("end1").where(_.getName().equals("b")); + .followedBy("end1").where(_.getName().equals("b")) {% endhighlight %}

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

@@ -786,7 +786,7 @@ Pattern.begin("start").where(_.getName().equals("c")) Pattern.begin("start").where(_.getName().equals("c")) .followedBy("middle").where(_.getName().equals("a")) .oneOrMore().allowCombinations() - .followedBy("end1").where(_.getName().equals("b")); + .followedBy("end1").where(_.getName().equals("b")) {% endhighlight %}

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

@@ -1491,7 +1491,7 @@ val result: SingleOutputStreamOperator[ComplexEvent] = patternStream.select(outp pattern: Map[String, Iterable[Event]] => ComplexEvent() } -val timeoutResult: DataStream = result.getSideOutput(outputTag); +val timeoutResult: DataStream = result.getSideOutput(outputTag) ~~~ The `flatSelect` API call offers the same overloaded version which takes as the first parameter a timeout function and as second parameter a selection function. @@ -1510,7 +1510,7 @@ val result: SingleOutputStreamOperator[ComplexEvent] = patternStream.flatSelect( out.collect(ComplexEvent()) } -val timeoutResult: DataStream = result.getSideOutput(outputTag); +val timeoutResult: DataStream = result.getSideOutput(outputTag) ~~~ diff --git a/docs/dev/libs/gelly/graph_api.md b/docs/dev/libs/gelly/graph_api.md index 465c24f83893a..f00275e1cfb78 100644 --- a/docs/dev/libs/gelly/graph_api.md +++ b/docs/dev/libs/gelly/graph_api.md @@ -773,7 +773,7 @@ final class SelectLargeWeightNeighbors extends NeighborsFunctionWithVertexValue[ for (neighbor <- neighbors) { if (neighbor._1.getValue() > 0.5) { - out.collect(vertex, neighbor._2); + out.collect(vertex, neighbor._2) } } } diff --git a/docs/dev/libs/gelly/library_methods.md b/docs/dev/libs/gelly/library_methods.md index 93a2c5dc61181..015f85a66a282 100644 --- a/docs/dev/libs/gelly/library_methods.md +++ b/docs/dev/libs/gelly/library_methods.md @@ -55,7 +55,7 @@ val graph: Graph[java.lang.Long, java.lang.Long, NullValue] = ... val verticesWithCommunity = graph.run(new LabelPropagation[java.lang.Long, java.lang.Long, NullValue](30)) // print the result -verticesWithCommunity.print +verticesWithCommunity.print() {% endhighlight %} diff --git a/docs/dev/stream/operators/asyncio.md b/docs/dev/stream/operators/asyncio.md index 32945e4ac868c..c4736381ff4f5 100644 --- a/docs/dev/stream/operators/asyncio.md +++ b/docs/dev/stream/operators/asyncio.md @@ -150,7 +150,7 @@ class AsyncDatabaseRequest extends AsyncFunction[String, (String, String)] { // set the callback to be executed once the request by the client is complete // the callback simply forwards the result to the result future resultFuture.onSuccess { - case result: String => resultFuture.complete(Iterable((str, result))); + case result: String => resultFuture.complete(Iterable((str, result))) } } } diff --git a/docs/dev/stream/state/custom_serialization.md b/docs/dev/stream/state/custom_serialization.md index ca6b07dd00ae1..7f886d2025fdd 100644 --- a/docs/dev/stream/state/custom_serialization.md +++ b/docs/dev/stream/state/custom_serialization.md @@ -61,7 +61,7 @@ val descriptor = new ListStateDescriptor[(String, Integer)]( new CustomTypeSerializer) ) -checkpointedState = getRuntimeContext.getListState(descriptor); +checkpointedState = getRuntimeContext.getListState(descriptor) {% endhighlight %} diff --git a/docs/dev/stream/testing.md b/docs/dev/stream/testing.md index e5bc02414244c..ce31629ee74a1 100644 --- a/docs/dev/stream/testing.md +++ b/docs/dev/stream/testing.md @@ -247,8 +247,8 @@ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100));
{% highlight scala %} -env.enableCheckpointing(500); -env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100)); +env.enableCheckpointing(500) +env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100)) {% endhighlight %}
diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index f5a20592985eb..da6d2e4d31ab9 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -131,7 +131,7 @@ val result: Table = orders .select('a.lowerCase(), 'b, 'rowtime) .window(Tumble over 1.hour on 'rowtime as 'hourlyWindow) .groupBy('hourlyWindow, 'a) - .select('a, 'hourlyWindow.end as 'hour, 'b.avg as 'avgBillingAmount); + .select('a, 'hourlyWindow.end as 'hour, 'b.avg as 'avgBillingAmount) {% endhighlight %} @@ -355,7 +355,7 @@ Table result = orders
@@ -448,7 +448,7 @@ val result: Table = orders preceding UNBOUNDED_RANGE following CURRENT_RANGE as 'w) - .select('a, 'b.avg over 'w, 'b.max over 'w, 'b.min over 'w,) // sliding aggregate + .select('a, 'b.avg over 'w, 'b.max over 'w, 'b.min over 'w) // sliding aggregate {% endhighlight %}

Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute.

@@ -614,9 +614,9 @@ Table result = orders @@ -656,12 +656,12 @@ val fullOuterResult = left.fullOuterJoin(right, 'a === 'd).select('a, 'b, 'e)

Note: Currently, only INNER time-windowed joins are supported.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c, 'ltime.rowtime); -val right = ds2.toTable(tableEnv, 'd, 'e, 'f, 'rtime.rowtime); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c, 'ltime.rowtime) +val right = ds2.toTable(tableEnv, 'd, 'e, 'f, 'rtime.rowtime) val result = left.join(right) .where('a === 'd && 'ltime >= 'rtime - 5.minutes && 'ltime < 'rtime + 10.minutes) - .select('a, 'b, 'e, 'ltime); + .select('a, 'b, 'e, 'ltime) {% endhighlight %} @@ -856,9 +856,9 @@ Table result = left.select("a, b, c").where("a.in(RightTable)"); @@ -872,9 +872,9 @@ val result = left.union(right); @@ -887,9 +887,9 @@ val result = left.unionAll(right); @@ -902,9 +902,9 @@ val result = left.intersect(right); @@ -917,9 +917,9 @@ val result = left.intersectAll(right); @@ -932,9 +932,9 @@ val result = left.minus(right); @@ -947,9 +947,9 @@ val result = left.minusAll(right); @@ -1030,8 +1030,8 @@ Table result3 = in.orderBy("a.asc").offset(10).fetch(5); diff --git a/docs/dev/table/udfs.md b/docs/dev/table/udfs.md index 71567d880564c..0e0930211b171 100644 --- a/docs/dev/table/udfs.md +++ b/docs/dev/table/udfs.md @@ -93,7 +93,7 @@ myTable.select('string, hashCode('string)) // register and use the function in SQL tableEnv.registerFunction("hashCode", new HashCode(10)) -tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable"); +tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable") {% endhighlight %} @@ -198,17 +198,17 @@ val myTable = ... // table schema: [a: String] // Use the table function in the Scala Table API (Note: No registration required in Scala Table API). val split = new Split("#") // "as" specifies the field names of the generated table. -myTable.join(split('a) as ('word, 'length)).select('a, 'word, 'length); -myTable.leftOuterJoin(split('a) as ('word, 'length)).select('a, 'word, 'length); +myTable.join(split('a) as ('word, 'length)).select('a, 'word, 'length) +myTable.leftOuterJoin(split('a) as ('word, 'length)).select('a, 'word, 'length) // Register the table function to use it in SQL queries. tableEnv.registerFunction("split", new Split("#")) // Use the table function in SQL with LATERAL and TABLE keywords. // CROSS JOIN a table function (equivalent to "join" in Table API) -tableEnv.sqlQuery("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)"); +tableEnv.sqlQuery("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)") // LEFT JOIN a table function (equivalent to "leftOuterJoin" in Table API) -tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN TABLE(split(a)) as T(word, length) ON TRUE"); +tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN TABLE(split(a)) as T(word, length) ON TRUE") {% endhighlight %} **IMPORTANT:** Do not implement TableFunction as a Scala object. Scala object is a singleton and will cause concurrency issues. @@ -723,7 +723,7 @@ tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable"); {% highlight scala %} object hashCode extends ScalarFunction { - var hashcode_factor = 12; + var hashcode_factor = 12 override def open(context: FunctionContext): Unit = { // access "hashcode_factor" parameter @@ -743,7 +743,7 @@ myTable.select('string, hashCode('string)) // register and use the function in SQL tableEnv.registerFunction("hashCode", hashCode) -tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable"); +tableEnv.sqlQuery("SELECT string, HASHCODE(string) FROM MyTable") {% endhighlight %} From e7395efece72ca4d3e13b0517444e547bed120ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=87=91=E7=AB=B9?= Date: Thu, 28 Dec 2017 22:45:08 +0800 Subject: [PATCH 169/367] [FLINK-8323] [table] Fix modulo scalar function bug This closes #5212. --- .../table/codegen/calls/ScalarOperators.scala | 19 ++- .../expressions/ScalarOperatorsTest.scala | 117 +++++++++++++++++- .../utils/ScalarOperatorsTestBase.scala | 6 +- 3 files changed, 131 insertions(+), 11 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala index bd5b1f75fee63..3b7dfc73f6126 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala @@ -46,9 +46,22 @@ object ScalarOperators { nullCheck: Boolean, resultType: TypeInformation[_], left: GeneratedExpression, - right: GeneratedExpression) - : GeneratedExpression = { - val leftCasting = numericCasting(left.resultType, resultType) + right: GeneratedExpression): GeneratedExpression = { + + val leftCasting = operator match { + case "%" => + if (left.resultType == right.resultType) { + numericCasting(left.resultType, resultType) + } else { + val castedType = if (isDecimal(left.resultType)) { + Types.LONG + } else { + left.resultType + } + numericCasting(left.resultType, castedType) + } + case _ => numericCasting(left.resultType, resultType) + } val rightCasting = numericCasting(right.resultType, resultType) val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala index 6dd2afcca9036..cb75f424bc28d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala @@ -76,12 +76,117 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { @Test def testArithmetic(): Unit = { - // math arthmetic - testTableApi('f8 - 5, "f8 - 5", "0") - testTableApi('f8 + 5, "f8 + 5", "10") - testTableApi('f8 / 2, "f8 / 2", "2") - testTableApi('f8 * 2, "f8 * 2", "10") - testTableApi('f8 % 2, "f8 % 2", "1") + + // math arithmetic + + // test addition + testAllApis( + 1514356320000L + 6000, + "1514356320000L + 6000", + "1514356320000 + 6000", + "1514356326000") + + testAllApis( + 'f20 + 6, + "f20 + 6", + "f20 + 6", + "1514356320006") + + testAllApis( + 'f20 + 'f20, + "f20 + f20", + "f20 + f20", + "3028712640000") + + // test subtraction + testAllApis( + 1514356320000L - 6000, + "1514356320000L - 6000", + "1514356320000 - 6000", + "1514356314000") + + testAllApis( + 'f20 - 6, + "f20 - 6", + "f20 - 6", + "1514356319994") + + testAllApis( + 'f20 - 'f20, + "f20 - f20", + "f20 - f20", + "0") + + // test multiplication + testAllApis( + 1514356320000L * 60000, + "1514356320000L * 60000", + "1514356320000 * 60000", + "90861379200000000") + + testAllApis( + 'f20 * 6, + "f20 * 6", + "f20 * 6", + "9086137920000") + + testAllApis( + 'f20 * 'f20, + "f20 * f20", + "f20 * f20", + "2293275063923942400000000") + + // test division + testAllApis( + 1514356320000L / 60000, + "1514356320000L / 60000", + "1514356320000 / 60000", + "25239272") + + testAllApis( + 'f20 / 6, + "f20 / 6", + "f20 / 6", + "252392720000") + + testAllApis( + 'f20 / 'f20, + "f20 / f20", + "f20 / f20", + "1") + + // test modulo + testAllApis( + 1514356320000L % 60000, + "1514356320000L % 60000", + "mod(1514356320000,60000)", + "0") + + testAllApis( + 'f20.mod('f20), + "f20.mod(f20)", + "mod(f20,f20)", + "0") + + testAllApis( + 'f20.mod(6), + "f20.mod(6)", + "mod(f20,6)", + "0") + + testAllApis( + 'f3.mod('f2), + "f3.mod(f2)", + "MOD(f3, f2)", + "0") + + testAllApis( + 'f3.mod(3), + "mod(f3, 3)", + "MOD(f3, 3)", + "1") + + // other math arithmetic testTableApi(-'f8, "-f8", "-5") testTableApi( +'f8, "+f8", "5") // additional space before "+" required because of checkstyle testTableApi(3.toExpr + 'f8, "3 + f8", "8") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala index 3eeb215955484..57fff3b9ac347 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala @@ -30,7 +30,7 @@ import org.apache.flink.types.Row class ScalarOperatorsTestBase extends ExpressionTestBase { def testData: Row = { - val testData = new Row(20) + val testData = new Row(21) testData.setField(0, 1: Byte) testData.setField(1, 1: Short) testData.setField(2, 1) @@ -51,6 +51,7 @@ class ScalarOperatorsTestBase extends ExpressionTestBase { testData.setField(17, BigDecimal("10.0").bigDecimal) testData.setField(18, Array[Integer](1,2)) testData.setField(19, Array[(Int, String)]((1,"a"), (2, "b"))) + testData.setField(20, BigDecimal("1514356320000").bigDecimal) testData } @@ -75,7 +76,8 @@ class ScalarOperatorsTestBase extends ExpressionTestBase { Types.DECIMAL, Types.DECIMAL, Types.OBJECT_ARRAY(Types.INT), - Types.OBJECT_ARRAY(createTypeInformation[(Int, String)]) + Types.OBJECT_ARRAY(createTypeInformation[(Int, String)]), + Types.DECIMAL ).asInstanceOf[TypeInformation[Any]] } From 1c7a7fa706e9b373bbe395398507378a61759190 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=87=91=E7=AB=B9?= Date: Sat, 30 Dec 2017 11:10:18 +0800 Subject: [PATCH 170/367] [FLINK-8331][core] FieldParser do not correctly set EMPT_COLUMN error state. This closes #5218 --- .../flink/types/parser/BooleanParser.java | 21 +--- .../apache/flink/types/parser/ByteParser.java | 6 + .../flink/types/parser/ByteValueParser.java | 8 +- .../flink/types/parser/FieldParser.java | 8 +- .../apache/flink/types/parser/IntParser.java | 9 +- .../flink/types/parser/IntValueParser.java | 8 +- .../apache/flink/types/parser/LongParser.java | 6 + .../flink/types/parser/LongValueParser.java | 6 + .../flink/types/parser/ShortParser.java | 7 ++ .../flink/types/parser/ShortValueParser.java | 8 +- .../flink/types/parser/StringParser.java | 10 +- .../flink/types/parser/StringValueParser.java | 10 +- .../flink/types/parser/FieldParserTest.java | 105 +++++++++++++++++- .../flink/types/parser/ParserTestBase.java | 27 +++-- 14 files changed, 194 insertions(+), 45 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/BooleanParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/BooleanParser.java index 908c05f41a601..3a6178a493277 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/BooleanParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/BooleanParser.java @@ -37,34 +37,25 @@ public class BooleanParser extends FieldParser { }; @Override - public int parseField(byte[] bytes, int startPos, int limit, byte[] delim, Boolean reuse) { + public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Boolean reuse) { - final int delimLimit = limit - delim.length + 1; + final int i = nextStringEndPos(bytes, startPos, limit, delimiter); - int i = startPos; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delim)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; + if (i < 0) { + return -1; } for (byte[] aTRUE : TRUE) { if (byteArrayEquals(bytes, startPos, i - startPos, aTRUE)) { result = true; - return (i == limit) ? limit : i + delim.length; + return (i == limit) ? limit : i + delimiter.length; } } for (byte[] aFALSE : FALSE) { if (byteArrayEquals(bytes, startPos, i - startPos, aFALSE)) { result = false; - return (i == limit) ? limit : i + delim.length; + return (i == limit) ? limit : i + delimiter.length; } } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/ByteParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/ByteParser.java index 7ee257ed85d64..79e9f1f4a939b 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/ByteParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/ByteParser.java @@ -28,6 +28,12 @@ public class ByteParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Byte reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + int val = 0; boolean neg = false; diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/ByteValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/ByteValueParser.java index c79f5d4dc9b74..e5b8e476ca64e 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/ByteValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/ByteValueParser.java @@ -33,6 +33,12 @@ public class ByteValueParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, ByteValue reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + int val = 0; boolean neg = false; @@ -73,7 +79,7 @@ public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, B return -1; } } - + reusable.setValue((byte) (neg ? -val : val)); return limit; } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java index c45f82032d6d0..59cc2e7d4601d 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java @@ -210,15 +210,15 @@ protected final int nextStringEndPos(byte[] bytes, int startPos, int limit, byte while (endPos < limit) { if (endPos < delimLimit && delimiterNext(bytes, endPos, delimiter)) { - if (endPos == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } break; } endPos++; } + if (endPos == startPos) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } return endPos; } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/IntParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/IntParser.java index 4e5d43fa4f745..5135a99c8f5e1 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/IntParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/IntParser.java @@ -35,8 +35,13 @@ public class IntParser extends FieldParser { private int result; @Override - public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Integer - reusable) { + public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Integer reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + long val = 0; boolean neg = false; diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/IntValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/IntValueParser.java index 0229bc7381ede..0c15f90c5312f 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/IntValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/IntValueParser.java @@ -36,6 +36,12 @@ public class IntValueParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, IntValue reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + long val = 0; boolean neg = false; @@ -75,7 +81,7 @@ public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, I return -1; } } - + reusable.setValue((int) (neg ? -val : val)); return limit; } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/LongParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/LongParser.java index 79eb0803eb141..11ed1a49fadf5 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/LongParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/LongParser.java @@ -32,6 +32,12 @@ public class LongParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Long reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + long val = 0; boolean neg = false; diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/LongValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/LongValueParser.java index 5ddd40c098de6..91b8aab52d0fb 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/LongValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/LongValueParser.java @@ -33,6 +33,12 @@ public class LongValueParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, LongValue reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + long val = 0; boolean neg = false; diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/ShortParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/ShortParser.java index c458a3f6fae6f..20f727d5c2674 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/ShortParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/ShortParser.java @@ -36,6 +36,12 @@ public class ShortParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Short reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + int val = 0; boolean neg = false; @@ -148,6 +154,7 @@ public static final short parseField(byte[] bytes, int startPos, int length, cha throw new NumberFormatException("Value overflow/underflow"); } } + return (short) (neg ? -val : val); } } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/ShortValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/ShortValueParser.java index 47471a300af51..0332ba84c139d 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/ShortValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/ShortValueParser.java @@ -36,6 +36,12 @@ public class ShortValueParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, ShortValue reusable) { + + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + int val = 0; boolean neg = false; @@ -75,7 +81,7 @@ public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, S return -1; } } - + reusable.setValue((short) (neg ? -val : val)); return limit; } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/StringParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/StringParser.java index 7b46a7e60712c..bb1c04f0f4b10 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/StringParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/StringParser.java @@ -42,6 +42,12 @@ public void enableQuotedStringParsing(byte quoteCharacter) { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, String reusable) { + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + this.result = ""; + return limit; + } + int i = startPos; final int delimLimit = limit - delimiter.length + 1; @@ -83,10 +89,6 @@ public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, S } if (i >= delimLimit) { - // no delimiter found. Take the full string - if (limit == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); // mark empty column - } this.result = new String(bytes, startPos, limit - startPos, getCharset()); return limit; } else { diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/StringValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/StringValueParser.java index c72b02962cb35..193babb0ef6e7 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/StringValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/StringValueParser.java @@ -45,6 +45,12 @@ public void enableQuotedStringParsing(byte quoteCharacter) { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, StringValue reusable) { + if (startPos == limit) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + reusable.setValueAscii(bytes, startPos, 0); + return limit; + } + this.result = reusable; int i = startPos; @@ -89,10 +95,6 @@ public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, S } if (i >= delimLimit) { - // no delimiter found. Take the full string - if (limit == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); // mark empty column - } reusable.setValueAscii(bytes, startPos, limit - startPos); return limit; } else { diff --git a/flink-core/src/test/java/org/apache/flink/types/parser/FieldParserTest.java b/flink-core/src/test/java/org/apache/flink/types/parser/FieldParserTest.java index bcb2bfb154774..ad46742f0d10d 100644 --- a/flink-core/src/test/java/org/apache/flink/types/parser/FieldParserTest.java +++ b/flink-core/src/test/java/org/apache/flink/types/parser/FieldParserTest.java @@ -18,6 +18,9 @@ package org.apache.flink.types.parser; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.types.parser.FieldParser.ParseErrorState; + import org.junit.Test; import static org.junit.Assert.*; @@ -43,4 +46,104 @@ public void testEndsWithDelimiter() throws Exception { assertFalse(FieldParser.endsWithDelimiter(bytes, 3, delim)); } -} \ No newline at end of file + @Test + public void testNextStringEndPos() throws Exception { + + FieldParser parser = new TestFieldParser(); + // single-char delimiter + byte[] singleCharDelim = "|".getBytes(ConfigConstants.DEFAULT_CHARSET); + + byte[] bytes1 = "a|".getBytes(ConfigConstants.DEFAULT_CHARSET); + assertEquals(1, parser.nextStringEndPos(bytes1, 0, bytes1.length, singleCharDelim)); + assertEquals(-1, parser.nextStringEndPos(bytes1, 1, bytes1.length, singleCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(bytes1, 1, 1, singleCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(bytes1, 2, bytes1.length, singleCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + byte[] bytes2 = "a||".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(bytes2, 1, bytes2.length, singleCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + byte[] bytes3 = "a|c".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(bytes3, 1, bytes3.length, singleCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + parser.resetParserState(); + assertEquals(3, parser.nextStringEndPos(bytes3, 2, bytes3.length, singleCharDelim)); + assertEquals(ParseErrorState.NONE, parser.getErrorState()); + + byte[] bytes4 = "a|c|".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(3, parser.nextStringEndPos(bytes4, 2, bytes4.length, singleCharDelim)); + assertEquals(ParseErrorState.NONE, parser.getErrorState()); + + // multi-char delimiter + byte[] multiCharDelim = "|#|".getBytes(ConfigConstants.DEFAULT_CHARSET); + byte[] mBytes1 = "a|#|".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(1, parser.nextStringEndPos(mBytes1, 0, mBytes1.length, multiCharDelim)); + assertEquals(-1, parser.nextStringEndPos(mBytes1, 1, mBytes1.length, multiCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(mBytes1, 1, 1, multiCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + byte[] mBytes2 = "a|#||#|".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(mBytes2, 1, mBytes2.length, multiCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + byte[] mBytes3 = "a|#|b".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(-1, parser.nextStringEndPos(mBytes3, 1, mBytes3.length, multiCharDelim)); + assertEquals(ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); + + parser.resetParserState(); + assertEquals(5, parser.nextStringEndPos(mBytes3, 2, mBytes3.length, multiCharDelim)); + assertEquals(ParseErrorState.NONE, parser.getErrorState()); + + byte[] mBytes4 = "a|#|b|#|".getBytes(ConfigConstants.DEFAULT_CHARSET); + parser.resetParserState(); + assertEquals(5, parser.nextStringEndPos(mBytes4, 2, mBytes4.length, multiCharDelim)); + assertEquals(ParseErrorState.NONE, parser.getErrorState()); + + } + +} + +/** + * A FieldParser just for nextStringEndPos test. + * + * @param The type that is parsed. + */ +class TestFieldParser extends FieldParser{ + + @Override + protected int parseField(byte[] bytes, int startPos, int limit, byte[] delim, T reuse) { + return 0; + } + + @Override + public T getLastResult() { + return null; + } + + @Override + public T createValue() { + return null; + } + + @Override + protected void resetParserState() { + super.resetParserState(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java b/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java index 51ace12fe39eb..c0e01b1b15945 100644 --- a/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/types/parser/ParserTestBase.java @@ -405,28 +405,31 @@ private static byte[] concatenate(String[] values, char[] delimiter, boolean del } @Test - public void testEmptyFieldInIsolation() { + public void testTrailingEmptyField() { try { - String [] emptyStrings = new String[] {"|"}; - FieldParser parser = getParser(); - for (String emptyString : emptyStrings) { - byte[] bytes = emptyString.getBytes(ConfigConstants.DEFAULT_CHARSET); - int numRead = parser.parseField(bytes, 0, bytes.length, new byte[]{'|'}, parser.createValue()); + byte[] bytes = "||".getBytes(ConfigConstants.DEFAULT_CHARSET); + + for (int i = 0; i < 2; i++) { + + // test empty field with trailing delimiter when i = 0, + // test empty field with out trailing delimiter when i= 1. + int numRead = parser.parseField(bytes, i, bytes.length, new byte[]{'|'}, parser.createValue()); assertEquals(FieldParser.ParseErrorState.EMPTY_COLUMN, parser.getErrorState()); - if(this.allowsEmptyField()) { + if (this.allowsEmptyField()) { assertTrue("Parser declared the empty string as invalid.", numRead != -1); - assertEquals("Invalid number of bytes read returned.", bytes.length, numRead); - } - else { + assertEquals("Invalid number of bytes read returned.", i + 1, numRead); + } else { assertTrue("Parser accepted the empty string.", numRead == -1); } + + parser.resetParserState(); } - } - catch (Exception e) { + + } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); fail("Test erroneous: " + e.getMessage()); From 4b2178677e9bae00de2ac201ac3642c803c29064 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 2 Jan 2018 14:06:18 +0100 Subject: [PATCH 171/367] [FLINK-8346][docs] add v4 signature workaround for manual S3 setups This closes #5231 --- docs/ops/deployment/aws.md | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/docs/ops/deployment/aws.md b/docs/ops/deployment/aws.md index d9d26478ee30c..7ef95e7399c85 100644 --- a/docs/ops/deployment/aws.md +++ b/docs/ops/deployment/aws.md @@ -397,15 +397,29 @@ Caused by: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native If you have configured everything properly, but get a `Bad Request` Exception **and** your S3 bucket is located in region `eu-central-1`, you might be running an S3 client, which does not support [Amazon's signature version 4](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html). -Currently, this includes all Hadoop versions up to 2.7.2 running `NativeS3FileSystem`, which depend on `JetS3t 0.9.0` instead of a version [>= 0.9.4](http://www.jets3t.org/RELEASE_NOTES.html). - -The only workaround is to change the bucket region. - ``` [...] Caused by: java.io.IOException: s3:/// : 400 : Bad Request [...] Caused by: org.jets3t.service.impl.rest.HttpException [...] ``` +or +``` +com.amazonaws.services.s3.model.AmazonS3Exception: Status Code: 400, AWS Service: Amazon S3, AWS Request ID: [...], AWS Error Code: null, AWS Error Message: Bad Request, S3 Extended Request ID: [...] + +``` + +This should not apply to our shaded Hadoop/Presto S3 file systems but can occur for Hadoop-provided +S3 file systems. In particular, all Hadoop versions up to 2.7.2 running `NativeS3FileSystem` (which +depend on `JetS3t 0.9.0` instead of a version [>= 0.9.4](http://www.jets3t.org/RELEASE_NOTES.html)) +are affected but users also reported this happening with the `S3AFileSystem`. + +Except for changing the bucket region, you may also be able to solve this by +[requesting signature version 4 for request authentication](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingAWSSDK.html#specify-signature-version), +e.g. by adding this to Flink's JVM options in `flink-conf.yaml` (see +[configuration](../config.html#common-options)): +``` +env.java.opts: -Dcom.amazonaws.services.s3.enableV4 +``` {% top %} From 6884ce98f3936248a534e259789b9adde1ff9514 Mon Sep 17 00:00:00 2001 From: Ankit Parashar Date: Mon, 4 Dec 2017 23:46:16 +0530 Subject: [PATCH 172/367] [FLINK-8116] [DataStream] Fix stale comments referring to Checkpointed interface --- docs/ops/state/state_backends.md | 2 +- .../api/functions/source/SourceFunction.java | 14 +++++++------- .../api/scala/StreamExecutionEnvironment.scala | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md index b32ad9f1c8a27..cc2ffdee7281c 100644 --- a/docs/ops/state/state_backends.md +++ b/docs/ops/state/state_backends.md @@ -26,7 +26,7 @@ Programs written in the [Data Stream API]({{ site.baseurl }}/dev/datastream_api. - Windows gather elements or aggregates until they are triggered - Transformation functions may use the key/value state interface to store values -- Transformation functions may implement the `Checkpointed` interface to make their local variables fault tolerant +- Transformation functions may implement the `CheckpointedFunction` interface to make their local variables fault tolerant See also [state section]({{ site.baseurl }}/dev/stream/state/index.html) in the streaming API guide. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java index 4665cc6005cff..cb2e15fd48a23 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java @@ -34,9 +34,9 @@ * The run method can run for as long as necessary. The source must, however, react to an * invocation of {@link #cancel()} by breaking out of its main loop. * - *

Checkpointed Sources

+ *

CheckpointedFunction Sources

* - *

Sources that also implement the {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} + *

Sources that also implement the {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction} * interface must ensure that state checkpointing, updating of internal state and emission of * elements are not done concurrently. This is achieved by using the provided checkpointing lock * object to protect update of state and emission of elements in a synchronized block. @@ -44,7 +44,7 @@ *

This is the basic pattern one should follow when implementing a (checkpointed) source: * *

{@code
- *  public class ExampleSource implements SourceFunction, Checkpointed {
+ *  public class ExampleSource implements SourceFunction, CheckpointedFunction {
  *      private long count = 0L;
  *      private volatile boolean isRunning = true;
  *
@@ -61,9 +61,9 @@
  *          isRunning = false;
  *      }
  *
- *      public Long snapshotState(long checkpointId, long checkpointTimestamp) { return count; }
+ *      public void snapshotState(FunctionSnapshotContext context) {  }
  *
- *      public void restoreState(Long state) { this.count = state; }
+ *      public void initializeState(FunctionInitializationContext context) {  }
  * }
  * }
* @@ -96,12 +96,12 @@ public interface SourceFunction extends Function, Serializable { * Starts the source. Implementations can use the {@link SourceContext} emit * elements. * - *

Sources that implement {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} + *

Sources that implement {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction} * must lock on the checkpoint lock (using a synchronized block) before updating internal * state and emitting elements, to make both an atomic operation: * *

{@code
-	 *  public class ExampleSource implements SourceFunction, Checkpointed {
+	 *  public class ExampleSource implements SourceFunction, CheckpointedFunction {
 	 *      private long count = 0L;
 	 *      private volatile boolean isRunning = true;
 	 *
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 9fd03c3d8c290..3bba5055445f5 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -229,7 +229,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * [[KeyedStream]] is maintained (heap, managed memory, externally), and where state
    * snapshots/checkpoints are stored, both for the key/value state, and for checkpointed
    * functions (implementing the interface 
-   * [[org.apache.flink.streaming.api.checkpoint.Checkpointed]].
+   * [[org.apache.flink.streaming.api.checkpoint.CheckpointedFunction]].
    *
    * 

The [[org.apache.flink.runtime.state.memory.MemoryStateBackend]] for example * maintains the state in heap memory, as objects. It is lightweight without extra From 1e637c54c2ad1b9a8d9ad6d3f9c8aa55605d7e8e Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 14:57:01 -0800 Subject: [PATCH 173/367] [FLINK-8116] [DataStream] Provide proper checkpointed source function example in Javadocs This closes #5121. --- .../api/functions/source/SourceFunction.java | 48 ++++++++++++++++--- 1 file changed, 41 insertions(+), 7 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java index cb2e15fd48a23..5a15df7d7a44a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java @@ -41,15 +41,19 @@ * elements are not done concurrently. This is achieved by using the provided checkpointing lock * object to protect update of state and emission of elements in a synchronized block. * - *

This is the basic pattern one should follow when implementing a (checkpointed) source: + *

This is the basic pattern one should follow when implementing a checkpointed source: * *

{@code
- *  public class ExampleSource implements SourceFunction, CheckpointedFunction {
+ *  public class ExampleCountSource implements SourceFunction, CheckpointedFunction {
  *      private long count = 0L;
  *      private volatile boolean isRunning = true;
  *
+ *      private transient ListState checkpointedCount;
+ *
  *      public void run(SourceContext ctx) {
  *          while (isRunning && count < 1000) {
+ *              // this synchronized block ensures that state checkpointing,
+ *              // internal state updates and emission of elements are an atomic operation
  *              synchronized (ctx.getCheckpointLock()) {
  *                  ctx.collect(count);
  *                  count++;
@@ -61,9 +65,22 @@
  *          isRunning = false;
  *      }
  *
- *      public void snapshotState(FunctionSnapshotContext context) {  }
+ *      public void initializeState(FunctionInitializationContext context) {
+ *          this.checkpointedCount = context
+ *              .getOperatorStateStore()
+ *              .getListState(new ListStateDescriptor<>("count", Long.class));
+ *
+ *          if (context.isRestored()) {
+ *              for (Long count : this.checkpointedCount.get()) {
+ *                  this.count = count;
+ *              }
+ *          }
+ *      }
  *
- *      public void initializeState(FunctionInitializationContext context) {  }
+ *      public void snapshotState(FunctionSnapshotContext context) {
+ *          this.checkpointedCount.clear();
+ *          this.checkpointedCount.add(count);
+ *      }
  * }
  * }
* @@ -101,12 +118,16 @@ public interface SourceFunction extends Function, Serializable { * state and emitting elements, to make both an atomic operation: * *
{@code
-	 *  public class ExampleSource implements SourceFunction, CheckpointedFunction {
+	 *  public class ExampleCountSource implements SourceFunction, CheckpointedFunction {
 	 *      private long count = 0L;
 	 *      private volatile boolean isRunning = true;
 	 *
+	 *      private transient ListState checkpointedCount;
+	 *
 	 *      public void run(SourceContext ctx) {
 	 *          while (isRunning && count < 1000) {
+	 *              // this synchronized block ensures that state checkpointing,
+	 *              // internal state updates and emission of elements are an atomic operation
 	 *              synchronized (ctx.getCheckpointLock()) {
 	 *                  ctx.collect(count);
 	 *                  count++;
@@ -118,9 +139,22 @@ public interface SourceFunction extends Function, Serializable {
 	 *          isRunning = false;
 	 *      }
 	 *
-	 *      public Long snapshotState(long checkpointId, long checkpointTimestamp) { return count; }
+	 *      public void initializeState(FunctionInitializationContext context) {
+	 *          this.checkpointedCount = context
+	 *              .getOperatorStateStore()
+	 *              .getListState(new ListStateDescriptor<>("count", Long.class));
+	 *
+	 *          if (context.isRestored()) {
+	 *              for (Long count : this.checkpointedCount.get()) {
+	 *                  this.count = count;
+	 *              }
+	 *          }
+	 *      }
 	 *
-	 *      public void restoreState(Long state) { this.count = state; }
+	 *      public void snapshotState(FunctionSnapshotContext context) {
+	 *          this.checkpointedCount.clear();
+	 *          this.checkpointedCount.add(count);
+	 *      }
 	 * }
 	 * }
* From c454ee3fa9a14cfe28dbfc641134659252d2c80b Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 15:21:31 -0800 Subject: [PATCH 174/367] [FLINK-8260] [kafka] Fix usage of deprecated instantiation methods in FlinkKafkaProducer docs --- docs/dev/connectors/kafka.md | 46 +++++++++--------------------------- 1 file changed, 11 insertions(+), 35 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index daf1903502ffa..c6461f948e596 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -447,17 +447,17 @@ if a new watermark should be emitted and with which timestamp. ## Kafka Producer -Flink’s Kafka Producer is called `FlinkKafkaProducer08` (or `09` for Kafka 0.9.0.x versions, etc.). +Flink’s Kafka Producer is called `FlinkKafkaProducer011` (or `010` for Kafka 0.10.0.x versions, etc.). It allows writing a stream of records to one or more Kafka topics. Example:
-
+
{% highlight java %} DataStream stream = ...; -FlinkKafkaProducer08 myProducer = new FlinkKafkaProducer08( +FlinkKafkaProducer011 myProducer = new FlinkKafkaProducer011( "localhost:9092", // broker list "my-topic", // target topic new SimpleStringSchema()); // serialization schema @@ -466,29 +466,17 @@ FlinkKafkaProducer08 myProducer = new FlinkKafkaProducer08( myProducer.setLogFailuresOnly(false); // "false" by default myProducer.setFlushOnCheckpoint(true); // "false" by default -stream.addSink(myProducer); -{% endhighlight %} -
-
-{% highlight java %} -DataStream stream = ...; - -FlinkKafkaProducer010Configuration myProducerConfig = FlinkKafkaProducer010.writeToKafkaWithTimestamps( - stream, // input stream - "my-topic", // target topic - new SimpleStringSchema(), // serialization schema - properties); // custom configuration for KafkaProducer (including broker list) +// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka +myProducer.setWriteTimestampToKafka(true); -// the following is necessary for at-least-once delivery guarantee -myProducerConfig.setLogFailuresOnly(false); // "false" by default -myProducerConfig.setFlushOnCheckpoint(true); // "false" by default +stream.addSink(myProducer); {% endhighlight %}
-
+
{% highlight scala %} val stream: DataStream[String] = ... -val myProducer = new FlinkKafkaProducer08[String]( +val myProducer = new FlinkKafkaProducer011[String]( "localhost:9092", // broker list "my-topic", // target topic new SimpleStringSchema) // serialization schema @@ -497,22 +485,10 @@ val myProducer = new FlinkKafkaProducer08[String]( myProducer.setLogFailuresOnly(false) // "false" by default myProducer.setFlushOnCheckpoint(true) // "false" by default -stream.addSink(myProducer) -{% endhighlight %} -
-
-{% highlight scala %} -val stream: DataStream[String] = ... +// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka +myProducer.setWriteTimestampToKafka(true) -val myProducerConfig = FlinkKafkaProducer010.writeToKafkaWithTimestamps( - stream, // input stream - "my-topic", // target topic - new SimpleStringSchema, // serialization schema - properties) // custom configuration for KafkaProducer (including broker list) - -// the following is necessary for at-least-once delivery guarantee -myProducerConfig.setLogFailuresOnly(false) // "false" by default -myProducerConfig.setFlushOnCheckpoint(true) // "false" by default +stream.addSink(myProducer) {% endhighlight %}
From 71974895da966478f2e24fd36c08d7cf386a7050 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 20:21:20 -0800 Subject: [PATCH 175/367] [FLINK-8287] [kafka] Improve Kafka producer Javadocs / doc to clarify partitioning behaviour --- docs/dev/connectors/kafka.md | 46 +++++++--- .../kafka/FlinkKafkaProducer010.java | 83 +++++++++++++++--- .../kafka/FlinkKafkaProducer011.java | 87 ++++++++++++++++--- .../kafka/FlinkKafkaProducer08.java | 82 ++++++++++++++--- .../kafka/FlinkKafkaProducer09.java | 83 +++++++++++++++--- 5 files changed, 324 insertions(+), 57 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index c6461f948e596..e2df5fdebd8fd 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -462,11 +462,8 @@ FlinkKafkaProducer011 myProducer = new FlinkKafkaProducer011( "my-topic", // target topic new SimpleStringSchema()); // serialization schema -// the following is necessary for at-least-once delivery guarantee -myProducer.setLogFailuresOnly(false); // "false" by default -myProducer.setFlushOnCheckpoint(true); // "false" by default - -// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka +// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka; +// this method is not available for earlier Kafka versions myProducer.setWriteTimestampToKafka(true); stream.addSink(myProducer); @@ -481,11 +478,8 @@ val myProducer = new FlinkKafkaProducer011[String]( "my-topic", // target topic new SimpleStringSchema) // serialization schema -// the following is necessary for at-least-once delivery guarantee -myProducer.setLogFailuresOnly(false) // "false" by default -myProducer.setFlushOnCheckpoint(true) // "false" by default - -// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka +// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka; +// this method is not available for earlier Kafka versions myProducer.setWriteTimestampToKafka(true) stream.addSink(myProducer) @@ -505,11 +499,30 @@ are other constructor variants that allow providing the following: partitions, you can provide an implementation of a `FlinkKafkaPartitioner` to the constructor. This partitioner will be called for each record in the stream to determine which exact partition of the target topic the record should be sent to. + Please see [Kafka Producer Partitioning Scheme](#kafka-producer-partitioning-scheme) for more details. * *Advanced serialization schema*: Similar to the consumer, the producer also allows using an advanced serialization schema called `KeyedSerializationSchema`, which allows serializing the key and value separately. It also allows to override the target topic, so that one producer instance can send data to multiple topics. +### Kafka Producer Partitioning Scheme + +By default, if a custom partitioner is not specified for the Flink Kafka Producer, the producer will use +a `FlinkFixedPartitioner` that maps each Flink Kafka Producer parallel subtask to a single Kafka partition +(i.e., all records received by a sink subtask will end up in the same Kafka partition). + +A custom partitioner can be implemented by extending the `FlinkKafkaPartitioner` class. All +Kafka versions' constructors allow providing a custom partitioner when instantiating the producer. +Note that the partitioner implementation must be serializable, as they will be transferred across Flink nodes. +Also, keep in mind that any state in the partitioner will be lost on job failures since the partitioner +is not part of the producer's checkpointed state. + +It is also possible to completely avoid using and kind of partitioner, and simply let Kafka partition +the written records by their attached key (as determined for each record using the provided serialization schema). +To do this, provide a `null` custom partitioner when instantiating the producer. It is important +to provide `null` as the custom partitioner; as explained above, if a custom partitioner is not specified +the `FlinkFixedPartitioner` is used instead. + ### Kafka Producers and Fault Tolerance #### Kafka 0.8 @@ -522,17 +535,22 @@ With Flink's checkpointing enabled, the `FlinkKafkaProducer09` and `FlinkKafkaPr can provide at-least-once delivery guarantees. Besides enabling Flink's checkpointing, you should also configure the setter -methods `setLogFailuresOnly(boolean)` and `setFlushOnCheckpoint(boolean)` appropriately, -as shown in the above examples in the previous section: +methods `setLogFailuresOnly(boolean)` and `setFlushOnCheckpoint(boolean)` appropriately. - * `setLogFailuresOnly(boolean)`: enabling this will let the producer log failures only + * `setLogFailuresOnly(boolean)`: by default, this is set to `false`. + Enabling this will let the producer only log failures instead of catching and rethrowing them. This essentially accounts the record to have succeeded, even if it was never written to the target Kafka topic. This must be disabled for at-least-once. - * `setFlushOnCheckpoint(boolean)`: with this enabled, Flink's checkpoints will wait for any + * `setFlushOnCheckpoint(boolean)`: by default, this is set to `false`. + With this enabled, Flink's checkpoints will wait for any on-the-fly records at the time of the checkpoint to be acknowledged by Kafka before succeeding the checkpoint. This ensures that all records before the checkpoint have been written to Kafka. This must be enabled for at-least-once. + +In conclusion, to configure the Kafka producer to have at-least-once guarantees for versions +0.9 and 0.10, `setLogFailureOnly` must be set to `false` and `setFlushOnCheckpoint` must be set +to `true`. **Note**: By default, the number of retries is set to "0". This means that when `setLogFailuresOnly` is set to `false`, the producer fails immediately on errors, including leader changes. The value is set to "0" by default to avoid diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 184a2e714536d..21e3a10964fdb 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -32,6 +32,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; +import javax.annotation.Nullable; + import java.util.Properties; /** @@ -124,12 +126,20 @@ public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestam * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer010(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. */ public FlinkKafkaProducer010(String brokerList, String topicId, SerializationSchema serializationSchema) { this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner()); @@ -139,10 +149,18 @@ public FlinkKafkaProducer010(String brokerList, String topicId, SerializationSch * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer010(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. * @param producerConfig * Properties with the producer configuration. */ @@ -151,15 +169,26 @@ public FlinkKafkaProducer010(String topicId, SerializationSchema serializatio } /** - * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to - * the topic. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a key-less {@link SerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

Since a key-less {@link SerializationSchema} is used, all records sent to Kafka will not have an + * attached key. Therefore, if a partitioner is also not provided, records will be distributed to Kafka + * partitions in a round-robin fashion. * * @param topicId The topic to write data to - * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] + * @param serializationSchema A key-less serializable serialization schema for turning user objects into a kafka-consumable byte[] * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner) + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If set to {@code null}, records will be distributed to Kafka partitions + * in a round-robin fashion. */ - public FlinkKafkaProducer010(String topicId, SerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { + public FlinkKafkaProducer010( + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig, + @Nullable FlinkKafkaPartitioner customPartitioner) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); } @@ -169,6 +198,14 @@ public FlinkKafkaProducer010(String topicId, SerializationSchema serializatio * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId @@ -184,6 +221,14 @@ public FlinkKafkaProducer010(String brokerList, String topicId, KeyedSerializati * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema @@ -196,11 +241,29 @@ public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema seriali } /** - * Create Kafka producer. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. * - *

This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above) + *

If a partitioner is not provided, written records will be partitioned by the attached key of each + * record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If written records do not + * have a key (i.e., {@link KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they + * will be distributed to Kafka partitions in a round-robin fashion. + * + * @param topicId The topic to write data to + * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages + * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If set to {@code null}, records will be partitioned by the key of each record + * (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the keys + * are {@code null}, then records will be distributed to Kafka partitions in a + * round-robin fashion. */ - public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { + public FlinkKafkaProducer010( + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig, + @Nullable FlinkKafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index b14e4871287e5..58355c9198ff2 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -277,13 +277,21 @@ public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSch } /** - * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer011(String, SerializationSchema, Properties, Optional)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. * @param producerConfig * Properties with the producer configuration. */ @@ -296,15 +304,26 @@ public FlinkKafkaProducer011(String topicId, SerializationSchema serializati } /** - * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to - * the topic. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a key-less {@link SerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

Since a key-less {@link SerializationSchema} is used, all records sent to Kafka will not have an + * attached key. Therefore, if a partitioner is also not provided, records will be distributed to Kafka + * partitions in a round-robin fashion. * * @param topicId The topic to write data to - * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] + * @param serializationSchema A key-less serializable serialization schema for turning user objects into a kafka-consumable byte[] * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner) + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If a partitioner is not provided, records will be distributed to Kafka partitions + * in a round-robin fashion. */ - public FlinkKafkaProducer011(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner) { + public FlinkKafkaProducer011( + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig, + Optional> customPartitioner) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); } @@ -314,6 +333,14 @@ public FlinkKafkaProducer011(String topicId, SerializationSchema serializati * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer011(String, KeyedSerializationSchema, Properties, Optional)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId @@ -333,6 +360,14 @@ public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializati * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer011(String, KeyedSerializationSchema, Properties, Optional)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema @@ -352,12 +387,22 @@ public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema serial * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer011(String, KeyedSerializationSchema, Properties, Optional, Semantic, int)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema * User defined serialization schema supporting key/value messages * @param producerConfig * Properties with the producer configuration. + * @param semantic + * Defines semantic that will be used by this producer (see {@link Semantic}). */ public FlinkKafkaProducer011( String topicId, @@ -374,12 +419,22 @@ public FlinkKafkaProducer011( /** - * The main constructor for creating a FlinkKafkaProducer. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

If a partitioner is not provided, written records will be partitioned by the attached key of each + * record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If written records do not + * have a key (i.e., {@link KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they + * will be distributed to Kafka partitions in a round-robin fashion. * * @param defaultTopicId The default topic to write data to * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If a partitioner is not provided, records will be partitioned by the key of each record + * (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the keys + * are {@code null}, then records will be distributed to Kafka partitions in a + * round-robin fashion. */ public FlinkKafkaProducer011( String defaultTopicId, @@ -396,12 +451,22 @@ public FlinkKafkaProducer011( } /** - * The main constructor for creating a FlinkKafkaProducer. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

If a partitioner is not provided, written records will be partitioned by the attached key of each + * record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If written records do not + * have a key (i.e., {@link KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they + * will be distributed to Kafka partitions in a round-robin fashion. * * @param defaultTopicId The default topic to write data to * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If a partitioner is not provided, records will be partitioned by the key of each record + * (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the keys + * are {@code null}, then records will be distributed to Kafka partitions in a + * round-robin fashion. * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}). */ diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java index d2f17d22c68aa..42fb892829ef4 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java @@ -25,6 +25,8 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import javax.annotation.Nullable; + import java.util.Properties; /** @@ -38,17 +40,26 @@ public class FlinkKafkaProducer08 extends FlinkKafkaProducerBase { private static final long serialVersionUID = 1L; - // ------------------- Keyless serialization schema constructors ---------------------- + // ------------------- Key-less serialization schema constructors ---------------------- + /** * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer08(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. */ public FlinkKafkaProducer08(String brokerList, String topicId, SerializationSchema serializationSchema) { this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner()); @@ -58,10 +69,18 @@ public FlinkKafkaProducer08(String brokerList, String topicId, SerializationSche * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer08(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. * @param producerConfig * Properties with the producer configuration. */ @@ -70,14 +89,26 @@ public FlinkKafkaProducer08(String topicId, SerializationSchema serializatio } /** - * The main constructor for creating a FlinkKafkaProducer. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a key-less {@link SerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

Since a key-less {@link SerializationSchema} is used, all records sent to Kafka will not have an + * attached key. Therefore, if a partitioner is also not provided, records will be distributed to Kafka + * partitions in a round-robin fashion. * * @param topicId The topic to write data to - * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] + * @param serializationSchema A key-less serializable serialization schema for turning user objects into a kafka-consumable byte[] * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If set to {@code null}, records will be distributed to Kafka partitions + * in a round-robin fashion. */ - public FlinkKafkaProducer08(String topicId, SerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { + public FlinkKafkaProducer08( + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig, + @Nullable FlinkKafkaPartitioner customPartitioner) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); } @@ -88,6 +119,14 @@ public FlinkKafkaProducer08(String topicId, SerializationSchema serializatio * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer08(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId @@ -103,6 +142,14 @@ public FlinkKafkaProducer08(String brokerList, String topicId, KeyedSerializatio * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer08(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema @@ -115,14 +162,29 @@ public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema seriali } /** - * The main constructor for creating a FlinkKafkaProducer. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

If a partitioner is not provided, written records will be partitioned by the attached key of each + * record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If written records do not + * have a key (i.e., {@link KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they + * will be distributed to Kafka partitions in a round-robin fashion. * * @param topicId The topic to write data to * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If set to {@code null}, records will be partitioned by the key of each record + * (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the keys + * are {@code null}, then records will be distributed to Kafka partitions in a + * round-robin fashion. */ - public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { + public FlinkKafkaProducer08( + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig, + @Nullable FlinkKafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java index 407bad5674341..19f445f6b6fde 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java @@ -25,6 +25,8 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import javax.annotation.Nullable; + import java.util.Properties; /** @@ -38,31 +40,47 @@ public class FlinkKafkaProducer09 extends FlinkKafkaProducerBase { private static final long serialVersionUID = 1L; - // ------------------- Keyless serialization schema constructors ---------------------- + // ------------------- Key-less serialization schema constructors ---------------------- /** * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer09(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. */ public FlinkKafkaProducer09(String brokerList, String topicId, SerializationSchema serializationSchema) { this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner()); } /** - * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer09(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema - * User defined (keyless) serialization schema. + * User defined key-less serialization schema. * @param producerConfig * Properties with the producer configuration. */ @@ -71,15 +89,26 @@ public FlinkKafkaProducer09(String topicId, SerializationSchema serializatio } /** - * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to - * the topic. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a key-less {@link SerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

Since a key-less {@link SerializationSchema} is used, all records sent to Kafka will not have an + * attached key. Therefore, if a partitioner is also not provided, records will be distributed to Kafka + * partitions in a round-robin fashion. * * @param topicId The topic to write data to - * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] + * @param serializationSchema A key-less serializable serialization schema for turning user objects into a kafka-consumable byte[] * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner) + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If set to {@code null}, records will be distributed to Kafka partitions + * in a round-robin fashion. */ - public FlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { + public FlinkKafkaProducer09( + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig, + @Nullable FlinkKafkaPartitioner customPartitioner) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); } @@ -90,6 +119,14 @@ public FlinkKafkaProducer09(String topicId, SerializationSchema serializatio * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer09(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param brokerList * Comma separated addresses of the brokers * @param topicId @@ -105,6 +142,14 @@ public FlinkKafkaProducer09(String brokerList, String topicId, KeyedSerializatio * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + *

Using this constructor, the default {@link FlinkFixedPartitioner} will be used as + * the partitioner. This default partitioner maps each sink subtask to a single Kafka + * partition (i.e. all records received by a sink subtask will end up in the same + * Kafka partition). + * + *

To use a custom partitioner, please use + * {@link #FlinkKafkaProducer09(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * * @param topicId * ID of the Kafka topic. * @param serializationSchema @@ -117,15 +162,29 @@ public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema seriali } /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to - * the topic. + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. It accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link FlinkKafkaPartitioner}. + * + *

If a partitioner is not provided, written records will be partitioned by the attached key of each + * record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If written records do not + * have a key (i.e., {@link KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they + * will be distributed to Kafka partitions in a round-robin fashion. * * @param topicId The topic to write data to * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * If set to {@code null}, records will be partitioned by the key of each record + * (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the keys + * are {@code null}, then records will be distributed to Kafka partitions in a + * round-robin fashion. */ - public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { + public FlinkKafkaProducer09( + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig, + @Nullable FlinkKafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); } From 10f1acf92313cde7bf4ac8aa1403b19405d2ed25 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 20:29:38 -0800 Subject: [PATCH 176/367] [FLINK-8260] [kafka] Reorder deprecated / regular constructors of FlinkKafkaProducer010 This commit moves deprecated factory methods of the FlinkKafkaProducer010 behind regular constructors, for better navigation and readability of the code. This closes #5179. --- .../kafka/FlinkKafkaProducer010.java | 167 +++++++++--------- 1 file changed, 82 insertions(+), 85 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 21e3a10964fdb..0e64aa5338c3a 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -46,80 +46,6 @@ public class FlinkKafkaProducer010 extends FlinkKafkaProducer09 { */ private boolean writeTimestampToKafka = false; - // ---------------------- "Constructors" for timestamp writing ------------------ - - /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to - * the topic. - * - *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) - * - * @param inStream The stream to write to Kafka - * @param topicId ID of the Kafka topic. - * @param serializationSchema User defined serialization schema supporting key/value messages - * @param producerConfig Properties with the producer configuration. - * - * @deprecated Use {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties)} - * and call {@link #setWriteTimestampToKafka(boolean)}. - */ - @Deprecated - public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, - String topicId, - KeyedSerializationSchema serializationSchema, - Properties producerConfig) { - return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner()); - } - - /** - * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to - * the topic. - * - *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) - * - * @param inStream The stream to write to Kafka - * @param topicId ID of the Kafka topic. - * @param serializationSchema User defined (keyless) serialization schema. - * @param producerConfig Properties with the producer configuration. - * - * @deprecated Use {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties)} - * and call {@link #setWriteTimestampToKafka(boolean)}. - */ - @Deprecated - public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, - String topicId, - SerializationSchema serializationSchema, - Properties producerConfig) { - return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner()); - } - - /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to - * the topic. - * - *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) - * - * @param inStream The stream to write to Kafka - * @param topicId The name of the target topic - * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages - * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. - * - * @deprecated Use {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} - * and call {@link #setWriteTimestampToKafka(boolean)}. - */ - @Deprecated - public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, - String topicId, - KeyedSerializationSchema serializationSchema, - Properties producerConfig, - FlinkKafkaPartitioner customPartitioner) { - - FlinkKafkaProducer010 kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner); - DataStreamSink streamSink = inStream.addSink(kafkaProducer); - return new FlinkKafkaProducer010Configuration<>(streamSink, inStream, kafkaProducer); - - } - // ---------------------- Regular constructors------------------ /** @@ -267,8 +193,90 @@ public FlinkKafkaProducer010( super(topicId, serializationSchema, producerConfig, customPartitioner); } + // ------------------- User configuration ---------------------- + + /** + * If set to true, Flink will write the (event time) timestamp attached to each record into Kafka. + * Timestamps must be positive for Kafka to accept them. + * + * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to Kafka. + */ + public void setWriteTimestampToKafka(boolean writeTimestampToKafka) { + this.writeTimestampToKafka = writeTimestampToKafka; + } + // ----------------------------- Deprecated constructors / factory methods --------------------------- + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to + * the topic. + * + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + * + * @param inStream The stream to write to Kafka + * @param topicId ID of the Kafka topic. + * @param serializationSchema User defined serialization schema supporting key/value messages + * @param producerConfig Properties with the producer configuration. + * + * @deprecated Use {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties)} + * and call {@link #setWriteTimestampToKafka(boolean)}. + */ + @Deprecated + public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig) { + return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to + * the topic. + * + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + * + * @param inStream The stream to write to Kafka + * @param topicId ID of the Kafka topic. + * @param serializationSchema User defined (keyless) serialization schema. + * @param producerConfig Properties with the producer configuration. + * + * @deprecated Use {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties)} + * and call {@link #setWriteTimestampToKafka(boolean)}. + */ + @Deprecated + public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig) { + return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to + * the topic. + * + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + * + * @param inStream The stream to write to Kafka + * @param topicId The name of the target topic + * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages + * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + * + * @deprecated Use {@link #FlinkKafkaProducer010(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} + * and call {@link #setWriteTimestampToKafka(boolean)}. + */ + @Deprecated + public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig, + FlinkKafkaPartitioner customPartitioner) { + FlinkKafkaProducer010 kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner); + DataStreamSink streamSink = inStream.addSink(kafkaProducer); + return new FlinkKafkaProducer010Configuration<>(streamSink, inStream, kafkaProducer); + } + /** * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. @@ -332,17 +340,6 @@ public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema seriali super(topicId, serializationSchema, producerConfig, customPartitioner); } - /** - * If set to true, Flink will write the (event time) timestamp attached to each record into Kafka. - * Timestamps must be positive for Kafka to accept them. - * - * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to Kafka. - */ - public void setWriteTimestampToKafka(boolean writeTimestampToKafka) { - this.writeTimestampToKafka = writeTimestampToKafka; - } - - // ----------------------------- Generic element processing --------------------------- @Override From 24d449353e89ae319b81c4d59da5403a243f8b07 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 20:34:17 -0800 Subject: [PATCH 177/367] [hotfix] [kafka] Properly deprecate FlinkKafkaProducer010Configuration FlinkKafkaProducer010Configuration is the return type of the deprecated writeToKafkaWithTimestamp factory methods. Therefore, the class should also be deprecated as well. --- .../streaming/connectors/kafka/FlinkKafkaProducer010.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 0e64aa5338c3a..369ab89f5fb70 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -386,7 +386,11 @@ record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(value * *

To enable the settings, this fake sink must override all the public methods * in {@link DataStreamSink}.

+ * + * @deprecated This class is deprecated since the factory methods {@code writeToKafkaWithTimestamps} + * for the producer are also deprecated. */ + @Deprecated public static class FlinkKafkaProducer010Configuration extends DataStreamSink { private final FlinkKafkaProducer010 producer; From 16a49de3f66e29108f6fb8977a51a2652ed3f312 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 20:36:31 -0800 Subject: [PATCH 178/367] [hotfix] [kafka] Fix stale Javadoc link in FlinkKafkaProducer09 The previous link was referencing a non-existent constructor signature. --- .../flink/streaming/connectors/kafka/FlinkKafkaProducer09.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java index 19f445f6b6fde..7f00c92d2cb6e 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java @@ -219,7 +219,7 @@ public FlinkKafkaProducer09(String topicId, SerializationSchema serializatio * * @deprecated This is a deprecated constructor that does not correctly handle partitioning when * producing to multiple topics. Use - * {@link #FlinkKafkaProducer09(String, org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * {@link #FlinkKafkaProducer09(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. */ @Deprecated public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { From f462f77194b76139591d478c4b66ef0676449849 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Dec 2017 20:38:32 -0800 Subject: [PATCH 179/367] [hotfix] [kafka] Add serialVersionUID to FlinkKafkaProducer010 --- .../flink/streaming/connectors/kafka/FlinkKafkaProducer010.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 369ab89f5fb70..e72134029f0ae 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -41,6 +41,8 @@ */ public class FlinkKafkaProducer010 extends FlinkKafkaProducer09 { + private static final long serialVersionUID = 1L; + /** * Flag controlling whether we are writing the Flink record's timestamp into Kafka. */ From 74135c9db11728f2189b6b4ccae90b1d4ccb84c1 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 21 Dec 2017 13:41:48 -0800 Subject: [PATCH 180/367] [FLINK-8283] [kafka] Stabalize FlinkKafkaConsumerBaseTest::testScaleUp() Previously, the testScaleUp() test was taking too much resources and causing test resources to be terminated before the test could finish. This commit lowers the intensity of the test, while still retaining the verified behaviour (i.e., when restoring the Kafka consumer with higher parallelism and more Kafka partitions). This closes #5201. --- .../streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 168cfd5c94af8..6ccfeb1d0664f 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -516,7 +516,7 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception @Test public void testScaleUp() throws Exception { - testRescaling(5, 2, 15, 1000); + testRescaling(5, 2, 8, 30); } @Test From 78e3405becb13ec74d6edd4624727f7fd40fea07 Mon Sep 17 00:00:00 2001 From: "wenlong.lwl" Date: Tue, 5 Dec 2017 15:16:32 +0800 Subject: [PATCH 181/367] [FLINK-8200][tests] Use TemporaryFolder in RocksDBAsyncSnapshotTest This closes #5122. --- .../state/RocksDBAsyncSnapshotTest.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index 2ba0494f4b6ee..71c5e77cfa6d7 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; @@ -60,7 +59,9 @@ import org.apache.flink.util.TestLogger; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.modules.junit4.PowerMockRunner; @@ -70,7 +71,6 @@ import java.lang.reflect.Field; import java.util.Arrays; import java.util.Map; -import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.RunnableFuture; @@ -95,6 +95,12 @@ @SuppressWarnings("serial") public class RocksDBAsyncSnapshotTest extends TestLogger { + /** + * Temporary fold for test. + */ + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + /** * This ensures that asynchronous state handles are actually materialized asynchronously. * @@ -119,7 +125,7 @@ public String getKey(String value) throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); - File dbDir = new File(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString()), "state"); + File dbDir = temporaryFolder.newFolder(); RocksDBStateBackend backend = new RocksDBStateBackend(new MemoryStateBackend()); backend.setDbStoragePath(dbDir.getAbsolutePath()); @@ -225,7 +231,7 @@ public String getKey(String value) throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); - File dbDir = new File(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString()), "state"); + File dbDir = temporaryFolder.newFolder(); BlockingStreamMemoryStateBackend memoryStateBackend = new BlockingStreamMemoryStateBackend(); @@ -333,7 +339,7 @@ public void testCleanupOfSnapshotsInFailureCase() throws Exception { RocksDBStateBackend backend = new RocksDBStateBackend(stateBackend); - backend.setDbStoragePath("file:///tmp/foobar"); + backend.setDbStoragePath(temporaryFolder.newFolder().toURI().toString()); AbstractKeyedStateBackend keyedStateBackend = backend.createKeyedStateBackend( env, From 2f6e001172366427005975b6644194b238baff82 Mon Sep 17 00:00:00 2001 From: okumin Date: Sat, 6 Jan 2018 22:48:02 +0900 Subject: [PATCH 182/367] [hotfix][docs] Fix DataStream iterations documentation * Fix a scala example which is using a wrong variable * Remove partitioning descriptions * partitioning parameters are already removed from IterativeStream#closeWith/DateStream#iterate * https://github.com/apache/flink/pull/988 * https://github.com/apache/flink/pull/4655 This closes #5249. --- docs/dev/datastream_api.md | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md index 6bb755edf9148..09580b1dd5fda 100644 --- a/docs/dev/datastream_api.md +++ b/docs/dev/datastream_api.md @@ -375,9 +375,6 @@ iteration.closeWith(iterationBody.filter(/* one part of the stream */)); DataStream output = iterationBody.filter(/* some other part of the stream */); {% endhighlight %} -By default the partitioning of the feedback stream will be automatically set to be the same as the input of the -iteration head. To override this the user can set an optional boolean flag in the `closeWith` method. - For example, here is program that continuously subtracts 1 from a series of integers until they reach zero: {% highlight java %} @@ -424,14 +421,10 @@ are forwarded downstream using filters. val iteratedStream = someDataStream.iterate( iteration => { val iterationBody = iteration.map(/* this is executed many times */) - (tail.filter(/* one part of the stream */), tail.filter(/* some other part of the stream */)) + (iterationBody.filter(/* one part of the stream */), iterationBody.filter(/* some other part of the stream */)) }) {% endhighlight %} - -By default the partitioning of the feedback stream will be automatically set to be the same as the input of the -iteration head. To override this the user can set an optional boolean flag in the `closeWith` method. - For example, here is program that continuously subtracts 1 from a series of integers until they reach zero: {% highlight scala %} From 951d1b94d9eed37f879328bbf4ab6e748209ea0c Mon Sep 17 00:00:00 2001 From: yew1eb Date: Fri, 29 Dec 2017 01:15:40 +0800 Subject: [PATCH 183/367] [hotfix][docs][metrics] Fix Threads.Count metric reference This closes #5213. --- docs/monitoring/metrics.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index e4a7a735416da..2a352ccc240c5 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -865,8 +865,8 @@ Thus, in order to infer the metric identifier:
- - + + From 6f50ed256a8a2ced26159ca12f9cdfbbaa91e740 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 8 Jan 2018 17:25:43 +0100 Subject: [PATCH 184/367] [FLINK-8320][docs] Clarify that only Java 8 is supported --- docs/quickstart/java_api_quickstart.md | 2 +- docs/quickstart/scala_api_quickstart.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md index 7665a7552caab..baf14deb7c5cc 100644 --- a/docs/quickstart/java_api_quickstart.md +++ b/docs/quickstart/java_api_quickstart.md @@ -31,7 +31,7 @@ Start working on your Flink Java program in a few simple steps. ## Requirements -The only requirements are working __Maven 3.0.4__ (or higher) and __Java 8.x__ (or higher) installations. +The only requirements are working __Maven 3.0.4__ (or higher) and __Java 8.x__ installations. ## Create Project diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md index dce71dd8b454a..40c02a9b8913a 100644 --- a/docs/quickstart/scala_api_quickstart.md +++ b/docs/quickstart/scala_api_quickstart.md @@ -114,7 +114,7 @@ Now you can import the project into Eclipse via `File -> Import... -> Existing P ### Requirements -The only requirements are working __Maven 3.0.4__ (or higher) and __Java 8.x__ (or higher) installations. +The only requirements are working __Maven 3.0.4__ (or higher) and __Java 8.x__ installations. ### Create Project From 9398dafae6918488582bc89c1ab73e3e2bd5ea16 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 23 Nov 2017 14:59:18 +0100 Subject: [PATCH 185/367] [hotfix] add some more buffer recycling checks in SpillableSubpartitionTest --- .../partition/SpillableSubpartitionTest.java | 48 +++++++++++++++++-- 1 file changed, 45 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index 2b356a8f49986..3b5c49c1ce959 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -40,8 +40,12 @@ import java.util.concurrent.Future; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doAnswer; @@ -181,32 +185,53 @@ public void testConsumeSpilledPartition() throws Exception { partition.add(buffer); partition.add(buffer); + assertFalse(buffer.isRecycled()); assertEquals(3, partition.releaseMemory()); + // now the buffer may be freed, depending on the timing of the write operation + // -> let's do this check at the end of the test (to save some time) partition.finish(); - BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class); + BufferAvailabilityListener listener = spy(new AwaitableBufferAvailablityListener()); SpilledSubpartitionView reader = (SpilledSubpartitionView) partition.createReadView(listener); verify(listener, times(1)).notifyBuffersAvailable(eq(4L)); Buffer read = reader.getNextBuffer(); assertNotNull(read); + assertNotSame(buffer, read); + assertFalse(read.isRecycled()); read.recycle(); + assertTrue(read.isRecycled()); read = reader.getNextBuffer(); assertNotNull(read); + assertNotSame(buffer, read); + assertFalse(read.isRecycled()); read.recycle(); + assertTrue(read.isRecycled()); read = reader.getNextBuffer(); assertNotNull(read); + assertNotSame(buffer, read); + assertFalse(read.isRecycled()); read.recycle(); + assertTrue(read.isRecycled()); // End of partition read = reader.getNextBuffer(); assertNotNull(read); assertEquals(EndOfPartitionEvent.class, EventSerializer.fromBuffer(read, ClassLoader.getSystemClassLoader()).getClass()); + assertFalse(read.isRecycled()); read.recycle(); + assertTrue(read.isRecycled()); + + // finally check that the buffer has been freed after a successful (or failed) write + final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs + while (!buffer.isRecycled() && System.currentTimeMillis() < deadline) { + Thread.sleep(1); + } + assertTrue(buffer.isRecycled()); } /** @@ -231,31 +256,48 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception // Initial notification assertEquals(1, listener.getNumNotifiedBuffers()); + assertFalse(buffer.isRecycled()); Buffer read = reader.getNextBuffer(); - assertNotNull(read); + assertSame(buffer, read); read.recycle(); assertEquals(2, listener.getNumNotifiedBuffers()); + assertFalse(buffer.isRecycled()); // Spill now assertEquals(2, partition.releaseMemory()); + assertFalse(buffer.isRecycled()); // still one in the reader! listener.awaitNotifications(4, 30_000); assertEquals(4, listener.getNumNotifiedBuffers()); read = reader.getNextBuffer(); - assertNotNull(read); + assertSame(buffer, read); read.recycle(); + // now the buffer may be freed, depending on the timing of the write operation + // -> let's do this check at the end of the test (to save some time) read = reader.getNextBuffer(); assertNotNull(read); + assertNotSame(buffer, read); + assertFalse(read.isRecycled()); read.recycle(); + assertTrue(read.isRecycled()); // End of partition read = reader.getNextBuffer(); assertNotNull(read); assertEquals(EndOfPartitionEvent.class, EventSerializer.fromBuffer(read, ClassLoader.getSystemClassLoader()).getClass()); + assertFalse(read.isRecycled()); read.recycle(); + assertTrue(read.isRecycled()); + + // finally check that the buffer has been freed after a successful (or failed) write + final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs + while (!buffer.isRecycled() && System.currentTimeMillis() < deadline) { + Thread.sleep(1); + } + assertTrue(buffer.isRecycled()); } private static class AwaitableBufferAvailablityListener implements BufferAvailabilityListener { From 2c667dea8e711fc928a37838c9f6cd5f6a9f7685 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 23 Nov 2017 13:09:37 +0100 Subject: [PATCH 186/367] [hotfix] only update buffer statistics in SpillableSubpartition#add() if successful --- .../io/network/partition/SpillableSubpartition.java | 13 ++++++++----- .../partition/PipelinedSubpartitionTest.java | 5 +++++ .../partition/SpillableSubpartitionTest.java | 13 +++++++++++++ .../io/network/partition/SubpartitionTestBase.java | 10 ++++++++++ 4 files changed, 36 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java index 654d5289bba1b..065de8ee5c106 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -95,13 +95,12 @@ public boolean add(Buffer buffer) throws IOException { return false; } - // The number of buffers are needed later when creating - // the read views. If you ever remove this line here, - // make sure to still count the number of buffers. - updateStatistics(buffer); - if (spillWriter == null) { buffers.add(buffer); + // The number of buffers are needed later when creating + // the read views. If you ever remove this line here, + // make sure to still count the number of buffers. + updateStatistics(buffer); return true; } @@ -109,6 +108,10 @@ public boolean add(Buffer buffer) throws IOException { // Didn't return early => go to disk spillWriter.writeBlock(buffer); + synchronized (buffers) { + // See the note above, but only do this if the buffer was correctly added! + updateStatistics(buffer); + } return true; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index de1e8a07109d1..6d36aa6b87599 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -35,6 +35,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static org.apache.flink.runtime.io.network.util.TestBufferFactory.BUFFER_SIZE; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -102,6 +103,8 @@ public void testBasicPipelinedProduceConsumeLogic() throws Exception { // Add data to the queue... subpartition.add(createBuffer()); + assertEquals(1, subpartition.getTotalNumberOfBuffers()); + assertEquals(BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); // ...should have resulted in a notification verify(listener, times(1)).notifyBuffersAvailable(eq(1L)); @@ -112,6 +115,8 @@ public void testBasicPipelinedProduceConsumeLogic() throws Exception { // Add data to the queue... subpartition.add(createBuffer()); + assertEquals(2, subpartition.getTotalNumberOfBuffers()); + assertEquals(2 * BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); verify(listener, times(2)).notifyBuffersAvailable(eq(1L)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index 3b5c49c1ce959..05a364d08efdf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -184,13 +184,21 @@ public void testConsumeSpilledPartition() throws Exception { partition.add(buffer); partition.add(buffer); partition.add(buffer); + assertEquals(3, partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 3, partition.getTotalNumberOfBytes()); assertFalse(buffer.isRecycled()); assertEquals(3, partition.releaseMemory()); // now the buffer may be freed, depending on the timing of the write operation // -> let's do this check at the end of the test (to save some time) + // stil same statistics + assertEquals(3, partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 3, partition.getTotalNumberOfBytes()); partition.finish(); + // + one EndOfPartitionEvent + assertEquals(4, partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 3 + 4, partition.getTotalNumberOfBytes()); BufferAvailabilityListener listener = spy(new AwaitableBufferAvailablityListener()); SpilledSubpartitionView reader = (SpilledSubpartitionView) partition.createReadView(listener); @@ -250,6 +258,8 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception partition.add(buffer); partition.add(buffer); partition.finish(); + assertEquals(4, partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 3 + 4, partition.getTotalNumberOfBytes()); AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener(); SpillableSubpartitionView reader = (SpillableSubpartitionView) partition.createReadView(listener); @@ -267,6 +277,9 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception // Spill now assertEquals(2, partition.releaseMemory()); assertFalse(buffer.isRecycled()); // still one in the reader! + // still same statistics: + assertEquals(4, partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 3 + 4, partition.getTotalNumberOfBytes()); listener.awaitNotifications(4, 30_000); assertEquals(4, listener.getNumNotifiedBuffers()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index 800542e08633e..d084f6212e0bf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -21,8 +21,10 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.util.TestBufferFactory; import org.apache.flink.util.TestLogger; + import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -46,8 +48,12 @@ public void testAddAfterFinish() throws Exception { try { subpartition.finish(); + assertEquals(1, subpartition.getTotalNumberOfBuffers()); + assertEquals(4, subpartition.getTotalNumberOfBytes()); assertFalse(subpartition.add(mock(Buffer.class))); + assertEquals(1, subpartition.getTotalNumberOfBuffers()); + assertEquals(4, subpartition.getTotalNumberOfBytes()); } finally { if (subpartition != null) { subpartition.release(); @@ -61,8 +67,12 @@ public void testAddAfterRelease() throws Exception { try { subpartition.release(); + assertEquals(0, subpartition.getTotalNumberOfBuffers()); + assertEquals(0, subpartition.getTotalNumberOfBytes()); assertFalse(subpartition.add(mock(Buffer.class))); + assertEquals(0, subpartition.getTotalNumberOfBuffers()); + assertEquals(0, subpartition.getTotalNumberOfBytes()); } finally { if (subpartition != null) { subpartition.release(); From a316989e5dfbb1dc0d555193425a4d6bd5f42d8d Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 23 Nov 2017 14:58:21 +0100 Subject: [PATCH 187/367] [FLINK-7499][io] also let AsynchronousBufferFileWriter#writeBlock() recycle the buffer in case of failures This fixes a double-recycle in SpillableSubpartitionView and also makes sure that even if adding the (asynchronous) write operation fails, the buffer is properly freed in code that did not perform this cleanup. It avoids code duplication of this cleanup and it is also more consistent to take over responsibility of the given buffer even if an exception is thrown. [FLINK-7499][io] complete the idiom of ResultSubpartition#add() taking over ownership of the buffer The buffer will now always be released once and at the right time and the caller must not worry about the buffer release if a called function threw an exception. This closes #5275. --- .../AsynchronousBufferFileWriter.java | 20 +- .../BlockChannelWriterWithCallback.java | 4 +- .../partition/PipelinedSubpartition.java | 1 + .../io/network/partition/ResultPartition.java | 12 +- .../network/partition/ResultSubpartition.java | 12 + .../partition/SpillableSubpartition.java | 16 +- .../partition/SpillableSubpartitionView.java | 6 +- .../AsynchronousBufferFileWriterTest.java | 30 +++ ...OManagerAsyncWithNoOpBufferFileWriter.java | 53 ++++ .../partition/ResultPartitionTest.java | 63 ++++- .../partition/SpillableSubpartitionTest.java | 252 +++++++++++++++++- .../SpilledSubpartitionViewTest.java | 4 + 12 files changed, 449 insertions(+), 24 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncWithNoOpBufferFileWriter.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java index 14bb8f7510cbe..9a78d0adf151d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.event.NotificationListener; +import org.apache.flink.util.ExceptionUtils; import java.io.IOException; @@ -31,9 +32,26 @@ protected AsynchronousBufferFileWriter(ID channelID, RequestQueue super(channelID, requestQueue, CALLBACK, true); } + /** + * Writes the given block asynchronously. + * + * @param buffer + * the buffer to be written (will be recycled when done) + * + * @throws IOException + * thrown if adding the write operation fails + */ @Override public void writeBlock(Buffer buffer) throws IOException { - addRequest(new BufferWriteRequest(this, buffer)); + try { + // if successfully added, the buffer will be recycled after the write operation + addRequest(new BufferWriteRequest(this, buffer)); + } catch (Throwable e) { + // if not added, we need to recycle here + buffer.recycle(); + ExceptionUtils.rethrowIOException(e); + } + } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java index f7618e493993c..5738787b43895 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java @@ -26,8 +26,8 @@ public interface BlockChannelWriterWithCallback extends FileIOChannel { * Writes the given block. The request may be executed synchronously, or asynchronously, depending * on the implementation. * - * @param block The segment to be written. + * @param block The segment to be written (transferring ownership to this writer). * @throws IOException Thrown, when the writer encounters an I/O error. */ void writeBlock(T block) throws IOException; -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index ed72b51ad5292..c1d6f133cff16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -67,6 +67,7 @@ public boolean add(Buffer buffer) throws IOException { synchronized (buffers) { if (isFinished || isReleased) { + buffer.recycle(); return false; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index 9b02e4d603c30..540142959a3e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -267,6 +267,7 @@ public ResultPartitionType getPartitionType() { * first buffer has been added. */ public void add(Buffer buffer, int subpartitionIndex) throws IOException { + checkNotNull(buffer); boolean success = false; try { @@ -274,6 +275,8 @@ public void add(Buffer buffer, int subpartitionIndex) throws IOException { final ResultSubpartition subpartition = subpartitions[subpartitionIndex]; + // retain for buffer use after add() but also to have a simple path for recycle() + buffer.retain(); synchronized (subpartition) { success = subpartition.add(buffer); @@ -281,14 +284,11 @@ public void add(Buffer buffer, int subpartitionIndex) throws IOException { totalNumberOfBuffers++; totalNumberOfBytes += buffer.getSize(); } - } - finally { + } finally { if (success) { notifyPipelinedConsumers(); } - else { - buffer.recycle(); - } + buffer.recycle(); } } @@ -463,7 +463,7 @@ ResultSubpartition[] getAllPartitions() { // ------------------------------------------------------------------------ - private void checkInProduceState() { + private void checkInProduceState() throws IllegalStateException { checkState(!isFinished, "Partition already finished."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java index 3b4e3c9b1a39b..e73082a7d7288 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java @@ -70,6 +70,18 @@ protected Throwable getFailureCause() { return parent.getFailureCause(); } + /** + * Adds the given buffer. + * + *

The request may be executed synchronously, or asynchronously, depending on the + * implementation. + * + * @param buffer + * the buffer to add (transferring ownership to this writer) + * + * @throws IOException + * thrown in case of errors while adding the buffer + */ abstract public boolean add(Buffer buffer) throws IOException; abstract public void finish() throws IOException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java index 065de8ee5c106..4a8e165d11230 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -92,6 +92,7 @@ public boolean add(Buffer buffer) throws IOException { synchronized (buffers) { if (isFinished || isReleased) { + buffer.recycle(); return false; } @@ -107,10 +108,15 @@ public boolean add(Buffer buffer) throws IOException { } // Didn't return early => go to disk - spillWriter.writeBlock(buffer); - synchronized (buffers) { - // See the note above, but only do this if the buffer was correctly added! - updateStatistics(buffer); + try { + // retain buffer for updateStatistics() below + spillWriter.writeBlock(buffer.retain()); + synchronized (buffers) { + // See the note above, but only do this if the buffer was correctly added! + updateStatistics(buffer); + } + } finally { + buffer.recycle(); } return true; @@ -207,7 +213,7 @@ public int releaseMemory() throws IOException { ResultSubpartitionView view = readView; if (view != null && view.getClass() == SpillableSubpartitionView.class) { - // If there is a spilalble view, it's the responsibility of the + // If there is a spillable view, it's the responsibility of the // view to release memory. SpillableSubpartitionView spillableView = (SpillableSubpartitionView) view; return spillableView.releaseMemory(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index df8de54d6ca30..6781902ca9066 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -108,11 +108,7 @@ int releaseMemory() throws IOException { for (int i = 0; i < numBuffers; i++) { Buffer buffer = buffers.remove(); spilledBytes += buffer.getSize(); - try { - spillWriter.writeBlock(buffer); - } finally { - buffer.recycle(); - } + spillWriter.writeBlock(buffer); } spilledView = new SpilledSubpartitionView( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java index 40f3e32feabdb..4c25e0d68e09b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java @@ -18,13 +18,17 @@ package org.apache.flink.runtime.io.disk.iomanager; +import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.util.TestNotificationListener; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.io.IOException; import java.util.concurrent.Callable; @@ -39,7 +43,12 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; +/** + * Tests for {@link AsynchronousBufferFileWriter}. + */ public class AsynchronousBufferFileWriterTest { + @Rule + public ExpectedException exception = ExpectedException.none(); private static final IOManager ioManager = new IOManagerAsync(); @@ -66,6 +75,27 @@ public void testAddAndHandleRequest() throws Exception { assertEquals("Didn't decrement number of outstanding requests.", 0, writer.getNumberOfOutstandingRequests()); } + @Test + public void testAddWithFailingWriter() throws Exception { + AsynchronousBufferFileWriter writer = + new AsynchronousBufferFileWriter(ioManager.createChannel(), new RequestQueue<>()); + writer.close(); + + exception.expect(IOException.class); + + Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + try { + writer.writeBlock(buffer); + } finally { + if (!buffer.isRecycled()) { + buffer.recycle(); + Assert.fail("buffer not recycled"); + } + assertEquals("Shouln't increment number of outstanding requests.", 0, writer.getNumberOfOutstandingRequests()); + } + } + @Test public void testSubscribe() throws Exception { final TestNotificationListener listener = new TestNotificationListener(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncWithNoOpBufferFileWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncWithNoOpBufferFileWriter.java new file mode 100644 index 0000000000000..363e02b9b9774 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncWithNoOpBufferFileWriter.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.flink.runtime.io.disk.iomanager; + +import org.apache.flink.runtime.io.network.buffer.Buffer; + +import java.io.IOException; + +/** + * An {@link IOManagerAsync} that creates {@link BufferFileWriter} instances which do nothing in their {@link BufferFileWriter#writeBlock(Object)} method. + * + *

Beware: the passed {@link Buffer} instances must be cleaned up manually! + */ +public class IOManagerAsyncWithNoOpBufferFileWriter extends IOManagerAsync { + @Override + public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) + throws IOException { + return new NoOpAsynchronousBufferFileWriter(channelID, getWriteRequestQueue(channelID)); + } + + /** + * {@link BufferFileWriter} subclass with a no-op in {@link #writeBlock(Buffer)}. + */ + private static class NoOpAsynchronousBufferFileWriter extends AsynchronousBufferFileWriter { + + private NoOpAsynchronousBufferFileWriter( + ID channelID, + RequestQueue requestQueue) throws IOException { + super(channelID, requestQueue); + } + + @Override + public void writeBlock(Buffer buffer) throws IOException { + // do nothing + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 0cd359197efcb..caa034ed873d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -20,21 +20,37 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.util.TestBufferFactory; import org.apache.flink.runtime.taskmanager.TaskActions; + +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; +import static org.junit.Assert.assertFalse; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +/** + * Tests for {@link ResultPartition}. + */ public class ResultPartitionTest { + /** Asynchronous I/O manager. */ + private static final IOManager ioManager = new IOManagerAsync(); + + @AfterClass + public static void shutdown() { + ioManager.shutdown(); + } + /** * Tests the schedule or update consumers message sending behaviour depending on the relevant flags. */ @@ -45,7 +61,11 @@ public void testSendScheduleOrUpdateConsumersMessage() throws Exception { ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class); ResultPartition partition = createPartition(notifier, ResultPartitionType.PIPELINED, true); partition.add(TestBufferFactory.createBuffer(), 0); - verify(notifier, times(1)).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class)); + verify(notifier, times(1)) + .notifyPartitionConsumable( + eq(partition.getJobId()), + eq(partition.getPartitionId()), + any(TaskActions.class)); } { @@ -145,6 +165,45 @@ protected void testAddOnReleasedPartition(final ResultPartitionType pipelined) } } + @Test + public void testAddOnPipelinedPartition() throws Exception { + testAddOnPartition(ResultPartitionType.PIPELINED); + } + + @Test + public void testAddOnBlockingPartition() throws Exception { + testAddOnPartition(ResultPartitionType.BLOCKING); + } + + /** + * Tests {@link ResultPartition#add(Buffer, int)} on a working partition. + * + * @param pipelined the result partition type to set up + */ + protected void testAddOnPartition(final ResultPartitionType pipelined) + throws Exception { + ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class); + ResultPartition partition = createPartition(notifier, pipelined, true); + Buffer buffer = TestBufferFactory.createBuffer(); + try { + // partition.add() adds the buffer without recycling it (if not spilling) + partition.add(buffer, 0); + assertFalse("buffer should not be recycled (still in the queue)", buffer.isRecycled()); + } finally { + if (!buffer.isRecycled()) { + buffer.recycle(); + } + // should have been notified for pipelined partitions + if (pipelined.isPipelined()) { + verify(notifier, times(1)) + .notifyPartitionConsumable( + eq(partition.getJobId()), + eq(partition.getPartitionId()), + any(TaskActions.class)); + } + } + } + // ------------------------------------------------------------------------ private static ResultPartition createPartition( @@ -161,7 +220,7 @@ private static ResultPartition createPartition( 1, mock(ResultPartitionManager.class), notifier, - mock(IOManager.class), + ioManager, sendScheduleOrUpdateConsumersMessage); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index 05a364d08efdf..c50b36125cd31 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -20,19 +20,26 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.disk.iomanager.AsynchronousBufferFileWriter; +import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsyncWithNoOpBufferFileWriter; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; + import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import java.io.IOException; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -56,12 +63,17 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +/** + * Tests for {@link SpillableSubpartition}. + */ public class SpillableSubpartitionTest extends SubpartitionTestBase { + @Rule + public ExpectedException exception = ExpectedException.none(); - /** Executor service for concurrent produce/consume tests */ - private final static ExecutorService executorService = Executors.newCachedThreadPool(); + /** Executor service for concurrent produce/consume tests. */ + private static final ExecutorService executorService = Executors.newCachedThreadPool(); - /** Asynchronous I/O manager */ + /** Asynchronous I/O manager. */ private static final IOManager ioManager = new IOManagerAsync(); @AfterClass @@ -72,6 +84,10 @@ public static void shutdown() { @Override SpillableSubpartition createSubpartition() { + return createSubpartition(ioManager); + } + + private static SpillableSubpartition createSubpartition(IOManager ioManager) { ResultPartition parent = mock(ResultPartition.class); BufferProvider bufferProvider = mock(BufferProvider.class); when(parent.getBufferProvider()).thenReturn(bufferProvider); @@ -313,6 +329,218 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception assertTrue(buffer.isRecycled()); } + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a spillable finished partition. + */ + @Test + public void testAddOnFinishedSpillablePartition() throws Exception { + testAddOnFinishedPartition(false); + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled finished partition. + */ + @Test + public void testAddOnFinishedSpilledPartition() throws Exception { + testAddOnFinishedPartition(true); + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a finished partition. + * + * @param spilled + * whether the partition should be spilled to disk (true) or not (false, + * spillable). + */ + private void testAddOnFinishedPartition(boolean spilled) throws Exception { + SpillableSubpartition partition = createSubpartition(); + if (spilled) { + assertEquals(0, partition.releaseMemory()); + } + partition.finish(); + // finish adds an EndOfPartitionEvent + assertEquals(1, partition.getTotalNumberOfBuffers()); + assertEquals(4, partition.getTotalNumberOfBytes()); + + Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + try { + partition.add(buffer); + } finally { + if (!buffer.isRecycled()) { + buffer.recycle(); + Assert.fail("buffer not recycled"); + } + // still same statistics + assertEquals(1, partition.getTotalNumberOfBuffers()); + assertEquals(4, partition.getTotalNumberOfBytes()); + } + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a spillable released partition. + */ + @Test + public void testAddOnReleasedSpillablePartition() throws Exception { + testAddOnReleasedPartition(false); + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled released partition. + */ + @Test + public void testAddOnReleasedSpilledPartition() throws Exception { + testAddOnReleasedPartition(true); + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a released partition. + * + * @param spilled + * whether the partition should be spilled to disk (true) or not (false, + * spillable). + */ + private void testAddOnReleasedPartition(boolean spilled) throws Exception { + SpillableSubpartition partition = createSubpartition(); + partition.release(); + if (spilled) { + assertEquals(0, partition.releaseMemory()); + } + + Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + try { + partition.add(buffer); + } finally { + if (!buffer.isRecycled()) { + buffer.recycle(); + Assert.fail("buffer not recycled"); + } + assertEquals(0, partition.getTotalNumberOfBuffers()); + assertEquals(0, partition.getTotalNumberOfBytes()); + } + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled partition where adding the + * write request fails with an exception. + */ + @Test + public void testAddOnSpilledPartitionWithSlowWriter() throws Exception { + // simulate slow writer by a no-op write operation + IOManager ioManager = new IOManagerAsyncWithNoOpBufferFileWriter(); + SpillableSubpartition partition = createSubpartition(ioManager); + assertEquals(0, partition.releaseMemory()); + + Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + try { + partition.add(buffer); + } finally { + ioManager.shutdown(); + if (buffer.isRecycled()) { + Assert.fail("buffer recycled before the write operation completed"); + } + buffer.recycle(); + assertEquals(1, partition.getTotalNumberOfBuffers()); + assertEquals(4096, partition.getTotalNumberOfBytes()); + } + } + + /** + * Tests {@link SpillableSubpartition#releaseMemory()} with a spillable partition without a view + * but with a writer that does not do any write to check for correct buffer recycling. + */ + @Test + public void testReleaseOnSpillablePartitionWithoutViewWithSlowWriter() throws Exception { + testReleaseOnSpillablePartitionWithSlowWriter(false); + } + + /** + * Tests {@link SpillableSubpartition#releaseMemory()} with a spillable partition which has a + * view associated with it and a writer that does not do any write to check for correct buffer + * recycling. + */ + @Test + public void testReleaseOnSpillablePartitionWithViewWithSlowWriter() throws Exception { + testReleaseOnSpillablePartitionWithSlowWriter(true); + } + + /** + * Tests {@link SpillableSubpartition#releaseMemory()} with a spillable partition which has a a + * writer that does not do any write to check for correct buffer recycling. + */ + private void testReleaseOnSpillablePartitionWithSlowWriter(boolean createView) throws Exception { + // simulate slow writer by a no-op write operation + IOManager ioManager = new IOManagerAsyncWithNoOpBufferFileWriter(); + SpillableSubpartition partition = createSubpartition(ioManager); + + Buffer buffer1 = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + Buffer buffer2 = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + try { + // we need two buffers because the view will use one of them and not release it + partition.add(buffer1); + partition.add(buffer2); + assertFalse("buffer1 should not be recycled (still in the queue)", buffer1.isRecycled()); + assertFalse("buffer2 should not be recycled (still in the queue)", buffer2.isRecycled()); + assertEquals(2, partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 2, partition.getTotalNumberOfBytes()); + + if (createView) { + // Create a read view + partition.finish(); + partition.createReadView(numBuffers -> {}); + } + + // one instance of the buffers is placed in the view's nextBuffer and not released + // (if there is no view, there will be no additional EndOfPartitionEvent) + assertEquals(2, partition.releaseMemory()); + assertFalse("buffer1 should not be recycled (advertised as nextBuffer)", buffer1.isRecycled()); + assertFalse("buffer2 should not be recycled (not written yet)", buffer2.isRecycled()); + } finally { + ioManager.shutdown(); + if (!buffer1.isRecycled()) { + buffer1.recycle(); + } + if (!buffer2.isRecycled()) { + buffer2.recycle(); + } + // note: a view requires a finished partition which has an additional EndOfPartitionEvent + assertEquals(2 + (createView ? 1 : 0), partition.getTotalNumberOfBuffers()); + assertEquals(4096 * 2 + (createView ? 4 : 0), partition.getTotalNumberOfBytes()); + } + } + + /** + * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled partition where adding the + * write request fails with an exception. + */ + @Test + public void testAddOnSpilledPartitionWithFailingWriter() throws Exception { + IOManager ioManager = new IOManagerAsyncWithClosedBufferFileWriter(); + SpillableSubpartition partition = createSubpartition(ioManager); + assertEquals(0, partition.releaseMemory()); + + exception.expect(IOException.class); + + Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + try { + partition.add(buffer); + } finally { + ioManager.shutdown(); + + if (!buffer.isRecycled()) { + buffer.recycle(); + Assert.fail("buffer not recycled"); + } + assertEquals(0, partition.getTotalNumberOfBuffers()); + assertEquals(0, partition.getTotalNumberOfBytes()); + } + } + private static class AwaitableBufferAvailablityListener implements BufferAvailabilityListener { private long numNotifiedBuffers; @@ -333,4 +561,22 @@ void awaitNotifications(long awaitedNumNotifiedBuffers, long timeoutMillis) thro } } } + + /** + * An {@link IOManagerAsync} that creates closed {@link BufferFileWriter} instances in its + * {@link #createBufferFileWriter(FileIOChannel.ID)} method. + * + *

These {@link BufferFileWriter} objects will thus throw an exception when trying to add + * write requests, e.g. by calling {@link BufferFileWriter#writeBlock(Object)}. + */ + private static class IOManagerAsyncWithClosedBufferFileWriter extends IOManagerAsync { + @Override + public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) + throws IOException { + BufferFileWriter bufferFileWriter = super.createBufferFileWriter(channelID); + bufferFileWriter.close(); + return bufferFileWriter; + } + } + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java index fa62593387122..b748e1c30f8a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java @@ -44,6 +44,10 @@ import static org.mockito.Mockito.mock; +/** + * Tests for {@link SpillableSubpartitionView}, in addition to indirect tests via {@link + * SpillableSubpartitionTest}. + */ public class SpilledSubpartitionViewTest { private static final IOManager IO_MANAGER = new IOManagerAsync(); From 71ede3992afe8f6907dd3c6c1e232c5b745048b4 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 5 Jan 2018 18:18:35 +0100 Subject: [PATCH 188/367] [FLINK-8371][network] always recycle Buffers when releasing SpillableSubpartition There were places where Buffer instances were not released upon SpillableSubpartition#release() with a view attached to a non-spilled subpartition: 1) SpillableSubpartition#buffer: SpillableSubpartition#release() delegates the recycling to the view, but SpillableSubpartitionView does not clean up the 'buffers' queue (the recycling was only done by the subpartition if there was no view). 2) SpillableSubpartitionView#nextBuffer: If this field is populated when the subpartition is released, it will neither be given out in subsequent SpillableSubpartitionView#getNextBuffer() calls (there was a short path returning 'null' here), nor was it recycled -> similarly to the PipelinesSubpartition implementation, make SpillableSubpartition#release() always clean up and recycle the buffers -> recycle SpillableSubpartitionView#nextBuffer in SpillableSubpartitionView#releaseAllResources() This closes #5276. --- .../partition/PipelinedSubpartition.java | 2 - .../partition/SpillableSubpartition.java | 11 ++- .../partition/SpillableSubpartitionView.java | 7 ++ .../partition/PipelinedSubpartitionTest.java | 72 ++++++++++++++ .../partition/SpillableSubpartitionTest.java | 99 +++++++++++++++++++ 5 files changed, 185 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index c1d6f133cff16..92eb7ce572372 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -128,7 +128,6 @@ public void release() { buffer.recycle(); } - // Get the view... view = readView; readView = null; @@ -138,7 +137,6 @@ public void release() { LOG.debug("Released {}.", this); - // Release all resources of the view if (view != null) { view.releaseAllResources(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java index 4a8e165d11230..093e9c89847e3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -138,6 +138,7 @@ public void finish() throws IOException { @Override public void release() throws IOException { + // view reference accessible outside the lock, but assigned inside the locked scope final ResultSubpartitionView view; synchronized (buffers) { @@ -145,16 +146,18 @@ public void release() throws IOException { return; } + // Release all available buffers + for (Buffer buffer : buffers) { + buffer.recycle(); + } + buffers.clear(); + view = readView; // No consumer yet, we are responsible to clean everything up. If // one is available, the view is responsible is to clean up (see // below). if (view == null) { - for (Buffer buffer : buffers) { - buffer.recycle(); - } - buffers.clear(); // TODO This can block until all buffers are written out to // disk if a spill is in-progress before deleting the file. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index 6781902ca9066..f88a6b6e5a4f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -165,6 +165,13 @@ public void releaseAllResources() throws IOException { if (spilled != null) { spilled.releaseAllResources(); } + // we are never giving this buffer out in getNextBuffer(), so we need to clean it up + synchronized (buffers) { + if (nextBuffer != null) { + nextBuffer.recycle(); + nextBuffer = null; + } + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 6d36aa6b87599..81ac40a6e450c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -19,16 +19,20 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.util.TestConsumerCallback; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; import org.apache.flink.runtime.io.network.util.TestProducerSource; import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer; import org.apache.flink.runtime.io.network.util.TestSubpartitionProducer; + import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Test; import java.util.concurrent.ExecutorService; @@ -238,4 +242,72 @@ public void onEvent(AbstractEvent event) { producerResult.get(); consumerResult.get(); } + + + /** + * Tests cleanup of {@link PipelinedSubpartition#release()} with no read view attached. + */ + @Test + public void testCleanupReleasedPartitionNoView() throws Exception { + testCleanupReleasedPartition(false); + } + + /** + * Tests cleanup of {@link PipelinedSubpartition#release()} with a read view attached. + */ + @Test + public void testCleanupReleasedPartitionWithView() throws Exception { + testCleanupReleasedPartition(true); + } + + /** + * Tests cleanup of {@link PipelinedSubpartition#release()}. + * + * @param createView + * whether the partition should have a view attached to it (true) or not (false) + */ + private void testCleanupReleasedPartition(boolean createView) throws Exception { + PipelinedSubpartition partition = createSubpartition(); + + Buffer buffer1 = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + Buffer buffer2 = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + boolean buffer1Recycled; + boolean buffer2Recycled; + try { + partition.add(buffer1); + partition.add(buffer2); + // create the read view first + ResultSubpartitionView view = null; + if (createView) { + view = partition.createReadView(numBuffers -> {}); + } + + partition.release(); + + assertTrue(partition.isReleased()); + if (createView) { + assertTrue(view.isReleased()); + } + assertTrue(buffer1.isRecycled()); + } finally { + buffer1Recycled = buffer1.isRecycled(); + if (!buffer1Recycled) { + buffer1.recycle(); + } + buffer2Recycled = buffer2.isRecycled(); + if (!buffer2Recycled) { + buffer2.recycle(); + } + } + if (!buffer1Recycled) { + Assert.fail("buffer 1 not recycled"); + } + if (!buffer2Recycled) { + Assert.fail("buffer 2 not recycled"); + } + assertEquals(2, partition.getTotalNumberOfBuffers()); + assertEquals(2 * 4096, partition.getTotalNumberOfBytes()); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index c50b36125cd31..18169b6ceb78d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -541,6 +541,105 @@ public void testAddOnSpilledPartitionWithFailingWriter() throws Exception { } } + /** + * Tests cleanup of {@link SpillableSubpartition#release()} with a spillable partition and no + * read view attached. + */ + @Test + public void testCleanupReleasedSpillablePartitionNoView() throws Exception { + testCleanupReleasedPartition(false, false); + } + + /** + * Tests cleanup of {@link SpillableSubpartition#release()} with a spillable partition and a + * read view attached - [FLINK-8371]. + */ + @Test + public void testCleanupReleasedSpillablePartitionWithView() throws Exception { + testCleanupReleasedPartition(false, true); + } + + /** + * Tests cleanup of {@link SpillableSubpartition#release()} with a spilled partition and no + * read view attached. + */ + @Test + public void testCleanupReleasedSpilledPartitionNoView() throws Exception { + testCleanupReleasedPartition(true, false); + } + + /** + * Tests cleanup of {@link SpillableSubpartition#release()} with a spilled partition and a + * read view attached. + */ + @Test + public void testCleanupReleasedSpilledPartitionWithView() throws Exception { + testCleanupReleasedPartition(true, true); + } + + /** + * Tests cleanup of {@link SpillableSubpartition#release()}. + * + * @param spilled + * whether the partition should be spilled to disk (true) or not (false, + * spillable) + * @param createView + * whether the partition should have a view attached to it (true) or not (false) + */ + private void testCleanupReleasedPartition(boolean spilled, boolean createView) throws Exception { + SpillableSubpartition partition = createSubpartition(); + + Buffer buffer1 = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + Buffer buffer2 = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); + boolean buffer1Recycled; + boolean buffer2Recycled; + try { + partition.add(buffer1); + partition.add(buffer2); + // create the read view before spilling + // (tests both code paths since this view may then contain the spilled view) + ResultSubpartitionView view = null; + if (createView) { + partition.finish(); + view = partition.createReadView(numBuffers -> {}); + } + if (spilled) { + // note: in case we create a view, one buffer will already reside in the view and + // one EndOfPartitionEvent will be added instead (so overall the number of + // buffers to spill is the same + assertEquals(2, partition.releaseMemory()); + } + + partition.release(); + + assertTrue(partition.isReleased()); + if (createView) { + assertTrue(view.isReleased()); + } + assertTrue(buffer1.isRecycled()); + } finally { + buffer1Recycled = buffer1.isRecycled(); + if (!buffer1Recycled) { + buffer1.recycle(); + } + buffer2Recycled = buffer2.isRecycled(); + if (!buffer2Recycled) { + buffer2.recycle(); + } + } + if (!buffer1Recycled) { + Assert.fail("buffer 1 not recycled"); + } + if (!buffer2Recycled) { + Assert.fail("buffer 2 not recycled"); + } + // note: in case we create a view, there will be an additional EndOfPartitionEvent + assertEquals(createView ? 3 : 2, partition.getTotalNumberOfBuffers()); + assertEquals((createView ? 4 : 0) + 2 * 4096, partition.getTotalNumberOfBytes()); + } + private static class AwaitableBufferAvailablityListener implements BufferAvailabilityListener { private long numNotifiedBuffers; From b32ac3aaf9dd7732f4c7f0ed43b6467d17c08444 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Sat, 9 Dec 2017 10:55:14 +0800 Subject: [PATCH 189/367] [FLINK-8226] [cep] Dangling reference generated after NFA clean up timed out SharedBufferEntry This closes #5141 --- .../apache/flink/cep/nfa/SharedBuffer.java | 48 ++++++++++++++----- .../org/apache/flink/cep/nfa/NFATest.java | 34 +++++++++++++ 2 files changed, 71 insertions(+), 11 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java index 0cf47ca1fcf69..b72bef263ac6e 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java @@ -191,14 +191,12 @@ public boolean isEmpty() { */ public boolean prune(long pruningTimestamp) { Iterator>> iter = pages.entrySet().iterator(); - boolean pruned = false; + List> prunedEntries = new ArrayList<>(); while (iter.hasNext()) { SharedBufferPage page = iter.next().getValue(); - if (page.prune(pruningTimestamp)) { - pruned = true; - } + page.prune(pruningTimestamp, prunedEntries); if (page.isEmpty()) { // delete page if it is empty @@ -206,7 +204,14 @@ public boolean prune(long pruningTimestamp) { } } - return pruned; + if (!prunedEntries.isEmpty()) { + for (Map.Entry> entry : pages.entrySet()) { + entry.getValue().removeEdges(prunedEntries); + } + return true; + } else { + return false; + } } /** @@ -451,25 +456,21 @@ public SharedBufferEntry get(final ValueTimeWrapper valueTime) { * Removes all entries from the map whose timestamp is smaller than the pruning timestamp. * * @param pruningTimestamp Timestamp for the pruning - * @return {@code true} if pruning happened */ - public boolean prune(long pruningTimestamp) { + public void prune(long pruningTimestamp, List> prunedEntries) { Iterator, SharedBufferEntry>> iterator = entries.entrySet().iterator(); boolean continuePruning = true; - boolean pruned = false; while (iterator.hasNext() && continuePruning) { SharedBufferEntry entry = iterator.next().getValue(); if (entry.getValueTime().getTimestamp() <= pruningTimestamp) { + prunedEntries.add(entry); iterator.remove(); - pruned = true; } else { continuePruning = false; } } - - return pruned; } public boolean isEmpty() { @@ -480,6 +481,15 @@ public SharedBufferEntry remove(final ValueTimeWrapper valueTime) { return entries.remove(valueTime); } + /** + * Remove edges with the specified targets for the entries. + */ + private void removeEdges(final List> prunedEntries) { + for (Map.Entry, SharedBufferEntry> entry : entries.entrySet()) { + entry.getValue().removeEdges(prunedEntries); + } + } + @Override public String toString() { StringBuilder builder = new StringBuilder(); @@ -566,6 +576,22 @@ public void addEdge(SharedBufferEdge edge) { edges.add(edge); } + /** + * Remove edges with the specified targets. + */ + private void removeEdges(final List> prunedEntries) { + Iterator> itor = edges.iterator(); + while (itor.hasNext()) { + SharedBufferEdge edge = itor.next(); + for (SharedBufferEntry prunedEntry : prunedEntries) { + if (prunedEntry == edge.getTarget()) { + itor.remove(); + break; + } + } + } + } + public boolean remove() { if (page != null) { page.remove(valueTime); diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java index 0f4066fef54ae..2a12d37210973 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java @@ -26,6 +26,7 @@ import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; @@ -176,6 +177,26 @@ public void testTimeoutWindowPruningWindowBorders() { assertEquals(expectedPatterns, actualPatterns); } + @Test + public void testTimeoutWindowPruning2() throws IOException { + NFA nfa = createLoopingNFA(2); + List> streamEvents = new ArrayList<>(); + + streamEvents.add(new StreamRecord<>(new Event(1, "loop", 1.0), 101L)); + streamEvents.add(new StreamRecord<>(new Event(2, "loop", 2.0), 102L)); + streamEvents.add(new StreamRecord<>(new Event(3, "loop", 3.0), 103L)); + streamEvents.add(new StreamRecord<>(new Event(4, "loop", 4.0), 104L)); + streamEvents.add(new StreamRecord<>(new Event(5, "loop", 5.0), 105L)); + runNFA(nfa, streamEvents); + + NFA.NFASerializer serializer = new NFA.NFASerializer<>(Event.createTypeSerializer()); + + //serialize + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.serialize(nfa, new DataOutputViewStreamWrapper(baos)); + baos.close(); + } + public Collection>> runNFA(NFA nfa, List> inputs) { Set>> actualPatterns = new HashSet<>(); @@ -358,4 +379,17 @@ public boolean filter(Event value) throws Exception { return nfa; } + + private NFA createLoopingNFA(long windowLength) { + Pattern pattern = Pattern.begin("loop").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("loop"); + } + }).timesOrMore(3).within(Time.milliseconds(windowLength)); + + return NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + } } From b607a8b95d11d80f68cdd167289a52d1f4c49d24 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Wed, 21 Jun 2017 15:55:19 -0700 Subject: [PATCH 190/367] [FLINK-6951] [kinesis] Shade httpcomponents dependency for Kinesis connector This closes #4150. --- flink-connectors/flink-connector-kinesis/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 83934f64608be..f64b831eb3c85 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -154,6 +154,7 @@ under the License. com.amazonaws:* com.google.protobuf:* + org.apache.httpcomponents:* @@ -166,6 +167,10 @@ under the License. com.amazonaws org.apache.flink.kinesis.shaded.com.amazonaws + + org.apache.http + org.apache.flink.kinesis.shaded.org.apache.http + From ae00cad38c868ee294d1d0930731974bfefcb727 Mon Sep 17 00:00:00 2001 From: Alejandro Alcalde Date: Thu, 11 Jan 2018 13:49:29 +0100 Subject: [PATCH 191/367] [hotfix] [doc] Fixed doc typo in DataStream API This closes #5283. This closes #5191. --- docs/dev/datastream_api.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md index 09580b1dd5fda..d7ab2e7a6fbec 100644 --- a/docs/dev/datastream_api.md +++ b/docs/dev/datastream_api.md @@ -161,7 +161,7 @@ File-based: *IMPLEMENTATION:* - Under the hood, Flink splits the file reading process into two sub-tasks, namely *directory monitoring* and *data reading*. Each of these sub-tasks is implemented by a separate entity. Monitoring is implemented by a single, **non-parallel** (parallelism = 1) task, while reading is performed by multiple tasks running in parallel. The parallelism of the latter is equal to the job parallelism. The role of the single monitoring task is to scan the directory (periodically or only once depending on the `watchType`), find the files to be processed, divide them in *splits*, and assign these splits to the downstream readers. The readers are the ones who will read the actual data. Each split is read by only one reader, while a reader can read muplitple splits, one-by-one. + Under the hood, Flink splits the file reading process into two sub-tasks, namely *directory monitoring* and *data reading*. Each of these sub-tasks is implemented by a separate entity. Monitoring is implemented by a single, **non-parallel** (parallelism = 1) task, while reading is performed by multiple tasks running in parallel. The parallelism of the latter is equal to the job parallelism. The role of the single monitoring task is to scan the directory (periodically or only once depending on the `watchType`), find the files to be processed, divide them in *splits*, and assign these splits to the downstream readers. The readers are the ones who will read the actual data. Each split is read by only one reader, while a reader can read multiple splits, one-by-one. *IMPORTANT NOTES:* @@ -219,7 +219,7 @@ File-based: *IMPLEMENTATION:* - Under the hood, Flink splits the file reading process into two sub-tasks, namely *directory monitoring* and *data reading*. Each of these sub-tasks is implemented by a separate entity. Monitoring is implemented by a single, **non-parallel** (parallelism = 1) task, while reading is performed by multiple tasks running in parallel. The parallelism of the latter is equal to the job parallelism. The role of the single monitoring task is to scan the directory (periodically or only once depending on the `watchType`), find the files to be processed, divide them in *splits*, and assign these splits to the downstream readers. The readers are the ones who will read the actual data. Each split is read by only one reader, while a reader can read muplitple splits, one-by-one. + Under the hood, Flink splits the file reading process into two sub-tasks, namely *directory monitoring* and *data reading*. Each of these sub-tasks is implemented by a separate entity. Monitoring is implemented by a single, **non-parallel** (parallelism = 1) task, while reading is performed by multiple tasks running in parallel. The parallelism of the latter is equal to the job parallelism. The role of the single monitoring task is to scan the directory (periodically or only once depending on the `watchType`), find the files to be processed, divide them in *splits*, and assign these splits to the downstream readers. The readers are the ones who will read the actual data. Each split is read by only one reader, while a reader can read multiple splits, one-by-one. *IMPORTANT NOTES:* From b7f8a5a2b03d5a5c8ca2f119486e17c24befbc99 Mon Sep 17 00:00:00 2001 From: maqingxiang-it Date: Thu, 11 Jan 2018 13:44:04 +0800 Subject: [PATCH 192/367] [hotfix] Fix typo in AbstractMetricGroup.java This closes #5280. --- .../flink/runtime/metrics/groups/AbstractMetricGroup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java index 66eace588582a..23b969a8c4024 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java @@ -352,7 +352,7 @@ protected void addMetric(String name, Metric metric) { // add the metric only if the group is still open synchronized (this) { if (!closed) { - // immediately put without a 'contains' check to optimize the common case (no collition) + // immediately put without a 'contains' check to optimize the common case (no collision) // collisions are resolved later Metric prior = metrics.put(name, metric); From 968683f15378122478e34bbba6f9c021b8b086ae Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 19 Dec 2017 16:10:44 -0800 Subject: [PATCH 193/367] [FLINK-8296] [kafka] Rework FlinkKafkaConsumerBaseTest to not rely on Java reflection Reflection was mainly used to inject mocks into private fields of the FlinkKafkaConsumerBase, without the need to fully execute all operator life cycle methods. This, however, caused the unit tests to be too implementation-specific. This commit reworks the FlinkKafkaConsumerBaseTest to remove test consumer instantiation methods that rely on reflection for dependency injection. All tests now instantiate dummy test consumers normally, and let all tests properly execute all operator life cycle methods regardless of the tested logic. This closes #5188. --- .../kafka/FlinkKafkaConsumerBase.java | 5 + .../kafka/FlinkKafkaConsumerBaseTest.java | 532 +++++++++++------- 2 files changed, 338 insertions(+), 199 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 865d66c3c1412..44e0d5486f953 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -882,4 +882,9 @@ TreeMap getRestoredState() { OffsetCommitMode getOffsetCommitMode() { return offsetCommitMode; } + + @VisibleForTesting + LinkedMap getPendingOffsetsToCommit() { + return pendingOffsetsToCommit; + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 6ccfeb1d0664f..4361c6720f1e5 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -18,17 +18,27 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.state.KeyedStateStore; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; @@ -44,16 +54,13 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; -import org.apache.commons.collections.map.LinkedMap; import org.junit.Assert; import org.junit.Test; import org.mockito.Matchers; import org.mockito.Mockito; import java.io.Serializable; -import java.lang.reflect.Field; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -86,20 +93,19 @@ public class FlinkKafkaConsumerBaseTest { * Tests that not both types of timestamp extractors / watermark generators can be used. */ @Test + @SuppressWarnings("unchecked") public void testEitherWatermarkExtractor() { try { - new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPeriodicWatermarks) null); + new DummyFlinkKafkaConsumer().assignTimestampsAndWatermarks((AssignerWithPeriodicWatermarks) null); fail(); } catch (NullPointerException ignored) {} try { - new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPunctuatedWatermarks) null); + new DummyFlinkKafkaConsumer().assignTimestampsAndWatermarks((AssignerWithPunctuatedWatermarks) null); fail(); } catch (NullPointerException ignored) {} - @SuppressWarnings("unchecked") final AssignerWithPeriodicWatermarks periodicAssigner = mock(AssignerWithPeriodicWatermarks.class); - @SuppressWarnings("unchecked") final AssignerWithPunctuatedWatermarks punctuatedAssigner = mock(AssignerWithPunctuatedWatermarks.class); DummyFlinkKafkaConsumer c1 = new DummyFlinkKafkaConsumer<>(); @@ -123,17 +129,16 @@ public void testEitherWatermarkExtractor() { @Test public void ignoreCheckpointWhenNotRunning() throws Exception { @SuppressWarnings("unchecked") - final AbstractFetcher fetcher = mock(AbstractFetcher.class); + final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>(); - FlinkKafkaConsumerBase consumer = getConsumer(fetcher, new LinkedMap(), false); - OperatorStateStore operatorStateStore = mock(OperatorStateStore.class); - TestingListState> listState = new TestingListState<>(); - when(operatorStateStore.getListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); + final TestingListState> listState = new TestingListState<>(); + setupConsumer(consumer, false, listState, true, 0, 1); + // snapshot before the fetcher starts running consumer.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1)); + // no state should have been checkpointed assertFalse(listState.get().iterator().hasNext()); - consumer.notifyCheckpointComplete(66L); } /** @@ -142,32 +147,13 @@ public void ignoreCheckpointWhenNotRunning() throws Exception { */ @Test public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { - OperatorStateStore operatorStateStore = mock(OperatorStateStore.class); - - TestingListState restoredListState = new TestingListState<>(); - restoredListState.add(Tuple2.of(new KafkaTopicPartition("abc", 13), 16768L)); - restoredListState.add(Tuple2.of(new KafkaTopicPartition("def", 7), 987654321L)); - - FlinkKafkaConsumerBase consumer = getConsumer(null, new LinkedMap(), true); - StreamingRuntimeContext context = mock(StreamingRuntimeContext.class); - when(context.getNumberOfParallelSubtasks()).thenReturn(1); - when(context.getIndexOfThisSubtask()).thenReturn(0); - consumer.setRuntimeContext(context); - - // mock old 1.2 state (empty) - when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); - // mock 1.3 state - when(operatorStateStore.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(restoredListState); - - StateInitializationContext initializationContext = mock(StateInitializationContext.class); - - when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore); - when(initializationContext.isRestored()).thenReturn(true); - - consumer.initializeState(initializationContext); + @SuppressWarnings("unchecked") + final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>(); - consumer.open(new Configuration()); + final TestingListState> restoredListState = new TestingListState<>(); + setupConsumer(consumer, true, restoredListState, true, 0, 1); + // snapshot before the fetcher starts running consumer.snapshotState(new StateSnapshotContextSynchronousImpl(17, 17)); // ensure that the list was cleared and refilled. while this is an implementation detail, we use it here @@ -192,67 +178,68 @@ public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { @Test public void testConfigureOnCheckpointsCommitMode() throws Exception { + @SuppressWarnings("unchecked") + // auto-commit enabled; this should be ignored in this case + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); - DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer(); - consumer.setIsAutoCommitEnabled(true); // this should be ignored - - StreamingRuntimeContext context = mock(StreamingRuntimeContext.class); - when(context.getIndexOfThisSubtask()).thenReturn(0); - when(context.getNumberOfParallelSubtasks()).thenReturn(1); - when(context.isCheckpointingEnabled()).thenReturn(true); // enable checkpointing, auto commit should be ignored - consumer.setRuntimeContext(context); + setupConsumer( + consumer, + false, + null, + true, // enable checkpointing; auto commit should be ignored + 0, + 1); - consumer.open(new Configuration()); assertEquals(OffsetCommitMode.ON_CHECKPOINTS, consumer.getOffsetCommitMode()); } @Test public void testConfigureAutoCommitMode() throws Exception { + @SuppressWarnings("unchecked") + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); - DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer(); - consumer.setIsAutoCommitEnabled(true); - - StreamingRuntimeContext context = mock(StreamingRuntimeContext.class); - when(context.getIndexOfThisSubtask()).thenReturn(0); - when(context.getNumberOfParallelSubtasks()).thenReturn(1); - when(context.isCheckpointingEnabled()).thenReturn(false); // disable checkpointing, auto commit should be respected - consumer.setRuntimeContext(context); + setupConsumer( + consumer, + false, + null, + false, // disable checkpointing; auto commit should be respected + 0, + 1); - consumer.open(new Configuration()); assertEquals(OffsetCommitMode.KAFKA_PERIODIC, consumer.getOffsetCommitMode()); } @Test public void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception { - - DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer(); - consumer.setIsAutoCommitEnabled(true); // this should be ignored - - StreamingRuntimeContext context = mock(StreamingRuntimeContext.class); - when(context.getIndexOfThisSubtask()).thenReturn(0); - when(context.getNumberOfParallelSubtasks()).thenReturn(1); - when(context.isCheckpointingEnabled()).thenReturn(true); // enable checkpointing, auto commit should be ignored - consumer.setRuntimeContext(context); - + @SuppressWarnings("unchecked") + // auto-commit enabled; this should be ignored in this case + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); consumer.setCommitOffsetsOnCheckpoints(false); // disabling offset committing should override everything - consumer.open(new Configuration()); + setupConsumer( + consumer, + false, + null, + true, // enable checkpointing; auto commit should be ignored + 0, + 1); + assertEquals(OffsetCommitMode.DISABLED, consumer.getOffsetCommitMode()); } @Test public void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Exception { + @SuppressWarnings("unchecked") + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(false); - DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer(); - consumer.setIsAutoCommitEnabled(false); - - StreamingRuntimeContext context = mock(StreamingRuntimeContext.class); - when(context.getIndexOfThisSubtask()).thenReturn(0); - when(context.getNumberOfParallelSubtasks()).thenReturn(1); - when(context.isCheckpointingEnabled()).thenReturn(false); // disable checkpointing, auto commit should be respected - consumer.setRuntimeContext(context); + setupConsumer( + consumer, + false, + null, + false, // disable checkpointing; auto commit should be respected + 0, + 1); - consumer.open(new Configuration()); assertEquals(OffsetCommitMode.DISABLED, consumer.getOffsetCommitMode()); } @@ -278,36 +265,37 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { // -------------------------------------------------------------------- - final AbstractFetcher fetcher = mock(AbstractFetcher.class); + final OneShotLatch runLatch = new OneShotLatch(); + final OneShotLatch stopLatch = new OneShotLatch(); + final AbstractFetcher fetcher = getRunnableMockFetcher(runLatch, stopLatch); when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3); - final LinkedMap pendingOffsetsToCommit = new LinkedMap(); - - FlinkKafkaConsumerBase consumer = getConsumer(fetcher, pendingOffsetsToCommit, true); - StreamingRuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class); - when(mockRuntimeContext.isCheckpointingEnabled()).thenReturn(true); // enable checkpointing - when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0); - when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1); - consumer.setRuntimeContext(mockRuntimeContext); - - assertEquals(0, pendingOffsetsToCommit.size()); - - OperatorStateStore backend = mock(OperatorStateStore.class); + final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( + fetcher, + mock(AbstractPartitionDiscoverer.class), + false); - TestingListState listState = new TestingListState<>(); - // mock old 1.2 state (empty) - when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); - // mock 1.3 state - when(backend.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); + final TestingListState listState = new TestingListState<>(); - StateInitializationContext initializationContext = mock(StateInitializationContext.class); + // setup and run the consumer; wait until the consumer reaches the main fetch loop before continuing test + setupConsumer(consumer, false, listState, true, 0, 1); - when(initializationContext.getOperatorStateStore()).thenReturn(backend); - when(initializationContext.isRestored()).thenReturn(false, true, true, true); + final CheckedThread runThread = new CheckedThread() { + @Override + public void go() throws Exception { + consumer.run(mock(SourceFunction.SourceContext.class)); + } - consumer.initializeState(initializationContext); + @Override + public void sync() throws Exception { + stopLatch.trigger(); + super.sync(); + } + }; + runThread.start(); + runLatch.await(); - consumer.open(new Configuration()); + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // checkpoint 1 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(138, 138)); @@ -320,8 +308,8 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { } assertEquals(state1, snapshot1); - assertEquals(1, pendingOffsetsToCommit.size()); - assertEquals(state1, pendingOffsetsToCommit.get(138L)); + assertEquals(1, consumer.getPendingOffsetsToCommit().size()); + assertEquals(state1, consumer.getPendingOffsetsToCommit().get(138L)); // checkpoint 2 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140)); @@ -334,13 +322,13 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { } assertEquals(state2, snapshot2); - assertEquals(2, pendingOffsetsToCommit.size()); - assertEquals(state2, pendingOffsetsToCommit.get(140L)); + assertEquals(2, consumer.getPendingOffsetsToCommit().size()); + assertEquals(state2, consumer.getPendingOffsetsToCommit().get(140L)); // ack checkpoint 1 consumer.notifyCheckpointComplete(138L); - assertEquals(1, pendingOffsetsToCommit.size()); - assertTrue(pendingOffsetsToCommit.containsKey(140L)); + assertEquals(1, consumer.getPendingOffsetsToCommit().size()); + assertTrue(consumer.getPendingOffsetsToCommit().containsKey(140L)); // checkpoint 3 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141)); @@ -353,37 +341,35 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { } assertEquals(state3, snapshot3); - assertEquals(2, pendingOffsetsToCommit.size()); - assertEquals(state3, pendingOffsetsToCommit.get(141L)); + assertEquals(2, consumer.getPendingOffsetsToCommit().size()); + assertEquals(state3, consumer.getPendingOffsetsToCommit().get(141L)); // ack checkpoint 3, subsumes number 2 consumer.notifyCheckpointComplete(141L); - assertEquals(0, pendingOffsetsToCommit.size()); + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); consumer.notifyCheckpointComplete(666); // invalid checkpoint - assertEquals(0, pendingOffsetsToCommit.size()); - - OperatorStateStore operatorStateStore = mock(OperatorStateStore.class); - listState = new TestingListState<>(); - when(operatorStateStore.getListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // create 500 snapshots for (int i = 100; i < 600; i++) { consumer.snapshotState(new StateSnapshotContextSynchronousImpl(i, i)); listState.clear(); } - assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, pendingOffsetsToCommit.size()); + assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, consumer.getPendingOffsetsToCommit().size()); // commit only the second last consumer.notifyCheckpointComplete(598); - assertEquals(1, pendingOffsetsToCommit.size()); + assertEquals(1, consumer.getPendingOffsetsToCommit().size()); // access invalid checkpoint consumer.notifyCheckpointComplete(590); // and the last consumer.notifyCheckpointComplete(599); - assertEquals(0, pendingOffsetsToCommit.size()); + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); + + runThread.sync(); } @Test @@ -407,38 +393,38 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception // -------------------------------------------------------------------- - final AbstractFetcher fetcher = mock(AbstractFetcher.class); + final OneShotLatch runLatch = new OneShotLatch(); + final OneShotLatch stopLatch = new OneShotLatch(); + final AbstractFetcher fetcher = getRunnableMockFetcher(runLatch, stopLatch); when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3); - final LinkedMap pendingOffsetsToCommit = new LinkedMap(); - - FlinkKafkaConsumerBase consumer = getConsumer(fetcher, pendingOffsetsToCommit, true); - StreamingRuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class); - when(mockRuntimeContext.isCheckpointingEnabled()).thenReturn(true); // enable checkpointing - when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0); - when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1); - consumer.setRuntimeContext(mockRuntimeContext); - + final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( + fetcher, + mock(AbstractPartitionDiscoverer.class), + false); consumer.setCommitOffsetsOnCheckpoints(false); // disable offset committing - assertEquals(0, pendingOffsetsToCommit.size()); - - OperatorStateStore backend = mock(OperatorStateStore.class); + final TestingListState listState = new TestingListState<>(); - TestingListState listState = new TestingListState<>(); - // mock old 1.2 state (empty) - when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); - // mock 1.3 state - when(backend.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); + // setup and run the consumer; wait until the consumer reaches the main fetch loop before continuing test + setupConsumer(consumer, false, listState, true, 0, 1); - StateInitializationContext initializationContext = mock(StateInitializationContext.class); - - when(initializationContext.getOperatorStateStore()).thenReturn(backend); - when(initializationContext.isRestored()).thenReturn(false, true, true, true); + final CheckedThread runThread = new CheckedThread() { + @Override + public void go() throws Exception { + consumer.run(mock(SourceFunction.SourceContext.class)); + } - consumer.initializeState(initializationContext); + @Override + public void sync() throws Exception { + stopLatch.trigger(); + super.sync(); + } + }; + runThread.start(); + runLatch.await(); - consumer.open(new Configuration()); + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // checkpoint 1 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(138, 138)); @@ -451,7 +437,7 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception } assertEquals(state1, snapshot1); - assertEquals(0, pendingOffsetsToCommit.size()); // pending offsets to commit should not be updated + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // pending offsets to commit should not be updated // checkpoint 2 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140)); @@ -464,7 +450,7 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception } assertEquals(state2, snapshot2); - assertEquals(0, pendingOffsetsToCommit.size()); // pending offsets to commit should not be updated + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // pending offsets to commit should not be updated // ack checkpoint 1 consumer.notifyCheckpointComplete(138L); @@ -481,7 +467,7 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception } assertEquals(state3, snapshot3); - assertEquals(0, pendingOffsetsToCommit.size()); // pending offsets to commit should not be updated + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // pending offsets to commit should not be updated // ack checkpoint 3, subsumes number 2 consumer.notifyCheckpointComplete(141L); @@ -490,16 +476,12 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception consumer.notifyCheckpointComplete(666); // invalid checkpoint verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed - OperatorStateStore operatorStateStore = mock(OperatorStateStore.class); - listState = new TestingListState<>(); - when(operatorStateStore.getListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); - // create 500 snapshots for (int i = 100; i < 600; i++) { consumer.snapshotState(new StateSnapshotContextSynchronousImpl(i, i)); listState.clear(); } - assertEquals(0, pendingOffsetsToCommit.size()); // pending offsets to commit should not be updated + assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // pending offsets to commit should not be updated // commit only the second last consumer.notifyCheckpointComplete(598); @@ -532,7 +514,7 @@ public void testScaleDown() throws Exception { * of topics fetched from Kafka. */ @SuppressWarnings("unchecked") - void testRescaling( + private void testRescaling( final int initialParallelism, final int numPartitions, final int restoredParallelism, @@ -554,8 +536,14 @@ void testRescaling( new AbstractStreamOperatorTestHarness[initialParallelism]; for (int i = 0; i < initialParallelism; i++) { - consumers[i] = new DummyFlinkKafkaConsumer<>( - Collections.singletonList("test-topic"), mockFetchedPartitionsOnStartup); + TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer( + new KafkaTopicsDescriptor(Collections.singletonList("test-topic"), null), + i, + initialParallelism, + TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList("test-topic")), + TestPartitionDiscoverer.createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(mockFetchedPartitionsOnStartup)); + + consumers[i] = new DummyFlinkKafkaConsumer<>(mock(AbstractFetcher.class), partitionDiscoverer, false); testHarnesses[i] = createTestHarness(consumers[i], initialParallelism, i); // initializeState() is always called, null signals that we didn't restore @@ -602,8 +590,14 @@ void testRescaling( new AbstractStreamOperatorTestHarness[restoredParallelism]; for (int i = 0; i < restoredParallelism; i++) { - restoredConsumers[i] = new DummyFlinkKafkaConsumer<>( - Collections.singletonList("test-topic"), mockFetchedPartitionsAfterRestore); + TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer( + new KafkaTopicsDescriptor(Collections.singletonList("test-topic"), null), + i, + restoredParallelism, + TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList("test-topic")), + TestPartitionDiscoverer.createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(mockFetchedPartitionsAfterRestore)); + + restoredConsumers[i] = new DummyFlinkKafkaConsumer<>(mock(AbstractFetcher.class), partitionDiscoverer, false); restoredTestHarnesses[i] = createTestHarness(restoredConsumers[i], restoredParallelism, i); // initializeState() is always called, null signals that we didn't restore @@ -630,28 +624,6 @@ void testRescaling( // ------------------------------------------------------------------------ - private static FlinkKafkaConsumerBase getConsumer( - AbstractFetcher fetcher, LinkedMap pendingOffsetsToCommit, boolean running) throws Exception { - FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>(); - StreamingRuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class); - Mockito.when(mockRuntimeContext.isCheckpointingEnabled()).thenReturn(true); - consumer.setRuntimeContext(mockRuntimeContext); - - Field fetcherField = FlinkKafkaConsumerBase.class.getDeclaredField("kafkaFetcher"); - fetcherField.setAccessible(true); - fetcherField.set(consumer, fetcher); - - Field mapField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingOffsetsToCommit"); - mapField.setAccessible(true); - mapField.set(consumer, pendingOffsetsToCommit); - - Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running"); - runningField.setAccessible(true); - runningField.set(consumer, running); - - return consumer; - } - private static AbstractStreamOperatorTestHarness createTestHarness( SourceFunction source, int numSubtasks, int subtaskIndex) throws Exception { @@ -667,25 +639,43 @@ private static AbstractStreamOperatorTestHarness createTestHarness( // ------------------------------------------------------------------------ + /** + * An instantiable dummy {@link FlinkKafkaConsumerBase} that supports injecting + * mocks for {@link FlinkKafkaConsumerBase#kafkaFetcher}, {@link FlinkKafkaConsumerBase#partitionDiscoverer}, + * and {@link FlinkKafkaConsumerBase#getIsAutoCommitEnabled()}. + */ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase { private static final long serialVersionUID = 1L; - boolean isAutoCommitEnabled = false; + private AbstractFetcher testFetcher; + private AbstractPartitionDiscoverer testPartitionDiscoverer; + private boolean isAutoCommitEnabled; - private List fixedMockGetAllTopicsReturnSequence; - private List fixedMockGetAllPartitionsForTopicsReturnSequence; + @SuppressWarnings("unchecked") + DummyFlinkKafkaConsumer() { + this(false); + } - public DummyFlinkKafkaConsumer() { - this(Collections.singletonList("dummy-topic"), Collections.singletonList(new KafkaTopicPartition("dummy-topic", 0))); + @SuppressWarnings("unchecked") + DummyFlinkKafkaConsumer(boolean isAutoCommitEnabled) { + this(mock(AbstractFetcher.class), mock(AbstractPartitionDiscoverer.class), isAutoCommitEnabled); } @SuppressWarnings("unchecked") - public DummyFlinkKafkaConsumer( - List fixedMockGetAllTopicsReturnSequence, - List fixedMockGetAllPartitionsForTopicsReturnSequence) { - super(Arrays.asList("dummy-topic"), null, (KeyedDeserializationSchema < T >) mock(KeyedDeserializationSchema.class), 0); - this.fixedMockGetAllTopicsReturnSequence = Preconditions.checkNotNull(fixedMockGetAllTopicsReturnSequence); - this.fixedMockGetAllPartitionsForTopicsReturnSequence = Preconditions.checkNotNull(fixedMockGetAllPartitionsForTopicsReturnSequence); + DummyFlinkKafkaConsumer( + AbstractFetcher testFetcher, + AbstractPartitionDiscoverer testPartitionDiscoverer, + boolean isAutoCommitEnabled) { + + super( + Collections.singletonList("dummy-topic"), + null, + (KeyedDeserializationSchema < T >) mock(KeyedDeserializationSchema.class), + PARTITION_DISCOVERY_DISABLED); + + this.testFetcher = testFetcher; + this.testPartitionDiscoverer = testPartitionDiscoverer; + this.isAutoCommitEnabled = isAutoCommitEnabled; } @Override @@ -697,7 +687,7 @@ public DummyFlinkKafkaConsumer( SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode) throws Exception { - return mock(AbstractFetcher.class); + return this.testFetcher; } @Override @@ -705,21 +695,12 @@ protected AbstractPartitionDiscoverer createPartitionDiscoverer( KafkaTopicsDescriptor topicsDescriptor, int indexOfThisSubtask, int numParallelSubtasks) { - return new TestPartitionDiscoverer( - topicsDescriptor, - indexOfThisSubtask, - numParallelSubtasks, - TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(fixedMockGetAllTopicsReturnSequence), - TestPartitionDiscoverer.createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(fixedMockGetAllPartitionsForTopicsReturnSequence)); + return this.testPartitionDiscoverer; } @Override protected boolean getIsAutoCommitEnabled() { - return isAutoCommitEnabled; - } - - public void setIsAutoCommitEnabled(boolean isAutoCommitEnabled) { - this.isAutoCommitEnabled = isAutoCommitEnabled; + return this.isAutoCommitEnabled; } } @@ -748,8 +729,161 @@ public List getList() { return list; } - public boolean isClearCalled() { + boolean isClearCalled() { return clearCalled; } } + + /** + * Returns a mock {@link AbstractFetcher}, with run / stop latches injected in + * the {@link AbstractFetcher#runFetchLoop()} method. + */ + private static AbstractFetcher getRunnableMockFetcher( + OneShotLatch runLatch, + OneShotLatch stopLatch) throws Exception { + + @SuppressWarnings("unchecked") + final AbstractFetcher fetcher = mock(AbstractFetcher.class); + + Mockito.doAnswer(invocationOnMock -> { + runLatch.trigger(); + stopLatch.await(); + return null; + }).when(fetcher).runFetchLoop(); + + return fetcher; + } + + @SuppressWarnings("unchecked") + private static void setupConsumer( + FlinkKafkaConsumerBase consumer, + boolean isRestored, + ListState restoredListState, + boolean isCheckpointingEnabled, + int subtaskIndex, + int totalNumSubtasks) throws Exception { + + // run setup procedure in operator life cycle + consumer.setRuntimeContext(new MockRuntimeContext(isCheckpointingEnabled, totalNumSubtasks, subtaskIndex)); + consumer.initializeState(new MockFunctionInitializationContext(isRestored, new MockOperatorStateStore(restoredListState))); + consumer.open(new Configuration()); + } + + private static class MockRuntimeContext extends StreamingRuntimeContext { + + private final boolean isCheckpointingEnabled; + + private final int numParallelSubtasks; + private final int subtaskIndex; + + private MockRuntimeContext( + boolean isCheckpointingEnabled, + int numParallelSubtasks, + int subtaskIndex) { + + super( + new MockStreamOperator(), + new MockEnvironment("mockTask", 4 * MemoryManager.DEFAULT_PAGE_SIZE, null, 16), + Collections.>emptyMap()); + + this.isCheckpointingEnabled = isCheckpointingEnabled; + this.numParallelSubtasks = numParallelSubtasks; + this.subtaskIndex = subtaskIndex; + } + + @Override + public MetricGroup getMetricGroup() { + return new UnregisteredMetricsGroup(); + } + + @Override + public boolean isCheckpointingEnabled() { + return isCheckpointingEnabled; + } + + @Override + public int getIndexOfThisSubtask() { + return subtaskIndex; + } + + @Override + public int getNumberOfParallelSubtasks() { + return numParallelSubtasks; + } + + // ------------------------------------------------------------------------ + + private static class MockStreamOperator extends AbstractStreamOperator { + private static final long serialVersionUID = -1153976702711944427L; + + @Override + public ExecutionConfig getExecutionConfig() { + return new ExecutionConfig(); + } + } + } + + private static class MockOperatorStateStore implements OperatorStateStore { + + private final ListState mockRestoredUnionListState; + + private MockOperatorStateStore(ListState restoredUnionListState) { + this.mockRestoredUnionListState = restoredUnionListState; + } + + @Override + @SuppressWarnings("unchecked") + public ListState getUnionListState(ListStateDescriptor stateDescriptor) throws Exception { + return (ListState) mockRestoredUnionListState; + } + + @Override + public ListState getSerializableListState(String stateName) throws Exception { + // return empty state for the legacy 1.2 Kafka consumer state + return new TestingListState<>(); + } + + // ------------------------------------------------------------------------ + + @Override + public ListState getOperatorState(ListStateDescriptor stateDescriptor) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public ListState getListState(ListStateDescriptor stateDescriptor) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public Set getRegisteredStateNames() { + throw new UnsupportedOperationException(); + } + } + + private static class MockFunctionInitializationContext implements FunctionInitializationContext { + + private final boolean isRestored; + private final OperatorStateStore operatorStateStore; + + private MockFunctionInitializationContext(boolean isRestored, OperatorStateStore operatorStateStore) { + this.isRestored = isRestored; + this.operatorStateStore = operatorStateStore; + } + + @Override + public boolean isRestored() { + return isRestored; + } + + @Override + public OperatorStateStore getOperatorStateStore() { + return operatorStateStore; + } + + @Override + public KeyedStateStore getKeyedStateStore() { + throw new UnsupportedOperationException(); + } + } } From c4bfc7de36201d7a144ae995931ffd3a079ed649 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 12 Jan 2018 08:45:32 +0800 Subject: [PATCH 194/367] [FLINK-8306] [kafka, tests] Fix mock verifications on final method Previously, offset commit behavioural tests relied on verifying on AbstractFetcher::commitInternalOffsetsToKafka(). That method is actually final, and could not be mocked. This commit fixes that by implementing a proper mock AbstractFetcher, which keeps track of the offset commits that go through. This closes #5284. --- .../kafka/FlinkKafkaConsumerBaseTest.java | 203 ++++++++++-------- .../kafka/internals/AbstractFetcherTest.java | 63 +----- .../kafka/testutils/TestSourceContext.java | 87 ++++++++ 3 files changed, 199 insertions(+), 154 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 4361c6720f1e5..f091c0875fd31 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.state.KeyedStateStore; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -48,7 +47,9 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.connectors.kafka.testutils.TestPartitionDiscoverer; +import org.apache.flink.streaming.connectors.kafka.testutils.TestSourceContext; import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.Preconditions; @@ -56,11 +57,13 @@ import org.junit.Assert; import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.Mockito; + +import javax.annotation.Nonnull; import java.io.Serializable; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -68,6 +71,7 @@ import java.util.Map; import java.util.Set; +import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.collection.IsIn.isIn; @@ -75,14 +79,11 @@ import static org.hamcrest.core.IsNot.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.anyMap; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; /** * Tests for the {@link FlinkKafkaConsumerBase}. @@ -129,7 +130,11 @@ public void testEitherWatermarkExtractor() { @Test public void ignoreCheckpointWhenNotRunning() throws Exception { @SuppressWarnings("unchecked") - final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>(); + final MockFetcher fetcher = new MockFetcher<>(); + final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( + fetcher, + mock(AbstractPartitionDiscoverer.class), + false); final TestingListState> listState = new TestingListState<>(); setupConsumer(consumer, false, listState, true, 0, 1); @@ -139,6 +144,11 @@ public void ignoreCheckpointWhenNotRunning() throws Exception { // no state should have been checkpointed assertFalse(listState.get().iterator().hasNext()); + + // acknowledgement of the checkpoint should also not result in any offset commits + consumer.notifyCheckpointComplete(1L); + assertNull(fetcher.getAndClearLastCommittedOffsets()); + assertEquals(0, fetcher.getCommitCount()); } /** @@ -265,10 +275,7 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { // -------------------------------------------------------------------- - final OneShotLatch runLatch = new OneShotLatch(); - final OneShotLatch stopLatch = new OneShotLatch(); - final AbstractFetcher fetcher = getRunnableMockFetcher(runLatch, stopLatch); - when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3); + final MockFetcher fetcher = new MockFetcher<>(state1, state2, state3); final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( fetcher, @@ -283,17 +290,11 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { final CheckedThread runThread = new CheckedThread() { @Override public void go() throws Exception { - consumer.run(mock(SourceFunction.SourceContext.class)); - } - - @Override - public void sync() throws Exception { - stopLatch.trigger(); - super.sync(); + consumer.run(new TestSourceContext<>()); } }; runThread.start(); - runLatch.await(); + fetcher.waitUntilRun(); assertEquals(0, consumer.getPendingOffsetsToCommit().size()); @@ -329,6 +330,8 @@ public void sync() throws Exception { consumer.notifyCheckpointComplete(138L); assertEquals(1, consumer.getPendingOffsetsToCommit().size()); assertTrue(consumer.getPendingOffsetsToCommit().containsKey(140L)); + assertEquals(state1, fetcher.getAndClearLastCommittedOffsets()); + assertEquals(1, fetcher.getCommitCount()); // checkpoint 3 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141)); @@ -347,28 +350,15 @@ public void sync() throws Exception { // ack checkpoint 3, subsumes number 2 consumer.notifyCheckpointComplete(141L); assertEquals(0, consumer.getPendingOffsetsToCommit().size()); + assertEquals(state3, fetcher.getAndClearLastCommittedOffsets()); + assertEquals(2, fetcher.getCommitCount()); consumer.notifyCheckpointComplete(666); // invalid checkpoint assertEquals(0, consumer.getPendingOffsetsToCommit().size()); + assertNull(fetcher.getAndClearLastCommittedOffsets()); + assertEquals(2, fetcher.getCommitCount()); - // create 500 snapshots - for (int i = 100; i < 600; i++) { - consumer.snapshotState(new StateSnapshotContextSynchronousImpl(i, i)); - listState.clear(); - } - assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, consumer.getPendingOffsetsToCommit().size()); - - // commit only the second last - consumer.notifyCheckpointComplete(598); - assertEquals(1, consumer.getPendingOffsetsToCommit().size()); - - // access invalid checkpoint - consumer.notifyCheckpointComplete(590); - - // and the last - consumer.notifyCheckpointComplete(599); - assertEquals(0, consumer.getPendingOffsetsToCommit().size()); - + consumer.cancel(); runThread.sync(); } @@ -393,10 +383,7 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception // -------------------------------------------------------------------- - final OneShotLatch runLatch = new OneShotLatch(); - final OneShotLatch stopLatch = new OneShotLatch(); - final AbstractFetcher fetcher = getRunnableMockFetcher(runLatch, stopLatch); - when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3); + final MockFetcher fetcher = new MockFetcher<>(state1, state2, state3); final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( fetcher, @@ -412,17 +399,11 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception final CheckedThread runThread = new CheckedThread() { @Override public void go() throws Exception { - consumer.run(mock(SourceFunction.SourceContext.class)); - } - - @Override - public void sync() throws Exception { - stopLatch.trigger(); - super.sync(); + consumer.run(new TestSourceContext<>()); } }; runThread.start(); - runLatch.await(); + fetcher.waitUntilRun(); assertEquals(0, consumer.getPendingOffsetsToCommit().size()); @@ -454,7 +435,8 @@ public void sync() throws Exception { // ack checkpoint 1 consumer.notifyCheckpointComplete(138L); - verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed + assertEquals(0, fetcher.getCommitCount()); + assertNull(fetcher.getAndClearLastCommittedOffsets()); // no offsets should be committed // checkpoint 3 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141)); @@ -471,29 +453,15 @@ public void sync() throws Exception { // ack checkpoint 3, subsumes number 2 consumer.notifyCheckpointComplete(141L); - verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed + assertEquals(0, fetcher.getCommitCount()); + assertNull(fetcher.getAndClearLastCommittedOffsets()); // no offsets should be committed consumer.notifyCheckpointComplete(666); // invalid checkpoint - verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed - - // create 500 snapshots - for (int i = 100; i < 600; i++) { - consumer.snapshotState(new StateSnapshotContextSynchronousImpl(i, i)); - listState.clear(); - } - assertEquals(0, consumer.getPendingOffsetsToCommit().size()); // pending offsets to commit should not be updated + assertEquals(0, fetcher.getCommitCount()); + assertNull(fetcher.getAndClearLastCommittedOffsets()); // no offsets should be committed - // commit only the second last - consumer.notifyCheckpointComplete(598); - verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed - - // access invalid checkpoint - consumer.notifyCheckpointComplete(590); - verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed - - // and the last - consumer.notifyCheckpointComplete(599); - verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap(), Matchers.any(KafkaCommitCallback.class)); // no offsets should be committed + consumer.cancel(); + runThread.sync(); } @Test @@ -734,26 +702,6 @@ boolean isClearCalled() { } } - /** - * Returns a mock {@link AbstractFetcher}, with run / stop latches injected in - * the {@link AbstractFetcher#runFetchLoop()} method. - */ - private static AbstractFetcher getRunnableMockFetcher( - OneShotLatch runLatch, - OneShotLatch stopLatch) throws Exception { - - @SuppressWarnings("unchecked") - final AbstractFetcher fetcher = mock(AbstractFetcher.class); - - Mockito.doAnswer(invocationOnMock -> { - runLatch.trigger(); - stopLatch.await(); - return null; - }).when(fetcher).runFetchLoop(); - - return fetcher; - } - @SuppressWarnings("unchecked") private static void setupConsumer( FlinkKafkaConsumerBase consumer, @@ -769,6 +717,77 @@ private static void setupConsumer( consumer.open(new Configuration()); } + private static class MockFetcher extends AbstractFetcher { + + private final OneShotLatch runLatch = new OneShotLatch(); + private final OneShotLatch stopLatch = new OneShotLatch(); + + private final ArrayDeque> stateSnapshotsToReturn = new ArrayDeque<>(); + + private Map lastCommittedOffsets; + private int commitCount = 0; + + @SafeVarargs + private MockFetcher(HashMap... stateSnapshotsToReturn) throws Exception { + super( + new TestSourceContext<>(), + new HashMap<>(), + null, + null, + new TestProcessingTimeService(), + 0, + MockFetcher.class.getClassLoader(), + false); + + this.stateSnapshotsToReturn.addAll(Arrays.asList(stateSnapshotsToReturn)); + } + + @Override + protected void doCommitInternalOffsetsToKafka( + Map offsets, + @Nonnull KafkaCommitCallback commitCallback) throws Exception { + this.lastCommittedOffsets = offsets; + this.commitCount++; + commitCallback.onSuccess(); + } + + @Override + public void runFetchLoop() throws Exception { + runLatch.trigger(); + stopLatch.await(); + } + + @Override + public HashMap snapshotCurrentState() { + checkState(!stateSnapshotsToReturn.isEmpty()); + return stateSnapshotsToReturn.poll(); + } + + @Override + protected Object createKafkaPartitionHandle(KafkaTopicPartition partition) { + throw new UnsupportedOperationException(); + } + + @Override + public void cancel() { + stopLatch.trigger(); + } + + private void waitUntilRun() throws InterruptedException { + runLatch.await(); + } + + private Map getAndClearLastCommittedOffsets() { + Map offsets = this.lastCommittedOffsets; + this.lastCommittedOffsets = null; + return offsets; + } + + private int getCommitCount() { + return commitCount; + } + } + private static class MockRuntimeContext extends StreamingRuntimeContext { private final boolean isCheckpointingEnabled; @@ -784,7 +803,7 @@ private MockRuntimeContext( super( new MockStreamOperator(), new MockEnvironment("mockTask", 4 * MemoryManager.DEFAULT_PAGE_SIZE, null, 16), - Collections.>emptyMap()); + Collections.emptyMap()); this.isCheckpointingEnabled = isCheckpointingEnabled; this.numParallelSubtasks = numParallelSubtasks; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java index e4a58dd3e8324..6fe1d6f6326e0 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.connectors.kafka.testutils.TestSourceContext; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.util.SerializedValue; @@ -444,67 +444,6 @@ public Optional> getLastCommittedOffsets() { // ------------------------------------------------------------------------ - private static final class TestSourceContext implements SourceContext { - - private final Object checkpointLock = new Object(); - private final Object watermarkLock = new Object(); - - private volatile StreamRecord latestElement; - private volatile Watermark currentWatermark; - - @Override - public void collect(T element) { - this.latestElement = new StreamRecord<>(element); - } - - @Override - public void collectWithTimestamp(T element, long timestamp) { - this.latestElement = new StreamRecord<>(element, timestamp); - } - - @Override - public void emitWatermark(Watermark mark) { - synchronized (watermarkLock) { - currentWatermark = mark; - watermarkLock.notifyAll(); - } - } - - @Override - public void markAsTemporarilyIdle() { - throw new UnsupportedOperationException(); - } - - @Override - public Object getCheckpointLock() { - return checkpointLock; - } - - @Override - public void close() {} - - public StreamRecord getLatestElement() { - return latestElement; - } - - public boolean hasWatermark() { - return currentWatermark != null; - } - - public Watermark getLatestWatermark() throws InterruptedException { - synchronized (watermarkLock) { - while (currentWatermark == null) { - watermarkLock.wait(); - } - Watermark wm = currentWatermark; - currentWatermark = null; - return wm; - } - } - } - - // ------------------------------------------------------------------------ - private static class PeriodicTestExtractor implements AssignerWithPeriodicWatermarks { private volatile long maxTimestamp = Long.MIN_VALUE; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java new file mode 100644 index 0000000000000..2a96a68e603d8 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java @@ -0,0 +1,87 @@ +/* + * 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.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +/** + * Test {@link org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}. + */ +public final class TestSourceContext implements SourceFunction.SourceContext { + + private final Object checkpointLock = new Object(); + private final Object watermarkLock = new Object(); + + private volatile StreamRecord latestElement; + private volatile Watermark currentWatermark; + + @Override + public void collect(T element) { + this.latestElement = new StreamRecord<>(element); + } + + @Override + public void collectWithTimestamp(T element, long timestamp) { + this.latestElement = new StreamRecord<>(element, timestamp); + } + + @Override + public void emitWatermark(Watermark mark) { + synchronized (watermarkLock) { + currentWatermark = mark; + watermarkLock.notifyAll(); + } + } + + @Override + public void markAsTemporarilyIdle() { + // do nothing + } + + @Override + public Object getCheckpointLock() { + return checkpointLock; + } + + @Override + public void close() { + // do nothing + } + + public StreamRecord getLatestElement() { + return latestElement; + } + + public boolean hasWatermark() { + return currentWatermark != null; + } + + public Watermark getLatestWatermark() throws InterruptedException { + synchronized (watermarkLock) { + while (currentWatermark == null) { + watermarkLock.wait(); + } + Watermark wm = currentWatermark; + currentWatermark = null; + return wm; + } + } +} From c91544d140668a1b6c4c59909cc9374cc890ccf3 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 20 Dec 2017 11:54:40 -0800 Subject: [PATCH 195/367] [hotfix] [kafka] Remove stale comment on publishing procedures of AbstractFetcher The previous comment mentioned "only now will the fetcher return at least the restored offsets when calling snapshotCurrentState()". This is a remnant of the previous fetcher initialization behaviour, where in the past the fetcher wasn't directly seeded with restored offsets on instantiation. Since this is no longer true, this commit fixes the stale comment to avoid confusion. --- .../kafka/FlinkKafkaConsumerBase.java | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 44e0d5486f953..d71827fd80bb6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -545,8 +545,12 @@ public void onException(Throwable cause) { sourceContext.markAsTemporarilyIdle(); } - // create the fetcher that will communicate with the Kafka brokers - final AbstractFetcher fetcher = createFetcher( + // from this point forward: + // - 'snapshotState' will draw offsets from the fetcher, + // instead of being built from `subscribedPartitionsToStartOffsets` + // - 'notifyCheckpointComplete' will start to do work (i.e. commit offsets to + // Kafka through the fetcher, if configured to do so) + this.kafkaFetcher = createFetcher( sourceContext, subscribedPartitionsToStartOffsets, periodicWatermarkAssigner, @@ -554,12 +558,6 @@ public void onException(Throwable cause) { (StreamingRuntimeContext) getRuntimeContext(), offsetCommitMode); - // publish the reference, for snapshot-, commit-, and cancel calls - // IMPORTANT: We can only do that now, because only now will calls to - // the fetchers 'snapshotCurrentState()' method return at least - // the restored offsets - this.kafkaFetcher = fetcher; - if (!running) { return; } @@ -598,7 +596,7 @@ public void run() { // no need to add the discovered partitions if we were closed during the meantime if (running && !discoveredPartitions.isEmpty()) { - fetcher.addDiscoveredPartitions(discoveredPartitions); + kafkaFetcher.addDiscoveredPartitions(discoveredPartitions); } // do not waste any time sleeping if we're not running anymore @@ -621,7 +619,7 @@ public void run() { }); discoveryLoopThread.start(); - fetcher.runFetchLoop(); + kafkaFetcher.runFetchLoop(); // -------------------------------------------------------------------- @@ -638,7 +636,7 @@ public void run() { // won't be using the discoverer partitionDiscoverer.close(); - fetcher.runFetchLoop(); + kafkaFetcher.runFetchLoop(); } } From 0deaee835cc97bc09d1c3b78e2444b2da996be57 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Fri, 12 Jan 2018 09:29:28 -0500 Subject: [PATCH 196/367] [hotfix] [docs] Fix typos This closes #5289. --- docs/dev/api_concepts.md | 2 +- docs/dev/batch/hadoop_compatibility.md | 2 +- docs/dev/batch/index.md | 22 +++++++++---------- docs/dev/batch/iterations.md | 4 ++-- docs/dev/batch/python.md | 4 ++-- docs/dev/connectors/cassandra.md | 2 +- docs/dev/connectors/kafka.md | 2 +- docs/dev/connectors/kinesis.md | 2 +- docs/dev/connectors/rabbitmq.md | 2 +- docs/dev/datastream_api.md | 2 +- docs/dev/java8.md | 2 +- docs/dev/libs/cep.md | 8 +++---- docs/dev/libs/gelly/graph_generators.md | 6 ++--- docs/dev/libs/ml/cross_validation.md | 2 +- docs/dev/libs/storm_compatibility.md | 4 ++-- docs/dev/linking_with_flink.md | 2 +- docs/dev/migration.md | 2 +- docs/dev/packaging.md | 2 +- docs/dev/scala_api_extensions.md | 2 +- docs/dev/stream/operators/asyncio.md | 10 ++++----- docs/dev/stream/operators/index.md | 4 ++-- docs/dev/stream/operators/windows.md | 2 +- docs/dev/stream/state/checkpointing.md | 2 +- docs/dev/stream/state/index.md | 4 ++-- docs/dev/stream/testing.md | 4 ++-- docs/dev/table/common.md | 4 ++-- docs/dev/table/sourceSinks.md | 4 ++-- docs/dev/table/sql.md | 2 +- docs/dev/table/udfs.md | 2 +- docs/dev/types_serialization.md | 2 +- docs/internals/ide_setup.md | 2 +- docs/internals/job_scheduling.md | 2 +- docs/monitoring/checkpoint_monitoring.md | 4 ++-- docs/ops/config.md | 2 +- docs/ops/deployment/mesos.md | 2 +- docs/ops/filesystems.md | 6 ++--- docs/ops/jobmanager_high_availability.md | 2 +- docs/ops/security-ssl.md | 2 +- docs/ops/state/large_state_tuning.md | 2 +- docs/ops/upgrading.md | 2 +- docs/start/flink_on_windows.md | 2 +- .../flink/api/java/operators/UdfOperator.java | 2 +- ...ionExecutionWithBroadcastVariableTest.java | 2 +- .../flink/graph/gsa/GSAConfiguration.java | 6 ++--- .../pregel/VertexCentricConfiguration.java | 2 +- .../spargel/ScatterGatherConfiguration.java | 4 ++-- .../flink/optimizer/UnionReplacementTest.java | 4 ++-- .../BroadcastVariableMaterialization.java | 2 +- .../runtime/state/OperatorStateHandle.java | 2 +- .../org/apache/flink/api/scala/DataSet.scala | 2 +- .../streaming/api/datastream/DataStream.java | 2 +- 51 files changed, 85 insertions(+), 85 deletions(-) diff --git a/docs/dev/api_concepts.md b/docs/dev/api_concepts.md index cb43a42bc8e9a..c6752254cd666 100644 --- a/docs/dev/api_concepts.md +++ b/docs/dev/api_concepts.md @@ -460,7 +460,7 @@ The following example shows a key selector function that simply returns the fiel // some ordinary POJO public class WC {public String word; public int count;} DataStream words = // [...] -KeyedStream kyed = words +KeyedStream keyed = words .keyBy(new KeySelector() { public String getKey(WC wc) { return wc.word; } }); diff --git a/docs/dev/batch/hadoop_compatibility.md b/docs/dev/batch/hadoop_compatibility.md index bbeea09e370dc..9f1478a243e17 100644 --- a/docs/dev/batch/hadoop_compatibility.md +++ b/docs/dev/batch/hadoop_compatibility.md @@ -42,7 +42,7 @@ This document shows how to use existing Hadoop MapReduce code with Flink. Please ### Project Configuration -Support for Haddop input/output formats is part of the `flink-java` and +Support for Hadoop input/output formats is part of the `flink-java` and `flink-scala` Maven modules that are always required when writing Flink jobs. The code is located in `org.apache.flink.api.java.hadoop` and `org.apache.flink.api.scala.hadoop` in an additional sub-package for the diff --git a/docs/dev/batch/index.md b/docs/dev/batch/index.md index cb3b42c6bbad5..f0fab8bab6c77 100644 --- a/docs/dev/batch/index.md +++ b/docs/dev/batch/index.md @@ -293,7 +293,7 @@ result = input1.join(input2) pick the best strategy according to those estimates. {% highlight java %} // This executes a join by broadcasting the first data set -// using a hash table for the broadcasted data +// using a hash table for the broadcast data result = input1.join(input2, JoinHint.BROADCAST_HASH_FIRST) .where(0).equalTo(1); {% endhighlight %} @@ -613,7 +613,7 @@ val result = input1.join(input2).where(0).equalTo(1) pick the best strategy according to those estimates. {% highlight scala %} // This executes a join by broadcasting the first data set -// using a hash table for the broadcasted data +// using a hash table for the broadcast data val result = input1.join(input2, JoinHint.BROADCAST_HASH_FIRST) .where(0).equalTo(1) {% endhighlight %} @@ -658,7 +658,7 @@ val data1: DataSet[Int] = // [...] val data2: DataSet[String] = // [...] val result: DataSet[(Int, String)] = data1.cross(data2) {% endhighlight %} -

Note: Cross is potentially a very compute-intensive operation which can challenge even large compute clusters! It is adviced to hint the system with the DataSet sizes by using crossWithTiny() and crossWithHuge().

+

Note: Cross is potentially a very compute-intensive operation which can challenge even large compute clusters! It is advised to hint the system with the DataSet sizes by using crossWithTiny() and crossWithHuge().

@@ -994,7 +994,7 @@ Collection-based: - `fromParallelCollection(SplittableIterator)` - Creates a data set from an iterator, in parallel. The class specifies the data type of the elements returned by the iterator. -- `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in +- `generateSequence(from, to)` - Generates the sequence of numbers in the given interval, in parallel. Generic: @@ -1146,7 +1146,7 @@ using an Flink comes with a variety of built-in output formats that are encapsulated behind operations on the DataSet: -- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are +- `writeAsText()` / `TextOutputFormat` - Writes elements line-wise as Strings. The Strings are obtained by calling the *toString()* method of each element. - `writeAsFormattedText()` / `TextOutputFormat` - Write elements line-wise as Strings. The Strings are obtained by calling a user-defined *format()* method for each element. @@ -1972,7 +1972,7 @@ Collection.
{% highlight java %} -// 1. The DataSet to be broadcasted +// 1. The DataSet to be broadcast DataSet toBroadcast = env.fromElements(1, 2, 3); DataSet data = env.fromElements("a", "b"); @@ -1980,7 +1980,7 @@ DataSet data = env.fromElements("a", "b"); data.map(new RichMapFunction() { @Override public void open(Configuration parameters) throws Exception { - // 3. Access the broadcasted DataSet as a Collection + // 3. Access the broadcast DataSet as a Collection Collection broadcastSet = getRuntimeContext().getBroadcastVariable("broadcastSetName"); } @@ -1993,13 +1993,13 @@ data.map(new RichMapFunction() { {% endhighlight %} Make sure that the names (`broadcastSetName` in the previous example) match when registering and -accessing broadcasted data sets. For a complete example program, have a look at +accessing broadcast data sets. For a complete example program, have a look at {% gh_link /flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java "K-Means Algorithm" %}.
{% highlight scala %} -// 1. The DataSet to be broadcasted +// 1. The DataSet to be broadcast val toBroadcast = env.fromElements(1, 2, 3) val data = env.fromElements("a", "b") @@ -2008,7 +2008,7 @@ data.map(new RichMapFunction[String, String]() { var broadcastSet: Traversable[String] = null override def open(config: Configuration): Unit = { - // 3. Access the broadcasted DataSet as a Collection + // 3. Access the broadcast DataSet as a Collection broadcastSet = getRuntimeContext().getBroadcastVariable[String]("broadcastSetName").asScala } @@ -2019,7 +2019,7 @@ data.map(new RichMapFunction[String, String]() { {% endhighlight %} Make sure that the names (`broadcastSetName` in the previous example) match when registering and -accessing broadcasted data sets. For a complete example program, have a look at +accessing broadcast data sets. For a complete example program, have a look at {% gh_link /flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala "KMeans Algorithm" %}.
diff --git a/docs/dev/batch/iterations.md b/docs/dev/batch/iterations.md index 67f2615cd0c65..f5b65ac8d79b4 100644 --- a/docs/dev/batch/iterations.md +++ b/docs/dev/batch/iterations.md @@ -119,13 +119,13 @@ setFinalState(state); ### Example: Incrementing Numbers -In the following example, we **iteratively incremenet a set numbers**: +In the following example, we **iteratively increment a set numbers**:

Iterate Operator Example

- 1. **Iteration Input**: The inital input is read from a data source and consists of five single-field records (integers `1` to `5`). + 1. **Iteration Input**: The initial input is read from a data source and consists of five single-field records (integers `1` to `5`). 2. **Step function**: The step function is a single `map` operator, which increments the integer field from `i` to `i+1`. It will be applied to every record of the input. 3. **Next Partial Solution**: The output of the step function will be the output of the map operator, i.e. records with incremented integers. 4. **Iteration Result**: After ten iterations, the initial numbers will have been incremented ten times, resulting in integers `11` to `15`. diff --git a/docs/dev/batch/python.md b/docs/dev/batch/python.md index 0383f54b9407a..22111027ee88e 100644 --- a/docs/dev/batch/python.md +++ b/docs/dev/batch/python.md @@ -560,7 +560,7 @@ class MapperBcv(MapFunction): factor = self.context.get_broadcast_variable("bcv")[0][0] return value * factor -# 1. The DataSet to be broadcasted +# 1. The DataSet to be broadcast toBroadcast = env.from_elements(1, 2, 3) data = env.from_elements("a", "b") @@ -569,7 +569,7 @@ data.map(MapperBcv()).with_broadcast_set("bcv", toBroadcast) {% endhighlight %} Make sure that the names (`bcv` in the previous example) match when registering and -accessing broadcasted data sets. +accessing broadcast data sets. **Note**: As the content of broadcast variables is kept in-memory on each node, it should not become too large. For simpler things like scalar values you can simply parameterize the rich function. diff --git a/docs/dev/connectors/cassandra.md b/docs/dev/connectors/cassandra.md index 64d81b5c96752..6e1470a4367ad 100644 --- a/docs/dev/connectors/cassandra.md +++ b/docs/dev/connectors/cassandra.md @@ -30,7 +30,7 @@ under the License. This connector provides sinks that writes data into a [Apache Cassandra](https://cassandra.apache.org/) database. To use this connector, add the following dependency to your project: diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index e2df5fdebd8fd..8e381466fea0b 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -633,7 +633,7 @@ the consumers until `transaction1` is committed or aborted. This has two implica * First of all, during normal working of Flink applications, user can expect a delay in visibility of the records produced into Kafka topics, equal to average time between completed checkpoints. - * Secondly in case of Flink application failure, topics into which this application was writting, + * Secondly in case of Flink application failure, topics into which this application was writing, will be blocked for the readers until the application restarts or the configured transaction timeout time will pass. This remark only applies for the cases when there are multiple agents/applications writing to the same Kafka topic. diff --git a/docs/dev/connectors/kinesis.md b/docs/dev/connectors/kinesis.md index ff22ee038a770..9bcd70a753440 100644 --- a/docs/dev/connectors/kinesis.md +++ b/docs/dev/connectors/kinesis.md @@ -331,7 +331,7 @@ Otherwise, the returned stream name is used. ### Threading Model -Since Flink 1.4.0, `FlinkKinesisProducer` switches its default underlying KPL from a one-thread-per-request mode to a thread-pool mode. KPL in thread-pool mode uses a queue and thread pool to execute requests to Kinesis. This limits the number of threads that KPL's native process may create, and therefore greatly lowers CPU utilizations and improves efficiency. **Thus, We highly recommend Flink users use thread-pool model.** The default thread pool size is `10`. Users can set the pool size in `java.util.Properties` instance with key `ThreadPoolSize`, as shown in the above example. +Since Flink 1.4.0, `FlinkKinesisProducer` switches its default underlying KPL from a one-thread-per-request mode to a thread-pool mode. KPL in thread-pool mode uses a queue and thread pool to execute requests to Kinesis. This limits the number of threads that KPL's native process may create, and therefore greatly lowers CPU utilization and improves efficiency. **Thus, We highly recommend Flink users use thread-pool model.** The default thread pool size is `10`. Users can set the pool size in `java.util.Properties` instance with key `ThreadPoolSize`, as shown in the above example. Users can still switch back to one-thread-per-request mode by setting a key-value pair of `ThreadingModel` and `PER_REQUEST` in `java.util.Properties`, as shown in the code commented out in above example. diff --git a/docs/dev/connectors/rabbitmq.md b/docs/dev/connectors/rabbitmq.md index c3ad4b789281a..2a698c14a375d 100644 --- a/docs/dev/connectors/rabbitmq.md +++ b/docs/dev/connectors/rabbitmq.md @@ -66,7 +66,7 @@ RabbitMQ source, the following is required - - *Use correlation ids*: Correlation ids are a RabbitMQ application feature. You have to set it in the message properties when injecting messages into RabbitMQ. The correlation id is used by the source to deduplicate any messages that - have been reproccessed when restoring from a checkpoint. + have been reprocessed when restoring from a checkpoint. - *Non-parallel source*: The source must be non-parallel (parallelism set to 1) in order to achieve exactly-once. This limitation is mainly due to RabbitMQ's approach to dispatching messages from a single queue to multiple diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md index d7ab2e7a6fbec..32cb519592c20 100644 --- a/docs/dev/datastream_api.md +++ b/docs/dev/datastream_api.md @@ -490,7 +490,7 @@ env.generateSequence(1,10).map(new MyMapper()).setBufferTimeout(timeoutMillis); LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment env.setBufferTimeout(timeoutMillis) -env.genereateSequence(1,10).map(myMap).setBufferTimeout(timeoutMillis) +env.generateSequence(1,10).map(myMap).setBufferTimeout(timeoutMillis) {% endhighlight %} diff --git a/docs/dev/java8.md b/docs/dev/java8.md index eb9a290669eb0..494983364786d 100644 --- a/docs/dev/java8.md +++ b/docs/dev/java8.md @@ -169,7 +169,7 @@ Create/Import your Eclipse project. If you are using Maven, you also need to change the Java version in your `pom.xml` for the `maven-compiler-plugin`. Otherwise right click the `JRE System Library` section of your project and open the `Properties` window in order to switch to a Java 8 JRE (or above) that supports Lambda Expressions. -The Eclipse JDT compiler needs a special compiler flag in order to store type information in `.class` files. Open the JDT configuration file at `{project directoy}/.settings/org.eclipse.jdt.core.prefs` with your favorite text editor and add the following line: +The Eclipse JDT compiler needs a special compiler flag in order to store type information in `.class` files. Open the JDT configuration file at `{project directory}/.settings/org.eclipse.jdt.core.prefs` with your favorite text editor and add the following line: ~~~ org.eclipse.jdt.core.compiler.codegen.lambda.genericSignature=generate diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md index d814a38f126ff..16f29eaafefee 100644 --- a/docs/dev/libs/cep.md +++ b/docs/dev/libs/cep.md @@ -150,7 +150,7 @@ it to a looping one by using [Quantifiers](#quantifiers). Each pattern can have #### Quantifiers -In FlinkCEP, you can specifiy looping patterns using these methods: `pattern.oneOrMore()`, for patterns that expect one or more occurrences of a given event (e.g. the `b+` mentioned before); and `pattern.times(#ofTimes)`, for patterns that +In FlinkCEP, you can specify looping patterns using these methods: `pattern.oneOrMore()`, for patterns that expect one or more occurrences of a given event (e.g. the `b+` mentioned before); and `pattern.times(#ofTimes)`, for patterns that expect a specific number of occurrences of a given type of event, e.g. 4 `a`'s; and `pattern.times(#fromTimes, #toTimes)`, for patterns that expect a specific minimum number of occurrences and a maximum number of occurrences of a given type of event, e.g. 2-4 `a`s. You can make looping patterns greedy using the `pattern.greedy()` method, but you cannot yet make group patterns greedy. You can make all patterns, looping or not, optional using the `pattern.optional()` method. @@ -1089,7 +1089,7 @@ Pattern notNext = start.notNext("not"); if other events occur between the matching (negative) event and the previous matching event (relaxed contiguity):

{% highlight java %} -Pattern notFollowedBy = start.notFllowedBy("not"); +Pattern notFollowedBy = start.notFollowedBy("not"); {% endhighlight %}
@@ -1211,7 +1211,7 @@ val notNext = start.notNext("not") if other events occur between the matching (negative) event and the previous matching event (relaxed contiguity):

{% highlight scala %} -val notFollowedBy = start.notFllowedBy("not") +val notFollowedBy = start.notFollowedBy("not") {% endhighlight %} @@ -1448,7 +1448,7 @@ To treat partial patterns, the `select` and `flatSelect` API calls offer an over parameters * `PatternTimeoutFunction`/`PatternFlatTimeoutFunction` - * [OutputTag]({{ site.baseurl }}/dev/stream/side_output.html) for the side output in which the timeouted matches will be returned + * [OutputTag]({{ site.baseurl }}/dev/stream/side_output.html) for the side output in which the timed out matches will be returned * and the known `PatternSelectFunction`/`PatternFlatSelectFunction`.
diff --git a/docs/dev/libs/gelly/graph_generators.md b/docs/dev/libs/gelly/graph_generators.md index d4ad22931a1ce..cbd65da7a5cb0 100644 --- a/docs/dev/libs/gelly/graph_generators.md +++ b/docs/dev/libs/gelly/graph_generators.md @@ -555,10 +555,10 @@ val graph = new RMatGraph(env.getJavaEnv, rnd, vertexCount, edgeCount).generate(
-The default RMat contants can be overridden as shown in the following example. -The contants define the interdependence of bits from each generated edge's source +The default RMat constants can be overridden as shown in the following example. +The constants define the interdependence of bits from each generated edge's source and target labels. The RMat noise can be enabled and progressively perturbs the -contants while generating each edge. +constants while generating each edge. The RMat generator can be configured to produce a simple graph by removing self-loops and duplicate edges. Symmetrization is performed either by a "clip-and-flip" throwing away diff --git a/docs/dev/libs/ml/cross_validation.md b/docs/dev/libs/ml/cross_validation.md index ef3d2ffc177c1..e2642829833aa 100644 --- a/docs/dev/libs/ml/cross_validation.md +++ b/docs/dev/libs/ml/cross_validation.md @@ -54,7 +54,7 @@ Traditionally, training and testing would be done to train an algorithms as norm In a train-test-holdout strategy we sacrifice the sample size of the initial fitting algorithm for increased confidence that our model is not over-fit. -When using `trainTestHoldout` splitter, the *fraction* `Double` is replaced by a *fraction* array of length three. The first element coresponds to the portion to be used for training, second for testing, and third for holdout. The weights of this array are *relative*, e.g. an array `Array(3.0, 2.0, 1.0)` would results in approximately 50% of the observations being in the training set, 33% of the observations in the testing set, and 17% of the observations in holdout set. +When using `trainTestHoldout` splitter, the *fraction* `Double` is replaced by a *fraction* array of length three. The first element corresponds to the portion to be used for training, second for testing, and third for holdout. The weights of this array are *relative*, e.g. an array `Array(3.0, 2.0, 1.0)` would results in approximately 50% of the observations being in the training set, 33% of the observations in the testing set, and 17% of the observations in holdout set. ### K-Fold Splits diff --git a/docs/dev/libs/storm_compatibility.md b/docs/dev/libs/storm_compatibility.md index 4f499f10292ec..853b8e119ceaf 100644 --- a/docs/dev/libs/storm_compatibility.md +++ b/docs/dev/libs/storm_compatibility.md @@ -54,10 +54,10 @@ Add the following dependency to your `pom.xml` if you want to execute Storm code **Please note**: Do not add `storm-core` as a dependency. It is already included via `flink-storm`. **Please note**: `flink-storm` is not part of the provided binary Flink distribution. -Thus, you need to include `flink-storm` classes (and their dependencies) in your program jar (also called ueber-jar or fat-jar) that is submitted to Flink's JobManager. +Thus, you need to include `flink-storm` classes (and their dependencies) in your program jar (also called uber-jar or fat-jar) that is submitted to Flink's JobManager. See *WordCount Storm* within `flink-storm-examples/pom.xml` for an example how to package a jar correctly. -If you want to avoid large ueber-jars, you can manually copy `storm-core-0.9.4.jar`, `json-simple-1.1.jar` and `flink-storm-{{site.version}}.jar` into Flink's `lib/` folder of each cluster node (*before* the cluster is started). +If you want to avoid large uber-jars, you can manually copy `storm-core-0.9.4.jar`, `json-simple-1.1.jar` and `flink-storm-{{site.version}}.jar` into Flink's `lib/` folder of each cluster node (*before* the cluster is started). For this case, it is sufficient to include only your own Spout and Bolt classes (and their internal dependencies) into the program jar. # Execute Storm Topologies diff --git a/docs/dev/linking_with_flink.md b/docs/dev/linking_with_flink.md index 3f55b9ee5d33e..f2380b23a07f6 100644 --- a/docs/dev/linking_with_flink.md +++ b/docs/dev/linking_with_flink.md @@ -109,7 +109,7 @@ import org.apache.flink.api.scala.createTypeInformation {% endhighlight %} The reason is that Flink analyzes the types that are used in a program and generates serializers -and comparaters for them. By having either of those imports you enable an implicit conversion +and comparators for them. By having either of those imports you enable an implicit conversion that creates the type information for Flink operations. If you would rather use SBT, see [here]({{ site.baseurl }}/quickstart/scala_api_quickstart.html#sbt). diff --git a/docs/dev/migration.md b/docs/dev/migration.md index 5ac69617a8490..dea8b7a7d7e44 100644 --- a/docs/dev/migration.md +++ b/docs/dev/migration.md @@ -165,7 +165,7 @@ public class BufferingSink implements SinkFunction>, {% endhighlight %} -The `CountMapper` is a `RichFlatMapFuction` which assumes a grouped-by-key input stream of the form +The `CountMapper` is a `RichFlatMapFunction` which assumes a grouped-by-key input stream of the form `(word, 1)`. The function keeps a counter for each incoming key (`ValueState counter`) and if the number of occurrences of a certain word surpasses the user-provided threshold, a tuple is emitted containing the word itself and the number of occurrences. diff --git a/docs/dev/packaging.md b/docs/dev/packaging.md index 769a675b7bf8d..f43ff87d12a72 100644 --- a/docs/dev/packaging.md +++ b/docs/dev/packaging.md @@ -48,7 +48,7 @@ automatically when exporting JAR files. ### Packaging Programs through Plans -Additionally, we support packaging programs as *Plans*. Instead of defining a progam in the main +Additionally, we support packaging programs as *Plans*. Instead of defining a program in the main method and calling `execute()` on the environment, plan packaging returns the *Program Plan*, which is a description of the program's data flow. To do that, the program must implement the diff --git a/docs/dev/scala_api_extensions.md b/docs/dev/scala_api_extensions.md index 41836f9c09959..7c1ae67b54b87 100644 --- a/docs/dev/scala_api_extensions.md +++ b/docs/dev/scala_api_extensions.md @@ -61,7 +61,7 @@ data.map { {% endhighlight %} This extension introduces new methods in both the DataSet and DataStream Scala API -that have a one-to-one correspondance in the extended API. These delegating methods +that have a one-to-one correspondence in the extended API. These delegating methods do support anonymous pattern matching functions. #### DataSet API diff --git a/docs/dev/stream/operators/asyncio.md b/docs/dev/stream/operators/asyncio.md index c4736381ff4f5..702d2ae993b1a 100644 --- a/docs/dev/stream/operators/asyncio.md +++ b/docs/dev/stream/operators/asyncio.md @@ -142,7 +142,7 @@ class AsyncDatabaseRequest extends AsyncFunction[String, (String, String)] { implicit lazy val executor: ExecutionContext = ExecutionContext.fromExecutor(Executors.directExecutor()) - override def asyncInvoke(str: String, resultFutre: ResultFuture[(String, String)]): Unit = { + override def asyncInvoke(str: String, resultFuture: ResultFuture[(String, String)]): Unit = { // issue the asynchronous request, receive a future for the result val resultFuture: Future[String] = client.query(str) @@ -193,7 +193,7 @@ To control in which order the resulting records are emitted, Flink offers two mo - **Ordered**: In that case, the stream order is preserved. Result records are emitted in the same order as the asynchronous requests are triggered (the order of the operators input records). To achieve that, the operator buffers a result record - until all its preceeding records are emitted (or timed out). + until all its preceding records are emitted (or timed out). This usually introduces some amount of extra latency and some overhead in checkpointing, because records or results are maintained in the checkpointed state for a longer time, compared to the unordered mode. Use `AsyncDataStream.orderedWait(...)` for this mode. @@ -227,10 +227,10 @@ asynchronous requests in checkpoints and restores/re-triggers the requests when ### Implementation Tips -For implementations with *Futures* that have an *Executor* (or *ExecutionContext* in Scala) for callbacks, we suggets to use a `DirectExecutor`, because the +For implementations with *Futures* that have an *Executor* (or *ExecutionContext* in Scala) for callbacks, we suggests to use a `DirectExecutor`, because the callback typically does minimal work, and a `DirectExecutor` avoids an additional thread-to-thread handover overhead. The callback typically only hands the result to the `ResultFuture`, which adds it to the output buffer. From there, the heavy logic that includes record emission and interaction -with the checkpoint bookkeepting happens in a dedicated thread-pool anyways. +with the checkpoint bookkeeping happens in a dedicated thread-pool anyways. A `DirectExecutor` can be obtained via `org.apache.flink.runtime.concurrent.Executors.directExecutor()` or `com.google.common.util.concurrent.MoreExecutors.directExecutor()`. @@ -249,6 +249,6 @@ For example, the following patterns result in a blocking `asyncInvoke(...)` func - Using a database client whose lookup/query method call blocks until the result has been received back - - Blocking/waiting on the future-type objects returned by an aynchronous client inside the `asyncInvoke(...)` method + - Blocking/waiting on the future-type objects returned by an asynchronous client inside the `asyncInvoke(...)` method {% top %} diff --git a/docs/dev/stream/operators/index.md b/docs/dev/stream/operators/index.md index 0ed0b2aae8b05..b0325ca46e3f7 100644 --- a/docs/dev/stream/operators/index.md +++ b/docs/dev/stream/operators/index.md @@ -271,7 +271,7 @@ windowedStream.fold("start", new FoldFunction() {
\s*$/g,rb={option:[1,""],legend:[1,"
","
"],area:[1,"",""],param:[1,"",""],thead:[1,"
+ {% highlight text %} +ARRAY ‘[’ value [, value ]* ‘]’ +{% endhighlight %} + +

Creates an array from a list of values.

+
{% highlight text %} @@ -2294,6 +2278,17 @@ CARDINALITY(ARRAY)
+ {% highlight text %} +array ‘[’ index ‘]’ +{% endhighlight %} + +

Returns the element at a particular position in an array. The index starts at 1.

+
{% highlight text %} diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index 2b45b5a71fd18..7cce042063fdf 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -1544,7 +1544,7 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov Data Types ---------- -The Table API is built on top of Flink's DataSet and DataStream API. Internally, it also uses Flink's `TypeInformation` to distinguish between types. The Table API does not support all Flink types so far. All supported simple types are listed in `org.apache.flink.table.api.Types`. The following table summarizes the relation between Table API types, SQL types, and the resulting Java class. +The Table API is built on top of Flink's DataSet and DataStream APIs. Internally, it also uses Flink's `TypeInformation` to define data types. Fully supported types are listed in `org.apache.flink.table.api.Types`. The following table summarizes the relation between Table API types, SQL types, and the resulting Java class. | Table API | SQL | Java type | | :--------------------- | :-------------------------- | :--------------------- | @@ -1565,15 +1565,9 @@ The Table API is built on top of Flink's DataSet and DataStream API. Internally, | `Types.PRIMITIVE_ARRAY`| `ARRAY` | e.g. `int[]` | | `Types.OBJECT_ARRAY` | `ARRAY` | e.g. `java.lang.Byte[]`| | `Types.MAP` | `MAP` | `java.util.HashMap` | +| `Types.MULTISET` | `MULTISET` | e.g. `java.util.HashMap` for a multiset of `String` | - -Advanced types such as generic types, composite types (e.g. POJOs or Tuples), and array types (object or primitive arrays) can be fields of a row. - -Generic types are treated as a black box within Table API and SQL yet. - -Composite types, however, are fully supported types where fields of a composite type can be accessed using the `.get()` operator in Table API and dot operator (e.g. `MyTable.pojoColumn.myField`) in SQL. Composite types can also be flattened using `.flatten()` in Table API or `MyTable.pojoColumn.*` in SQL. - -Array types can be accessed using the `myArray.at(1)` operator in Table API and `myArray[1]` operator in SQL. Array literals can be created using `array(1, 2, 3)` in Table API and `ARRAY[1, 2, 3]` in SQL. +Generic types and composite types (e.g., POJOs or Tuples) can be fields of a row as well. Generic types are treated as a black box and can be passed on or processed by [user-defined functions](udfs.html). Composite types can be accessed with [built-in functions](#built-in-functions) (see *Value access functions* section). {% top %} @@ -1654,7 +1648,7 @@ Temporal intervals can be represented as number of months (`Types.INTERVAL_MONTH Built-In Functions ------------------ -Both the Table API and SQL come with a set of built-in functions for data transformations. This section gives a brief overview of the available functions so far. +The Table API comes with a set of built-in functions for data transformations. This section gives a brief overview of the available functions.
@@ -2468,28 +2462,6 @@ ANY.cast(TYPE)
- {% highlight java %} -ARRAY.at(INT) -{% endhighlight %} - -

Returns the element at a particular position in an array. The index starts at 1.

-
- {% highlight java %} -array(ANY [, ANY ]*) -{% endhighlight %} - -

Creates an array from a list of values. The array will be an array of objects (not primitives).

-
{% highlight java %} @@ -2926,6 +2898,17 @@ ANY.flatten()
+ {% highlight java %} +array(ANY [, ANY ]*) +{% endhighlight %} + +

Creates an array from a list of values. The array will be an array of objects (not primitives).

+
{% highlight java %} @@ -2937,6 +2920,17 @@ ARRAY.cardinality()
+ {% highlight java %} +ARRAY.at(INT) +{% endhighlight %} + +

Returns the element at a particular position in an array. The index starts at 1.

+
{% highlight java %} @@ -3764,28 +3758,6 @@ ANY.cast(TYPE)
- {% highlight scala %} -ARRAY.at(INT) -{% endhighlight %} - -

Returns the element at a particular position in an array. The index starts at 1.

-
- {% highlight scala %} -array(ANY [, ANY ]*) -{% endhighlight %} - -

Creates an array from a list of values. The array will be an array of objects (not primitives).

-
{% highlight scala %} @@ -4220,6 +4192,17 @@ dateFormat(TIMESTAMP, STRING)
+ {% highlight scala %} +array(ANY [, ANY ]*) +{% endhighlight %} + +

Creates an array from a list of values. The array will be an array of objects (not primitives).

+
{% highlight scala %} @@ -4231,6 +4214,17 @@ ARRAY.cardinality()
+ {% highlight scala %} +ARRAY.at(INT) +{% endhighlight %} + +

Returns the element at a particular position in an array. The index starts at 1.

+
{% highlight scala %} From dc1ca78a4e4cb339e9fbf0c90700f3204e091c53 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 8 Nov 2017 00:12:49 +0100 Subject: [PATCH 009/367] [hotfix] [docs] Fix UDTF join description in SQL docs. --- docs/dev/table/sql.md | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 8525ecfb720d9..23182714cf6db 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -440,11 +440,18 @@ FROM Orders CROSS JOIN UNNEST(tags) AS t (tag)

UDTFs must be registered in the TableEnvironment. See the UDF documentation for details on how to specify and register UDTFs.

-

Note: Currently only literal TRUE can be accepted as the predicate for the left outer join against a lateral table.

+

Inner Join

{% highlight sql %} SELECT users, tag -FROM Orders LATERAL VIEW UNNEST_UDTF(tags) t AS tag +FROM Orders, LATERAL TABLE(unnest_udtf(tags)) t AS tag {% endhighlight %} +

Left Outer Join

+{% highlight sql %} +SELECT users, tag +FROM Orders LEFT JOIN LATERAL TABLE(unnest_udtf(tags)) t AS tag ON TRUE +{% endhighlight %} + +

Note: Currently, only literal TRUE is supported as predicate for a left outer join against a lateral table.

Note: Time-windowed joins are a subset of regular joins that can be processed in a streaming fashion.

-

A time-windowed join requires at least one equi-join predicate and a special join - condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a BETWEEN predicate (which is not available in Table API yet) that compares the time attributes of both input tables. The following rules apply for time predicates: -

    -
  • The time attribute of a table must be compared to a bounded interval on a time attribute of the opposite table.
  • -
  • The compared time attributes must be of the same type, i.e., both are processing time or event time.
  • -
-

- +

A time-windowed join requires at least one equi-join predicate and a join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >), a BETWEEN predicate, or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

+

For example, the following predicates are valid window join conditions:

+ +
    +
  • ltime = rtime
  • +
  • ltime >= rtime AND ltime < rtime + INTERVAL '10' MINUTE
  • +
  • ltime BETWEEN rtime - INTERVAL '10' SECOND AND rtime + INTERVAL '5' SECOND
  • +
+

Note: Currently, only INNER time-windowed joins are supported.

{% highlight sql %} diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index 7cce042063fdf..f5a20592985eb 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -527,13 +527,14 @@ Table fullOuterResult = left.fullOuterJoin(right, "a = d").select("a, b, e");

Note: Time-windowed joins are a subset of regular joins that can be processed in a streaming fashion.

-

A time-windowed join requires at least one equi-join predicate and a special join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a BETWEEN predicate (which is not available in Table API yet) that compares the time attributes of both input tables. The following rules apply for time predicates: -

    -
  • The time attribute of a table must be compared to a bounded interval on a time attribute of the opposite table.
  • -
  • The compared time attributes must be of the same type, i.e., both are processing time or event time.
  • -
-

+

A time-windowed join requires at least one equi-join predicate and a join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

+

For example, the following predicates are valid window join conditions:

+
    +
  • ltime === rtime
  • +
  • ltime >= rtime && ltime < rtime + 10.minutes
  • +
+

Note: Currently, only INNER time-windowed joins are supported.

{% highlight java %} @@ -644,13 +645,14 @@ val fullOuterResult = left.fullOuterJoin(right, 'a === 'd).select('a, 'b, 'e)

Note: Time-windowed joins are a subset of regular joins that can be processed in a streaming fashion.

-

A time-windowed join requires at least one equi-join predicate and a special join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a BETWEEN predicate (which is not available in Table API yet) that compares the time attributes of both input tables. The following rules apply for time predicates: -

    -
  • The time attribute of a table must be compared to a bounded interval on a time attribute of the opposite table.
  • -
  • The compared time attributes must be of the same type, i.e., both are processing time or event time.
  • -
-

+

A time-windowed join requires at least one equi-join predicate and a join condition that bounds the time on both sides. Such a condition can be defined by two appropriate range predicates (<, <=, >=, >) or a single equality predicate that compares time attributes of the same type (i.e., processing time or event time) of both input tables.

+

For example, the following predicates are valid window join conditions:

+
    +
  • 'ltime === 'rtime
  • +
  • 'ltime >= 'rtime && 'ltime < 'rtime + 10.minutes
  • +
+

Note: Currently, only INNER time-windowed joins are supported.

{% highlight scala %} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala index 863f34251d2dd..1693c41e7fb2e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala @@ -78,29 +78,45 @@ object WindowJoinUtil { // Converts the condition to conjunctive normal form (CNF) val cnfCondition = RexUtil.toCnf(rexBuilder, predicate) - // split the condition into time indicator condition and other condition + // split the condition into time predicates and other predicates + // We need two range predicates or an equality predicate for a properly bounded window join. val (timePreds, otherPreds) = cnfCondition match { - // We need at least two comparison predicates for a properly bounded window join. - // So we need an AND expression for a valid window join. - case c: RexCall if cnfCondition.getKind == SqlKind.AND => - c.getOperands.asScala - .map(identifyTimePredicate(_, leftLogicalFieldCnt, inputSchema)) - .foldLeft((Seq[TimePredicate](), Seq[RexNode]()))((preds, analyzed) => { - analyzed match { - case Left(timePred) => (preds._1 :+ timePred, preds._2) - case Right(otherPred) => (preds._1, preds._2 :+ otherPred) - } - }) - case _ => - // No valid window bounds. A windowed stream join requires two comparison predicates that - // bound the time in both directions. - return (None, Some(predicate)) + case c: RexCall if cnfCondition.getKind == SqlKind.AND => + // extract all time predicates from conjunctive predicate + c.getOperands.asScala + .map(identifyTimePredicate(_, leftLogicalFieldCnt, inputSchema)) + .foldLeft((Seq[TimePredicate](), Seq[RexNode]()))((preds, analyzed) => { + analyzed match { + case Left(timePred) => (preds._1 :+ timePred, preds._2) + case Right(otherPred) => (preds._1, preds._2 :+ otherPred) + } + }) + case c: RexCall => + // extract time predicate if it exists + identifyTimePredicate(c, leftLogicalFieldCnt, inputSchema) match { + case Left(timePred) => (Seq[TimePredicate](timePred), Seq[RexNode]()) + case Right(otherPred) => (Seq[TimePredicate](), Seq[RexNode](otherPred)) + } + case _ => + // No valid window bounds. + return (None, Some(predicate)) } - if (timePreds.size != 2) { - // No valid window bounds. A windowed stream join requires two comparison predicates that - // bound the time in both directions. - return (None, Some(predicate)) + timePreds match { + case Seq() => + return (None, Some(predicate)) + case Seq(t) if t.pred.getKind != SqlKind.EQUALS => + // single predicate must be equality predicate + return (None, Some(predicate)) + case s@Seq(_, _) if s.exists(_.pred.getKind == SqlKind.EQUALS) => + // pair of range predicate must not include equals predicate + return (None, Some(predicate)) + case Seq(_) => + // Single equality predicate is OK + case Seq(_, _) => + // Two range (i.e., non-equality predicates are OK + case _ => + return (None, Some(predicate)) } // assemble window bounds from predicates @@ -108,9 +124,14 @@ object WindowJoinUtil { val (leftLowerBound, leftUpperBound) = streamTimeOffsets match { case Seq(Some(x: WindowBound), Some(y: WindowBound)) if x.isLeftLower && !y.isLeftLower => + // two range predicates (x.bound, y.bound) case Seq(Some(x: WindowBound), Some(y: WindowBound)) if !x.isLeftLower && y.isLeftLower => + // two range predicates (y.bound, x.bound) + case Seq(Some(x: WindowBound)) => + // single equality predicate + (x.bound, x.bound) case _ => // Window join requires two comparison predicate that bound the time in both directions. return (None, Some(predicate)) @@ -118,12 +139,12 @@ object WindowJoinUtil { // compose the remain condition list into one condition val remainCondition = - otherPreds match { - case Seq() => - None - case _ => - Some(otherPreds.reduceLeft((l, r) => RelOptUtil.andJoinFilters(rexBuilder, l, r))) - } + otherPreds match { + case Seq() => + None + case _ => + Some(otherPreds.reduceLeft((l, r) => RelOptUtil.andJoinFilters(rexBuilder, l, r))) + } val bounds = if (timePreds.head.leftInputOnLeftSide) { Some(WindowBounds( @@ -146,14 +167,15 @@ object WindowJoinUtil { /** * Analyzes a predicate and identifies whether it is a valid predicate for a window join. - * A valid window join predicate is a comparison predicate (<, <=, =>, >) that accesses - * time attributes of both inputs, each input on a different side of the condition. + * + * A valid window join predicate is a range or equality predicate (<, <=, ==, =>, >) that + * accesses time attributes of both inputs, each input on a different side of the condition. * Both accessed time attributes must be of the same time type, i.e., row-time or proc-time. * * Examples: * - left.rowtime > right.rowtime + 2.minutes => valid + * - left.rowtime == right.rowtime => valid * - left.proctime < right.rowtime + 2.minutes => invalid: different time type - * - left.rowtime == right.rowtime + 2.minutes => invalid: not a comparison predicate * - left.rowtime - right.rowtime < 2.minutes => invalid: both time attributes on same side * * If the predicate is a regular join predicate, i.e., it accesses no time attribute it is @@ -172,7 +194,8 @@ object WindowJoinUtil { case SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL | SqlKind.LESS_THAN | - SqlKind.LESS_THAN_OR_EQUAL => + SqlKind.LESS_THAN_OR_EQUAL | + SqlKind.EQUALS => val leftTerm = c.getOperands.get(0) val rightTerm = c.getOperands.get(1) @@ -235,7 +258,7 @@ object WindowJoinUtil { * * @return A Seq of all time attribute accessed in the expression. */ - def extractTimeAttributeAccesses( + private def extractTimeAttributeAccesses( expr: RexNode, leftFieldCount: Int, inputType: RelDataType): Seq[TimeAttributeAccess] = { @@ -248,9 +271,9 @@ object WindowJoinUtil { case t: TimeIndicatorRelDataType => // time attribute access. Remember time type and side of input if (idx < leftFieldCount) { - Seq(TimeAttributeAccess(t.isEventTime, true, idx)) + Seq(TimeAttributeAccess(t.isEventTime, isLeftInput = true, idx)) } else { - Seq(TimeAttributeAccess(t.isEventTime, false, idx - leftFieldCount)) + Seq(TimeAttributeAccess(t.isEventTime, isLeftInput = false, idx - leftFieldCount)) } case _ => // not a time attribute access. @@ -272,7 +295,7 @@ object WindowJoinUtil { * @param inputType The input type of the expression. * @return True, if the expression accesses a non-time attribute. False otherwise. */ - def accessesNonTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = { + private def accessesNonTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = { expr match { case i: RexInputRef => val accessedType = inputType.getFieldList.get(i.getIndex).getType @@ -292,7 +315,7 @@ object WindowJoinUtil { * * @return window boundary, is left lower bound */ - def computeWindowBoundFromPredicate( + private def computeWindowBoundFromPredicate( timePred: TimePredicate, rexBuilder: RexBuilder, config: TableConfig): Option[WindowBound] = { @@ -303,6 +326,8 @@ object WindowJoinUtil { timePred.leftInputOnLeftSide case (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) => !timePred.leftInputOnLeftSide + case (SqlKind.EQUALS) => + true // We don't care about this since there's only one bound value. case _ => return None } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index 53aff8265d344..ded5c51c2b98c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -184,6 +184,66 @@ class JoinTest extends TableTestBase { streamUtil.verifySql(sqlQuery, expected) } + @Test + def testJoinWithEquiProcTime(): Unit = { + val sqlQuery = + """ + |SELECT t1.a, t2.b + |FROM MyTable t1, MyTable2 t2 + |WHERE t1.a = t2.a AND + | t1.proctime = t2.proctime + |""".stripMargin + + val expected = + unaryNode("DataStreamCalc", + binaryNode("DataStreamWindowJoin", + unaryNode("DataStreamCalc", + streamTableNode(0), + term("select", "a", "proctime") + ), + unaryNode("DataStreamCalc", + streamTableNode(1), + term("select", "a", "b", "proctime") + ), + term("where", "AND(=(a, a0), =(proctime, proctime0))"), + term("join", "a", "proctime", "a0", "b", "proctime0"), + term("joinType", "InnerJoin") + ), + term("select", "a", "b0 AS b") + ) + streamUtil.verifySql(sqlQuery, expected) + } + + @Test + def testJoinWithEquiRowTime(): Unit = { + val sqlQuery = + """ + |SELECT t1.a, t2.b + |FROM MyTable t1, MyTable2 t2 + |WHERE t1.a = t2.a AND + | t1.c = t2.c + |""".stripMargin + + val expected = + unaryNode("DataStreamCalc", + binaryNode("DataStreamWindowJoin", + unaryNode("DataStreamCalc", + streamTableNode(0), + term("select", "a", "c") + ), + unaryNode("DataStreamCalc", + streamTableNode(1), + term("select", "a", "b", "c") + ), + term("where", "AND(=(a, a0), =(c, c0))"), + term("join", "a", "c", "a0", "b", "c0"), + term("joinType", "InnerJoin") + ), + term("select", "a", "b0 AS b") + ) + streamUtil.verifySql(sqlQuery, expected) + } + @Test def testRowTimeInnerJoinAndWindowAggregationOnFirst(): Unit = { @@ -332,6 +392,12 @@ class JoinTest extends TableTestBase { -10000, -5000, "rowtime") + + verifyTimeBoundary( + "t1.c = t2.c", + 0, + 0, + "rowtime") } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala index 9cce37eb1a561..9f7078c240e94 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala @@ -92,4 +92,33 @@ class JoinValidationTest extends TableTestBase { streamUtil.verifySql(sql, "n/a") } + /** Validates that range and equality predicate are not accepted **/ + @Test(expected = classOf[TableException]) + def testRangeAndEqualityPredicates(): Unit = { + val sql = + """ + |SELECT * + |FROM MyTable t1, MyTable2 t2 + |WHERE t1.a = t2.a AND + | t1.proctime > t2.proctime - INTERVAL '5' SECOND AND + | t1.proctime = t2.proctime + | """.stripMargin + + streamUtil.verifySql(sql, "n/a") + } + + /** Validates that equality predicate with offset are not accepted **/ + @Test(expected = classOf[TableException]) + def testEqualityPredicateWithOffset(): Unit = { + val sql = + """ + |SELECT * + |FROM MyTable t1, MyTable2 t2 + |WHERE t1.a = t2.a AND + | t1.proctime = t2.proctime - INTERVAL '5' SECOND + | """.stripMargin + + streamUtil.verifySql(sql, "n/a") + } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala index 07e879fceafbe..79b413ccd3feb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala @@ -100,6 +100,40 @@ class JoinTest extends TableTestBase { util.verifyTable(resultTable, expected) } + @Test + def testProcTimeWindowInnerJoinWithEquiTimeAttrs(): Unit = { + val util = streamTestUtil() + val left = util.addTable[(Long, Int, String)]('a, 'b, 'c, 'ltime.proctime) + val right = util.addTable[(Long, Int, String)]('d, 'e, 'f, 'rtime.proctime) + + val resultTable = left.join(right) + .where('a === 'd && 'ltime === 'rtime) + .select('a, 'e, 'ltime) + + val expected = + unaryNode( + "DataStreamCalc", + binaryNode( + "DataStreamWindowJoin", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "ltime") + ), + unaryNode( + "DataStreamCalc", + streamTableNode(1), + term("select", "d", "e", "rtime") + ), + term("where", "AND(=(a, d), =(ltime, rtime))"), + term("join", "a", "ltime", "d", "e", "rtime"), + term("joinType", "InnerJoin") + ), + term("select", "a", "e", "PROCTIME(ltime) AS ltime") + ) + util.verifyTable(resultTable, expected) + } + /** * The time indicator can be accessed from non-time predicates now. */ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala index 1d7bab663d458..85929e8d72704 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala @@ -187,6 +187,53 @@ class JoinITCase extends StreamingWithStateTestBase { StreamITCase.compareWithList(expected) } + /** test rowtime inner join with equi-times **/ + @Test + def testRowTimeInnerJoinWithEquiTimeAttrs(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStateBackend(getStateBackend) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.clear + + val sqlQuery = + """ + |SELECT t2.key, t2.id, t1.id + |FROM T1 as t1 join T2 as t2 ON + | t1.key = t2.key AND + | t2.rt = t1.rt + |""".stripMargin + + val data1 = new mutable.MutableList[(Int, Long, String, Long)] + + data1.+=((4, 4000L, "A", 4000L)) + data1.+=((5, 5000L, "A", 5000L)) + data1.+=((6, 6000L, "A", 6000L)) + data1.+=((6, 6000L, "B", 6000L)) + + val data2 = new mutable.MutableList[(String, String, Long)] + data2.+=(("A", "R-5", 5000L)) + data2.+=(("B", "R-6", 6000L)) + + val t1 = env.fromCollection(data1) + .assignTimestampsAndWatermarks(new Row4WatermarkExtractor) + .toTable(tEnv, 'id, 'tm, 'key, 'rt.rowtime) + val t2 = env.fromCollection(data2) + .assignTimestampsAndWatermarks(new Row3WatermarkExtractor2) + .toTable(tEnv, 'key, 'id, 'rt.rowtime) + + tEnv.registerTable("T1", t1) + tEnv.registerTable("T2", t2) + + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + val expected = new java.util.ArrayList[String] + expected.add("A,R-5,5") + expected.add("B,R-6,6") + StreamITCase.compareWithList(expected) + } + /** test rowtime inner join with other conditions **/ @Test def testRowTimeInnerJoinWithOtherConditions(): Unit = { From 51657fc6deaf28115020db86d031d536b09bf384 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Tue, 7 Nov 2017 17:57:39 +0100 Subject: [PATCH 017/367] [FLINK-8012] [table] Fix TableSink config for tables with time attributes. This closes #4974. --- .../scala/org/apache/flink/table/api/table.scala | 7 ++++++- .../runtime/stream/table/TableSinkITCase.scala | 15 ++++++++++++--- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala index 0430e497d6bfd..7349a0e1ae5ad 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala @@ -848,7 +848,12 @@ class Table( val rowType = getRelNode.getRowType val fieldNames: Array[String] = rowType.getFieldNames.asScala.toArray val fieldTypes: Array[TypeInformation[_]] = rowType.getFieldList.asScala - .map(field => FlinkTypeFactory.toTypeInfo(field.getType)).toArray + .map(field => FlinkTypeFactory.toTypeInfo(field.getType)) + .map { + // replace time indicator types by SQL_TIMESTAMP + case t: TypeInformation[_] if FlinkTypeFactory.isTimeIndicatorType(t) => Types.SQL_TIMESTAMP + case t: TypeInformation[_] => t + }.toArray // configure the table sink val configuredSink = sink.configure(fieldNames, fieldTypes) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index 07934b89c0adf..b44d8eff90c9e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -88,13 +88,16 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase { val env = StreamExecutionEnvironment.getExecutionEnvironment env.getConfig.enableObjectReuse() + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) env.setParallelism(4) val input = StreamTestData.get3TupleDataStream(env) + .assignAscendingTimestamps(_._2) .map(x => x).setParallelism(4) // increase DOP to 4 - val results = input.toTable(tEnv, 'a, 'b, 'c) + val results = input.toTable(tEnv, 'a, 'b.rowtime, 'c) .where('a < 5 || 'a > 17) .select('c, 'b) .writeToSink(new CsvTableSink(path)) @@ -102,8 +105,14 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase { env.execute() val expected = Seq( - "Hi,1", "Hello,2", "Hello world,2", "Hello world, how are you?,3", - "Comment#12,6", "Comment#13,6", "Comment#14,6", "Comment#15,6").mkString("\n") + "Hi,1970-01-01 00:00:00.001", + "Hello,1970-01-01 00:00:00.002", + "Hello world,1970-01-01 00:00:00.002", + "Hello world, how are you?,1970-01-01 00:00:00.003", + "Comment#12,1970-01-01 00:00:00.006", + "Comment#13,1970-01-01 00:00:00.006", + "Comment#14,1970-01-01 00:00:00.006", + "Comment#15,1970-01-01 00:00:00.006").mkString("\n") TestBaseUtils.compareResultsByLinesInMemory(expected, path) } From c7943291599260003304f003e89725352ae7d836 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 6 Nov 2017 21:22:35 +0100 Subject: [PATCH 018/367] [FLINK-8002] [table] Fix join window boundary for LESS_THAN and GREATER_THAN predicates. This closes #4962. --- .../table/runtime/join/WindowJoinUtil.scala | 8 +++- .../flink/table/api/stream/sql/JoinTest.scala | 38 ++++++++++++++++++- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala index 1693c41e7fb2e..7006476abd0ef 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala @@ -346,10 +346,14 @@ object WindowJoinUtil { leftLiteral.get - rightLiteral.get } val boundary = timePred.pred.getKind match { - case SqlKind.LESS_THAN => + case SqlKind.LESS_THAN if timePred.leftInputOnLeftSide => tmpTimeOffset - 1 - case SqlKind.GREATER_THAN => + case SqlKind.LESS_THAN if !timePred.leftInputOnLeftSide => tmpTimeOffset + 1 + case SqlKind.GREATER_THAN if timePred.leftInputOnLeftSide => + tmpTimeOffset + 1 + case SqlKind.GREATER_THAN if !timePred.leftInputOnLeftSide => + tmpTimeOffset - 1 case _ => tmpTimeOffset } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index ded5c51c2b98c..8c1865c69643b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -379,13 +379,26 @@ class JoinTest extends TableTestBase { "rowtime") verifyTimeBoundary( - "t1.c - interval '2' second >= t2.c + interval '1' second -" + - "interval '10' second and " + + "t1.c >= t2.c - interval '1' second and " + + "t1.c <= t2.c + interval '10' second", + -1000, + 10000, + "rowtime") + + verifyTimeBoundary( + "t1.c - interval '2' second >= t2.c + interval '1' second - interval '10' second and " + "t1.c <= t2.c + interval '10' second", -7000, 10000, "rowtime") + verifyTimeBoundary( + "t2.c + interval '1' second - interval '10' second <= t1.c - interval '2' second and " + + "t2.c + interval '10' second >= t1.c", + -7000, + 10000, + "rowtime") + verifyTimeBoundary( "t1.c >= t2.c - interval '10' second and " + "t1.c <= t2.c - interval '5' second", @@ -393,6 +406,27 @@ class JoinTest extends TableTestBase { -5000, "rowtime") + verifyTimeBoundary( + "t2.c - interval '10' second <= t1.c and " + + "t2.c - interval '5' second >= t1.c", + -10000, + -5000, + "rowtime") + + verifyTimeBoundary( + "t1.c > t2.c - interval '2' second and " + + "t1.c < t2.c + interval '2' second", + -1999, + 1999, + "rowtime") + + verifyTimeBoundary( + "t2.c > t1.c - interval '2' second and " + + "t2.c < t1.c + interval '2' second", + -1999, + 1999, + "rowtime") + verifyTimeBoundary( "t1.c = t2.c", 0, From 02a19a14fad1ef928038f4971bdcacf4d0642d88 Mon Sep 17 00:00:00 2001 From: Dan Kelley Date: Tue, 7 Nov 2017 17:27:44 -0800 Subject: [PATCH 019/367] [FLINK-8017] Fix High availability cluster-id key in documentation --- docs/ops/jobmanager_high_availability.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ops/jobmanager_high_availability.md b/docs/ops/jobmanager_high_availability.md index 25701a1510276..2b590cc070b6a 100644 --- a/docs/ops/jobmanager_high_availability.md +++ b/docs/ops/jobmanager_high_availability.md @@ -80,7 +80,7 @@ In order to start an HA-cluster add the following configuration keys to `conf/fl - **ZooKeeper cluster-id** (recommended): The *cluster-id ZooKeeper node*, under which all required coordination data for a cluster is placed. -
high-availability.zookeeper.path.cluster-id: /default_ns # important: customize per cluster
+
high-availability.cluster-id: /default_ns # important: customize per cluster
**Important**: You should not set this value manually when runnig a YARN cluster, a per-job YARN session, or on another cluster manager. In those @@ -108,7 +108,7 @@ After configuring the masters and the ZooKeeper quorum, you can use the provided high-availability: zookeeper high-availability.zookeeper.quorum: localhost:2181 high-availability.zookeeper.path.root: /flink -high-availability.zookeeper.path.cluster-id: /cluster_one # important: customize per cluster +high-availability.cluster-id: /cluster_one # important: customize per cluster high-availability.zookeeper.storageDir: hdfs:///flink/recovery 2. **Configure masters** in `conf/masters`: From d302c652f1b52aac29ff6d09c817bd7f9e5e00e7 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 9 Nov 2017 15:34:44 +0100 Subject: [PATCH 020/367] [hotfix] Fix formatting in windowing documentation --- docs/dev/stream/operators/windows.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 798b7bbc97bf0..7966ec81417a6 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -1293,6 +1293,7 @@ Before continuing in this section you might want to take a look at our section a [event time and watermarks]({{ site.baseurl }}/dev/event_time.html). When watermarks arrive at the window operator this triggers two things: + - the watermark triggers computation of all windows where the maximum timestamp (which is *end-timestamp - 1*) is smaller than the new watermark - the watermark is forwarded (as is) to downstream operations From 7df7fc457618d371b4c1f9623ac7fc2cab37cb1f Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 5 Oct 2017 15:17:13 +0200 Subject: [PATCH 021/367] [hotfix][build] Deduplicate maven-enforcer version --- pom.xml | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index d9d0a686c37c2..7b9309c046daf 100644 --- a/pom.xml +++ b/pom.xml @@ -885,7 +885,6 @@ under the License. org.apache.maven.plugins maven-enforcer-plugin - 1.4.1 enforce-maven @@ -1194,7 +1193,6 @@ under the License. org.apache.maven.plugins maven-enforcer-plugin - 1.4.1 enforce-maven @@ -1324,6 +1322,11 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + From 005a871771ce73bef9c78ee04a61817fa9a31e99 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Tue, 7 Nov 2017 12:13:59 +0100 Subject: [PATCH 022/367] [FLINK-7765][build] Enable dependency convergence by default Disable it in most modules. --- flink-connectors/pom.xml | 20 +++++++++++ flink-examples/pom.xml | 19 ++++++++++ flink-filesystems/pom.xml | 20 +++++++++++ flink-fs-tests/pom.xml | 16 +++++++++ flink-java/pom.xml | 1 - flink-libraries/pom.xml | 19 ++++++++++ flink-runtime/pom.xml | 15 ++++++++ flink-scala/pom.xml | 15 ++++++++ flink-shaded-hadoop/pom.xml | 15 ++++++++ flink-streaming-scala/pom.xml | 16 +++++++++ flink-tests/pom.xml | 18 +++++++++- flink-yarn-tests/pom.xml | 16 +++++++++ flink-yarn/pom.xml | 16 +++++++++ pom.xml | 65 +++++++++++++++++++++++++++++++++-- 14 files changed, 266 insertions(+), 5 deletions(-) diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 7468b676b064f..1f9f05f2f8a30 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -100,4 +100,24 @@ under the License. + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + + diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 21db5902352e2..a4e530f707807 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -71,4 +71,23 @@ under the License. + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + diff --git a/flink-filesystems/pom.xml b/flink-filesystems/pom.xml index e31ecef0a3915..075c0f7c2a7b5 100644 --- a/flink-filesystems/pom.xml +++ b/flink-filesystems/pom.xml @@ -42,4 +42,24 @@ under the License. flink-s3-fs-presto + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + + diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml index 3ac62dff589e2..b56cca5563a44 100644 --- a/flink-fs-tests/pom.xml +++ b/flink-fs-tests/pom.xml @@ -99,6 +99,22 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + org.apache.maven.plugins maven-surefire-plugin diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 6e2087d9c5bac..6b635bdc1c72c 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -71,7 +71,6 @@ under the License. - com.github.siom79.japicmp diff --git a/flink-libraries/pom.xml b/flink-libraries/pom.xml index 5b4cf7932810f..ed6591b3b7011 100644 --- a/flink-libraries/pom.xml +++ b/flink-libraries/pom.xml @@ -61,4 +61,23 @@ under the License. + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index a7d028478cab1..92a6f2f159950 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -241,6 +241,21 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + org.apache.maven.plugins maven-checkstyle-plugin diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index f132145cc4237..95bc6b771eed5 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -111,6 +111,21 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml index 3e7cb41f44129..c9549933e52e3 100644 --- a/flink-shaded-hadoop/pom.xml +++ b/flink-shaded-hadoop/pom.xml @@ -56,6 +56,21 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml index 316731a5de4a1..52537321035dd 100644 --- a/flink-streaming-scala/pom.xml +++ b/flink-streaming-scala/pom.xml @@ -106,6 +106,22 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + net.alchim31.maven diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 221719924d6f9..c21bdb47b8332 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -235,7 +235,23 @@ under the License. - + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + net.alchim31.maven diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 1232d15c98767..f01e8beaf16ec 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -138,6 +138,22 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml index feaace0e50dbe..db6ee88dc903b 100644 --- a/flink-yarn/pom.xml +++ b/flink-yarn/pom.xml @@ -157,6 +157,22 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + net.alchim31.maven diff --git a/pom.xml b/pom.xml index 7b9309c046daf..21ad7aa235a5a 100644 --- a/pom.xml +++ b/pom.xml @@ -123,6 +123,7 @@ under the License. 4.12 1.10.19 1.6.5 + 1.3 false + + org.hamcrest + hamcrest-core + ${hamcrest.version} + + + + + org.objenesis + objenesis + 2.1 + + + + + com.typesafe + config + 1.3.0 + + + + + commons-logging + commons-logging + 1.1.3 + + + + + junit + junit + ${junit.version} + + + + + org.tukaani + xz + 1.5 + + commons-cli @@ -901,7 +943,6 @@ under the License. 1.8.0 - @@ -1190,6 +1231,7 @@ under the License. true + org.apache.maven.plugins maven-enforcer-plugin @@ -1213,6 +1255,23 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + + + + + + + Date: Fri, 10 Nov 2017 13:57:51 +0100 Subject: [PATCH 025/367] [hotfix][build] Disable dependency convergence in flink-dist Previously mvn javadoc:aggregate goal was failing --- flink-dist/pom.xml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index b2af9e8a898b9..d009ead5ce25a 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -426,6 +426,21 @@ under the License. + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + From da435f121821fd1107c41352a54ee804f10cf7e3 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 10 Nov 2017 10:54:16 +0100 Subject: [PATCH 026/367] [FLINK-6163] Document per-window state in ProcessWindowFunction --- docs/dev/stream/operators/windows.md | 32 ++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 7966ec81417a6..3c0cd8509d3fa 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -978,6 +978,38 @@ input +### Using per-window state in ProcessWindowFunction + +In addition to accessing keyed state (as any rich function can) a `ProcessWindowFunction` can +also use keyed state that is scoped to the window that the function is currently processing. In this +context it is important to understand what the window that *per-window* state is referring to is. +There are different "windows" involved: + + - The window that was defined when specifying the windowed operation: This might be *tumbling + windows of 1 hour* or *sliding windows of 2 hours that slide by 1 hour*. + - An actual instance of a defined window for a given key: This might be *time window from 12:00 + to 13:00 for user-id xyz*. This is based on the window definition and there will be many windows + based on the number of keys that the job is currently processing and based on what time slots + the events fall into. + +Per-window state is tied to the latter of those two. Meaning that if we process events for 1000 +different keys and events for all of them currently fall into the *[12:00, 13:00)* time window +then there will be 1000 window instances that each have their own keyed per-window state. + +There are two methods on the `Context` object that a `process()` invocation receives that allow +access two the two types of state: + + - `globalState()`, which allows access to keyed state that is not scoped to a window + - `windowState()`, which allows access to keyed state that is also scoped to the window + +This feature is helpful if you anticipate multiple firing for the same window, as can happen when +you have late firings for data that arrives late or when you have a custom trigger that does +speculative early firings. In such a case you would store information about previous firings or +the number of firings in per-window state. + +When using windowed state it is important to also clean up that state when a window is cleared. This +should happen in the `clear()` method. + ### WindowFunction (Legacy) In some places where a `ProcessWindowFunction` can be used you can also use a `WindowFunction`. This From b3df579f0fd36f8b4a235a994caaaffe6f2b2a0d Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 10 Nov 2017 15:15:11 +0100 Subject: [PATCH 027/367] [hotfix][docs] Change mailing list link in quickstart to flink-user Previously it was pointing to flink-dev --- docs/quickstart/java_api_quickstart.md | 7 ++++++- docs/quickstart/scala_api_quickstart.md | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md index 38a2aa9896c98..7665a7552caab 100644 --- a/docs/quickstart/java_api_quickstart.md +++ b/docs/quickstart/java_api_quickstart.md @@ -197,6 +197,11 @@ public static final class LineSplitter implements FlatMapFunction Date: Fri, 10 Nov 2017 16:28:46 +0100 Subject: [PATCH 028/367] [FLINK-7702] Remove Javadoc aggregation for Scala code genjavadoc generated some Java code that was making Javadoc fail. --- pom.xml | 94 ++------------------------------------------------------- 1 file changed, 2 insertions(+), 92 deletions(-) diff --git a/pom.xml b/pom.xml index 21ad7aa235a5a..5c04f5f7a23e3 100644 --- a/pom.xml +++ b/pom.xml @@ -738,99 +738,9 @@ under the License. - + aggregate-scaladoc - - - - com.typesafe.akka - akka-testkit_${scala.binary.version} - ${akka.version} - provided - - - - - - - - maven-clean-plugin - 2.5 - - - clean-target - generate-sources - - clean - - - true - - - ${project.build.directory} - - **/*.class - **/classes.*.timestamp - - - - - - - - - - net.alchim31.maven - scala-maven-plugin - - - doc - generate-sources - - compile - - - - - - -P:genjavadoc:out=${project.build.directory}/genjavadoc - - - - com.typesafe.genjavadoc - genjavadoc-plugin_${scala.version} - 0.10 - - - - - - org.codehaus.mojo - build-helper-maven-plugin - 1.7 - - - generate-sources - - add-source - - - - ${project.build.directory}/genjavadoc - - - - - - - From 431ae36f787adcaac2e1071753d2dc2af299f528 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 10 Nov 2017 18:13:26 +0100 Subject: [PATCH 029/367] [FLINK-7702] Add maven-bundle-plugin to root pom Before, we had it in places that require it. This doesn't work when running mvn javadoc:aggregate because this will only run for the root pom and can then not find the "bundle" dependencies. --- .../flink-connector-filesystem/pom.xml | 13 ------------- .../flink-connector-kafka-0.9/pom.xml | 11 ----------- .../flink-connector-kafka-base/pom.xml | 11 ----------- flink-test-utils-parent/flink-test-utils/pom.xml | 12 ------------ flink-yarn-tests/pom.xml | 12 ------------ pom.xml | 15 +++++++++++++++ 6 files changed, 15 insertions(+), 59 deletions(-) diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml index d1904a2e7b36f..4f5de2798eae2 100644 --- a/flink-connectors/flink-connector-filesystem/pom.xml +++ b/flink-connectors/flink-connector-filesystem/pom.xml @@ -139,19 +139,6 @@ under the License. - - - - org.apache.felix - maven-bundle-plugin - 3.0.1 - true - true - - org.apache.maven.plugins maven-surefire-plugin diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index 94f407863babc..06bd399eaf6ce 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -209,17 +209,6 @@ under the License. 1 - - - org.apache.felix - maven-bundle-plugin - 3.0.1 - true - true - diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml index aae5716cbbf14..5cafcf99143d9 100644 --- a/flink-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-connectors/flink-connector-kafka-base/pom.xml @@ -214,17 +214,6 @@ under the License. - - - org.apache.felix - maven-bundle-plugin - 3.0.1 - true - true - diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 4dcf6df94b75e..59cc4287844aa 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -113,18 +113,6 @@ under the License. - - - org.apache.felix - maven-bundle-plugin - 3.0.1 - true - true - - org.apache.maven.plugins maven-shade-plugin diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index f01e8beaf16ec..41de9c8bfea74 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -345,18 +345,6 @@ under the License. - - - org.apache.felix - maven-bundle-plugin - 3.0.1 - true - true - - + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + org.apache.maven.plugins maven-jar-plugin From e2b92f22c2686f8d842d371a17c36c5d28f9b247 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 13 Nov 2017 11:50:07 +0100 Subject: [PATCH 030/367] [FLINK-8040] [tests] Fix test instability in ResourceGuardTest (cherry picked from commit ad8ef6d) --- .../apache/flink/util/ResourceGuardTest.java | 53 ++++++++++++------- 1 file changed, 34 insertions(+), 19 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java b/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java index 98aae4defb1d2..ade8d0fec3bf8 100644 --- a/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/ResourceGuardTest.java @@ -53,30 +53,37 @@ public void testAcquireReleaseClose() throws IOException { @Test public void testCloseBlockIfAcquired() throws Exception { ResourceGuard resourceGuard = new ResourceGuard(); - ResourceGuard.Lease lease_1 = resourceGuard.acquireResource(); + ResourceGuard.Lease lease = resourceGuard.acquireResource(); AtomicBoolean checker = new AtomicBoolean(true); Thread closerThread = new Thread() { @Override public void run() { - try { - // this line should block until all acquires are matched by releases. - resourceGuard.close(); - checker.set(false); - } catch (Exception ignore) { - checker.set(false); - } + // this line should block until all acquires are matched by releases. + resourceGuard.close(); + checker.set(false); } }; closerThread.start(); - ResourceGuard.Lease lease_2 = resourceGuard.acquireResource(); - lease_2.close(); + // we wait until the close()-call in the other thread happened. + while (!resourceGuard.isClosed()) { + Thread.yield(); + } + + // validate that the close()-call is still blocked. Assert.assertTrue(checker.get()); - // this matches the first acquire and will unblock the close. - lease_1.close(); + // validate that the closed-status is already effective. + try { + resourceGuard.acquireResource(); + Assert.fail("Resource guard is expected to be already closed."); + } catch (IOException ignore) { + } + + // this matches the first acquire and will unblock the close()-call in the other thread. + lease.close(); closerThread.join(60_000); Assert.assertFalse(checker.get()); } @@ -90,21 +97,29 @@ public void testInterruptHandledCorrectly() throws Exception { Thread closerThread = new Thread() { @Override public void run() { - try { - // this line should block until all acquires are matched by releases. - resourceGuard.close(); - checker.set(false); - } catch (Exception ignore) { - checker.set(false); - } + // this line should block until all acquires are matched by releases. + resourceGuard.close(); + checker.set(false); } }; closerThread.start(); + + // we wait until the close()-call in the other thread happened. + while (!resourceGuard.isClosed()) { + Thread.yield(); + } + + // attempt to unblock the resource guard via interrupt. closerThread.interrupt(); + // wait some time. + closerThread.join(100); + + // check that unblock through interrupting failed. Assert.assertTrue(checker.get()); + // proper unblocking by closing the lease. lease.close(); closerThread.join(60_000); Assert.assertFalse(checker.get()); From bb04187efffc74e17ded8b6199b023d7416ad5a3 Mon Sep 17 00:00:00 2001 From: Kent Murra Date: Wed, 27 Sep 2017 13:48:55 -0700 Subject: [PATCH 031/367] [FLINK-7657] [table] Add time types FilterableTableSource push down This closes #4746. --- .../flink/table/expressions/literals.scala | 34 ++- .../table/plan/util/RexProgramExtractor.scala | 33 ++- .../flink/table/api/TableSourceTest.scala | 65 ++++- .../table/plan/RexProgramExtractorTest.scala | 53 +++- .../flink/table/plan/RexProgramTestBase.scala | 5 + .../batch/table/TableSourceITCase.scala | 117 ++++++++- .../stream/table/TableSourceITCase.scala | 2 +- .../flink/table/utils/TableTestBase.scala | 2 - .../utils/TestFilterableTableSource.scala | 226 ++++++++++++++++++ .../flink/table/utils/testTableSources.scala | 101 -------- 10 files changed, 511 insertions(+), 127 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala index eb9c4f518639e..d797cc490e4ea 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala @@ -17,21 +17,22 @@ */ package org.apache.flink.table.expressions -import java.sql.{Date, Time, Timestamp} -import java.util.{Calendar, TimeZone} - import org.apache.calcite.avatica.util.TimeUnit import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.SqlIntervalQualifier import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo} +import java.sql.{Date, Time, Timestamp} +import java.util.{Calendar, TimeZone} + object Literal { - private[flink] val GMT = TimeZone.getTimeZone("GMT") + private[flink] val UTC = TimeZone.getTimeZone("UTC") private[flink] def apply(l: Any): Literal = l match { case i: Int => Literal(i, BasicTypeInfo.INT_TYPE_INFO) @@ -52,7 +53,7 @@ object Literal { } case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpression { - override def toString = resultType match { + override def toString: String = resultType match { case _: BasicTypeInfo[_] => value.toString case _@SqlTimeTypeInfo.DATE => value.toString + ".toDate" case _@SqlTimeTypeInfo.TIME => value.toString + ".toTime" @@ -77,11 +78,14 @@ case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpre // date/time case SqlTimeTypeInfo.DATE => - relBuilder.getRexBuilder.makeDateLiteral(dateToCalendar) + val datestr = DateString.fromCalendarFields(valueAsCalendar) + relBuilder.getRexBuilder.makeDateLiteral(datestr) case SqlTimeTypeInfo.TIME => - relBuilder.getRexBuilder.makeTimeLiteral(dateToCalendar, 0) + val timestr = TimeString.fromCalendarFields(valueAsCalendar) + relBuilder.getRexBuilder.makeTimeLiteral(timestr, 0) case SqlTimeTypeInfo.TIMESTAMP => - relBuilder.getRexBuilder.makeTimestampLiteral(dateToCalendar, 3) + val timestampstr = TimestampString.fromCalendarFields(valueAsCalendar) + relBuilder.getRexBuilder.makeTimestampLiteral(timestampstr, 3) case TimeIntervalTypeInfo.INTERVAL_MONTHS => val interval = java.math.BigDecimal.valueOf(value.asInstanceOf[Int]) @@ -103,12 +107,16 @@ case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpre } } - private def dateToCalendar: Calendar = { + /** + * Convert a date value to a calendar. Calcite fromCalendarField functions use the Calendar.get + * methods, so the raw values of the individual fields are preserved when converted to the + * string formats. + * @return Get the Calendar value + */ + private def valueAsCalendar: Calendar = { val date = value.asInstanceOf[java.util.Date] - val cal = Calendar.getInstance(Literal.GMT) - val t = date.getTime - // according to Calcite's SqlFunctions.internalToXXX methods - cal.setTimeInMillis(t + TimeZone.getDefault.getOffset(t)) + val cal = Calendar.getInstance + cal.setTime(date) cal } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index 53bf8e777af33..d11a43d1f4f37 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -22,12 +22,16 @@ import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.sql.{SqlFunction, SqlPostfixOperator} +import org.apache.calcite.util.{DateString, TimeString, TimestampString} +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo} import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.expressions.{And, Expression, Literal, Or, ResolvedFieldReference} import org.apache.flink.table.validate.FunctionCatalog import org.apache.flink.util.Preconditions +import java.sql.{Date, Time, Timestamp} + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -158,7 +162,33 @@ class RexNodeToExpressionConverter( } override def visitLiteral(literal: RexLiteral): Option[Expression] = { - Some(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + val literalType = FlinkTypeFactory.toTypeInfo(literal.getType) + + val literalValue = literalType match { + // Chrono use cases. + case _@SqlTimeTypeInfo.DATE => + val rexValue = literal.getValueAs(classOf[DateString]) + Date.valueOf(rexValue.toString) + case _@SqlTimeTypeInfo.TIME => + val rexValue = literal.getValueAs(classOf[TimeString]) + Time.valueOf(rexValue.toString(0)) + case _@SqlTimeTypeInfo.TIMESTAMP => + val rexValue = literal.getValueAs(classOf[TimestampString]) + Timestamp.valueOf(rexValue.toString(3)) + + case _@BasicTypeInfo.INT_TYPE_INFO => + /* + Force integer conversion. RelDataType is INTEGER and SqlTypeName is DECIMAL, + meaning that it will assume that we are using a BigDecimal + and refuse to convert to Integer. + */ + val rexValue = literal.getValueAs(classOf[java.math.BigDecimal]) + rexValue.intValue() + + case _ => literal.getValue + } + + Some(Literal(literalValue, literalType)) } override def visitCall(call: RexCall): Option[Expression] = { @@ -209,7 +239,6 @@ class RexNodeToExpressionConverter( private def replace(str: String): String = { str.replaceAll("\\s|_", "") } - } /** diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala index 4b88bc34f6815..dc84c193a0370 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala @@ -18,13 +18,19 @@ package org.apache.flink.table.api +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.{BinaryComparison, Expression, Literal} import org.apache.flink.table.expressions.utils._ import org.apache.flink.table.runtime.utils.CommonTestData import org.apache.flink.table.sources.{CsvTableSource, TableSource} import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.{TableTestBase, TestFilterableTableSource} import org.junit.{Assert, Test} +import _root_.java.sql.{Date, Time, Timestamp} + +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.types.Row class TableSourceTest extends TableTestBase { @@ -374,13 +380,69 @@ class TableSourceTest extends TableTestBase { Assert.assertEquals(source1, source2) } + @Test + def testTimeLiteralExpressionPushdown(): Unit = { + val (tableSource, tableName) = filterableTableSourceTimeTypes + val util = batchTestUtil() + val tableEnv = util.tableEnv + + tableEnv.registerTableSource(tableName, tableSource) + + val sqlQuery = + s""" + |SELECT id from $tableName + |WHERE + | tv > TIME '14:25:02' AND + | dv > DATE '2017-02-03' AND + | tsv > TIMESTAMP '2017-02-03 14:25:02.000' + """.stripMargin + + val result = tableEnv.sqlQuery(sqlQuery) + + val expectedFilter = + "'tv > 14:25:02.toTime && " + + "'dv > 2017-02-03.toDate && " + + "'tsv > 2017-02-03 14:25:02.0.toTimestamp" + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("id", "dv", "tv", "tsv"), + expectedFilter), + term("select", "id") + ) + util.verifyTable(result, expected) + } + // utils def filterableTableSource:(TableSource[_], String) = { - val tableSource = new TestFilterableTableSource + val tableSource = TestFilterableTableSource() (tableSource, "filterableTable") } + def filterableTableSourceTimeTypes:(TableSource[_], String) = { + val rowTypeInfo = new RowTypeInfo( + Array[TypeInformation[_]]( + BasicTypeInfo.INT_TYPE_INFO, + SqlTimeTypeInfo.DATE, + SqlTimeTypeInfo.TIME, + SqlTimeTypeInfo.TIMESTAMP + ), + Array("id", "dv", "tv", "tsv") + ) + + val row = new Row(4) + row.setField(0, 1) + row.setField(1, Date.valueOf("2017-01-23")) + row.setField(2, Time.valueOf("14:23:02")) + row.setField(3, Timestamp.valueOf("2017-01-24 12:45:01.234")) + + val tableSource = TestFilterableTableSource(rowTypeInfo, Seq(row), Set("dv", "tv", "tsv")) + (tableSource, "filterableTable") + } + + def csvTable: (CsvTableSource, String) = { val csvTable = CommonTestData.getCsvTableSource val tableName = "csvTable" @@ -414,4 +476,5 @@ class TableSourceTest extends TableTestBase { "StreamTableSourceScan(" + s"table=[[$sourceName]], fields=[${fields.mkString(", ")}], source=[filter=[$exp]])" } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala index c2a01c68ad441..6ed94552620ba 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala @@ -19,12 +19,15 @@ package org.apache.flink.table.plan import java.math.BigDecimal +import java.sql.{Date, Time, Timestamp} -import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex._ import org.apache.calcite.sql.SqlPostfixOperator +import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, INTEGER, VARCHAR} import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.apache.flink.table.expressions._ import org.apache.flink.table.plan.util.{RexNodeToExpressionConverter, RexProgramExtractor} import org.apache.flink.table.utils.InputTypeBuilder.inputOf @@ -199,6 +202,54 @@ class RexProgramExtractorTest extends RexProgramTestBase { } @Test + def testLiteralConversions(): Unit = { + val fieldNames = List("timestamp_col", "date_col", "time_col").asJava + val fieldTypes = makeTypes(SqlTypeName.TIMESTAMP, SqlTypeName.DATE, SqlTypeName.TIME) + + val inputRowType = typeFactory.createStructType(fieldTypes, fieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + val timestampString = new TimestampString("2017-09-10 14:23:01.245") + val rexTimestamp = rexBuilder.makeTimestampLiteral(timestampString, 3) + val rexDate = rexBuilder.makeDateLiteral(new DateString("2017-09-12")) + val rexTime = rexBuilder.makeTimeLiteral(new TimeString("14:23:01"), 0) + + val allRexNodes = List(rexTimestamp, rexDate, rexTime) + + val condition = fieldTypes.asScala.zipWithIndex + .map((t: (RelDataType, Int)) => rexBuilder.makeInputRef(t._1, t._2)) + .zip(allRexNodes) + .map(t => rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t._1, t._2)) + .map(builder.addExpr) + .asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + + val (converted, _) = RexProgramExtractor.extractConjunctiveConditions( + builder.getProgram, + new RexBuilder(typeFactory), + functionCatalog) + + + val expected = Array[Expression]( + EqualTo( + UnresolvedFieldReference("timestamp_col"), + Literal(Timestamp.valueOf("2017-09-10 14:23:01.245")) + ), + EqualTo( + UnresolvedFieldReference("date_col"), + Literal(Date.valueOf("2017-09-12")) + ), + EqualTo( + UnresolvedFieldReference("time_col"), + Literal(Time.valueOf("14:23:01")) + ) + ) + + assertExpressionArrayEquals(expected, converted) + } + + @Test def testExtractArithmeticConditions(): Unit = { val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) val builder = new RexProgramBuilder(inputRowType, rexBuilder) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala index b7116043a4d86..728694fffa8ff 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala @@ -24,6 +24,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory import org.apache.calcite.jdbc.JavaTypeFactoryImpl import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeSystem} import org.apache.calcite.rex.{RexBuilder, RexProgram, RexProgramBuilder} +import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.`type`.SqlTypeName._ import org.apache.calcite.sql.fun.SqlStdOperatorTable @@ -76,4 +77,8 @@ abstract class RexProgramTestBase { builder.getProgram } + protected def makeTypes(fieldTypes: SqlTypeName*): java.util.List[RelDataType] = { + fieldTypes.toList.map(typeFactory.createSqlType).asJava + } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala index 2292e17f7a7bf..f0fe896e92f74 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala @@ -19,17 +19,17 @@ package org.apache.flink.table.runtime.batch.table import java.lang.{Boolean => JBool, Integer => JInt, Long => JLong} +import java.sql.{Date, Time, Timestamp} import org.apache.calcite.runtime.SqlFunctions.{internalToTimestamp => toTimestamp} -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.DataSet -import org.apache.flink.api.java.{ExecutionEnvironment => JExecEnv} +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo} +import org.apache.flink.api.java.{DataSet, ExecutionEnvironment => JExecEnv} import org.apache.flink.api.scala.ExecutionEnvironment -import org.apache.flink.table.api.{TableEnvironment, TableException, TableSchema, Types} import org.apache.flink.table.api.scala._ -import org.apache.flink.table.runtime.utils.{CommonTestData, TableProgramsCollectionTestBase} +import org.apache.flink.table.api.{TableEnvironment, TableException, TableSchema, Types} import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode +import org.apache.flink.table.runtime.utils.{CommonTestData, TableProgramsCollectionTestBase} import org.apache.flink.table.sources.BatchTableSource import org.apache.flink.table.utils._ import org.apache.flink.test.util.TestBaseUtils @@ -101,7 +101,7 @@ class TableSourceITCase( val tableName = "MyTable" val env = ExecutionEnvironment.getExecutionEnvironment val tableEnv = TableEnvironment.getTableEnvironment(env, config) - tableEnv.registerTableSource(tableName, new TestFilterableTableSource) + tableEnv.registerTableSource(tableName, TestFilterableTableSource()) val results = tableEnv .scan(tableName) .where("amount > 4 && price < 9") @@ -250,6 +250,37 @@ class TableSourceITCase( "Mary,1970-01-01 00:00:00.0,40", "Bob,1970-01-01 00:00:00.0,20", "Liz,1970-01-01 00:00:02.0,40").mkString("\n") + } + + @Test + def testTableSourceWithFilterableDate(): Unit = { + val tableName = "MyTable" + val env = ExecutionEnvironment.getExecutionEnvironment + val tableEnv = TableEnvironment.getTableEnvironment(env, config) + + val rowTypeInfo = new RowTypeInfo( + Array[TypeInformation[_]](BasicTypeInfo.INT_TYPE_INFO, SqlTimeTypeInfo.DATE), + Array("id", "date_val")) + + val rows = Seq( + makeRow(23, Date.valueOf("2017-04-23")), + makeRow(24, Date.valueOf("2017-04-24")), + makeRow(25, Date.valueOf("2017-04-25")), + makeRow(26, Date.valueOf("2017-04-26")) + ) + + val query = + """ + |select id from MyTable + |where date_val >= DATE '2017-04-24' and date_val < DATE '2017-04-26' + """.stripMargin + val tableSource = TestFilterableTableSource(rowTypeInfo, rows, Set("date_val")) + tableEnv.registerTableSource(tableName, tableSource) + val results = tableEnv + .sqlQuery(query) + .collect() + + val expected = Seq(24, 25).mkString("\n") TestBaseUtils.compareResultAsText(results.asJava, expected) } @@ -473,6 +504,7 @@ class TableSourceITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test def testProjectOnlyProctime(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -504,6 +536,7 @@ class TableSourceITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test def testProjectOnlyRowtime(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -643,4 +676,76 @@ class TableSourceITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test + def testTableSourceWithFilterableTime(): Unit = { + val tableName = "MyTable" + val env = ExecutionEnvironment.getExecutionEnvironment + val tableEnv = TableEnvironment.getTableEnvironment(env, config) + + val rowTypeInfo = new RowTypeInfo( + Array[TypeInformation[_]](BasicTypeInfo.INT_TYPE_INFO, SqlTimeTypeInfo.TIME), + Array("id", "time_val")) + + val rows = Seq( + makeRow(1, Time.valueOf("7:23:19")), + makeRow(2, Time.valueOf("11:45:00")), + makeRow(3, Time.valueOf("11:45:01")), + makeRow(4, Time.valueOf("12:14:23")), + makeRow(5, Time.valueOf("13:33:12")) + ) + + val query = + """ + |select id from MyTable + |where time_val >= TIME '11:45:00' and time_val < TIME '12:14:23' + """.stripMargin + val tableSource = TestFilterableTableSource(rowTypeInfo, rows, Set("time_val")) + tableEnv.registerTableSource(tableName, tableSource) + val results = tableEnv + .sqlQuery(query) + .collect() + + val expected = Seq(2, 3).mkString("\n") + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testTableSourceWithFilterableTimestamp(): Unit = { + val tableName = "MyTable" + val env = ExecutionEnvironment.getExecutionEnvironment + val tableEnv = TableEnvironment.getTableEnvironment(env, config) + + val rowTypeInfo = new RowTypeInfo( + Array[TypeInformation[_]](BasicTypeInfo.INT_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP), + Array("id", "ts")) + + val rows = Seq( + makeRow(1, Timestamp.valueOf("2017-07-11 7:23:19")), + makeRow(2, Timestamp.valueOf("2017-07-12 11:45:00")), + makeRow(3, Timestamp.valueOf("2017-07-13 11:45:01")), + makeRow(4, Timestamp.valueOf("2017-07-14 12:14:23")), + makeRow(5, Timestamp.valueOf("2017-07-13 13:33:12")) + ) + + val query = + """ + |select id from MyTable + |where ts >= TIMESTAMP '2017-07-12 11:45:00' and ts < TIMESTAMP '2017-07-14 12:14:23' + """.stripMargin + val tableSource = TestFilterableTableSource(rowTypeInfo, rows, Set("ts")) + tableEnv.registerTableSource(tableName, tableSource) + val results = tableEnv + .sqlQuery(query) + .collect() + + val expected = Seq(2, 3, 5).mkString("\n") + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + private def makeRow(fields: Any*): Row = { + val row = new Row(fields.length) + val addField = (value: Any, pos: Int) => row.setField(pos, value) + fields.zipWithIndex.foreach(addField.tupled) + row + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala index a9e9632015630..77c1e081686d5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala @@ -104,7 +104,7 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase { val tableName = "MyTable" val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) - tEnv.registerTableSource(tableName, new TestFilterableTableSource) + tEnv.registerTableSource(tableName, TestFilterableTableSource()) tEnv.scan(tableName) .where("amount > 4 && price < 9") .select("id, name") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index 4042f504cb232..5f8f5d6a40d87 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -212,11 +212,9 @@ case class BatchTableTestUtil() extends TableTestUtil { def printSql(query: String): Unit = { printTable(tableEnv.sqlQuery(query)) } - } case class StreamTableTestUtil() extends TableTestUtil { - val javaEnv = mock(classOf[JStreamExecutionEnvironment]) when(javaEnv.getStreamTimeCharacteristic).thenReturn(TimeCharacteristic.EventTime) val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala new file mode 100644 index 0000000000000..ae2b1d6138231 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala @@ -0,0 +1,226 @@ +/* + * 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.flink.table.utils + +import java.util.{List => JList} + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.table.api.TableSchema +import org.apache.flink.table.api.Types._ +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.{BatchTableSource, FilterableTableSource, StreamTableSource, TableSource} +import org.apache.flink.types.Row + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object TestFilterableTableSource { + /** + * @return The default filterable table source. + */ + def apply(): TestFilterableTableSource = { + apply(defaultTypeInfo, defaultRows, defaultFilterableFields) + } + + /** + * A filterable data source with custom data. + * @param rowTypeInfo The type of the data. + * Its expected that both types and field names are provided + * @param rows The data as a sequence of rows. + * @param filterableFields The fields that are allowed to be filtered on. + * @return The table source. + */ + def apply(rowTypeInfo: RowTypeInfo, + rows: Seq[Row], + filterableFields: Set[String]): TestFilterableTableSource = { + new TestFilterableTableSource(rowTypeInfo, rows, filterableFields) + } + + private lazy val defaultFilterableFields = Set("amount") + + private lazy val defaultTypeInfo: RowTypeInfo = { + val fieldNames: Array[String] = Array("name", "id", "amount", "price") + val fieldTypes: Array[TypeInformation[_]] = Array(STRING, LONG, INT, DOUBLE) + new RowTypeInfo(fieldTypes, fieldNames) + } + + + private lazy val defaultRows: Seq[Row] = { + for { + cnt <- 0 until 33 + } yield { + Row.of( + s"Record_$cnt", + cnt.toLong.asInstanceOf[Object], + cnt.toInt.asInstanceOf[Object], + cnt.toDouble.asInstanceOf[Object]) + } + } +} + + +/** + * + * + * A data source that implements some very basic filtering in-memory in order to test + * expression push-down logic. + * + * @param rowTypeInfo The type info for the rows. + * @param data The data that filtering is applied to in order to get the final dataset. + * @param filterableFields The fields that are allowed to be filtered. + * @param filterPredicates The predicates that should be used to filter. + * @param filterPushedDown Whether predicates have been pushed down yet. + */ +class TestFilterableTableSource(rowTypeInfo: RowTypeInfo, + data: Seq[Row], + filterableFields: Set[String] = Set(), + filterPredicates: Seq[Expression] = Seq(), + val filterPushedDown: Boolean = false) + extends BatchTableSource[Row] + with StreamTableSource[Row] + with FilterableTableSource[Row] { + + val fieldNames: Array[String] = rowTypeInfo.getFieldNames + + val fieldTypes: Array[TypeInformation[_]] = rowTypeInfo.getFieldTypes + + // all comparing values for field "amount" + private val filterValues = new mutable.ArrayBuffer[Int] + + override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] = { + execEnv.fromCollection[Row](applyPredicatesToRows(data).asJava, getReturnType) + } + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + execEnv.fromCollection[Row](applyPredicatesToRows(data).asJava, getReturnType) + } + + override def explainSource(): String = { + if (filterPredicates.nonEmpty) { + s"filter=[${filterPredicates.reduce((l, r) => And(l, r)).toString}]" + } else { + "" + } + } + + override def getReturnType: TypeInformation[Row] = rowTypeInfo + + override def applyPredicate(predicates: JList[Expression]): TableSource[Row] = { + val predicatesToUse = new mutable.ListBuffer[Expression]() + val iterator = predicates.iterator() + while (iterator.hasNext) { + val expr = iterator.next() + if (shouldPushDown(expr)) { + predicatesToUse += expr + iterator.remove() + } + } + + new TestFilterableTableSource( + rowTypeInfo, + data, + filterableFields, + predicatesToUse, + filterPushedDown = true) + } + + override def isFilterPushedDown: Boolean = filterPushedDown + + private def applyPredicatesToRows(rows: Seq[Row]): Seq[Row] = { + rows.filter(shouldKeep) + } + + private def shouldPushDown(expr: Expression): Boolean = { + expr match { + case binExpr: BinaryComparison => shouldPushDown(binExpr) + case _ => false + } + } + + private def shouldPushDown(expr: BinaryComparison): Boolean = { + (expr.left, expr.right) match { + case (f: ResolvedFieldReference, v: Literal) => + filterableFields.contains(f.name) + case (v: Literal, f: ResolvedFieldReference) => + filterableFields.contains(f.name) + case (f1: ResolvedFieldReference, f2: ResolvedFieldReference) => + filterableFields.contains(f1.name) && filterableFields.contains(f2.name) + case (_, _) => false + } + } + + private def shouldKeep(row: Row): Boolean = { + filterPredicates.isEmpty || filterPredicates.forall { + case expr: BinaryComparison => binaryFilterApplies(expr, row) + case expr => throw new RuntimeException(expr + " not supported!") + } + } + + private def binaryFilterApplies(expr: BinaryComparison, row: Row): Boolean = { + val (lhsValue, rhsValue) = extractValues(expr, row) + + expr match { + case _: GreaterThan => + lhsValue.compareTo(rhsValue) > 0 + case LessThan(l: ResolvedFieldReference, r: Literal) => + lhsValue.compareTo(rhsValue) < 0 + case GreaterThanOrEqual(l: ResolvedFieldReference, r: Literal) => + lhsValue.compareTo(rhsValue) >= 0 + case LessThanOrEqual(l: ResolvedFieldReference, r: Literal) => + lhsValue.compareTo(rhsValue) <= 0 + case EqualTo(l: ResolvedFieldReference, r: Literal) => + lhsValue.compareTo(rhsValue) == 0 + case NotEqualTo(l: ResolvedFieldReference, r: Literal) => + lhsValue.compareTo(rhsValue) != 0 + } + } + + private def extractValues(expr: BinaryComparison, + row: Row): (Comparable[Any], Comparable[Any]) = { + (expr.left, expr.right) match { + case (l: ResolvedFieldReference, r: Literal) => + val idx = rowTypeInfo.getFieldIndex(l.name) + val lv = row.getField(idx).asInstanceOf[Comparable[Any]] + val rv = r.value.asInstanceOf[Comparable[Any]] + (lv, rv) + case (l: Literal, r: ResolvedFieldReference) => + val idx = rowTypeInfo.getFieldIndex(r.name) + val lv = l.value.asInstanceOf[Comparable[Any]] + val rv = row.getField(idx).asInstanceOf[Comparable[Any]] + (lv, rv) + case (l: Literal, r: Literal) => + val lv = l.value.asInstanceOf[Comparable[Any]] + val rv = r.value.asInstanceOf[Comparable[Any]] + (lv, rv) + case (l: ResolvedFieldReference, r: ResolvedFieldReference) => + val lidx = rowTypeInfo.getFieldIndex(l.name) + val ridx = rowTypeInfo.getFieldIndex(r.name) + val lv = row.getField(lidx).asInstanceOf[Comparable[Any]] + val rv = row.getField(ridx).asInstanceOf[Comparable[Any]] + (lv, rv) + case _ => throw new RuntimeException(expr + " not supported!") + } + } + + override def getTableSchema: TableSchema = new TableSchema(fieldNames, fieldTypes) +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala index f11f0ca3e165c..c2eba32981a30 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala @@ -147,107 +147,6 @@ class TestProjectableTableSource( } } -/** - * This source can only handle simple comparision with field "amount". - * Supports ">, <, >=, <=, =, <>" with an integer. - */ -class TestFilterableTableSource( - val recordNum: Int = 33) - extends BatchTableSource[Row] - with StreamTableSource[Row] - with FilterableTableSource[Row] { - - var filterPushedDown: Boolean = false - - val fieldNames: Array[String] = Array("name", "id", "amount", "price") - - val fieldTypes: Array[TypeInformation[_]] = Array(STRING, LONG, INT, DOUBLE) - - // all predicates with field "amount" - private var filterPredicates = new mutable.ArrayBuffer[Expression] - - // all comparing values for field "amount" - private val filterValues = new mutable.ArrayBuffer[Int] - - override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] = { - execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) - } - - override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { - execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) - } - - override def explainSource(): String = { - if (filterPredicates.nonEmpty) { - s"filter=[${filterPredicates.reduce((l, r) => And(l, r)).toString}]" - } else { - "" - } - } - - override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes, fieldNames) - - override def applyPredicate(predicates: JList[Expression]): TableSource[Row] = { - val newSource = new TestFilterableTableSource(recordNum) - newSource.filterPushedDown = true - - val iterator = predicates.iterator() - while (iterator.hasNext) { - iterator.next() match { - case expr: BinaryComparison => - (expr.left, expr.right) match { - case (f: ResolvedFieldReference, v: Literal) if f.name.equals("amount") => - newSource.filterPredicates += expr - newSource.filterValues += v.value.asInstanceOf[Number].intValue() - iterator.remove() - case (_, _) => - } - case _ => - } - } - - newSource - } - - override def isFilterPushedDown: Boolean = filterPushedDown - - private def generateDynamicCollection(): Seq[Row] = { - Preconditions.checkArgument(filterPredicates.length == filterValues.length) - - for { - cnt <- 0 until recordNum - if shouldCreateRow(cnt) - } yield { - Row.of( - s"Record_$cnt", - cnt.toLong.asInstanceOf[Object], - cnt.toInt.asInstanceOf[Object], - cnt.toDouble.asInstanceOf[Object]) - } - } - - private def shouldCreateRow(value: Int): Boolean = { - filterPredicates.zip(filterValues).forall { - case (_: GreaterThan, v) => - value > v - case (_: LessThan, v) => - value < v - case (_: GreaterThanOrEqual, v) => - value >= v - case (_: LessThanOrEqual, v) => - value <= v - case (_: EqualTo, v) => - value == v - case (_: NotEqualTo, v) => - value != v - case (expr, _) => - throw new RuntimeException(expr + " not supported!") - } - } - - override def getTableSchema: TableSchema = new TableSchema(fieldNames, fieldTypes) -} - class TestNestedProjectableTableSource( tableSchema: TableSchema, returnType: TypeInformation[Row], From ce1cb8fd6d667713be9b5f9ec8f1c394b9ca4644 Mon Sep 17 00:00:00 2001 From: twalthr Date: Mon, 13 Nov 2017 14:09:45 +0100 Subject: [PATCH 032/367] [FLINK-7657] [table] Add all basic types to RexProgramExtractor --- .../flink/table/expressions/literals.scala | 9 +++-- .../table/plan/util/RexProgramExtractor.scala | 33 ++++++++++++---- .../flink/table/api/TableSourceTest.scala | 25 ++++++------ .../flink/table/plan/RexProgramTestBase.scala | 1 - .../batch/table/TableSourceITCase.scala | 2 +- .../utils/TestFilterableTableSource.scala | 39 ++++++++++--------- .../flink/table/utils/testTableSources.scala | 6 +-- 7 files changed, 63 insertions(+), 52 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala index d797cc490e4ea..e6905ef998956 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala @@ -108,10 +108,11 @@ case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpre } /** - * Convert a date value to a calendar. Calcite fromCalendarField functions use the Calendar.get - * methods, so the raw values of the individual fields are preserved when converted to the - * string formats. - * @return Get the Calendar value + * Convert a Date value to a Calendar. Calcite's fromCalendarField functions use the + * Calendar.get methods, so the raw values of the individual fields are preserved when + * converted to the String formats. + * + * @return get the Calendar value */ private def valueAsCalendar: Calendar = { val date = value.asInstanceOf[java.util.Date] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index d11a43d1f4f37..9c06135e46806 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -165,25 +165,42 @@ class RexNodeToExpressionConverter( val literalType = FlinkTypeFactory.toTypeInfo(literal.getType) val literalValue = literalType match { - // Chrono use cases. + case _@SqlTimeTypeInfo.DATE => val rexValue = literal.getValueAs(classOf[DateString]) Date.valueOf(rexValue.toString) + case _@SqlTimeTypeInfo.TIME => val rexValue = literal.getValueAs(classOf[TimeString]) Time.valueOf(rexValue.toString(0)) + case _@SqlTimeTypeInfo.TIMESTAMP => val rexValue = literal.getValueAs(classOf[TimestampString]) Timestamp.valueOf(rexValue.toString(3)) + case _@BasicTypeInfo.BYTE_TYPE_INFO => + // convert from BigDecimal to Byte + literal.getValueAs(classOf[java.lang.Byte]) + + case _@BasicTypeInfo.SHORT_TYPE_INFO => + // convert from BigDecimal to Short + literal.getValueAs(classOf[java.lang.Short]) + case _@BasicTypeInfo.INT_TYPE_INFO => - /* - Force integer conversion. RelDataType is INTEGER and SqlTypeName is DECIMAL, - meaning that it will assume that we are using a BigDecimal - and refuse to convert to Integer. - */ - val rexValue = literal.getValueAs(classOf[java.math.BigDecimal]) - rexValue.intValue() + // convert from BigDecimal to Integer + literal.getValueAs(classOf[java.lang.Integer]) + + case _@BasicTypeInfo.LONG_TYPE_INFO => + // convert from BigDecimal to Long + literal.getValueAs(classOf[java.lang.Long]) + + case _@BasicTypeInfo.FLOAT_TYPE_INFO => + // convert from BigDecimal to Float + literal.getValueAs(classOf[java.lang.Float]) + + case _@BasicTypeInfo.DOUBLE_TYPE_INFO => + // convert from BigDecimal to Double + literal.getValueAs(classOf[java.lang.Double]) case _ => literal.getValue } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala index dc84c193a0370..42f0769c63a34 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala @@ -18,19 +18,18 @@ package org.apache.flink.table.api +import _root_.java.sql.{Date, Time, Timestamp} + import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.scala._ -import org.apache.flink.table.expressions.{BinaryComparison, Expression, Literal} import org.apache.flink.table.expressions.utils._ import org.apache.flink.table.runtime.utils.CommonTestData import org.apache.flink.table.sources.{CsvTableSource, TableSource} import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.{TableTestBase, TestFilterableTableSource} -import org.junit.{Assert, Test} -import _root_.java.sql.{Date, Time, Timestamp} - -import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.types.Row +import org.junit.{Assert, Test} class TableSourceTest extends TableTestBase { @@ -225,7 +224,8 @@ class TableSourceTest extends TableTestBase { val result = tableEnv .scan(tableName) .select('price, 'id, 'amount) - .where("amount > 2 && (amount < 32 || amount.cast(LONG) > 10)") // cast can not be converted + .where("amount > 2 && id < 1.2 && " + + "(amount < 32 || amount.cast(LONG) > 10)") // cast can not be converted val expected = unaryNode( "DataSetCalc", @@ -234,7 +234,7 @@ class TableSourceTest extends TableTestBase { Array("price", "id", "amount"), "'amount > 2"), term("select", "price", "id", "amount"), - term("where", "OR(<(amount, 32), >(CAST(amount), 10))") + term("where", "AND(<(id, 1.2E0), OR(<(amount, 32), >(CAST(amount), 10)))") ) util.verifyTable(result, expected) } @@ -403,13 +403,10 @@ class TableSourceTest extends TableTestBase { "'tv > 14:25:02.toTime && " + "'dv > 2017-02-03.toDate && " + "'tsv > 2017-02-03 14:25:02.0.toTimestamp" - val expected = unaryNode( - "DataSetCalc", - batchFilterableSourceTableNode( - tableName, - Array("id", "dv", "tv", "tsv"), - expectedFilter), - term("select", "id") + val expected = batchFilterableSourceTableNode( + tableName, + Array("id"), + expectedFilter ) util.verifyTable(result, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala index 728694fffa8ff..05870ca514ff1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramTestBase.scala @@ -80,5 +80,4 @@ abstract class RexProgramTestBase { protected def makeTypes(fieldTypes: SqlTypeName*): java.util.List[RelDataType] = { fieldTypes.toList.map(typeFactory.createSqlType).asJava } - } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala index f0fe896e92f74..eb73f1b5ca70f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableSourceITCase.scala @@ -528,7 +528,7 @@ class TableSourceITCase( new TestProjectableTableSource(tableSchema, returnType, data, "rtime", "ptime")) val results = tEnv.scan("T") - .select('ptime > 0) + .select('ptime.cast(Types.LONG) > 0) .select(1.count) .collect() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala index ae2b1d6138231..da8220da0ee22 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala @@ -35,6 +35,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable object TestFilterableTableSource { + /** * @return The default filterable table source. */ @@ -44,15 +45,17 @@ object TestFilterableTableSource { /** * A filterable data source with custom data. - * @param rowTypeInfo The type of the data. - * Its expected that both types and field names are provided + * @param rowTypeInfo The type of the data. Its expected that both types and field + * names are provided. * @param rows The data as a sequence of rows. * @param filterableFields The fields that are allowed to be filtered on. * @return The table source. */ - def apply(rowTypeInfo: RowTypeInfo, - rows: Seq[Row], - filterableFields: Set[String]): TestFilterableTableSource = { + def apply( + rowTypeInfo: RowTypeInfo, + rows: Seq[Row], + filterableFields: Set[String]) + : TestFilterableTableSource = { new TestFilterableTableSource(rowTypeInfo, rows, filterableFields) } @@ -64,24 +67,20 @@ object TestFilterableTableSource { new RowTypeInfo(fieldTypes, fieldNames) } - private lazy val defaultRows: Seq[Row] = { for { cnt <- 0 until 33 } yield { Row.of( s"Record_$cnt", - cnt.toLong.asInstanceOf[Object], - cnt.toInt.asInstanceOf[Object], - cnt.toDouble.asInstanceOf[Object]) + cnt.toLong.asInstanceOf[AnyRef], + cnt.toInt.asInstanceOf[AnyRef], + cnt.toDouble.asInstanceOf[AnyRef]) } } } - /** - * - * * A data source that implements some very basic filtering in-memory in order to test * expression push-down logic. * @@ -91,11 +90,12 @@ object TestFilterableTableSource { * @param filterPredicates The predicates that should be used to filter. * @param filterPushedDown Whether predicates have been pushed down yet. */ -class TestFilterableTableSource(rowTypeInfo: RowTypeInfo, - data: Seq[Row], - filterableFields: Set[String] = Set(), - filterPredicates: Seq[Expression] = Seq(), - val filterPushedDown: Boolean = false) +class TestFilterableTableSource( + rowTypeInfo: RowTypeInfo, + data: Seq[Row], + filterableFields: Set[String] = Set(), + filterPredicates: Seq[Expression] = Seq(), + val filterPushedDown: Boolean = false) extends BatchTableSource[Row] with StreamTableSource[Row] with FilterableTableSource[Row] { @@ -195,8 +195,9 @@ class TestFilterableTableSource(rowTypeInfo: RowTypeInfo, } } - private def extractValues(expr: BinaryComparison, - row: Row): (Comparable[Any], Comparable[Any]) = { + private def extractValues(expr: BinaryComparison, row: Row) + : (Comparable[Any], Comparable[Any]) = { + (expr.left, expr.right) match { case (l: ResolvedFieldReference, r: Literal) => val idx = rowTypeInfo.getFieldIndex(l.name) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala index c2eba32981a30..a546919189e2e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala @@ -22,21 +22,17 @@ import java.util import java.util.{Collections, List => JList} import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import org.apache.flink.table.api.Types.{DOUBLE, INT, LONG, STRING} import org.apache.flink.table.api.TableSchema -import org.apache.flink.table.expressions._ import org.apache.flink.table.sources._ import org.apache.flink.table.sources.tsextractors.ExistingField import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps import org.apache.flink.types.Row -import org.apache.flink.util.Preconditions import scala.collection.JavaConverters._ -import scala.collection.mutable class TestTableSourceWithTime[T]( tableSchema: TableSchema, From 25a28ab32609c45fb8c40f717148e32fb453d2fc Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Mon, 6 Nov 2017 19:53:37 +0100 Subject: [PATCH 033/367] [FLINK-7973] Fix shading and relocating Hadoop for the S3 filesystems - do not shade everything, especially not JDK classes! -> instead define include patterns explicitly - do not shade core Flink classes (only those imported from flink-hadoop-fs) - hack around Hadoop loading (unshaded/non-relocated) classes based on names in the core-default.xml by overwriting the Configuration class (we may need to extend this for the mapred-default.xml and hdfs-defaults.xml): -> provide a core-default-shaded.xml file with shaded class names and copy and adapt the Configuration class of the respective Hadoop version to load this file instead of core-default.xml. Add checkstyle suppression pattern for the Hadoop Configuration classes Also fix the (integration) tests not working because they tried to load the relocated classes which are apparently not available there Remove minimizeJar from shading of flink-s3-fs-presto because this was causing "java.lang.ClassNotFoundException: org.apache.flink.fs.s3presto.shaded.org.apache.commons.logging.impl.LogFactoryImpl" since these classes are not statically imported and thus removed when minimizing. Fix s3-fs-presto not shading org.HdrHistogram Fix log4j being relocated in the S3 fs implementations Add shading checks to travis --- .../flink-s3-fs-hadoop/README.md | 27 + flink-filesystems/flink-s3-fs-hadoop/pom.xml | 84 +- .../org/apache/hadoop/conf/Configuration.java | 3002 +++++++++++++++++ .../main/resources/core-default-shaded.xml | 2312 +++++++++++++ .../src/test/resources/core-site.xml | 2312 +++++++++++++ .../flink-s3-fs-presto/README.md | 28 + flink-filesystems/flink-s3-fs-presto/pom.xml | 73 +- .../org/apache/hadoop/conf/Configuration.java | 2951 ++++++++++++++++ .../main/resources/core-default-shaded.xml | 1978 +++++++++++ .../src/test/resources/core-site.xml | 1978 +++++++++++ tools/maven/suppressions.xml | 4 + tools/travis_mvn_watchdog.sh | 53 +- 12 files changed, 14778 insertions(+), 24 deletions(-) create mode 100644 flink-filesystems/flink-s3-fs-hadoop/README.md create mode 100644 flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java create mode 100644 flink-filesystems/flink-s3-fs-hadoop/src/main/resources/core-default-shaded.xml create mode 100644 flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml create mode 100644 flink-filesystems/flink-s3-fs-presto/README.md create mode 100644 flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java create mode 100644 flink-filesystems/flink-s3-fs-presto/src/main/resources/core-default-shaded.xml create mode 100644 flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml diff --git a/flink-filesystems/flink-s3-fs-hadoop/README.md b/flink-filesystems/flink-s3-fs-hadoop/README.md new file mode 100644 index 0000000000000..3ad90e36e2c3a --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/README.md @@ -0,0 +1,27 @@ +This project is a wrapper around Hadoop's s3a file system. By pulling a smaller dependency tree and +shading all dependencies away, this keeps the appearance of Flink being Hadoop-free, +from a dependency perspective. + +We also relocate the shaded Hadoop version to allow running in a different +setup. For this to work, however, we needed to adapt Hadoop's `Configuration` +class to load a (shaded) `core-default-shaded.xml` configuration with the +relocated class names of classes loaded via reflection +(in the future, we may need to extend this to `mapred-default.xml` and `hdfs-defaults.xml` and their respective configuration classes). + +# Changing the Hadoop Version + +If you want to change the Hadoop version this project depends on, the following +steps are required to keep the shading correct: + +1. copy `org/apache/hadoop/conf/Configuration.java` from the respective Hadoop jar file to this project + - adapt the `Configuration` class by replacing `core-default.xml` with `core-default-shaded.xml`. +2. copy `core-default.xml` from the respective Hadoop jar file to this project as + - `src/main/resources/core-default-shaded.xml` (replacing every occurence of `org.apache.hadoop` with `org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop`) + - `src/test/resources/core-site.xml` (as is) +3. verify the shaded jar: + - does not contain any unshaded classes except for `org.apache.flink.fs.s3hadoop.S3FileSystemFactory` + - all other classes should be under `org.apache.flink.fs.s3hadoop.shaded` + - there should be a `META-INF/services/org.apache.flink.fs.s3hadoop.S3FileSystemFactory` file pointing to the `org.apache.flink.fs.s3hadoop.S3FileSystemFactory` class + - other service files under `META-INF/services` should have their names and contents in the relocated `org.apache.flink.fs.s3hadoop.shaded` package + - contains a `core-default-shaded.xml` file + - does not contain a `core-default.xml` or `core-site.xml` file diff --git a/flink-filesystems/flink-s3-fs-hadoop/pom.xml b/flink-filesystems/flink-s3-fs-hadoop/pom.xml index de7921ddff566..7ffe8216fbfb0 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-s3-fs-hadoop/pom.xml @@ -33,6 +33,7 @@ under the License. jar + 2.8.1 1.11.95 @@ -234,28 +235,87 @@ under the License. - org - org.apache.flink.fs.s3hadoop.shaded.org + com.amazonaws + org.apache.flink.fs.s3hadoop.shaded.com.amazonaws + + + com.fasterxml + org.apache.flink.fs.s3hadoop.shaded.com.fasterxml + + + com.google + org.apache.flink.fs.s3hadoop.shaded.com.google + + + com.google.code.findbugs.** + + + + com.nimbusds + org.apache.flink.fs.s3hadoop.shaded.com.nimbusds + + + com.squareup + org.apache.flink.fs.s3hadoop.shaded.com.squareup + + + net.jcip + org.apache.flink.fs.s3hadoop.shaded.net.jcip + + + net.minidev + org.apache.flink.fs.s3hadoop.shaded.net.minidev + + + + + org.apache.flink.runtime.fs.hdfs + org.apache.flink.fs.s3hadoop.shaded.org.apache.flink.runtime.fs.hdfs + + + org.apache.flink.runtime.util + org.apache.flink.fs.s3hadoop.shaded.org.apache.flink.runtime.util + + org.apache.flink.runtime.util.**Hadoop* + + + + + org.apache + org.apache.flink.fs.s3hadoop.shaded.org.apache - org.apache.flink.core.fs.FileSystemFactory - org.apache.flink.fs.s3hadoop.** + + org.apache.flink.** + org.apache.log4j.** - com - org.apache.flink.fs.s3hadoop.shaded.com + org.codehaus + org.apache.flink.fs.s3hadoop.shaded.org.codehaus + + + org.joda + org.apache.flink.fs.s3hadoop.shaded.org.joda + + + org.mortbay + org.apache.flink.fs.s3hadoop.shaded.org.mortbay + + + org.tukaani + org.apache.flink.fs.s3hadoop.shaded.org.tukaani - net - org.apache.flink.fs.s3hadoop.shaded.net + org.znerd + org.apache.flink.fs.s3hadoop.shaded.org.znerd okio org.apache.flink.fs.s3hadoop.shaded.okio - software - org.apache.flink.fs.s3hadoop.shaded.software + software.amazon + org.apache.flink.fs.s3hadoop.shaded.software.amazon @@ -277,6 +337,10 @@ under the License. META-INF/maven/org.apache.commons/** META-INF/maven/org.apache.flink/flink-hadoop-fs/** META-INF/maven/org.apache.flink/force-shading/** + + core-default.xml + + core-site.xml diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java new file mode 100644 index 0000000000000..16f162ddbfe30 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -0,0 +1,3002 @@ +/** + * 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.hadoop.conf; + +import com.google.common.annotations.VisibleForTesting; + +import java.io.BufferedInputStream; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.Reader; +import java.io.Writer; +import java.lang.ref.WeakReference; +import java.net.InetSocketAddress; +import java.net.JarURLConnection; +import java.net.URL; +import java.net.URLConnection; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.StringTokenizer; +import java.util.WeakHashMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerException; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; + +import com.google.common.base.Charsets; +import org.apache.commons.collections.map.UnmodifiableMap; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.alias.CredentialProvider; +import org.apache.hadoop.security.alias.CredentialProvider.CredentialEntry; +import org.apache.hadoop.security.alias.CredentialProviderFactory; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.StringInterner; +import org.apache.hadoop.util.StringUtils; +import org.codehaus.jackson.JsonFactory; +import org.codehaus.jackson.JsonGenerator; +import org.w3c.dom.DOMException; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.w3c.dom.Text; +import org.xml.sax.SAXException; + +import com.google.common.base.Preconditions; + +/** + * Provides access to configuration parameters. + * + *

Resources

+ * + *

Configurations are specified by resources. A resource contains a set of + * name/value pairs as XML data. Each resource is named by either a + * String or by a {@link Path}. If named by a String, + * then the classpath is examined for a file with that name. If named by a + * Path, then the local filesystem is examined directly, without + * referring to the classpath. + * + *

Unless explicitly turned off, Hadoop by default specifies two + * resources, loaded in-order from the classpath:

    + *
  1. + * + * core-default-shaded.xml: Read-only defaults for hadoop.
  2. + *
  3. core-site.xml: Site-specific configuration for a given hadoop + * installation.
  4. + *
+ * Applications may add additional resources, which are loaded + * subsequent to these resources in the order they are added. + * + *

Final Parameters

+ * + *

Configuration parameters may be declared final. + * Once a resource declares a value final, no subsequently-loaded + * resource can alter that value. + * For example, one might define a final parameter with: + *

+ *  <property>
+ *    <name>dfs.hosts.include</name>
+ *    <value>/etc/hadoop/conf/hosts.include</value>
+ *    <final>true</final>
+ *  </property>
+ * + * Administrators typically define parameters as final in + * core-site.xml for values that user applications may not alter. + * + *

Variable Expansion

+ * + *

Value strings are first processed for variable expansion. The + * available properties are:

    + *
  1. Other properties defined in this Configuration; and, if a name is + * undefined here,
  2. + *
  3. Properties in {@link System#getProperties()}.
  4. + *
+ * + *

For example, if a configuration resource contains the following property + * definitions: + *

+ *  <property>
+ *    <name>basedir</name>
+ *    <value>/user/${user.name}</value>
+ *  </property>
+ *  
+ *  <property>
+ *    <name>tempdir</name>
+ *    <value>${basedir}/tmp</value>
+ *  </property>
+ * + * When conf.get("tempdir") is called, then ${basedir} + * will be resolved to another property in this Configuration, while + * ${user.name} would then ordinarily be resolved to the value + * of the System property with that name. + *

When conf.get("otherdir") is called, then ${env.BASE_DIR} + * will be resolved to the value of the ${BASE_DIR} environment variable. + * It supports ${env.NAME:-default} and ${env.NAME-default} notations. + * The former is resolved to "default" if ${NAME} environment variable is undefined + * or its value is empty. + * The latter behaves the same way only if ${NAME} is undefined. + *

By default, warnings will be given to any deprecated configuration + * parameters and these are suppressible by configuring + * log4j.logger.org.apache.hadoop.conf.Configuration.deprecation in + * log4j.properties file. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class Configuration implements Iterable>, + Writable { + private static final Log LOG = + LogFactory.getLog(Configuration.class); + + private static final Log LOG_DEPRECATION = + LogFactory.getLog("org.apache.hadoop.conf.Configuration.deprecation"); + + private boolean quietmode = true; + + private static final String DEFAULT_STRING_CHECK = + "testingforemptydefaultvalue"; + + private boolean allowNullValueProperties = false; + + private static class Resource { + private final Object resource; + private final String name; + + public Resource(Object resource) { + this(resource, resource.toString()); + } + + public Resource(Object resource, String name) { + this.resource = resource; + this.name = name; + } + + public String getName(){ + return name; + } + + public Object getResource() { + return resource; + } + + @Override + public String toString() { + return name; + } + } + + /** + * List of configuration resources. + */ + private ArrayList resources = new ArrayList(); + + /** + * The value reported as the setting resource when a key is set + * by code rather than a file resource by dumpConfiguration. + */ + static final String UNKNOWN_RESOURCE = "Unknown"; + + + /** + * List of configuration parameters marked final. + */ + private Set finalParameters = Collections.newSetFromMap( + new ConcurrentHashMap()); + + private boolean loadDefaults = true; + + /** + * Configuration objects + */ + private static final WeakHashMap REGISTRY = + new WeakHashMap(); + + /** + * List of default Resources. Resources are loaded in the order of the list + * entries + */ + private static final CopyOnWriteArrayList defaultResources = + new CopyOnWriteArrayList(); + + private static final Map>>> + CACHE_CLASSES = new WeakHashMap>>>(); + + /** + * Sentinel value to store negative cache results in {@link #CACHE_CLASSES}. + */ + private static final Class NEGATIVE_CACHE_SENTINEL = + NegativeCacheSentinel.class; + + /** + * Stores the mapping of key to the resource which modifies or loads + * the key most recently + */ + private Map updatingResource; + + /** + * Class to keep the information about the keys which replace the deprecated + * ones. + * + * This class stores the new keys which replace the deprecated keys and also + * gives a provision to have a custom message for each of the deprecated key + * that is being replaced. It also provides method to get the appropriate + * warning message which can be logged whenever the deprecated key is used. + */ + private static class DeprecatedKeyInfo { + private final String[] newKeys; + private final String customMessage; + private final AtomicBoolean accessed = new AtomicBoolean(false); + + DeprecatedKeyInfo(String[] newKeys, String customMessage) { + this.newKeys = newKeys; + this.customMessage = customMessage; + } + + /** + * Method to provide the warning message. It gives the custom message if + * non-null, and default message otherwise. + * @param key the associated deprecated key. + * @return message that is to be logged when a deprecated key is used. + */ + private final String getWarningMessage(String key) { + String warningMessage; + if(customMessage == null) { + StringBuilder message = new StringBuilder(key); + String deprecatedKeySuffix = " is deprecated. Instead, use "; + message.append(deprecatedKeySuffix); + for (int i = 0; i < newKeys.length; i++) { + message.append(newKeys[i]); + if(i != newKeys.length-1) { + message.append(", "); + } + } + warningMessage = message.toString(); + } + else { + warningMessage = customMessage; + } + return warningMessage; + } + + boolean getAndSetAccessed() { + return accessed.getAndSet(true); + } + + public void clearAccessed() { + accessed.set(false); + } + } + + /** + * A pending addition to the global set of deprecated keys. + */ + public static class DeprecationDelta { + private final String key; + private final String[] newKeys; + private final String customMessage; + + DeprecationDelta(String key, String[] newKeys, String customMessage) { + Preconditions.checkNotNull(key); + Preconditions.checkNotNull(newKeys); + Preconditions.checkArgument(newKeys.length > 0); + this.key = key; + this.newKeys = newKeys; + this.customMessage = customMessage; + } + + public DeprecationDelta(String key, String newKey, String customMessage) { + this(key, new String[] { newKey }, customMessage); + } + + public DeprecationDelta(String key, String newKey) { + this(key, new String[] { newKey }, null); + } + + public String getKey() { + return key; + } + + public String[] getNewKeys() { + return newKeys; + } + + public String getCustomMessage() { + return customMessage; + } + } + + /** + * The set of all keys which are deprecated. + * + * DeprecationContext objects are immutable. + */ + private static class DeprecationContext { + /** + * Stores the deprecated keys, the new keys which replace the deprecated keys + * and custom message(if any provided). + */ + private final Map deprecatedKeyMap; + + /** + * Stores a mapping from superseding keys to the keys which they deprecate. + */ + private final Map reverseDeprecatedKeyMap; + + /** + * Create a new DeprecationContext by copying a previous DeprecationContext + * and adding some deltas. + * + * @param other The previous deprecation context to copy, or null to start + * from nothing. + * @param deltas The deltas to apply. + */ + @SuppressWarnings("unchecked") + DeprecationContext(DeprecationContext other, DeprecationDelta[] deltas) { + HashMap newDeprecatedKeyMap = + new HashMap(); + HashMap newReverseDeprecatedKeyMap = + new HashMap(); + if (other != null) { + for (Entry entry : + other.deprecatedKeyMap.entrySet()) { + newDeprecatedKeyMap.put(entry.getKey(), entry.getValue()); + } + for (Entry entry : + other.reverseDeprecatedKeyMap.entrySet()) { + newReverseDeprecatedKeyMap.put(entry.getKey(), entry.getValue()); + } + } + for (DeprecationDelta delta : deltas) { + if (!newDeprecatedKeyMap.containsKey(delta.getKey())) { + DeprecatedKeyInfo newKeyInfo = + new DeprecatedKeyInfo(delta.getNewKeys(), delta.getCustomMessage()); + newDeprecatedKeyMap.put(delta.key, newKeyInfo); + for (String newKey : delta.getNewKeys()) { + newReverseDeprecatedKeyMap.put(newKey, delta.key); + } + } + } + this.deprecatedKeyMap = + UnmodifiableMap.decorate(newDeprecatedKeyMap); + this.reverseDeprecatedKeyMap = + UnmodifiableMap.decorate(newReverseDeprecatedKeyMap); + } + + Map getDeprecatedKeyMap() { + return deprecatedKeyMap; + } + + Map getReverseDeprecatedKeyMap() { + return reverseDeprecatedKeyMap; + } + } + + private static DeprecationDelta[] defaultDeprecations = + new DeprecationDelta[] { + new DeprecationDelta("topology.script.file.name", + CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY), + new DeprecationDelta("topology.script.number.args", + CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY), + new DeprecationDelta("hadoop.configured.node.mapping", + CommonConfigurationKeys.NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY), + new DeprecationDelta("topology.node.switch.mapping.impl", + CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY), + new DeprecationDelta("dfs.df.interval", + CommonConfigurationKeys.FS_DF_INTERVAL_KEY), + new DeprecationDelta("hadoop.native.lib", + CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY), + new DeprecationDelta("fs.default.name", + CommonConfigurationKeys.FS_DEFAULT_NAME_KEY), + new DeprecationDelta("dfs.umaskmode", + CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY), + new DeprecationDelta("dfs.nfs.exports.allowed.hosts", + CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY) + }; + + /** + * The global DeprecationContext. + */ + private static AtomicReference deprecationContext = + new AtomicReference( + new DeprecationContext(null, defaultDeprecations)); + + /** + * Adds a set of deprecated keys to the global deprecations. + * + * This method is lockless. It works by means of creating a new + * DeprecationContext based on the old one, and then atomically swapping in + * the new context. If someone else updated the context in between us reading + * the old context and swapping in the new one, we try again until we win the + * race. + * + * @param deltas The deprecations to add. + */ + public static void addDeprecations(DeprecationDelta[] deltas) { + DeprecationContext prev, next; + do { + prev = deprecationContext.get(); + next = new DeprecationContext(prev, deltas); + } while (!deprecationContext.compareAndSet(prev, next)); + } + + /** + * Adds the deprecated key to the global deprecation map. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If a key is deprecated in favor of multiple keys, they are all treated as + * aliases of each other, and setting any one of them resets all the others + * to the new value. + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key + * @param newKeys + * @param customMessage + * @deprecated use {@link #addDeprecation(String key, String newKey, + String customMessage)} instead + */ + @Deprecated + public static void addDeprecation(String key, String[] newKeys, + String customMessage) { + addDeprecations(new DeprecationDelta[] { + new DeprecationDelta(key, newKeys, customMessage) + }); + } + + /** + * Adds the deprecated key to the global deprecation map. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key + * @param newKey + * @param customMessage + */ + public static void addDeprecation(String key, String newKey, + String customMessage) { + addDeprecation(key, new String[] {newKey}, customMessage); + } + + /** + * Adds the deprecated key to the global deprecation map when no custom + * message is provided. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If a key is deprecated in favor of multiple keys, they are all treated as + * aliases of each other, and setting any one of them resets all the others + * to the new value. + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key Key that is to be deprecated + * @param newKeys list of keys that take up the values of deprecated key + * @deprecated use {@link #addDeprecation(String key, String newKey)} instead + */ + @Deprecated + public static void addDeprecation(String key, String[] newKeys) { + addDeprecation(key, newKeys, null); + } + + /** + * Adds the deprecated key to the global deprecation map when no custom + * message is provided. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key Key that is to be deprecated + * @param newKey key that takes up the value of deprecated key + */ + public static void addDeprecation(String key, String newKey) { + addDeprecation(key, new String[] {newKey}, null); + } + + /** + * checks whether the given key is deprecated. + * + * @param key the parameter which is to be checked for deprecation + * @return true if the key is deprecated and + * false otherwise. + */ + public static boolean isDeprecated(String key) { + return deprecationContext.get().getDeprecatedKeyMap().containsKey(key); + } + + /** + * Sets all deprecated properties that are not currently set but have a + * corresponding new property that is set. Useful for iterating the + * properties when all deprecated properties for currently set properties + * need to be present. + */ + public void setDeprecatedProperties() { + DeprecationContext deprecations = deprecationContext.get(); + Properties props = getProps(); + Properties overlay = getOverlay(); + for (Map.Entry entry : + deprecations.getDeprecatedKeyMap().entrySet()) { + String depKey = entry.getKey(); + if (!overlay.contains(depKey)) { + for (String newKey : entry.getValue().newKeys) { + String val = overlay.getProperty(newKey); + if (val != null) { + props.setProperty(depKey, val); + overlay.setProperty(depKey, val); + break; + } + } + } + } + } + + /** + * Checks for the presence of the property name in the + * deprecation map. Returns the first of the list of new keys if present + * in the deprecation map or the name itself. If the property + * is not presently set but the property map contains an entry for the + * deprecated key, the value of the deprecated key is set as the value for + * the provided property name. + * + * @param name the property name + * @return the first property in the list of properties mapping + * the name or the name itself. + */ + private String[] handleDeprecation(DeprecationContext deprecations, + String name) { + if (null != name) { + name = name.trim(); + } + ArrayList names = new ArrayList(); + if (isDeprecated(name)) { + DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name); + warnOnceIfDeprecated(deprecations, name); + for (String newKey : keyInfo.newKeys) { + if(newKey != null) { + names.add(newKey); + } + } + } + if(names.size() == 0) { + names.add(name); + } + for(String n : names) { + String deprecatedKey = deprecations.getReverseDeprecatedKeyMap().get(n); + if (deprecatedKey != null && !getOverlay().containsKey(n) && + getOverlay().containsKey(deprecatedKey)) { + getProps().setProperty(n, getOverlay().getProperty(deprecatedKey)); + getOverlay().setProperty(n, getOverlay().getProperty(deprecatedKey)); + } + } + return names.toArray(new String[names.size()]); + } + + private void handleDeprecation() { + LOG.debug("Handling deprecation for all properties in config..."); + DeprecationContext deprecations = deprecationContext.get(); + Set keys = new HashSet(); + keys.addAll(getProps().keySet()); + for (Object item: keys) { + LOG.debug("Handling deprecation for " + (String)item); + handleDeprecation(deprecations, (String)item); + } + } + + static{ + //print deprecation warning if hadoop-site.xml is found in classpath + ClassLoader cL = Thread.currentThread().getContextClassLoader(); + if (cL == null) { + cL = Configuration.class.getClassLoader(); + } + if(cL.getResource("hadoop-site.xml")!=null) { + LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " + + "Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, " + + "mapred-site.xml and hdfs-site.xml to override properties of " + + "core-default-shaded.xml, mapred-default.xml and hdfs-default.xml " + + "respectively"); + } + addDefaultResource("core-default-shaded.xml"); + addDefaultResource("core-site.xml"); + } + + private Properties properties; + private Properties overlay; + private ClassLoader classLoader; + { + classLoader = Thread.currentThread().getContextClassLoader(); + if (classLoader == null) { + classLoader = Configuration.class.getClassLoader(); + } + } + + /** A new configuration. */ + public Configuration() { + this(true); + } + + /** A new configuration where the behavior of reading from the default + * resources can be turned off. + * + * If the parameter {@code loadDefaults} is false, the new instance + * will not load resources from the default files. + * @param loadDefaults specifies whether to load from the default files + */ + public Configuration(boolean loadDefaults) { + this.loadDefaults = loadDefaults; + updatingResource = new ConcurrentHashMap(); + synchronized(Configuration.class) { + REGISTRY.put(this, null); + } + } + + /** + * A new configuration with the same settings cloned from another. + * + * @param other the configuration from which to clone settings. + */ + @SuppressWarnings("unchecked") + public Configuration(Configuration other) { + this.resources = (ArrayList) other.resources.clone(); + synchronized(other) { + if (other.properties != null) { + this.properties = (Properties)other.properties.clone(); + } + + if (other.overlay!=null) { + this.overlay = (Properties)other.overlay.clone(); + } + + this.updatingResource = new ConcurrentHashMap( + other.updatingResource); + this.finalParameters = Collections.newSetFromMap( + new ConcurrentHashMap()); + this.finalParameters.addAll(other.finalParameters); + } + + synchronized(Configuration.class) { + REGISTRY.put(this, null); + } + this.classLoader = other.classLoader; + this.loadDefaults = other.loadDefaults; + setQuietMode(other.getQuietMode()); + } + + /** + * Add a default resource. Resources are loaded in the order of the resources + * added. + * @param name file name. File should be present in the classpath. + */ + public static synchronized void addDefaultResource(String name) { + if(!defaultResources.contains(name)) { + defaultResources.add(name); + for(Configuration conf : REGISTRY.keySet()) { + if(conf.loadDefaults) { + conf.reloadConfiguration(); + } + } + } + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param name resource to be added, the classpath is examined for a file + * with that name. + */ + public void addResource(String name) { + addResourceObject(new Resource(name)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param url url of the resource to be added, the local filesystem is + * examined directly to find the resource, without referring to + * the classpath. + */ + public void addResource(URL url) { + addResourceObject(new Resource(url)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param file file-path of resource to be added, the local filesystem is + * examined directly to find the resource, without referring to + * the classpath. + */ + public void addResource(Path file) { + addResourceObject(new Resource(file)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * WARNING: The contents of the InputStream will be cached, by this method. + * So use this sparingly because it does increase the memory consumption. + * + * @param in InputStream to deserialize the object from. In will be read from + * when a get or set is called next. After it is read the stream will be + * closed. + */ + public void addResource(InputStream in) { + addResourceObject(new Resource(in)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param in InputStream to deserialize the object from. + * @param name the name of the resource because InputStream.toString is not + * very descriptive some times. + */ + public void addResource(InputStream in, String name) { + addResourceObject(new Resource(in, name)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param conf Configuration object from which to load properties + */ + public void addResource(Configuration conf) { + addResourceObject(new Resource(conf.getProps())); + } + + + + /** + * Reload configuration from previously added resources. + * + * This method will clear all the configuration read from the added + * resources, and final parameters. This will make the resources to + * be read again before accessing the values. Values that are added + * via set methods will overlay values read from the resources. + */ + public synchronized void reloadConfiguration() { + properties = null; // trigger reload + finalParameters.clear(); // clear site-limits + } + + private synchronized void addResourceObject(Resource resource) { + resources.add(resource); // add to resources + reloadConfiguration(); + } + + private static final int MAX_SUBST = 20; + + private static final int SUB_START_IDX = 0; + private static final int SUB_END_IDX = SUB_START_IDX + 1; + + /** + * This is a manual implementation of the following regex + * "\\$\\{[^\\}\\$\u0020]+\\}". It can be 15x more efficient than + * a regex matcher as demonstrated by HADOOP-11506. This is noticeable with + * Hadoop apps building on the assumption Configuration#get is an O(1) + * hash table lookup, especially when the eval is a long string. + * + * @param eval a string that may contain variables requiring expansion. + * @return a 2-element int array res such that + * eval.substring(res[0], res[1]) is "var" for the left-most occurrence of + * ${var} in eval. If no variable is found -1, -1 is returned. + */ + private static int[] findSubVariable(String eval) { + int[] result = {-1, -1}; + + int matchStart; + int leftBrace; + + // scanning for a brace first because it's less frequent than $ + // that can occur in nested class names + // + match_loop: + for (matchStart = 1, leftBrace = eval.indexOf('{', matchStart); + // minimum left brace position (follows '$') + leftBrace > 0 + // right brace of a smallest valid expression "${c}" + && leftBrace + "{c".length() < eval.length(); + leftBrace = eval.indexOf('{', matchStart)) { + int matchedLen = 0; + if (eval.charAt(leftBrace - 1) == '$') { + int subStart = leftBrace + 1; // after '{' + for (int i = subStart; i < eval.length(); i++) { + switch (eval.charAt(i)) { + case '}': + if (matchedLen > 0) { // match + result[SUB_START_IDX] = subStart; + result[SUB_END_IDX] = subStart + matchedLen; + break match_loop; + } + // fall through to skip 1 char + case ' ': + case '$': + matchStart = i + 1; + continue match_loop; + default: + matchedLen++; + } + } + // scanned from "${" to the end of eval, and no reset via ' ', '$': + // no match! + break match_loop; + } else { + // not a start of a variable + // + matchStart = leftBrace + 1; + } + } + return result; + } + + /** + * Attempts to repeatedly expand the value {@code expr} by replacing the + * left-most substring of the form "${var}" in the following precedence order + *
    + *
  1. by the value of the Java system property "var" if defined
  2. + *
  3. by the value of the configuration key "var" if defined
  4. + *
+ * + * If var is unbounded the current state of expansion "prefix${var}suffix" is + * returned. + * + * @param expr the literal value of a config key + * @return null if expr is null, otherwise the value resulting from expanding + * expr using the algorithm above. + * @throws IllegalArgumentException when more than + * {@link Configuration#MAX_SUBST} replacements are required + */ + private String substituteVars(String expr) { + if (expr == null) { + return null; + } + String eval = expr; + for (int s = 0; s < MAX_SUBST; s++) { + final int[] varBounds = findSubVariable(eval); + if (varBounds[SUB_START_IDX] == -1) { + return eval; + } + final String var = eval.substring(varBounds[SUB_START_IDX], + varBounds[SUB_END_IDX]); + String val = null; + try { + val = System.getProperty(var); + } catch(SecurityException se) { + LOG.warn("Unexpected SecurityException in Configuration", se); + } + if (val == null) { + val = getRaw(var); + } + if (val == null) { + return eval; // return literal ${var}: var is unbound + } + final int dollar = varBounds[SUB_START_IDX] - "${".length(); + final int afterRightBrace = varBounds[SUB_END_IDX] + "}".length(); + // substitute + eval = eval.substring(0, dollar) + + val + + eval.substring(afterRightBrace); + } + throw new IllegalStateException("Variable substitution depth too large: " + + MAX_SUBST + " " + expr); + } + + /** + * Get the value of the name property, null if + * no such property exists. If the key is deprecated, it returns the value of + * the first key which replaces the deprecated key and is not null. + * + * Values are processed for variable expansion + * before being returned. + * + * @param name the property name, will be trimmed before get value. + * @return the value of the name or its replacing property, + * or null if no such property exists. + */ + public String get(String name) { + String[] names = handleDeprecation(deprecationContext.get(), name); + String result = null; + for(String n : names) { + result = substituteVars(getProps().getProperty(n)); + } + return result; + } + + /** + * Set Configuration to allow keys without values during setup. Intended + * for use during testing. + * + * @param val If true, will allow Configuration to store keys without values + */ + @VisibleForTesting + public void setAllowNullValueProperties( boolean val ) { + this.allowNullValueProperties = val; + } + + /** + * Return existence of the name property, but only for + * names which have no valid value, usually non-existent or commented + * out in XML. + * + * @param name the property name + * @return true if the property name exists without value + */ + @VisibleForTesting + public boolean onlyKeyExists(String name) { + String[] names = handleDeprecation(deprecationContext.get(), name); + for(String n : names) { + if ( getProps().getProperty(n,DEFAULT_STRING_CHECK) + .equals(DEFAULT_STRING_CHECK) ) { + return true; + } + } + return false; + } + + /** + * Get the value of the name property as a trimmed String, + * null if no such property exists. + * If the key is deprecated, it returns the value of + * the first key which replaces the deprecated key and is not null + * + * Values are processed for variable expansion + * before being returned. + * + * @param name the property name. + * @return the value of the name or its replacing property, + * or null if no such property exists. + */ + public String getTrimmed(String name) { + String value = get(name); + + if (null == value) { + return null; + } else { + return value.trim(); + } + } + + /** + * Get the value of the name property as a trimmed String, + * defaultValue if no such property exists. + * See @{Configuration#getTrimmed} for more details. + * + * @param name the property name. + * @param defaultValue the property default value. + * @return the value of the name or defaultValue + * if it is not set. + */ + public String getTrimmed(String name, String defaultValue) { + String ret = getTrimmed(name); + return ret == null ? defaultValue : ret; + } + + /** + * Get the value of the name property, without doing + * variable expansion.If the key is + * deprecated, it returns the value of the first key which replaces + * the deprecated key and is not null. + * + * @param name the property name. + * @return the value of the name property or + * its replacing property and null if no such property exists. + */ + public String getRaw(String name) { + String[] names = handleDeprecation(deprecationContext.get(), name); + String result = null; + for(String n : names) { + result = getProps().getProperty(n); + } + return result; + } + + /** + * Returns alternative names (non-deprecated keys or previously-set deprecated keys) + * for a given non-deprecated key. + * If the given key is deprecated, return null. + * + * @param name property name. + * @return alternative names. + */ + private String[] getAlternativeNames(String name) { + String altNames[] = null; + DeprecatedKeyInfo keyInfo = null; + DeprecationContext cur = deprecationContext.get(); + String depKey = cur.getReverseDeprecatedKeyMap().get(name); + if(depKey != null) { + keyInfo = cur.getDeprecatedKeyMap().get(depKey); + if(keyInfo.newKeys.length > 0) { + if(getProps().containsKey(depKey)) { + //if deprecated key is previously set explicitly + List list = new ArrayList(); + list.addAll(Arrays.asList(keyInfo.newKeys)); + list.add(depKey); + altNames = list.toArray(new String[list.size()]); + } + else { + altNames = keyInfo.newKeys; + } + } + } + return altNames; + } + + /** + * Set the value of the name property. If + * name is deprecated or there is a deprecated name associated to it, + * it sets the value to both names. Name will be trimmed before put into + * configuration. + * + * @param name property name. + * @param value property value. + */ + public void set(String name, String value) { + set(name, value, null); + } + + /** + * Set the value of the name property. If + * name is deprecated, it also sets the value to + * the keys that replace the deprecated key. Name will be trimmed before put + * into configuration. + * + * @param name property name. + * @param value property value. + * @param source the place that this configuration value came from + * (For debugging). + * @throws IllegalArgumentException when the value or name is null. + */ + public void set(String name, String value, String source) { + Preconditions.checkArgument( + name != null, + "Property name must not be null"); + Preconditions.checkArgument( + value != null, + "The value of property " + name + " must not be null"); + name = name.trim(); + DeprecationContext deprecations = deprecationContext.get(); + if (deprecations.getDeprecatedKeyMap().isEmpty()) { + getProps(); + } + getOverlay().setProperty(name, value); + getProps().setProperty(name, value); + String newSource = (source == null ? "programatically" : source); + + if (!isDeprecated(name)) { + updatingResource.put(name, new String[] {newSource}); + String[] altNames = getAlternativeNames(name); + if(altNames != null) { + for(String n: altNames) { + if(!n.equals(name)) { + getOverlay().setProperty(n, value); + getProps().setProperty(n, value); + updatingResource.put(n, new String[] {newSource}); + } + } + } + } + else { + String[] names = handleDeprecation(deprecationContext.get(), name); + String altSource = "because " + name + " is deprecated"; + for(String n : names) { + getOverlay().setProperty(n, value); + getProps().setProperty(n, value); + updatingResource.put(n, new String[] {altSource}); + } + } + } + + private void warnOnceIfDeprecated(DeprecationContext deprecations, String name) { + DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name); + if (keyInfo != null && !keyInfo.getAndSetAccessed()) { + LOG_DEPRECATION.info(keyInfo.getWarningMessage(name)); + } + } + + /** + * Unset a previously set property. + */ + public synchronized void unset(String name) { + String[] names = null; + if (!isDeprecated(name)) { + names = getAlternativeNames(name); + if(names == null) { + names = new String[]{name}; + } + } + else { + names = handleDeprecation(deprecationContext.get(), name); + } + + for(String n: names) { + getOverlay().remove(n); + getProps().remove(n); + } + } + + /** + * Sets a property if it is currently unset. + * @param name the property name + * @param value the new value + */ + public synchronized void setIfUnset(String name, String value) { + if (get(name) == null) { + set(name, value); + } + } + + private synchronized Properties getOverlay() { + if (overlay==null){ + overlay=new Properties(); + } + return overlay; + } + + /** + * Get the value of the name. If the key is deprecated, + * it returns the value of the first key which replaces the deprecated key + * and is not null. + * If no such property exists, + * then defaultValue is returned. + * + * @param name property name, will be trimmed before get value. + * @param defaultValue default value. + * @return property value, or defaultValue if the property + * doesn't exist. + */ + public String get(String name, String defaultValue) { + String[] names = handleDeprecation(deprecationContext.get(), name); + String result = null; + for(String n : names) { + result = substituteVars(getProps().getProperty(n, defaultValue)); + } + return result; + } + + /** + * Get the value of the name property as an int. + * + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid int, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as an int, + * or defaultValue. + */ + public int getInt(String name, int defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + String hexString = getHexDigits(valueString); + if (hexString != null) { + return Integer.parseInt(hexString, 16); + } + return Integer.parseInt(valueString); + } + + /** + * Get the value of the name property as a set of comma-delimited + * int values. + * + * If no such property exists, an empty array is returned. + * + * @param name property name + * @return property value interpreted as an array of comma-delimited + * int values + */ + public int[] getInts(String name) { + String[] strings = getTrimmedStrings(name); + int[] ints = new int[strings.length]; + for (int i = 0; i < strings.length; i++) { + ints[i] = Integer.parseInt(strings[i]); + } + return ints; + } + + /** + * Set the value of the name property to an int. + * + * @param name property name. + * @param value int value of the property. + */ + public void setInt(String name, int value) { + set(name, Integer.toString(value)); + } + + + /** + * Get the value of the name property as a long. + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid long, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a long, + * or defaultValue. + */ + public long getLong(String name, long defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + String hexString = getHexDigits(valueString); + if (hexString != null) { + return Long.parseLong(hexString, 16); + } + return Long.parseLong(valueString); + } + + /** + * Get the value of the name property as a long or + * human readable format. If no such property exists, the provided default + * value is returned, or if the specified value is not a valid + * long or human readable format, then an error is thrown. You + * can use the following suffix (case insensitive): k(kilo), m(mega), g(giga), + * t(tera), p(peta), e(exa) + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a long, + * or defaultValue. + */ + public long getLongBytes(String name, long defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + return StringUtils.TraditionalBinaryPrefix.string2long(valueString); + } + + private String getHexDigits(String value) { + boolean negative = false; + String str = value; + String hexString = null; + if (value.startsWith("-")) { + negative = true; + str = value.substring(1); + } + if (str.startsWith("0x") || str.startsWith("0X")) { + hexString = str.substring(2); + if (negative) { + hexString = "-" + hexString; + } + return hexString; + } + return null; + } + + /** + * Set the value of the name property to a long. + * + * @param name property name. + * @param value long value of the property. + */ + public void setLong(String name, long value) { + set(name, Long.toString(value)); + } + + /** + * Get the value of the name property as a float. + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid float, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a float, + * or defaultValue. + */ + public float getFloat(String name, float defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + return Float.parseFloat(valueString); + } + + /** + * Set the value of the name property to a float. + * + * @param name property name. + * @param value property value. + */ + public void setFloat(String name, float value) { + set(name,Float.toString(value)); + } + + /** + * Get the value of the name property as a double. + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid double, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a double, + * or defaultValue. + */ + public double getDouble(String name, double defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + return Double.parseDouble(valueString); + } + + /** + * Set the value of the name property to a double. + * + * @param name property name. + * @param value property value. + */ + public void setDouble(String name, double value) { + set(name,Double.toString(value)); + } + + /** + * Get the value of the name property as a boolean. + * If no such property is specified, or if the specified value is not a valid + * boolean, then defaultValue is returned. + * + * @param name property name. + * @param defaultValue default value. + * @return property value as a boolean, + * or defaultValue. + */ + public boolean getBoolean(String name, boolean defaultValue) { + String valueString = getTrimmed(name); + if (null == valueString || valueString.isEmpty()) { + return defaultValue; + } + + if (StringUtils.equalsIgnoreCase("true", valueString)) + return true; + else if (StringUtils.equalsIgnoreCase("false", valueString)) + return false; + else return defaultValue; + } + + /** + * Set the value of the name property to a boolean. + * + * @param name property name. + * @param value boolean value of the property. + */ + public void setBoolean(String name, boolean value) { + set(name, Boolean.toString(value)); + } + + /** + * Set the given property, if it is currently unset. + * @param name property name + * @param value new value + */ + public void setBooleanIfUnset(String name, boolean value) { + setIfUnset(name, Boolean.toString(value)); + } + + /** + * Set the value of the name property to the given type. This + * is equivalent to set(<name>, value.toString()). + * @param name property name + * @param value new value + */ + public > void setEnum(String name, T value) { + set(name, value.toString()); + } + + /** + * Return value matching this enumerated type. + * Note that the returned value is trimmed by this method. + * @param name Property name + * @param defaultValue Value returned if no mapping exists + * @throws IllegalArgumentException If mapping is illegal for the type + * provided + */ + public > T getEnum(String name, T defaultValue) { + final String val = getTrimmed(name); + return null == val + ? defaultValue + : Enum.valueOf(defaultValue.getDeclaringClass(), val); + } + + enum ParsedTimeDuration { + NS { + TimeUnit unit() { return TimeUnit.NANOSECONDS; } + String suffix() { return "ns"; } + }, + US { + TimeUnit unit() { return TimeUnit.MICROSECONDS; } + String suffix() { return "us"; } + }, + MS { + TimeUnit unit() { return TimeUnit.MILLISECONDS; } + String suffix() { return "ms"; } + }, + S { + TimeUnit unit() { return TimeUnit.SECONDS; } + String suffix() { return "s"; } + }, + M { + TimeUnit unit() { return TimeUnit.MINUTES; } + String suffix() { return "m"; } + }, + H { + TimeUnit unit() { return TimeUnit.HOURS; } + String suffix() { return "h"; } + }, + D { + TimeUnit unit() { return TimeUnit.DAYS; } + String suffix() { return "d"; } + }; + abstract TimeUnit unit(); + abstract String suffix(); + static ParsedTimeDuration unitFor(String s) { + for (ParsedTimeDuration ptd : values()) { + // iteration order is in decl order, so SECONDS matched last + if (s.endsWith(ptd.suffix())) { + return ptd; + } + } + return null; + } + static ParsedTimeDuration unitFor(TimeUnit unit) { + for (ParsedTimeDuration ptd : values()) { + if (ptd.unit() == unit) { + return ptd; + } + } + return null; + } + } + + /** + * Set the value of name to the given time duration. This + * is equivalent to set(<name>, value + <time suffix>). + * @param name Property name + * @param value Time duration + * @param unit Unit of time + */ + public void setTimeDuration(String name, long value, TimeUnit unit) { + set(name, value + ParsedTimeDuration.unitFor(unit).suffix()); + } + + /** + * Return time duration in the given time unit. Valid units are encoded in + * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds + * (ms), seconds (s), minutes (m), hours (h), and days (d). + * @param name Property name + * @param defaultValue Value returned if no mapping exists. + * @param unit Unit to convert the stored property, if it exists. + * @throws NumberFormatException If the property stripped of its unit is not + * a number + */ + public long getTimeDuration(String name, long defaultValue, TimeUnit unit) { + String vStr = get(name); + if (null == vStr) { + return defaultValue; + } + vStr = vStr.trim(); + return getTimeDurationHelper(name, vStr, unit); + } + + private long getTimeDurationHelper(String name, String vStr, TimeUnit unit) { + ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr); + if (null == vUnit) { + LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit); + vUnit = ParsedTimeDuration.unitFor(unit); + } else { + vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix())); + } + return unit.convert(Long.parseLong(vStr), vUnit.unit()); + } + + public long[] getTimeDurations(String name, TimeUnit unit) { + String[] strings = getTrimmedStrings(name); + long[] durations = new long[strings.length]; + for (int i = 0; i < strings.length; i++) { + durations[i] = getTimeDurationHelper(name, strings[i], unit); + } + return durations; + } + + /** + * Get the value of the name property as a Pattern. + * If no such property is specified, or if the specified value is not a valid + * Pattern, then DefaultValue is returned. + * Note that the returned value is NOT trimmed by this method. + * + * @param name property name + * @param defaultValue default value + * @return property value as a compiled Pattern, or defaultValue + */ + public Pattern getPattern(String name, Pattern defaultValue) { + String valString = get(name); + if (null == valString || valString.isEmpty()) { + return defaultValue; + } + try { + return Pattern.compile(valString); + } catch (PatternSyntaxException pse) { + LOG.warn("Regular expression '" + valString + "' for property '" + + name + "' not valid. Using default", pse); + return defaultValue; + } + } + + /** + * Set the given property to Pattern. + * If the pattern is passed as null, sets the empty pattern which results in + * further calls to getPattern(...) returning the default value. + * + * @param name property name + * @param pattern new value + */ + public void setPattern(String name, Pattern pattern) { + assert pattern != null : "Pattern cannot be null"; + set(name, pattern.pattern()); + } + + /** + * Gets information about why a property was set. Typically this is the + * path to the resource objects (file, URL, etc.) the property came from, but + * it can also indicate that it was set programatically, or because of the + * command line. + * + * @param name - The property name to get the source of. + * @return null - If the property or its source wasn't found. Otherwise, + * returns a list of the sources of the resource. The older sources are + * the first ones in the list. So for example if a configuration is set from + * the command line, and then written out to a file that is read back in the + * first entry would indicate that it was set from the command line, while + * the second one would indicate the file that the new configuration was read + * in from. + */ + @InterfaceStability.Unstable + public synchronized String[] getPropertySources(String name) { + if (properties == null) { + // If properties is null, it means a resource was newly added + // but the props were cleared so as to load it upon future + // requests. So lets force a load by asking a properties list. + getProps(); + } + // Return a null right away if our properties still + // haven't loaded or the resource mapping isn't defined + if (properties == null || updatingResource == null) { + return null; + } else { + String[] source = updatingResource.get(name); + if(source == null) { + return null; + } else { + return Arrays.copyOf(source, source.length); + } + } + } + + /** + * A class that represents a set of positive integer ranges. It parses + * strings of the form: "2-3,5,7-" where ranges are separated by comma and + * the lower/upper bounds are separated by dash. Either the lower or upper + * bound may be omitted meaning all values up to or over. So the string + * above means 2, 3, 5, and 7, 8, 9, ... + */ + public static class IntegerRanges implements Iterable{ + private static class Range { + int start; + int end; + } + + private static class RangeNumberIterator implements Iterator { + Iterator internal; + int at; + int end; + + public RangeNumberIterator(List ranges) { + if (ranges != null) { + internal = ranges.iterator(); + } + at = -1; + end = -2; + } + + @Override + public boolean hasNext() { + if (at <= end) { + return true; + } else if (internal != null){ + return internal.hasNext(); + } + return false; + } + + @Override + public Integer next() { + if (at <= end) { + at++; + return at - 1; + } else if (internal != null){ + Range found = internal.next(); + if (found != null) { + at = found.start; + end = found.end; + at++; + return at - 1; + } + } + return null; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + + List ranges = new ArrayList(); + + public IntegerRanges() { + } + + public IntegerRanges(String newValue) { + StringTokenizer itr = new StringTokenizer(newValue, ","); + while (itr.hasMoreTokens()) { + String rng = itr.nextToken().trim(); + String[] parts = rng.split("-", 3); + if (parts.length < 1 || parts.length > 2) { + throw new IllegalArgumentException("integer range badly formed: " + + rng); + } + Range r = new Range(); + r.start = convertToInt(parts[0], 0); + if (parts.length == 2) { + r.end = convertToInt(parts[1], Integer.MAX_VALUE); + } else { + r.end = r.start; + } + if (r.start > r.end) { + throw new IllegalArgumentException("IntegerRange from " + r.start + + " to " + r.end + " is invalid"); + } + ranges.add(r); + } + } + + /** + * Convert a string to an int treating empty strings as the default value. + * @param value the string value + * @param defaultValue the value for if the string is empty + * @return the desired integer + */ + private static int convertToInt(String value, int defaultValue) { + String trim = value.trim(); + if (trim.length() == 0) { + return defaultValue; + } + return Integer.parseInt(trim); + } + + /** + * Is the given value in the set of ranges + * @param value the value to check + * @return is the value in the ranges? + */ + public boolean isIncluded(int value) { + for(Range r: ranges) { + if (r.start <= value && value <= r.end) { + return true; + } + } + return false; + } + + /** + * @return true if there are no values in this range, else false. + */ + public boolean isEmpty() { + return ranges == null || ranges.isEmpty(); + } + + @Override + public String toString() { + StringBuilder result = new StringBuilder(); + boolean first = true; + for(Range r: ranges) { + if (first) { + first = false; + } else { + result.append(','); + } + result.append(r.start); + result.append('-'); + result.append(r.end); + } + return result.toString(); + } + + @Override + public Iterator iterator() { + return new RangeNumberIterator(ranges); + } + + } + + /** + * Parse the given attribute as a set of integer ranges + * @param name the attribute name + * @param defaultValue the default value if it is not set + * @return a new set of ranges from the configured value + */ + public IntegerRanges getRange(String name, String defaultValue) { + return new IntegerRanges(get(name, defaultValue)); + } + + /** + * Get the comma delimited values of the name property as + * a collection of Strings. + * If no such property is specified then empty collection is returned. + *

+ * This is an optimized version of {@link #getStrings(String)} + * + * @param name property name. + * @return property value as a collection of Strings. + */ + public Collection getStringCollection(String name) { + String valueString = get(name); + return StringUtils.getStringCollection(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings. + * If no such property is specified then null is returned. + * + * @param name property name. + * @return property value as an array of Strings, + * or null. + */ + public String[] getStrings(String name) { + String valueString = get(name); + return StringUtils.getStrings(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings. + * If no such property is specified then default value is returned. + * + * @param name property name. + * @param defaultValue The default value + * @return property value as an array of Strings, + * or default value. + */ + public String[] getStrings(String name, String... defaultValue) { + String valueString = get(name); + if (valueString == null) { + return defaultValue; + } else { + return StringUtils.getStrings(valueString); + } + } + + /** + * Get the comma delimited values of the name property as + * a collection of Strings, trimmed of the leading and trailing whitespace. + * If no such property is specified then empty Collection is returned. + * + * @param name property name. + * @return property value as a collection of Strings, or empty Collection + */ + public Collection getTrimmedStringCollection(String name) { + String valueString = get(name); + if (null == valueString) { + Collection empty = new ArrayList(); + return empty; + } + return StringUtils.getTrimmedStringCollection(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings, trimmed of the leading and trailing whitespace. + * If no such property is specified then an empty array is returned. + * + * @param name property name. + * @return property value as an array of trimmed Strings, + * or empty array. + */ + public String[] getTrimmedStrings(String name) { + String valueString = get(name); + return StringUtils.getTrimmedStrings(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings, trimmed of the leading and trailing whitespace. + * If no such property is specified then default value is returned. + * + * @param name property name. + * @param defaultValue The default value + * @return property value as an array of trimmed Strings, + * or default value. + */ + public String[] getTrimmedStrings(String name, String... defaultValue) { + String valueString = get(name); + if (null == valueString) { + return defaultValue; + } else { + return StringUtils.getTrimmedStrings(valueString); + } + } + + /** + * Set the array of string values for the name property as + * as comma delimited values. + * + * @param name property name. + * @param values The values + */ + public void setStrings(String name, String... values) { + set(name, StringUtils.arrayToString(values)); + } + + /** + * Get the value for a known password configuration element. + * In order to enable the elimination of clear text passwords in config, + * this method attempts to resolve the property name as an alias through + * the CredentialProvider API and conditionally fallsback to config. + * @param name property name + * @return password + */ + public char[] getPassword(String name) throws IOException { + char[] pass = null; + + pass = getPasswordFromCredentialProviders(name); + + if (pass == null) { + pass = getPasswordFromConfig(name); + } + + return pass; + } + + /** + * Try and resolve the provided element name as a credential provider + * alias. + * @param name alias of the provisioned credential + * @return password or null if not found + * @throws IOException + */ + protected char[] getPasswordFromCredentialProviders(String name) + throws IOException { + char[] pass = null; + try { + List providers = + CredentialProviderFactory.getProviders(this); + + if (providers != null) { + for (CredentialProvider provider : providers) { + try { + CredentialEntry entry = provider.getCredentialEntry(name); + if (entry != null) { + pass = entry.getCredential(); + break; + } + } + catch (IOException ioe) { + throw new IOException("Can't get key " + name + " from key provider" + + "of type: " + provider.getClass().getName() + ".", ioe); + } + } + } + } + catch (IOException ioe) { + throw new IOException("Configuration problem with provider path.", ioe); + } + + return pass; + } + + /** + * Fallback to clear text passwords in configuration. + * @param name + * @return clear text password or null + */ + protected char[] getPasswordFromConfig(String name) { + char[] pass = null; + if (getBoolean(CredentialProvider.CLEAR_TEXT_FALLBACK, true)) { + String passStr = get(name); + if (passStr != null) { + pass = passStr.toCharArray(); + } + } + return pass; + } + + /** + * Get the socket address for hostProperty as a + * InetSocketAddress. If hostProperty is + * null, addressProperty will be used. This + * is useful for cases where we want to differentiate between host + * bind address and address clients should use to establish connection. + * + * @param hostProperty bind host property name. + * @param addressProperty address property name. + * @param defaultAddressValue the default value + * @param defaultPort the default port + * @return InetSocketAddress + */ + public InetSocketAddress getSocketAddr( + String hostProperty, + String addressProperty, + String defaultAddressValue, + int defaultPort) { + + InetSocketAddress bindAddr = getSocketAddr( + addressProperty, defaultAddressValue, defaultPort); + + final String host = get(hostProperty); + + if (host == null || host.isEmpty()) { + return bindAddr; + } + + return NetUtils.createSocketAddr( + host, bindAddr.getPort(), hostProperty); + } + + /** + * Get the socket address for name property as a + * InetSocketAddress. + * @param name property name. + * @param defaultAddress the default value + * @param defaultPort the default port + * @return InetSocketAddress + */ + public InetSocketAddress getSocketAddr( + String name, String defaultAddress, int defaultPort) { + final String address = getTrimmed(name, defaultAddress); + return NetUtils.createSocketAddr(address, defaultPort, name); + } + + /** + * Set the socket address for the name property as + * a host:port. + */ + public void setSocketAddr(String name, InetSocketAddress addr) { + set(name, NetUtils.getHostPortString(addr)); + } + + /** + * Set the socket address a client can use to connect for the + * name property as a host:port. The wildcard + * address is replaced with the local host's address. If the host and address + * properties are configured the host component of the address will be combined + * with the port component of the addr to generate the address. This is to allow + * optional control over which host name is used in multi-home bind-host + * cases where a host can have multiple names + * @param hostProperty the bind-host configuration name + * @param addressProperty the service address configuration name + * @param defaultAddressValue the service default address configuration value + * @param addr InetSocketAddress of the service listener + * @return InetSocketAddress for clients to connect + */ + public InetSocketAddress updateConnectAddr( + String hostProperty, + String addressProperty, + String defaultAddressValue, + InetSocketAddress addr) { + + final String host = get(hostProperty); + final String connectHostPort = getTrimmed(addressProperty, defaultAddressValue); + + if (host == null || host.isEmpty() || connectHostPort == null || connectHostPort.isEmpty()) { + //not our case, fall back to original logic + return updateConnectAddr(addressProperty, addr); + } + + final String connectHost = connectHostPort.split(":")[0]; + // Create connect address using client address hostname and server port. + return updateConnectAddr(addressProperty, NetUtils.createSocketAddrForHost( + connectHost, addr.getPort())); + } + + /** + * Set the socket address a client can use to connect for the + * name property as a host:port. The wildcard + * address is replaced with the local host's address. + * @param name property name. + * @param addr InetSocketAddress of a listener to store in the given property + * @return InetSocketAddress for clients to connect + */ + public InetSocketAddress updateConnectAddr(String name, + InetSocketAddress addr) { + final InetSocketAddress connectAddr = NetUtils.getConnectAddress(addr); + setSocketAddr(name, connectAddr); + return connectAddr; + } + + /** + * Load a class by name. + * + * @param name the class name. + * @return the class object. + * @throws ClassNotFoundException if the class is not found. + */ + public Class getClassByName(String name) throws ClassNotFoundException { + Class ret = getClassByNameOrNull(name); + if (ret == null) { + throw new ClassNotFoundException("Class " + name + " not found"); + } + return ret; + } + + /** + * Load a class by name, returning null rather than throwing an exception + * if it couldn't be loaded. This is to avoid the overhead of creating + * an exception. + * + * @param name the class name + * @return the class object, or null if it could not be found. + */ + public Class getClassByNameOrNull(String name) { + Map>> map; + + synchronized (CACHE_CLASSES) { + map = CACHE_CLASSES.get(classLoader); + if (map == null) { + map = Collections.synchronizedMap( + new WeakHashMap>>()); + CACHE_CLASSES.put(classLoader, map); + } + } + + Class clazz = null; + WeakReference> ref = map.get(name); + if (ref != null) { + clazz = ref.get(); + } + + if (clazz == null) { + try { + clazz = Class.forName(name, true, classLoader); + } catch (ClassNotFoundException e) { + // Leave a marker that the class isn't found + map.put(name, new WeakReference>(NEGATIVE_CACHE_SENTINEL)); + return null; + } + // two putters can race here, but they'll put the same class + map.put(name, new WeakReference>(clazz)); + return clazz; + } else if (clazz == NEGATIVE_CACHE_SENTINEL) { + return null; // not found + } else { + // cache hit + return clazz; + } + } + + /** + * Get the value of the name property + * as an array of Class. + * The value of the property specifies a list of comma separated class names. + * If no such property is specified, then defaultValue is + * returned. + * + * @param name the property name. + * @param defaultValue default value. + * @return property value as a Class[], + * or defaultValue. + */ + public Class[] getClasses(String name, Class ... defaultValue) { + String[] classnames = getTrimmedStrings(name); + if (classnames == null) + return defaultValue; + try { + Class[] classes = new Class[classnames.length]; + for(int i = 0; i < classnames.length; i++) { + classes[i] = getClassByName(classnames[i]); + } + return classes; + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + /** + * Get the value of the name property as a Class. + * If no such property is specified, then defaultValue is + * returned. + * + * @param name the class name. + * @param defaultValue default value. + * @return property value as a Class, + * or defaultValue. + */ + public Class getClass(String name, Class defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + try { + return getClassByName(valueString); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + /** + * Get the value of the name property as a Class + * implementing the interface specified by xface. + * + * If no such property is specified, then defaultValue is + * returned. + * + * An exception is thrown if the returned class does not implement the named + * interface. + * + * @param name the class name. + * @param defaultValue default value. + * @param xface the interface implemented by the named class. + * @return property value as a Class, + * or defaultValue. + */ + public Class getClass(String name, + Class defaultValue, + Class xface) { + try { + Class theClass = getClass(name, defaultValue); + if (theClass != null && !xface.isAssignableFrom(theClass)) + throw new RuntimeException(theClass+" not "+xface.getName()); + else if (theClass != null) + return theClass.asSubclass(xface); + else + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Get the value of the name property as a List + * of objects implementing the interface specified by xface. + * + * An exception is thrown if any of the classes does not exist, or if it does + * not implement the named interface. + * + * @param name the property name. + * @param xface the interface implemented by the classes named by + * name. + * @return a List of objects implementing xface. + */ + @SuppressWarnings("unchecked") + public List getInstances(String name, Class xface) { + List ret = new ArrayList(); + Class[] classes = getClasses(name); + for (Class cl: classes) { + if (!xface.isAssignableFrom(cl)) { + throw new RuntimeException(cl + " does not implement " + xface); + } + ret.add((U)ReflectionUtils.newInstance(cl, this)); + } + return ret; + } + + /** + * Set the value of the name property to the name of a + * theClass implementing the given interface xface. + * + * An exception is thrown if theClass does not implement the + * interface xface. + * + * @param name property name. + * @param theClass property value. + * @param xface the interface implemented by the named class. + */ + public void setClass(String name, Class theClass, Class xface) { + if (!xface.isAssignableFrom(theClass)) + throw new RuntimeException(theClass+" not "+xface.getName()); + set(name, theClass.getName()); + } + + /** + * Get a local file under a directory named by dirsProp with + * the given path. If dirsProp contains multiple directories, + * then one is chosen based on path's hash code. If the selected + * directory does not exist, an attempt is made to create it. + * + * @param dirsProp directory in which to locate the file. + * @param path file-path. + * @return local file under the directory with the given path. + */ + public Path getLocalPath(String dirsProp, String path) + throws IOException { + String[] dirs = getTrimmedStrings(dirsProp); + int hashCode = path.hashCode(); + FileSystem fs = FileSystem.getLocal(this); + for (int i = 0; i < dirs.length; i++) { // try each local dir + int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length; + Path file = new Path(dirs[index], path); + Path dir = file.getParent(); + if (fs.mkdirs(dir) || fs.exists(dir)) { + return file; + } + } + LOG.warn("Could not make " + path + + " in local directories from " + dirsProp); + for(int i=0; i < dirs.length; i++) { + int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length; + LOG.warn(dirsProp + "[" + index + "]=" + dirs[index]); + } + throw new IOException("No valid local directories in property: "+dirsProp); + } + + /** + * Get a local file name under a directory named in dirsProp with + * the given path. If dirsProp contains multiple directories, + * then one is chosen based on path's hash code. If the selected + * directory does not exist, an attempt is made to create it. + * + * @param dirsProp directory in which to locate the file. + * @param path file-path. + * @return local file under the directory with the given path. + */ + public File getFile(String dirsProp, String path) + throws IOException { + String[] dirs = getTrimmedStrings(dirsProp); + int hashCode = path.hashCode(); + for (int i = 0; i < dirs.length; i++) { // try each local dir + int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length; + File file = new File(dirs[index], path); + File dir = file.getParentFile(); + if (dir.exists() || dir.mkdirs()) { + return file; + } + } + throw new IOException("No valid local directories in property: "+dirsProp); + } + + /** + * Get the {@link URL} for the named resource. + * + * @param name resource name. + * @return the url for the named resource. + */ + public URL getResource(String name) { + return classLoader.getResource(name); + } + + /** + * Get an input stream attached to the configuration resource with the + * given name. + * + * @param name configuration resource name. + * @return an input stream attached to the resource. + */ + public InputStream getConfResourceAsInputStream(String name) { + try { + URL url= getResource(name); + + if (url == null) { + LOG.info(name + " not found"); + return null; + } else { + LOG.info("found resource " + name + " at " + url); + } + + return url.openStream(); + } catch (Exception e) { + return null; + } + } + + /** + * Get a {@link Reader} attached to the configuration resource with the + * given name. + * + * @param name configuration resource name. + * @return a reader attached to the resource. + */ + public Reader getConfResourceAsReader(String name) { + try { + URL url= getResource(name); + + if (url == null) { + LOG.info(name + " not found"); + return null; + } else { + LOG.info("found resource " + name + " at " + url); + } + + return new InputStreamReader(url.openStream(), Charsets.UTF_8); + } catch (Exception e) { + return null; + } + } + + /** + * Get the set of parameters marked final. + * + * @return final parameter set. + */ + public Set getFinalParameters() { + Set setFinalParams = Collections.newSetFromMap( + new ConcurrentHashMap()); + setFinalParams.addAll(finalParameters); + return setFinalParams; + } + + protected synchronized Properties getProps() { + if (properties == null) { + properties = new Properties(); + Map backup = + new ConcurrentHashMap(updatingResource); + loadResources(properties, resources, quietmode); + + if (overlay != null) { + properties.putAll(overlay); + for (Map.Entry item: overlay.entrySet()) { + String key = (String)item.getKey(); + String[] source = backup.get(key); + if(source != null) { + updatingResource.put(key, source); + } + } + } + } + return properties; + } + + /** + * Return the number of keys in the configuration. + * + * @return number of keys in the configuration. + */ + public int size() { + return getProps().size(); + } + + /** + * Clears all keys from the configuration. + */ + public void clear() { + getProps().clear(); + getOverlay().clear(); + } + + /** + * Get an {@link Iterator} to go through the list of String + * key-value pairs in the configuration. + * + * @return an iterator over the entries. + */ + @Override + public Iterator> iterator() { + // Get a copy of just the string to string pairs. After the old object + // methods that allow non-strings to be put into configurations are removed, + // we could replace properties with a Map and get rid of this + // code. + Map result = new HashMap(); + for(Map.Entry item: getProps().entrySet()) { + if (item.getKey() instanceof String && + item.getValue() instanceof String) { + result.put((String) item.getKey(), (String) item.getValue()); + } + } + return result.entrySet().iterator(); + } + + /** + * Constructs a mapping of configuration and includes all properties that + * start with the specified configuration prefix. Property names in the + * mapping are trimmed to remove the configuration prefix. + * + * @param confPrefix configuration prefix + * @return mapping of configuration properties with prefix stripped + */ + public Map getPropsWithPrefix(String confPrefix) { + Map configMap = new HashMap<>(); + for (Map.Entry entry : this) { + String name = entry.getKey(); + if (name.startsWith(confPrefix)) { + String value = this.get(name); + name = name.substring(confPrefix.length()); + configMap.put(name, value); + } + } + return configMap; + } + + private Document parse(DocumentBuilder builder, URL url) + throws IOException, SAXException { + if (!quietmode) { + if (LOG.isDebugEnabled()) { + LOG.debug("parsing URL " + url); + } + } + if (url == null) { + return null; + } + + URLConnection connection = url.openConnection(); + if (connection instanceof JarURLConnection) { + // Disable caching for JarURLConnection to avoid sharing JarFile + // with other users. + connection.setUseCaches(false); + } + return parse(builder, connection.getInputStream(), url.toString()); + } + + private Document parse(DocumentBuilder builder, InputStream is, + String systemId) throws IOException, SAXException { + if (!quietmode) { + LOG.debug("parsing input stream " + is); + } + if (is == null) { + return null; + } + try { + return (systemId == null) ? builder.parse(is) : builder.parse(is, + systemId); + } finally { + is.close(); + } + } + + private void loadResources(Properties properties, + ArrayList resources, + boolean quiet) { + if(loadDefaults) { + for (String resource : defaultResources) { + loadResource(properties, new Resource(resource), quiet); + } + + //support the hadoop-site.xml as a deprecated case + if(getResource("hadoop-site.xml")!=null) { + loadResource(properties, new Resource("hadoop-site.xml"), quiet); + } + } + + for (int i = 0; i < resources.size(); i++) { + Resource ret = loadResource(properties, resources.get(i), quiet); + if (ret != null) { + resources.set(i, ret); + } + } + } + + private Resource loadResource(Properties properties, Resource wrapper, boolean quiet) { + String name = UNKNOWN_RESOURCE; + try { + Object resource = wrapper.getResource(); + name = wrapper.getName(); + + DocumentBuilderFactory docBuilderFactory + = DocumentBuilderFactory.newInstance(); + //ignore all comments inside the xml file + docBuilderFactory.setIgnoringComments(true); + + //allow includes in the xml file + docBuilderFactory.setNamespaceAware(true); + try { + docBuilderFactory.setXIncludeAware(true); + } catch (UnsupportedOperationException e) { + LOG.error("Failed to set setXIncludeAware(true) for parser " + + docBuilderFactory + + ":" + e, + e); + } + DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); + Document doc = null; + Element root = null; + boolean returnCachedProperties = false; + + if (resource instanceof URL) { // an URL resource + doc = parse(builder, (URL)resource); + } else if (resource instanceof String) { // a CLASSPATH resource + URL url = getResource((String)resource); + doc = parse(builder, url); + } else if (resource instanceof Path) { // a file resource + // Can't use FileSystem API or we get an infinite loop + // since FileSystem uses Configuration API. Use java.io.File instead. + File file = new File(((Path)resource).toUri().getPath()) + .getAbsoluteFile(); + if (file.exists()) { + if (!quiet) { + LOG.debug("parsing File " + file); + } + doc = parse(builder, new BufferedInputStream( + new FileInputStream(file)), ((Path)resource).toString()); + } + } else if (resource instanceof InputStream) { + doc = parse(builder, (InputStream) resource, null); + returnCachedProperties = true; + } else if (resource instanceof Properties) { + overlay(properties, (Properties)resource); + } else if (resource instanceof Element) { + root = (Element)resource; + } + + if (root == null) { + if (doc == null) { + if (quiet) { + return null; + } + throw new RuntimeException(resource + " not found"); + } + root = doc.getDocumentElement(); + } + Properties toAddTo = properties; + if(returnCachedProperties) { + toAddTo = new Properties(); + } + if (!"configuration".equals(root.getTagName())) + LOG.fatal("bad conf file: top-level element not "); + NodeList props = root.getChildNodes(); + DeprecationContext deprecations = deprecationContext.get(); + for (int i = 0; i < props.getLength(); i++) { + Node propNode = props.item(i); + if (!(propNode instanceof Element)) + continue; + Element prop = (Element)propNode; + if ("configuration".equals(prop.getTagName())) { + loadResource(toAddTo, new Resource(prop, name), quiet); + continue; + } + if (!"property".equals(prop.getTagName())) + LOG.warn("bad conf file: element not "); + NodeList fields = prop.getChildNodes(); + String attr = null; + String value = null; + boolean finalParameter = false; + LinkedList source = new LinkedList(); + for (int j = 0; j < fields.getLength(); j++) { + Node fieldNode = fields.item(j); + if (!(fieldNode instanceof Element)) + continue; + Element field = (Element)fieldNode; + if ("name".equals(field.getTagName()) && field.hasChildNodes()) + attr = StringInterner.weakIntern( + ((Text)field.getFirstChild()).getData().trim()); + if ("value".equals(field.getTagName()) && field.hasChildNodes()) + value = StringInterner.weakIntern( + ((Text)field.getFirstChild()).getData()); + if ("final".equals(field.getTagName()) && field.hasChildNodes()) + finalParameter = "true".equals(((Text)field.getFirstChild()).getData()); + if ("source".equals(field.getTagName()) && field.hasChildNodes()) + source.add(StringInterner.weakIntern( + ((Text)field.getFirstChild()).getData())); + } + source.add(name); + + // Ignore this parameter if it has already been marked as 'final' + if (attr != null) { + if (deprecations.getDeprecatedKeyMap().containsKey(attr)) { + DeprecatedKeyInfo keyInfo = + deprecations.getDeprecatedKeyMap().get(attr); + keyInfo.clearAccessed(); + for (String key:keyInfo.newKeys) { + // update new keys with deprecated key's value + loadProperty(toAddTo, name, key, value, finalParameter, + source.toArray(new String[source.size()])); + } + } + else { + loadProperty(toAddTo, name, attr, value, finalParameter, + source.toArray(new String[source.size()])); + } + } + } + + if (returnCachedProperties) { + overlay(properties, toAddTo); + return new Resource(toAddTo, name); + } + return null; + } catch (IOException e) { + LOG.fatal("error parsing conf " + name, e); + throw new RuntimeException(e); + } catch (DOMException e) { + LOG.fatal("error parsing conf " + name, e); + throw new RuntimeException(e); + } catch (SAXException e) { + LOG.fatal("error parsing conf " + name, e); + throw new RuntimeException(e); + } catch (ParserConfigurationException e) { + LOG.fatal("error parsing conf " + name , e); + throw new RuntimeException(e); + } + } + + private void overlay(Properties to, Properties from) { + for (Entry entry: from.entrySet()) { + to.put(entry.getKey(), entry.getValue()); + } + } + + private void loadProperty(Properties properties, String name, String attr, + String value, boolean finalParameter, String[] source) { + if (value != null || allowNullValueProperties) { + if (value == null) { + value = DEFAULT_STRING_CHECK; + } + if (!finalParameters.contains(attr)) { + properties.setProperty(attr, value); + if(source != null) { + updatingResource.put(attr, source); + } + } else if (!value.equals(properties.getProperty(attr))) { + LOG.warn(name+":an attempt to override final parameter: "+attr + +"; Ignoring."); + } + } + if (finalParameter && attr != null) { + finalParameters.add(attr); + } + } + + /** + * Write out the non-default properties in this configuration to the given + * {@link OutputStream} using UTF-8 encoding. + * + * @param out the output stream to write to. + */ + public void writeXml(OutputStream out) throws IOException { + writeXml(new OutputStreamWriter(out, "UTF-8")); + } + + /** + * Write out the non-default properties in this configuration to the given + * {@link Writer}. + * + * @param out the writer to write to. + */ + public void writeXml(Writer out) throws IOException { + Document doc = asXmlDocument(); + + try { + DOMSource source = new DOMSource(doc); + StreamResult result = new StreamResult(out); + TransformerFactory transFactory = TransformerFactory.newInstance(); + Transformer transformer = transFactory.newTransformer(); + + // Important to not hold Configuration log while writing result, since + // 'out' may be an HDFS stream which needs to lock this configuration + // from another thread. + transformer.transform(source, result); + } catch (TransformerException te) { + throw new IOException(te); + } + } + + /** + * Return the XML DOM corresponding to this Configuration. + */ + private synchronized Document asXmlDocument() throws IOException { + Document doc; + try { + doc = + DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument(); + } catch (ParserConfigurationException pe) { + throw new IOException(pe); + } + Element conf = doc.createElement("configuration"); + doc.appendChild(conf); + conf.appendChild(doc.createTextNode("\n")); + handleDeprecation(); //ensure properties is set and deprecation is handled + for (Enumeration e = properties.keys(); e.hasMoreElements();) { + String name = (String)e.nextElement(); + Object object = properties.get(name); + String value = null; + if (object instanceof String) { + value = (String) object; + }else { + continue; + } + Element propNode = doc.createElement("property"); + conf.appendChild(propNode); + + Element nameNode = doc.createElement("name"); + nameNode.appendChild(doc.createTextNode(name)); + propNode.appendChild(nameNode); + + Element valueNode = doc.createElement("value"); + valueNode.appendChild(doc.createTextNode(value)); + propNode.appendChild(valueNode); + + if (updatingResource != null) { + String[] sources = updatingResource.get(name); + if(sources != null) { + for(String s : sources) { + Element sourceNode = doc.createElement("source"); + sourceNode.appendChild(doc.createTextNode(s)); + propNode.appendChild(sourceNode); + } + } + } + + conf.appendChild(doc.createTextNode("\n")); + } + return doc; + } + + /** + * Writes out all the parameters and their properties (final and resource) to + * the given {@link Writer} + * The format of the output would be + * { "properties" : [ {key1,value1,key1.isFinal,key1.resource}, {key2,value2, + * key2.isFinal,key2.resource}... ] } + * It does not output the parameters of the configuration object which is + * loaded from an input stream. + * @param out the Writer to write to + * @throws IOException + */ + public static void dumpConfiguration(Configuration config, + Writer out) throws IOException { + JsonFactory dumpFactory = new JsonFactory(); + JsonGenerator dumpGenerator = dumpFactory.createJsonGenerator(out); + dumpGenerator.writeStartObject(); + dumpGenerator.writeFieldName("properties"); + dumpGenerator.writeStartArray(); + dumpGenerator.flush(); + synchronized (config) { + for (Map.Entry item: config.getProps().entrySet()) { + dumpGenerator.writeStartObject(); + dumpGenerator.writeStringField("key", (String) item.getKey()); + dumpGenerator.writeStringField("value", + config.get((String) item.getKey())); + dumpGenerator.writeBooleanField("isFinal", + config.finalParameters.contains(item.getKey())); + String[] resources = config.updatingResource.get(item.getKey()); + String resource = UNKNOWN_RESOURCE; + if(resources != null && resources.length > 0) { + resource = resources[0]; + } + dumpGenerator.writeStringField("resource", resource); + dumpGenerator.writeEndObject(); + } + } + dumpGenerator.writeEndArray(); + dumpGenerator.writeEndObject(); + dumpGenerator.flush(); + } + + /** + * Get the {@link ClassLoader} for this job. + * + * @return the correct class loader. + */ + public ClassLoader getClassLoader() { + return classLoader; + } + + /** + * Set the class loader that will be used to load the various objects. + * + * @param classLoader the new class loader. + */ + public void setClassLoader(ClassLoader classLoader) { + this.classLoader = classLoader; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("Configuration: "); + if(loadDefaults) { + toString(defaultResources, sb); + if(resources.size()>0) { + sb.append(", "); + } + } + toString(resources, sb); + return sb.toString(); + } + + private void toString(List resources, StringBuilder sb) { + ListIterator i = resources.listIterator(); + while (i.hasNext()) { + if (i.nextIndex() != 0) { + sb.append(", "); + } + sb.append(i.next()); + } + } + + /** + * Set the quietness-mode. + * + * In the quiet-mode, error and informational messages might not be logged. + * + * @param quietmode true to set quiet-mode on, false + * to turn it off. + */ + public synchronized void setQuietMode(boolean quietmode) { + this.quietmode = quietmode; + } + + synchronized boolean getQuietMode() { + return this.quietmode; + } + + /** For debugging. List non-default properties to the terminal and exit. */ + public static void main(String[] args) throws Exception { + new Configuration().writeXml(System.out); + } + + @Override + public void readFields(DataInput in) throws IOException { + clear(); + int size = WritableUtils.readVInt(in); + for(int i=0; i < size; ++i) { + String key = org.apache.hadoop.io.Text.readString(in); + String value = org.apache.hadoop.io.Text.readString(in); + set(key, value); + String sources[] = WritableUtils.readCompressedStringArray(in); + if(sources != null) { + updatingResource.put(key, sources); + } + } + } + + //@Override + @Override + public void write(DataOutput out) throws IOException { + Properties props = getProps(); + WritableUtils.writeVInt(out, props.size()); + for(Map.Entry item: props.entrySet()) { + org.apache.hadoop.io.Text.writeString(out, (String) item.getKey()); + org.apache.hadoop.io.Text.writeString(out, (String) item.getValue()); + WritableUtils.writeCompressedStringArray(out, + updatingResource.get(item.getKey())); + } + } + + /** + * get keys matching the the regex + * @param regex + * @return Map with matching keys + */ + public Map getValByRegex(String regex) { + Pattern p = Pattern.compile(regex); + + Map result = new HashMap(); + Matcher m; + + for(Map.Entry item: getProps().entrySet()) { + if (item.getKey() instanceof String && + item.getValue() instanceof String) { + m = p.matcher((String)item.getKey()); + if(m.find()) { // match + result.put((String) item.getKey(), + substituteVars(getProps().getProperty((String) item.getKey()))); + } + } + } + return result; + } + + /** + * A unique class which is used as a sentinel value in the caching + * for getClassByName. {@see Configuration#getClassByNameOrNull(String)} + */ + private static abstract class NegativeCacheSentinel {} + + public static void dumpDeprecatedKeys() { + DeprecationContext deprecations = deprecationContext.get(); + for (Map.Entry entry : + deprecations.getDeprecatedKeyMap().entrySet()) { + StringBuilder newKeys = new StringBuilder(); + for (String newKey : entry.getValue().newKeys) { + newKeys.append(newKey).append("\t"); + } + System.out.println(entry.getKey() + "\t" + newKeys.toString()); + } + } + + /** + * Returns whether or not a deprecated name has been warned. If the name is not + * deprecated then always return false + */ + public static boolean hasWarnedDeprecation(String name) { + DeprecationContext deprecations = deprecationContext.get(); + if(deprecations.getDeprecatedKeyMap().containsKey(name)) { + if(deprecations.getDeprecatedKeyMap().get(name).accessed.get()) { + return true; + } + } + return false; + } +} diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/core-default-shaded.xml b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/core-default-shaded.xml new file mode 100644 index 0000000000000..c9dff76625565 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/core-default-shaded.xml @@ -0,0 +1,2312 @@ + + + + + + + + + + + + + + + hadoop.common.configuration.version + 0.23.0 + version of this configuration file + + + + hadoop.tmp.dir + /tmp/hadoop-${user.name} + A base for other temporary directories. + + + + io.native.lib.available + true + Controls whether to use native libraries for bz2 and zlib + compression codecs or not. The property does not control any other native + libraries. + + + + + hadoop.http.filter.initializers + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.http.lib.StaticUserWebFilter + A comma separated list of class names. Each class in the list + must extend org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.http.FilterInitializer. The corresponding + Filter will be initialized. Then, the Filter will be applied to all user + facing jsp and servlet web pages. The ordering of the list defines the + ordering of the filters. + + + + + + hadoop.security.authorization + false + Is service-level authorization enabled? + + + + hadoop.security.instrumentation.requires.admin + false + + Indicates if administrator ACLs are required to access + instrumentation servlets (JMX, METRICS, CONF, STACKS). + + + + + hadoop.security.authentication + simple + Possible values are simple (no authentication), and kerberos + + + + + hadoop.security.group.mapping + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback + + Class for user to group mapping (get groups for a given user) for ACL. + The default implementation, + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback, + will determine if the Java Native Interface (JNI) is available. If JNI is + available the implementation will use the API within hadoop to resolve a + list of groups for a user. If JNI is not available then the shell + implementation, ShellBasedUnixGroupsMapping, is used. This implementation + shells out to the Linux/Unix environment with the + bash -c groups command to resolve a list of groups for a user. + + + + + hadoop.security.dns.interface + + The name of the Network Interface from which the service should determine + its host name for Kerberos login. e.g. eth2. In a multi-homed environment, + the setting can be used to affect the _HOST subsitution in the service + Kerberos principal. If this configuration value is not set, the service + will use its default hostname as returned by + InetAddress.getLocalHost().getCanonicalHostName(). + + Most clusters will not require this setting. + + + + + hadoop.security.dns.nameserver + + The host name or IP address of the name server (DNS) which a service Node + should use to determine its own host name for Kerberos Login. Requires + hadoop.security.dns.interface. + + Most clusters will not require this setting. + + + + + hadoop.security.dns.log-slow-lookups.enabled + false + + Time name lookups (via SecurityUtil) and log them if they exceed the + configured threshold. + + + + + hadoop.security.dns.log-slow-lookups.threshold.ms + 1000 + + If slow lookup logging is enabled, this threshold is used to decide if a + lookup is considered slow enough to be logged. + + + + + hadoop.security.groups.cache.secs + 300 + + This is the config controlling the validity of the entries in the cache + containing the user->group mapping. When this duration has expired, + then the implementation of the group mapping provider is invoked to get + the groups of the user and then cached back. + + + + + hadoop.security.groups.negative-cache.secs + 30 + + Expiration time for entries in the the negative user-to-group mapping + caching, in seconds. This is useful when invalid users are retrying + frequently. It is suggested to set a small value for this expiration, since + a transient error in group lookup could temporarily lock out a legitimate + user. + + Set this to zero or negative value to disable negative user-to-group caching. + + + + + hadoop.security.groups.cache.warn.after.ms + 5000 + + If looking up a single user to group takes longer than this amount of + milliseconds, we will log a warning message. + + + + + hadoop.security.groups.cache.background.reload + false + + Whether to reload expired user->group mappings using a background thread + pool. If set to true, a pool of + hadoop.security.groups.cache.background.reload.threads is created to + update the cache in the background. + + + + + hadoop.security.groups.cache.background.reload.threads + 3 + + Only relevant if hadoop.security.groups.cache.background.reload is true. + Controls the number of concurrent background user->group cache entry + refreshes. Pending refresh requests beyond this value are queued and + processed when a thread is free. + + + + + hadoop.security.group.mapping.ldap.connection.timeout.ms + 60000 + + This property is the connection timeout (in milliseconds) for LDAP + operations. If the LDAP provider doesn't establish a connection within the + specified period, it will abort the connect attempt. Non-positive value + means no LDAP connection timeout is specified in which case it waits for the + connection to establish until the underlying network times out. + + + + + hadoop.security.group.mapping.ldap.read.timeout.ms + 60000 + + This property is the read timeout (in milliseconds) for LDAP + operations. If the LDAP provider doesn't get a LDAP response within the + specified period, it will abort the read attempt. Non-positive value + means no read timeout is specified in which case it waits for the response + infinitely. + + + + + hadoop.security.group.mapping.ldap.url + + + The URL of the LDAP server to use for resolving user groups when using + the LdapGroupsMapping user to group mapping. + + + + + hadoop.security.group.mapping.ldap.ssl + false + + Whether or not to use SSL when connecting to the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore + + + File path to the SSL keystore that contains the SSL certificate required + by the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore.password.file + + + The path to a file containing the password of the LDAP SSL keystore. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.bind.user + + + The distinguished name of the user to bind as when connecting to the LDAP + server. This may be left blank if the LDAP server supports anonymous binds. + + + + + hadoop.security.group.mapping.ldap.bind.password.file + + + The path to a file containing the password of the bind user. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.base + + + The search base for the LDAP connection. This is a distinguished name, + and will typically be the root of the LDAP directory. + + + + + hadoop.security.group.mapping.ldap.search.filter.user + (&(objectClass=user)(sAMAccountName={0})) + + An additional filter to use when searching for LDAP users. The default will + usually be appropriate for Active Directory installations. If connecting to + an LDAP server with a non-AD schema, this should be replaced with + (&(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to + denote where the username fits into the filter. + + If the LDAP server supports posixGroups, Hadoop can enable the feature by + setting the value of this property to "posixAccount" and the value of + the hadoop.security.group.mapping.ldap.search.filter.group property to + "posixGroup". + + + + + hadoop.security.group.mapping.ldap.search.filter.group + (objectClass=group) + + An additional filter to use when searching for LDAP groups. This should be + changed when resolving groups against a non-Active Directory installation. + + See the description of hadoop.security.group.mapping.ldap.search.filter.user + to enable posixGroups support. + + + + + hadoop.security.group.mapping.ldap.search.attr.member + member + + The attribute of the group object that identifies the users that are + members of the group. The default will usually be appropriate for + any LDAP installation. + + + + + hadoop.security.group.mapping.ldap.search.attr.group.name + cn + + The attribute of the group object that identifies the group name. The + default will usually be appropriate for all LDAP systems. + + + + + hadoop.security.group.mapping.ldap.posix.attr.uid.name + uidNumber + + The attribute of posixAccount to use when groups for membership. + Mostly useful for schemas wherein groups have memberUids that use an + attribute other than uidNumber. + + + + + hadoop.security.group.mapping.ldap.posix.attr.gid.name + gidNumber + + The attribute of posixAccount indicating the group id. + + + + + hadoop.security.group.mapping.ldap.directory.search.timeout + 10000 + + The attribute applied to the LDAP SearchControl properties to set a + maximum time limit when searching and awaiting a result. + Set to 0 if infinite wait period is desired. + Default is 10 seconds. Units in milliseconds. + + + + + hadoop.security.group.mapping.providers + + + Comma separated of names of other providers to provide user to group + mapping. Used by CompositeGroupsMapping. + + + + + hadoop.security.group.mapping.providers.combined + true + + true or false to indicate whether groups from the providers are combined or + not. The default value is true. If true, then all the providers will be + tried to get groups and all the groups are combined to return as the final + results. Otherwise, providers are tried one by one in the configured list + order, and if any groups are retrieved from any provider, then the groups + will be returned without trying the left ones. + + + + + hadoop.security.service.user.name.key + + + For those cases where the same RPC protocol is implemented by multiple + servers, this configuration is required for specifying the principal + name to use for the service when the client wishes to make an RPC call. + + + + + + hadoop.security.uid.cache.secs + 14400 + + This is the config controlling the validity of the entries in the cache + containing the userId to userName and groupId to groupName used by + NativeIO getFstat(). + + + + + hadoop.rpc.protection + authentication + A comma-separated list of protection values for secured sasl + connections. Possible values are authentication, integrity and privacy. + authentication means authentication only and no integrity or privacy; + integrity implies authentication and integrity are enabled; and privacy + implies all of authentication, integrity and privacy are enabled. + hadoop.security.saslproperties.resolver.class can be used to override + the hadoop.rpc.protection for a connection at the server side. + + + + + hadoop.security.saslproperties.resolver.class + + SaslPropertiesResolver used to resolve the QOP used for a + connection. If not specified, the full set of values specified in + hadoop.rpc.protection is used while determining the QOP used for the + connection. If a class is specified, then the QOP values returned by + the class will be used while determining the QOP used for the connection. + + + + + hadoop.security.sensitive-config-keys + secret$,password$,ssl.keystore.pass$,fs.s3.*[Ss]ecret.?[Kk]ey,fs.azure.account.key.*,dfs.webhdfs.oauth2.[a-z]+.token,hadoop.security.sensitive-config-keys + A comma-separated list of regular expressions to match against + configuration keys that should be redacted where appropriate, for + example, when logging modified properties during a reconfiguration, + private credentials should not be logged. + + + + + hadoop.workaround.non.threadsafe.getpwuid + true + Some operating systems or authentication modules are known to + have broken implementations of getpwuid_r and getpwgid_r, such that these + calls are not thread-safe. Symptoms of this problem include JVM crashes + with a stack trace inside these functions. If your system exhibits this + issue, enable this configuration parameter to include a lock around the + calls as a workaround. + + An incomplete list of some systems known to have this issue is available + at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations + + + + + hadoop.kerberos.kinit.command + kinit + Used to periodically renew Kerberos credentials when provided + to Hadoop. The default setting assumes that kinit is in the PATH of users + running the Hadoop client. Change this to the absolute path to kinit if this + is not the case. + + + + + hadoop.kerberos.min.seconds.before.relogin + 60 + The minimum time between relogin attempts for Kerberos, in + seconds. + + + + + hadoop.security.auth_to_local + + Maps kerberos principals to local user names + + + + + io.file.buffer.size + 4096 + The size of buffer for use in sequence files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + io.bytes.per.checksum + 512 + The number of bytes per checksum. Must not be larger than + io.file.buffer.size. + + + + io.skip.checksum.errors + false + If true, when a checksum error is encountered while + reading a sequence file, entries are skipped, instead of throwing an + exception. + + + + io.compression.codecs + + A comma-separated list of the compression codec classes that can + be used for compression/decompression. In addition to any classes specified + with this property (which take precedence), codec classes on the classpath + are discovered using a Java ServiceLoader. + + + + io.compression.codec.bzip2.library + system-native + The native-code library to be used for compression and + decompression by the bzip2 codec. This library could be specified + either by by name or the full pathname. In the former case, the + library is located by the dynamic linker, usually searching the + directories specified in the environment variable LD_LIBRARY_PATH. + + The value of "system-native" indicates that the default system + library should be used. To indicate that the algorithm should + operate entirely in Java, specify "java-builtin". + + + + io.serializations + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.io.serializer.WritableSerialization, org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.io.serializer.avro.AvroReflectSerialization + A list of serialization classes that can be used for + obtaining serializers and deserializers. + + + + io.seqfile.local.dir + ${hadoop.tmp.dir}/io/local + The local directory where sequence file stores intermediate + data files during merge. May be a comma-separated list of + directories on different devices in order to spread disk i/o. + Directories that do not exist are ignored. + + + + + io.map.index.skip + 0 + Number of index entries to skip between each entry. + Zero by default. Setting this to values larger than zero can + facilitate opening large MapFiles using less memory. + + + + io.map.index.interval + 128 + + MapFile consist of two files - data file (tuples) and index file + (keys). For every io.map.index.interval records written in the + data file, an entry (record-key, data-file-position) is written + in the index file. This is to allow for doing binary search later + within the index file to look up records by their keys and get their + closest positions in the data file. + + + + + + + fs.defaultFS + file:/// + The name of the default file system. A URI whose + scheme and authority determine the FileSystem implementation. The + uri's scheme determines the config property (fs.SCHEME.impl) naming + the FileSystem implementation class. The uri's authority is used to + determine the host, port, etc. for a filesystem. + + + + fs.default.name + file:/// + Deprecated. Use (fs.defaultFS) property + instead + + + + fs.trash.interval + 0 + Number of minutes after which the checkpoint + gets deleted. If zero, the trash feature is disabled. + This option may be configured both on the server and the + client. If trash is disabled server side then the client + side configuration is checked. If trash is enabled on the + server side then the value configured on the server is + used and the client configuration value is ignored. + + + + + fs.trash.checkpoint.interval + 0 + Number of minutes between trash checkpoints. + Should be smaller or equal to fs.trash.interval. If zero, + the value is set to the value of fs.trash.interval. + Every time the checkpointer runs it creates a new checkpoint + out of current and removes checkpoints created more than + fs.trash.interval minutes ago. + + + + + fs.protected.directories + + A comma-separated list of directories which cannot + be deleted even by the superuser unless they are empty. This + setting can be used to guard important system directories + against accidental deletion due to administrator error. + + + + + fs.AbstractFileSystem.file.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.local.LocalFs + The AbstractFileSystem for file: uris. + + + + fs.AbstractFileSystem.har.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.HarFs + The AbstractFileSystem for har: uris. + + + + fs.AbstractFileSystem.hdfs.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.Hdfs + The FileSystem for hdfs: uris. + + + + fs.AbstractFileSystem.viewfs.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.viewfs.ViewFs + The AbstractFileSystem for view file system for viewfs: uris + (ie client side mount table:). + + + + fs.AbstractFileSystem.ftp.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.ftp.FtpFs + The FileSystem for Ftp: uris. + + + + fs.AbstractFileSystem.webhdfs.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.WebHdfs + The FileSystem for webhdfs: uris. + + + + fs.AbstractFileSystem.swebhdfs.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.SWebHdfs + The FileSystem for swebhdfs: uris. + + + + fs.ftp.host + 0.0.0.0 + FTP filesystem connects to this server + + + + fs.ftp.host.port + 21 + + FTP filesystem connects to fs.ftp.host on this port + + + + + fs.df.interval + 60000 + Disk usage statistics refresh interval in msec. + + + + fs.du.interval + 600000 + File space usage statistics refresh interval in msec. + + + + fs.s3.block.size + 67108864 + Block size to use when writing files to S3. + + + + fs.s3.buffer.dir + ${hadoop.tmp.dir}/s3 + Determines where on the local filesystem the s3:/s3n: filesystem + should store files before sending them to S3 + (or after retrieving them from S3). + + + + + fs.s3.maxRetries + 4 + The maximum number of retries for reading or writing files to S3, + before we signal failure to the application. + + + + + fs.s3.sleepTimeSeconds + 10 + The number of seconds to sleep between each S3 retry. + + + + + fs.automatic.close + true + By default, FileSystem instances are automatically closed at program + exit using a JVM shutdown hook. Setting this property to false disables this + behavior. This is an advanced option that should only be used by server applications + requiring a more carefully orchestrated shutdown sequence. + + + + + fs.s3n.block.size + 67108864 + Block size to use when reading files using the native S3 + filesystem (s3n: URIs). + + + + fs.s3n.multipart.uploads.enabled + false + Setting this property to true enables multiple uploads to + native S3 filesystem. When uploading a file, it is split into blocks + if the size is larger than fs.s3n.multipart.uploads.block.size. + + + + + fs.s3n.multipart.uploads.block.size + 67108864 + The block size for multipart uploads to native S3 filesystem. + Default size is 64MB. + + + + + fs.s3n.multipart.copy.block.size + 5368709120 + The block size for multipart copy in native S3 filesystem. + Default size is 5GB. + + + + + fs.s3n.server-side-encryption-algorithm + + Specify a server-side encryption algorithm for S3. + Unset by default, and the only other currently allowable value is AES256. + + + + + fs.s3a.access.key + AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication. + + + + fs.s3a.secret.key + AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication. + + + + fs.s3a.aws.credentials.provider + + Comma-separated class names of credential provider classes which implement + com.amazonaws.auth.AWSCredentialsProvider. + + These are loaded and queried in sequence for a valid set of credentials. + Each listed class must implement one of the following means of + construction, which are attempted in order: + 1. a public constructor accepting java.net.URI and + org.apache.hadoop.conf.Configuration, + 2. a public static method named getInstance that accepts no + arguments and returns an instance of + com.amazonaws.auth.AWSCredentialsProvider, or + 3. a public default constructor. + + Specifying + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider + allows + anonymous access to a publicly accessible S3 bucket without any credentials. + Please note that allowing anonymous access to an S3 bucket compromises + security and therefore is unsuitable for most use cases. It can be useful + for accessing public data sets without requiring AWS credentials. + + If unspecified, then the default list of credential provider classes, + queried in sequence, is: + 1. org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider: + supports static + configuration of AWS access key ID and secret access key. See also + fs.s3a.access.key and fs.s3a.secret.key. + 2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports + configuration of AWS access key ID and secret access key in + environment variables named AWS_ACCESS_KEY_ID and + AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK. + 3. + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.s3a.SharedInstanceProfileCredentialsProvider: + a shared instance of + com.amazonaws.auth.InstanceProfileCredentialsProvider from the AWS + SDK, which supports use of instance profile credentials if running + in an EC2 VM. Using this shared instance potentially reduces load + on the EC2 instance metadata service for multi-threaded + applications. + + + + + fs.s3a.session.token + Session token, when using org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider + as one of the providers. + + + + + fs.s3a.security.credential.provider.path + + + Optional comma separated list of credential providers, a list + which is prepended to that set in hadoop.security.credential.provider.path + + + + + fs.s3a.connection.maximum + 15 + Controls the maximum number of simultaneous connections to S3. + + + + fs.s3a.connection.ssl.enabled + true + Enables or disables SSL connections to S3. + + + + fs.s3a.endpoint + AWS S3 endpoint to connect to. An up-to-date list is + provided in the AWS Documentation: regions and endpoints. Without this + property, the standard region (s3.amazonaws.com) is assumed. + + + + + fs.s3a.path.style.access + false + Enable S3 path style access ie disabling the default virtual hosting behaviour. + Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting. + + + + + fs.s3a.proxy.host + Hostname of the (optional) proxy server for S3 connections. + + + + fs.s3a.proxy.port + Proxy server port. If this property is not set + but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with + the value of fs.s3a.connection.ssl.enabled). + + + + fs.s3a.proxy.username + Username for authenticating with proxy server. + + + + fs.s3a.proxy.password + Password for authenticating with proxy server. + + + + fs.s3a.proxy.domain + Domain for authenticating with proxy server. + + + + fs.s3a.proxy.workstation + Workstation for authenticating with proxy server. + + + + fs.s3a.attempts.maximum + 20 + How many times we should retry commands on transient errors. + + + + fs.s3a.connection.establish.timeout + 5000 + Socket connection setup timeout in milliseconds. + + + + fs.s3a.connection.timeout + 200000 + Socket connection timeout in milliseconds. + + + + fs.s3a.socket.send.buffer + 8192 + Socket send buffer hint to amazon connector. Represented in bytes. + + + + fs.s3a.socket.recv.buffer + 8192 + Socket receive buffer hint to amazon connector. Represented in bytes. + + + + fs.s3a.paging.maximum + 5000 + How many keys to request from S3 when doing + directory listings at a time. + + + + fs.s3a.threads.max + 10 + The total number of threads available in the filesystem for data + uploads *or any other queued filesystem operation*. + + + + fs.s3a.threads.keepalivetime + 60 + Number of seconds a thread can be idle before being + terminated. + + + + fs.s3a.max.total.tasks + 5 + The number of operations which can be queued for execution + + + + fs.s3a.multipart.size + 100M + How big (in bytes) to split upload or copy operations up into. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.multipart.threshold + 2147483647 + How big (in bytes) to split upload or copy operations up into. + This also controls the partition size in renamed files, as rename() involves + copying the source file(s). + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.multiobjectdelete.enable + true + When enabled, multiple single-object delete requests are replaced by + a single 'delete multiple objects'-request, reducing the number of requests. + Beware: legacy S3-compatible object stores might not support this request. + + + + + fs.s3a.acl.default + Set a canned ACL for newly created and copied objects. Value may be Private, + PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead, + or BucketOwnerFullControl. + + + + fs.s3a.multipart.purge + false + True if you want to purge existing multipart uploads that may not have been + completed/aborted correctly. The corresponding purge age is defined in + fs.s3a.multipart.purge.age. + If set, when the filesystem is instantiated then all outstanding uploads + older than the purge age will be terminated -across the entire bucket. + This will impact multipart uploads by other applications and users. so should + be used sparingly, with an age value chosen to stop failed uploads, without + breaking ongoing operations. + + + + + fs.s3a.multipart.purge.age + 86400 + Minimum age in seconds of multipart uploads to purge. + + + + + fs.s3a.server-side-encryption-algorithm + Specify a server-side encryption algorithm for s3a: file system. + Unset by default, and the only other currently allowable value is AES256. + + + + + fs.s3a.signing-algorithm + Override the default signing algorithm so legacy + implementations can still be used + + + + fs.s3a.block.size + 32M + Block size to use when reading files using s3a: file system. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.buffer.dir + ${hadoop.tmp.dir}/s3a + Comma separated list of directories that will be used to buffer file + uploads to. + + + + fs.s3a.fast.upload + false + + Use the incremental block-based fast upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + + + + + fs.s3a.fast.upload.buffer + disk + + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + + + + + fs.s3a.fast.upload.active.blocks + 4 + + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + + + + + fs.s3a.readahead.range + 64K + Bytes to read ahead during a seek() before closing and + re-opening the S3 HTTP connection. This option will be overridden if + any call to setReadahead() is made to an open stream. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.user.agent.prefix + + + Sets a custom value that will be prepended to the User-Agent header sent in + HTTP requests to the S3 back-end by S3AFileSystem. The User-Agent header + always includes the Hadoop version number followed by a string generated by + the AWS SDK. An example is "User-Agent: Hadoop 2.8.0, aws-sdk-java/1.10.6". + If this optional property is set, then its value is prepended to create a + customized User-Agent. For example, if this configuration property was set + to "MyApp", then an example of the resulting User-Agent would be + "User-Agent: MyApp, Hadoop 2.8.0, aws-sdk-java/1.10.6". + + + + + fs.s3a.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.s3a.S3AFileSystem + The implementation class of the S3A Filesystem + + + + fs.AbstractFileSystem.s3a.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.fs.s3a.S3A + The implementation class of the S3A AbstractFileSystem. + + + + io.seqfile.compress.blocksize + 1000000 + The minimum block size for compression in block compressed + SequenceFiles. + + + + + io.mapfile.bloom.size + 1048576 + The size of BloomFilter-s used in BloomMapFile. Each time this many + keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter). + Larger values minimize the number of filters, which slightly increases the performance, + but may waste too much space if the total number of keys is usually much smaller + than this number. + + + + + io.mapfile.bloom.error.rate + 0.005 + The rate of false positives in BloomFilter-s used in BloomMapFile. + As this value decreases, the size of BloomFilter-s increases exponentially. This + value is the probability of encountering false positives (default is 0.5%). + + + + + hadoop.util.hash.type + murmur + The default implementation of Hash. Currently this can take one of the + two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash. + + + + + + + + ipc.client.idlethreshold + 4000 + Defines the threshold number of connections after which + connections will be inspected for idleness. + + + + + ipc.client.kill.max + 10 + Defines the maximum number of clients to disconnect in one go. + + + + + ipc.client.connection.maxidletime + 10000 + The maximum time in msec after which a client will bring down the + connection to the server. + + + + + ipc.client.connect.max.retries + 10 + Indicates the number of retries a client will make to establish + a server connection. + + + + + ipc.client.connect.retry.interval + 1000 + Indicates the number of milliseconds a client will wait for + before retrying to establish a server connection. + + + + + ipc.client.connect.timeout + 20000 + Indicates the number of milliseconds a client will wait for the + socket to establish a server connection. + + + + + ipc.client.connect.max.retries.on.timeouts + 45 + Indicates the number of retries a client will make on socket timeout + to establish a server connection. + + + + + ipc.client.tcpnodelay + true + Use TCP_NODELAY flag to bypass Nagle's algorithm transmission delays. + + + + + ipc.client.low-latency + false + Use low-latency QoS markers for IPC connections. + + + + + ipc.client.ping + true + Send a ping to the server when timeout on reading the response, + if set to true. If no failure is detected, the client retries until at least + a byte is read or the time given by ipc.client.rpc-timeout.ms is passed. + + + + + ipc.ping.interval + 60000 + Timeout on waiting response from server, in milliseconds. + The client will send ping when the interval is passed without receiving bytes, + if ipc.client.ping is set to true. + + + + + ipc.client.rpc-timeout.ms + 0 + Timeout on waiting response from server, in milliseconds. + If ipc.client.ping is set to true and this rpc-timeout is greater than + the value of ipc.ping.interval, the effective value of the rpc-timeout is + rounded up to multiple of ipc.ping.interval. + + + + + ipc.server.listen.queue.size + 128 + Indicates the length of the listen queue for servers accepting + client connections. + + + + + ipc.server.log.slow.rpc + false + This setting is useful to troubleshoot performance issues for + various services. If this value is set to true then we log requests that + fall into 99th percentile as well as increment RpcSlowCalls counter. + + + + + ipc.maximum.data.length + 67108864 + This indicates the maximum IPC message length (bytes) that can be + accepted by the server. Messages larger than this value are rejected by the + immediately to avoid possible OOMs. This setting should rarely need to be + changed. + + + + + ipc.maximum.response.length + 134217728 + This indicates the maximum IPC message length (bytes) that can be + accepted by the client. Messages larger than this value are rejected + immediately to avoid possible OOMs. This setting should rarely need to be + changed. Set to 0 to disable. + + + + + + + hadoop.security.impersonation.provider.class + + A class which implements ImpersonationProvider interface, used to + authorize whether one user can impersonate a specific user. + If not specified, the DefaultImpersonationProvider will be used. + If a class is specified, then that class will be used to determine + the impersonation capability. + + + + + hadoop.rpc.socket.factory.class.default + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.net.StandardSocketFactory + Default SocketFactory to use. This parameter is expected to be + formatted as "package.FactoryClassName". + + + + + hadoop.rpc.socket.factory.class.ClientProtocol + + SocketFactory to use to connect to a DFS. If null or empty, use + hadoop.rpc.socket.class.default. This socket factory is also used by + DFSClient to create sockets to DataNodes. + + + + + + + hadoop.socks.server + + Address (host:port) of the SOCKS server to be used by the + SocksSocketFactory. + + + + + + net.topology.node.switch.mapping.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.net.ScriptBasedMapping + The default implementation of the DNSToSwitchMapping. It + invokes a script specified in net.topology.script.file.name to resolve + node names. If the value for net.topology.script.file.name is not set, the + default value of DEFAULT_RACK is returned for all node names. + + + + + net.topology.impl + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.net.NetworkTopology + The default implementation of NetworkTopology which is classic three layer one. + + + + + net.topology.script.file.name + + The script name that should be invoked to resolve DNS names to + NetworkTopology names. Example: the script would take host.foo.bar as an + argument, and return /rack1 as the output. + + + + + net.topology.script.number.args + 100 + The max number of args that the script configured with + net.topology.script.file.name should be run with. Each arg is an + IP address. + + + + + net.topology.table.file.name + + The file name for a topology file, which is used when the + net.topology.node.switch.mapping.impl property is set to + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.net.TableMapping. The file format is a two column text + file, with columns separated by whitespace. The first column is a DNS or + IP address and the second column specifies the rack where the address maps. + If no entry corresponding to a host in the cluster is found, then + /default-rack is assumed. + + + + + + file.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + file.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + file.stream-buffer-size + + + + file.client-write-packet-size + 65536 + Packet size for clients to write + + + + file.blocksize + 67108864 + Block size + + + + file.replication + 1 + Replication factor + + + + + + s3.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3.stream-buffer-size + + + + s3.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3.blocksize + 67108864 + Block size + + + + s3.replication + 3 + Replication factor + + + + + + s3native.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3native.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3native.stream-buffer-size + + + + s3native.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3native.blocksize + 67108864 + Block size + + + + s3native.replication + 3 + Replication factor + + + + + ftp.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + ftp.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + ftp.stream-buffer-size + + + + ftp.client-write-packet-size + 65536 + Packet size for clients to write + + + + ftp.blocksize + 67108864 + Block size + + + + ftp.replication + 3 + Replication factor + + + + + + tfile.io.chunk.size + 1048576 + + Value chunk size in bytes. Default to + 1MB. Values of the length less than the chunk size is + guaranteed to have known value length in read time (See also + TFile.Reader.Scanner.Entry.isValueLengthKnown()). + + + + + tfile.fs.output.buffer.size + 262144 + + Buffer size used for FSDataOutputStream in bytes. + + + + + tfile.fs.input.buffer.size + 262144 + + Buffer size used for FSDataInputStream in bytes. + + + + + + + hadoop.http.authentication.type + simple + + Defines authentication used for Oozie HTTP endpoint. + Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME# + + + + + hadoop.http.authentication.token.validity + 36000 + + Indicates how long (in seconds) an authentication token is valid before it has + to be renewed. + + + + + hadoop.http.authentication.signature.secret.file + ${user.home}/hadoop-http-auth-signature-secret + + The signature secret for signing the authentication tokens. + The same secret should be used for JT/NN/DN/TT configurations. + + + + + hadoop.http.authentication.cookie.domain + + + The domain to use for the HTTP cookie that stores the authentication token. + In order to authentiation to work correctly across all Hadoop nodes web-consoles + the domain must be correctly set. + IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings. + For this setting to work properly all nodes in the cluster must be configured + to generate URLs with hostname.domain names on it. + + + + + hadoop.http.authentication.simple.anonymous.allowed + true + + Indicates if anonymous requests are allowed when using 'simple' authentication. + + + + + hadoop.http.authentication.kerberos.principal + HTTP/_HOST@LOCALHOST + + Indicates the Kerberos principal to be used for HTTP endpoint. + The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification. + + + + + hadoop.http.authentication.kerberos.keytab + ${user.home}/hadoop.keytab + + Location of the keytab file with the credentials for the principal. + Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop. + + + + + + Enable/disable the cross-origin (CORS) filter. + hadoop.http.cross-origin.enabled + false + + + + Comma separated list of origins that are allowed for web + services needing cross-origin (CORS) support. Wildcards (*) and patterns + allowed + hadoop.http.cross-origin.allowed-origins + * + + + + Comma separated list of methods that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-methods + GET,POST,HEAD + + + + Comma separated list of headers that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-headers + X-Requested-With,Content-Type,Accept,Origin + + + + The number of seconds a pre-flighted request can be cached + for web services needing cross-origin (CORS) support. + hadoop.http.cross-origin.max-age + 1800 + + + + dfs.ha.fencing.methods + + + List of fencing methods to use for service fencing. May contain + builtin methods (eg shell and sshfence) or user-defined method. + + + + + dfs.ha.fencing.ssh.connect-timeout + 30000 + + SSH connection timeout, in milliseconds, to use with the builtin + sshfence fencer. + + + + + dfs.ha.fencing.ssh.private-key-files + + + The SSH private key files to use with the builtin sshfence fencer. + + + + + + + + The user name to filter as, on static web filters + while rendering content. An example use is the HDFS + web UI (user to be used for browsing files). + + hadoop.http.staticuser.user + dr.who + + + + ha.zookeeper.quorum + + A list of ZooKeeper server addresses, separated by commas, that are + to be used by the ZKFailoverController in automatic failover. + + + + + ha.zookeeper.session-timeout.ms + 5000 + + The session timeout to use when the ZKFC connects to ZooKeeper. + Setting this value to a lower value implies that server crashes + will be detected more quickly, but risks triggering failover too + aggressively in the case of a transient error or network blip. + + + + + ha.zookeeper.parent-znode + /hadoop-ha + + The ZooKeeper znode under which the ZK failover controller stores + its information. Note that the nameservice ID is automatically + appended to this znode, so it is not normally necessary to + configure this, even in a federated environment. + + + + + ha.zookeeper.acl + world:anyone:rwcda + + A comma-separated list of ZooKeeper ACLs to apply to the znodes + used by automatic failover. These ACLs are specified in the same + format as used by the ZooKeeper CLI. + + If the ACL itself contains secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + ha.zookeeper.auth + + + A comma-separated list of ZooKeeper authentications to add when + connecting to ZooKeeper. These are specified in the same format + as used by the "addauth" command in the ZK CLI. It is + important that the authentications specified here are sufficient + to access znodes with the ACL specified in ha.zookeeper.acl. + + If the auths contain secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + + + hadoop.ssl.keystores.factory.class + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory + + The keystores factory to use for retrieving certificates. + + + + + hadoop.ssl.require.client.cert + false + Whether client certificates are required + + + + hadoop.ssl.hostname.verifier + DEFAULT + + The hostname verifier to provide for HttpsURLConnections. + Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and + ALLOW_ALL + + + + + hadoop.ssl.server.conf + ssl-server.xml + + Resource file from which ssl server keystore information will be extracted. + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.client.conf + ssl-client.xml + + Resource file from which ssl client keystore information will be extracted + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.enabled + false + + Deprecated. Use dfs.http.policy and yarn.http.policy instead. + + + + + hadoop.ssl.enabled.protocols + TLSv1 + + Protocols supported by the ssl. + + + + + hadoop.jetty.logs.serve.aliases + true + + Enable/Disable aliases serving from jetty + + + + + fs.permissions.umask-mode + 022 + + The umask used when creating files and directories. + Can be in octal or in symbolic. Examples are: + "022" (octal for u=rwx,g=r-x,o=r-x in symbolic), + or "u=rwx,g=rwx,o=" (symbolic for 007 in octal). + + + + + + + ha.health-monitor.connect-retry-interval.ms + 1000 + + How often to retry connecting to the service. + + + + + ha.health-monitor.check-interval.ms + 1000 + + How often to check the service. + + + + + ha.health-monitor.sleep-after-disconnect.ms + 1000 + + How long to sleep after an unexpected RPC error. + + + + + ha.health-monitor.rpc-timeout.ms + 45000 + + Timeout for the actual monitorHealth() calls. + + + + + ha.failover-controller.new-active.rpc-timeout.ms + 60000 + + Timeout that the FC waits for the new active to become active + + + + + ha.failover-controller.graceful-fence.rpc-timeout.ms + 5000 + + Timeout that the FC waits for the old active to go to standby + + + + + ha.failover-controller.graceful-fence.connection.retries + 1 + + FC connection retries for graceful fencing + + + + + ha.failover-controller.cli-check.rpc-timeout.ms + 20000 + + Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState + + + + + ipc.client.fallback-to-simple-auth-allowed + false + + When a client is configured to attempt a secure connection, but attempts to + connect to an insecure server, that server may instruct the client to + switch to SASL SIMPLE (unsecure) authentication. This setting controls + whether or not the client will accept this instruction from the server. + When false (the default), the client will not allow the fallback to SIMPLE + authentication, and will abort the connection. + + + + + fs.client.resolve.remote.symlinks + true + + Whether to resolve symlinks when accessing a remote Hadoop filesystem. + Setting this to false causes an exception to be thrown upon encountering + a symlink. This setting does not apply to local filesystems, which + automatically resolve local symlinks. + + + + + nfs.exports.allowed.hosts + * rw + + By default, the export can be mounted by any client. The value string + contains machine name and access privilege, separated by whitespace + characters. The machine name format can be a single host, a Java regular + expression, or an IPv4 address. The access privilege uses rw or ro to + specify read/write or read-only access of the machines to exports. If the + access privilege is not provided, the default is read-only. Entries are separated by ";". + For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;". + Only the NFS gateway needs to restart after this property is updated. + + + + + hadoop.user.group.static.mapping.overrides + dr.who=; + + Static mapping of user to groups. This will override the groups if + available in the system for the specified user. In otherwords, groups + look-up will not happen for these users, instead groups mapped in this + configuration will be used. + Mapping should be in this format. + user1=group1,group2;user2=;user3=group2; + Default, "dr.who=;" will consider "dr.who" as user without groups. + + + + + rpc.metrics.quantile.enable + false + + Setting this property to true and rpc.metrics.percentiles.intervals + to a comma-separated list of the granularity in seconds, the + 50/75/90/95/99th percentile latency for rpc queue/processing time in + milliseconds are added to rpc metrics. + + + + + rpc.metrics.percentiles.intervals + + + A comma-separated list of the granularity in seconds for the metrics which + describe the 50/75/90/95/99th percentile latency for rpc queue/processing + time. The metrics are outputted if rpc.metrics.quantile.enable is set to + true. + + + + + hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE + + + The prefix for a given crypto codec, contains a comma-separated + list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE). + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.codec.classes.aes.ctr.nopadding + org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop.crypto.JceAesCtrCryptoCodec + + Comma-separated list of crypto codec implementations for AES/CTR/NoPadding. + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.cipher.suite + AES/CTR/NoPadding + + Cipher suite for crypto codec. + + + + + hadoop.security.crypto.jce.provider + + + The JCE provider name used in CryptoCodec. + + + + + hadoop.security.crypto.buffer.size + 8192 + + The buffer size used by CryptoInputStream and CryptoOutputStream. + + + + + hadoop.security.java.secure.random.algorithm + SHA1PRNG + + The java secure random algorithm. + + + + + hadoop.security.secure.random.impl + + + Implementation of secure random. + + + + + hadoop.security.random.device.file.path + /dev/urandom + + OS security random device file path. + + + + + hadoop.security.key.provider.path + + The KeyProvider to use when managing zone keys, and interacting with + encryption keys when reading and writing to an encryption zone. + + + + + fs.har.impl.disable.cache + true + Don't cache 'har' filesystem instances. + + + + + hadoop.security.kms.client.authentication.retry-count + 1 + + Number of time to retry connecting to KMS on authentication failure + + + + hadoop.security.kms.client.encrypted.key.cache.size + 500 + + Size of the EncryptedKeyVersion cache Queue for each key + + + + hadoop.security.kms.client.encrypted.key.cache.low-watermark + 0.3f + + If size of the EncryptedKeyVersion cache Queue falls below the + low watermark, this cache queue will be scheduled for a refill + + + + hadoop.security.kms.client.encrypted.key.cache.num.refill.threads + 2 + + Number of threads to use for refilling depleted EncryptedKeyVersion + cache Queues + + + + hadoop.security.kms.client.encrypted.key.cache.expiry + 43200000 + + Cache expiry time for a Key, after which the cache Queue for this + key will be dropped. Default = 12hrs + + + + + ipc.server.max.connections + 0 + The maximum number of concurrent connections a server is allowed + to accept. If this limit is exceeded, incoming connections will first fill + the listen queue and then may go to an OS-specific listen overflow queue. + The client may fail or timeout, but the server can avoid running out of file + descriptors using this feature. 0 means no limit. + + + + + + + + + Is the registry enabled in the YARN Resource Manager? + + If true, the YARN RM will, as needed. + create the user and system paths, and purge + service records when containers, application attempts + and applications complete. + + If false, the paths must be created by other means, + and no automatic cleanup of service records will take place. + + hadoop.registry.rm.enabled + false + + + + + The root zookeeper node for the registry + + hadoop.registry.zk.root + /registry + + + + + Zookeeper session timeout in milliseconds + + hadoop.registry.zk.session.timeout.ms + 60000 + + + + + Zookeeper connection timeout in milliseconds + + hadoop.registry.zk.connection.timeout.ms + 15000 + + + + + Zookeeper connection retry count before failing + + hadoop.registry.zk.retry.times + 5 + + + + + + hadoop.registry.zk.retry.interval.ms + 1000 + + + + + Zookeeper retry limit in milliseconds, during + exponential backoff. + + This places a limit even + if the retry times and interval limit, combined + with the backoff policy, result in a long retry + period + + hadoop.registry.zk.retry.ceiling.ms + 60000 + + + + + List of hostname:port pairs defining the + zookeeper quorum binding for the registry + + hadoop.registry.zk.quorum + localhost:2181 + + + + + Key to set if the registry is secure. Turning it on + changes the permissions policy from "open access" + to restrictions on kerberos with the option of + a user adding one or more auth key pairs down their + own tree. + + hadoop.registry.secure + false + + + + + A comma separated list of Zookeeper ACL identifiers with + system access to the registry in a secure cluster. + + These are given full access to all entries. + + If there is an "@" at the end of a SASL entry it + instructs the registry client to append the default kerberos domain. + + hadoop.registry.system.acls + sasl:yarn@, sasl:mapred@, sasl:hdfs@ + + + + + The kerberos realm: used to set the realm of + system principals which do not declare their realm, + and any other accounts that need the value. + + If empty, the default realm of the running process + is used. + + If neither are known and the realm is needed, then the registry + service/client will fail. + + hadoop.registry.kerberos.realm + + + + + + Key to define the JAAS context. Used in secure + mode + + hadoop.registry.jaas.context + Client + + + + + Enable hdfs shell commands to display warnings if (fs.defaultFS) property + is not set. + + hadoop.shell.missing.defaultFs.warning + false + + + + hadoop.shell.safely.delete.limit.num.files + 100 + Used by -safely option of hadoop fs shell -rm command to avoid + accidental deletion of large directories. When enabled, the -rm command + requires confirmation if the number of files to be deleted is greater than + this limit. The default limit is 100 files. The warning is disabled if + the limit is 0 or the -safely is not specified in -rm command. + + + + + fs.client.htrace.sampler.classes + + The class names of the HTrace Samplers to use for Hadoop + filesystem clients. + + + + + hadoop.htrace.span.receiver.classes + + The class names of the Span Receivers to use for Hadoop. + + + + + hadoop.caller.context.enabled + false + When the feature is enabled, additional fields are written into + name-node audit log records for auditing coarse granularity operations. + + + + hadoop.caller.context.max.size + 128 + The maximum bytes a caller context string can have. If the + passed caller context is longer than this maximum bytes, client will + truncate it before sending to server. Note that the server may have a + different maximum size, and will truncate the caller context to the + maximum size it allows. + + + + hadoop.caller.context.signature.max.size + 40 + + The caller's signature (optional) is for offline validation. If the + signature exceeds the maximum allowed bytes in server, the caller context + will be abandoned, in which case the caller context will not be recorded + in audit logs. + + + + diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml b/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml new file mode 100644 index 0000000000000..b875e97be2a29 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml @@ -0,0 +1,2312 @@ + + + + + + + + + + + + + + + hadoop.common.configuration.version + 0.23.0 + version of this configuration file + + + + hadoop.tmp.dir + /tmp/hadoop-${user.name} + A base for other temporary directories. + + + + io.native.lib.available + true + Controls whether to use native libraries for bz2 and zlib + compression codecs or not. The property does not control any other native + libraries. + + + + + hadoop.http.filter.initializers + org.apache.hadoop.http.lib.StaticUserWebFilter + A comma separated list of class names. Each class in the list + must extend org.apache.hadoop.http.FilterInitializer. The corresponding + Filter will be initialized. Then, the Filter will be applied to all user + facing jsp and servlet web pages. The ordering of the list defines the + ordering of the filters. + + + + + + hadoop.security.authorization + false + Is service-level authorization enabled? + + + + hadoop.security.instrumentation.requires.admin + false + + Indicates if administrator ACLs are required to access + instrumentation servlets (JMX, METRICS, CONF, STACKS). + + + + + hadoop.security.authentication + simple + Possible values are simple (no authentication), and kerberos + + + + + hadoop.security.group.mapping + org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback + + Class for user to group mapping (get groups for a given user) for ACL. + The default implementation, + org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback, + will determine if the Java Native Interface (JNI) is available. If JNI is + available the implementation will use the API within hadoop to resolve a + list of groups for a user. If JNI is not available then the shell + implementation, ShellBasedUnixGroupsMapping, is used. This implementation + shells out to the Linux/Unix environment with the + bash -c groups command to resolve a list of groups for a user. + + + + + hadoop.security.dns.interface + + The name of the Network Interface from which the service should determine + its host name for Kerberos login. e.g. eth2. In a multi-homed environment, + the setting can be used to affect the _HOST subsitution in the service + Kerberos principal. If this configuration value is not set, the service + will use its default hostname as returned by + InetAddress.getLocalHost().getCanonicalHostName(). + + Most clusters will not require this setting. + + + + + hadoop.security.dns.nameserver + + The host name or IP address of the name server (DNS) which a service Node + should use to determine its own host name for Kerberos Login. Requires + hadoop.security.dns.interface. + + Most clusters will not require this setting. + + + + + hadoop.security.dns.log-slow-lookups.enabled + false + + Time name lookups (via SecurityUtil) and log them if they exceed the + configured threshold. + + + + + hadoop.security.dns.log-slow-lookups.threshold.ms + 1000 + + If slow lookup logging is enabled, this threshold is used to decide if a + lookup is considered slow enough to be logged. + + + + + hadoop.security.groups.cache.secs + 300 + + This is the config controlling the validity of the entries in the cache + containing the user->group mapping. When this duration has expired, + then the implementation of the group mapping provider is invoked to get + the groups of the user and then cached back. + + + + + hadoop.security.groups.negative-cache.secs + 30 + + Expiration time for entries in the the negative user-to-group mapping + caching, in seconds. This is useful when invalid users are retrying + frequently. It is suggested to set a small value for this expiration, since + a transient error in group lookup could temporarily lock out a legitimate + user. + + Set this to zero or negative value to disable negative user-to-group caching. + + + + + hadoop.security.groups.cache.warn.after.ms + 5000 + + If looking up a single user to group takes longer than this amount of + milliseconds, we will log a warning message. + + + + + hadoop.security.groups.cache.background.reload + false + + Whether to reload expired user->group mappings using a background thread + pool. If set to true, a pool of + hadoop.security.groups.cache.background.reload.threads is created to + update the cache in the background. + + + + + hadoop.security.groups.cache.background.reload.threads + 3 + + Only relevant if hadoop.security.groups.cache.background.reload is true. + Controls the number of concurrent background user->group cache entry + refreshes. Pending refresh requests beyond this value are queued and + processed when a thread is free. + + + + + hadoop.security.group.mapping.ldap.connection.timeout.ms + 60000 + + This property is the connection timeout (in milliseconds) for LDAP + operations. If the LDAP provider doesn't establish a connection within the + specified period, it will abort the connect attempt. Non-positive value + means no LDAP connection timeout is specified in which case it waits for the + connection to establish until the underlying network times out. + + + + + hadoop.security.group.mapping.ldap.read.timeout.ms + 60000 + + This property is the read timeout (in milliseconds) for LDAP + operations. If the LDAP provider doesn't get a LDAP response within the + specified period, it will abort the read attempt. Non-positive value + means no read timeout is specified in which case it waits for the response + infinitely. + + + + + hadoop.security.group.mapping.ldap.url + + + The URL of the LDAP server to use for resolving user groups when using + the LdapGroupsMapping user to group mapping. + + + + + hadoop.security.group.mapping.ldap.ssl + false + + Whether or not to use SSL when connecting to the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore + + + File path to the SSL keystore that contains the SSL certificate required + by the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore.password.file + + + The path to a file containing the password of the LDAP SSL keystore. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.bind.user + + + The distinguished name of the user to bind as when connecting to the LDAP + server. This may be left blank if the LDAP server supports anonymous binds. + + + + + hadoop.security.group.mapping.ldap.bind.password.file + + + The path to a file containing the password of the bind user. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.base + + + The search base for the LDAP connection. This is a distinguished name, + and will typically be the root of the LDAP directory. + + + + + hadoop.security.group.mapping.ldap.search.filter.user + (&(objectClass=user)(sAMAccountName={0})) + + An additional filter to use when searching for LDAP users. The default will + usually be appropriate for Active Directory installations. If connecting to + an LDAP server with a non-AD schema, this should be replaced with + (&(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to + denote where the username fits into the filter. + + If the LDAP server supports posixGroups, Hadoop can enable the feature by + setting the value of this property to "posixAccount" and the value of + the hadoop.security.group.mapping.ldap.search.filter.group property to + "posixGroup". + + + + + hadoop.security.group.mapping.ldap.search.filter.group + (objectClass=group) + + An additional filter to use when searching for LDAP groups. This should be + changed when resolving groups against a non-Active Directory installation. + + See the description of hadoop.security.group.mapping.ldap.search.filter.user + to enable posixGroups support. + + + + + hadoop.security.group.mapping.ldap.search.attr.member + member + + The attribute of the group object that identifies the users that are + members of the group. The default will usually be appropriate for + any LDAP installation. + + + + + hadoop.security.group.mapping.ldap.search.attr.group.name + cn + + The attribute of the group object that identifies the group name. The + default will usually be appropriate for all LDAP systems. + + + + + hadoop.security.group.mapping.ldap.posix.attr.uid.name + uidNumber + + The attribute of posixAccount to use when groups for membership. + Mostly useful for schemas wherein groups have memberUids that use an + attribute other than uidNumber. + + + + + hadoop.security.group.mapping.ldap.posix.attr.gid.name + gidNumber + + The attribute of posixAccount indicating the group id. + + + + + hadoop.security.group.mapping.ldap.directory.search.timeout + 10000 + + The attribute applied to the LDAP SearchControl properties to set a + maximum time limit when searching and awaiting a result. + Set to 0 if infinite wait period is desired. + Default is 10 seconds. Units in milliseconds. + + + + + hadoop.security.group.mapping.providers + + + Comma separated of names of other providers to provide user to group + mapping. Used by CompositeGroupsMapping. + + + + + hadoop.security.group.mapping.providers.combined + true + + true or false to indicate whether groups from the providers are combined or + not. The default value is true. If true, then all the providers will be + tried to get groups and all the groups are combined to return as the final + results. Otherwise, providers are tried one by one in the configured list + order, and if any groups are retrieved from any provider, then the groups + will be returned without trying the left ones. + + + + + hadoop.security.service.user.name.key + + + For those cases where the same RPC protocol is implemented by multiple + servers, this configuration is required for specifying the principal + name to use for the service when the client wishes to make an RPC call. + + + + + + hadoop.security.uid.cache.secs + 14400 + + This is the config controlling the validity of the entries in the cache + containing the userId to userName and groupId to groupName used by + NativeIO getFstat(). + + + + + hadoop.rpc.protection + authentication + A comma-separated list of protection values for secured sasl + connections. Possible values are authentication, integrity and privacy. + authentication means authentication only and no integrity or privacy; + integrity implies authentication and integrity are enabled; and privacy + implies all of authentication, integrity and privacy are enabled. + hadoop.security.saslproperties.resolver.class can be used to override + the hadoop.rpc.protection for a connection at the server side. + + + + + hadoop.security.saslproperties.resolver.class + + SaslPropertiesResolver used to resolve the QOP used for a + connection. If not specified, the full set of values specified in + hadoop.rpc.protection is used while determining the QOP used for the + connection. If a class is specified, then the QOP values returned by + the class will be used while determining the QOP used for the connection. + + + + + hadoop.security.sensitive-config-keys + secret$,password$,ssl.keystore.pass$,fs.s3.*[Ss]ecret.?[Kk]ey,fs.azure.account.key.*,dfs.webhdfs.oauth2.[a-z]+.token,hadoop.security.sensitive-config-keys + A comma-separated list of regular expressions to match against + configuration keys that should be redacted where appropriate, for + example, when logging modified properties during a reconfiguration, + private credentials should not be logged. + + + + + hadoop.workaround.non.threadsafe.getpwuid + true + Some operating systems or authentication modules are known to + have broken implementations of getpwuid_r and getpwgid_r, such that these + calls are not thread-safe. Symptoms of this problem include JVM crashes + with a stack trace inside these functions. If your system exhibits this + issue, enable this configuration parameter to include a lock around the + calls as a workaround. + + An incomplete list of some systems known to have this issue is available + at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations + + + + + hadoop.kerberos.kinit.command + kinit + Used to periodically renew Kerberos credentials when provided + to Hadoop. The default setting assumes that kinit is in the PATH of users + running the Hadoop client. Change this to the absolute path to kinit if this + is not the case. + + + + + hadoop.kerberos.min.seconds.before.relogin + 60 + The minimum time between relogin attempts for Kerberos, in + seconds. + + + + + hadoop.security.auth_to_local + + Maps kerberos principals to local user names + + + + + io.file.buffer.size + 4096 + The size of buffer for use in sequence files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + io.bytes.per.checksum + 512 + The number of bytes per checksum. Must not be larger than + io.file.buffer.size. + + + + io.skip.checksum.errors + false + If true, when a checksum error is encountered while + reading a sequence file, entries are skipped, instead of throwing an + exception. + + + + io.compression.codecs + + A comma-separated list of the compression codec classes that can + be used for compression/decompression. In addition to any classes specified + with this property (which take precedence), codec classes on the classpath + are discovered using a Java ServiceLoader. + + + + io.compression.codec.bzip2.library + system-native + The native-code library to be used for compression and + decompression by the bzip2 codec. This library could be specified + either by by name or the full pathname. In the former case, the + library is located by the dynamic linker, usually searching the + directories specified in the environment variable LD_LIBRARY_PATH. + + The value of "system-native" indicates that the default system + library should be used. To indicate that the algorithm should + operate entirely in Java, specify "java-builtin". + + + + io.serializations + org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization + A list of serialization classes that can be used for + obtaining serializers and deserializers. + + + + io.seqfile.local.dir + ${hadoop.tmp.dir}/io/local + The local directory where sequence file stores intermediate + data files during merge. May be a comma-separated list of + directories on different devices in order to spread disk i/o. + Directories that do not exist are ignored. + + + + + io.map.index.skip + 0 + Number of index entries to skip between each entry. + Zero by default. Setting this to values larger than zero can + facilitate opening large MapFiles using less memory. + + + + io.map.index.interval + 128 + + MapFile consist of two files - data file (tuples) and index file + (keys). For every io.map.index.interval records written in the + data file, an entry (record-key, data-file-position) is written + in the index file. This is to allow for doing binary search later + within the index file to look up records by their keys and get their + closest positions in the data file. + + + + + + + fs.defaultFS + file:/// + The name of the default file system. A URI whose + scheme and authority determine the FileSystem implementation. The + uri's scheme determines the config property (fs.SCHEME.impl) naming + the FileSystem implementation class. The uri's authority is used to + determine the host, port, etc. for a filesystem. + + + + fs.default.name + file:/// + Deprecated. Use (fs.defaultFS) property + instead + + + + fs.trash.interval + 0 + Number of minutes after which the checkpoint + gets deleted. If zero, the trash feature is disabled. + This option may be configured both on the server and the + client. If trash is disabled server side then the client + side configuration is checked. If trash is enabled on the + server side then the value configured on the server is + used and the client configuration value is ignored. + + + + + fs.trash.checkpoint.interval + 0 + Number of minutes between trash checkpoints. + Should be smaller or equal to fs.trash.interval. If zero, + the value is set to the value of fs.trash.interval. + Every time the checkpointer runs it creates a new checkpoint + out of current and removes checkpoints created more than + fs.trash.interval minutes ago. + + + + + fs.protected.directories + + A comma-separated list of directories which cannot + be deleted even by the superuser unless they are empty. This + setting can be used to guard important system directories + against accidental deletion due to administrator error. + + + + + fs.AbstractFileSystem.file.impl + org.apache.hadoop.fs.local.LocalFs + The AbstractFileSystem for file: uris. + + + + fs.AbstractFileSystem.har.impl + org.apache.hadoop.fs.HarFs + The AbstractFileSystem for har: uris. + + + + fs.AbstractFileSystem.hdfs.impl + org.apache.hadoop.fs.Hdfs + The FileSystem for hdfs: uris. + + + + fs.AbstractFileSystem.viewfs.impl + org.apache.hadoop.fs.viewfs.ViewFs + The AbstractFileSystem for view file system for viewfs: uris + (ie client side mount table:). + + + + fs.AbstractFileSystem.ftp.impl + org.apache.hadoop.fs.ftp.FtpFs + The FileSystem for Ftp: uris. + + + + fs.AbstractFileSystem.webhdfs.impl + org.apache.hadoop.fs.WebHdfs + The FileSystem for webhdfs: uris. + + + + fs.AbstractFileSystem.swebhdfs.impl + org.apache.hadoop.fs.SWebHdfs + The FileSystem for swebhdfs: uris. + + + + fs.ftp.host + 0.0.0.0 + FTP filesystem connects to this server + + + + fs.ftp.host.port + 21 + + FTP filesystem connects to fs.ftp.host on this port + + + + + fs.df.interval + 60000 + Disk usage statistics refresh interval in msec. + + + + fs.du.interval + 600000 + File space usage statistics refresh interval in msec. + + + + fs.s3.block.size + 67108864 + Block size to use when writing files to S3. + + + + fs.s3.buffer.dir + ${hadoop.tmp.dir}/s3 + Determines where on the local filesystem the s3:/s3n: filesystem + should store files before sending them to S3 + (or after retrieving them from S3). + + + + + fs.s3.maxRetries + 4 + The maximum number of retries for reading or writing files to S3, + before we signal failure to the application. + + + + + fs.s3.sleepTimeSeconds + 10 + The number of seconds to sleep between each S3 retry. + + + + + fs.automatic.close + true + By default, FileSystem instances are automatically closed at program + exit using a JVM shutdown hook. Setting this property to false disables this + behavior. This is an advanced option that should only be used by server applications + requiring a more carefully orchestrated shutdown sequence. + + + + + fs.s3n.block.size + 67108864 + Block size to use when reading files using the native S3 + filesystem (s3n: URIs). + + + + fs.s3n.multipart.uploads.enabled + false + Setting this property to true enables multiple uploads to + native S3 filesystem. When uploading a file, it is split into blocks + if the size is larger than fs.s3n.multipart.uploads.block.size. + + + + + fs.s3n.multipart.uploads.block.size + 67108864 + The block size for multipart uploads to native S3 filesystem. + Default size is 64MB. + + + + + fs.s3n.multipart.copy.block.size + 5368709120 + The block size for multipart copy in native S3 filesystem. + Default size is 5GB. + + + + + fs.s3n.server-side-encryption-algorithm + + Specify a server-side encryption algorithm for S3. + Unset by default, and the only other currently allowable value is AES256. + + + + + fs.s3a.access.key + AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication. + + + + fs.s3a.secret.key + AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication. + + + + fs.s3a.aws.credentials.provider + + Comma-separated class names of credential provider classes which implement + com.amazonaws.auth.AWSCredentialsProvider. + + These are loaded and queried in sequence for a valid set of credentials. + Each listed class must implement one of the following means of + construction, which are attempted in order: + 1. a public constructor accepting java.net.URI and + org.apache.hadoop.conf.Configuration, + 2. a public static method named getInstance that accepts no + arguments and returns an instance of + com.amazonaws.auth.AWSCredentialsProvider, or + 3. a public default constructor. + + Specifying + org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider + allows + anonymous access to a publicly accessible S3 bucket without any credentials. + Please note that allowing anonymous access to an S3 bucket compromises + security and therefore is unsuitable for most use cases. It can be useful + for accessing public data sets without requiring AWS credentials. + + If unspecified, then the default list of credential provider classes, + queried in sequence, is: + 1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider: + supports static + configuration of AWS access key ID and secret access key. See also + fs.s3a.access.key and fs.s3a.secret.key. + 2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports + configuration of AWS access key ID and secret access key in + environment variables named AWS_ACCESS_KEY_ID and + AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK. + 3. + org.apache.hadoop.fs.s3a.SharedInstanceProfileCredentialsProvider: + a shared instance of + com.amazonaws.auth.InstanceProfileCredentialsProvider from the AWS + SDK, which supports use of instance profile credentials if running + in an EC2 VM. Using this shared instance potentially reduces load + on the EC2 instance metadata service for multi-threaded + applications. + + + + + fs.s3a.session.token + Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider + as one of the providers. + + + + + fs.s3a.security.credential.provider.path + + + Optional comma separated list of credential providers, a list + which is prepended to that set in hadoop.security.credential.provider.path + + + + + fs.s3a.connection.maximum + 15 + Controls the maximum number of simultaneous connections to S3. + + + + fs.s3a.connection.ssl.enabled + true + Enables or disables SSL connections to S3. + + + + fs.s3a.endpoint + AWS S3 endpoint to connect to. An up-to-date list is + provided in the AWS Documentation: regions and endpoints. Without this + property, the standard region (s3.amazonaws.com) is assumed. + + + + + fs.s3a.path.style.access + false + Enable S3 path style access ie disabling the default virtual hosting behaviour. + Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting. + + + + + fs.s3a.proxy.host + Hostname of the (optional) proxy server for S3 connections. + + + + fs.s3a.proxy.port + Proxy server port. If this property is not set + but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with + the value of fs.s3a.connection.ssl.enabled). + + + + fs.s3a.proxy.username + Username for authenticating with proxy server. + + + + fs.s3a.proxy.password + Password for authenticating with proxy server. + + + + fs.s3a.proxy.domain + Domain for authenticating with proxy server. + + + + fs.s3a.proxy.workstation + Workstation for authenticating with proxy server. + + + + fs.s3a.attempts.maximum + 20 + How many times we should retry commands on transient errors. + + + + fs.s3a.connection.establish.timeout + 5000 + Socket connection setup timeout in milliseconds. + + + + fs.s3a.connection.timeout + 200000 + Socket connection timeout in milliseconds. + + + + fs.s3a.socket.send.buffer + 8192 + Socket send buffer hint to amazon connector. Represented in bytes. + + + + fs.s3a.socket.recv.buffer + 8192 + Socket receive buffer hint to amazon connector. Represented in bytes. + + + + fs.s3a.paging.maximum + 5000 + How many keys to request from S3 when doing + directory listings at a time. + + + + fs.s3a.threads.max + 10 + The total number of threads available in the filesystem for data + uploads *or any other queued filesystem operation*. + + + + fs.s3a.threads.keepalivetime + 60 + Number of seconds a thread can be idle before being + terminated. + + + + fs.s3a.max.total.tasks + 5 + The number of operations which can be queued for execution + + + + fs.s3a.multipart.size + 100M + How big (in bytes) to split upload or copy operations up into. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.multipart.threshold + 2147483647 + How big (in bytes) to split upload or copy operations up into. + This also controls the partition size in renamed files, as rename() involves + copying the source file(s). + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.multiobjectdelete.enable + true + When enabled, multiple single-object delete requests are replaced by + a single 'delete multiple objects'-request, reducing the number of requests. + Beware: legacy S3-compatible object stores might not support this request. + + + + + fs.s3a.acl.default + Set a canned ACL for newly created and copied objects. Value may be Private, + PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead, + or BucketOwnerFullControl. + + + + fs.s3a.multipart.purge + false + True if you want to purge existing multipart uploads that may not have been + completed/aborted correctly. The corresponding purge age is defined in + fs.s3a.multipart.purge.age. + If set, when the filesystem is instantiated then all outstanding uploads + older than the purge age will be terminated -across the entire bucket. + This will impact multipart uploads by other applications and users. so should + be used sparingly, with an age value chosen to stop failed uploads, without + breaking ongoing operations. + + + + + fs.s3a.multipart.purge.age + 86400 + Minimum age in seconds of multipart uploads to purge. + + + + + fs.s3a.server-side-encryption-algorithm + Specify a server-side encryption algorithm for s3a: file system. + Unset by default, and the only other currently allowable value is AES256. + + + + + fs.s3a.signing-algorithm + Override the default signing algorithm so legacy + implementations can still be used + + + + fs.s3a.block.size + 32M + Block size to use when reading files using s3a: file system. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.buffer.dir + ${hadoop.tmp.dir}/s3a + Comma separated list of directories that will be used to buffer file + uploads to. + + + + fs.s3a.fast.upload + false + + Use the incremental block-based fast upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + + + + + fs.s3a.fast.upload.buffer + disk + + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + + + + + fs.s3a.fast.upload.active.blocks + 4 + + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + + + + + fs.s3a.readahead.range + 64K + Bytes to read ahead during a seek() before closing and + re-opening the S3 HTTP connection. This option will be overridden if + any call to setReadahead() is made to an open stream. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + + + + + fs.s3a.user.agent.prefix + + + Sets a custom value that will be prepended to the User-Agent header sent in + HTTP requests to the S3 back-end by S3AFileSystem. The User-Agent header + always includes the Hadoop version number followed by a string generated by + the AWS SDK. An example is "User-Agent: Hadoop 2.8.0, aws-sdk-java/1.10.6". + If this optional property is set, then its value is prepended to create a + customized User-Agent. For example, if this configuration property was set + to "MyApp", then an example of the resulting User-Agent would be + "User-Agent: MyApp, Hadoop 2.8.0, aws-sdk-java/1.10.6". + + + + + fs.s3a.impl + org.apache.hadoop.fs.s3a.S3AFileSystem + The implementation class of the S3A Filesystem + + + + fs.AbstractFileSystem.s3a.impl + org.apache.hadoop.fs.s3a.S3A + The implementation class of the S3A AbstractFileSystem. + + + + io.seqfile.compress.blocksize + 1000000 + The minimum block size for compression in block compressed + SequenceFiles. + + + + + io.mapfile.bloom.size + 1048576 + The size of BloomFilter-s used in BloomMapFile. Each time this many + keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter). + Larger values minimize the number of filters, which slightly increases the performance, + but may waste too much space if the total number of keys is usually much smaller + than this number. + + + + + io.mapfile.bloom.error.rate + 0.005 + The rate of false positives in BloomFilter-s used in BloomMapFile. + As this value decreases, the size of BloomFilter-s increases exponentially. This + value is the probability of encountering false positives (default is 0.5%). + + + + + hadoop.util.hash.type + murmur + The default implementation of Hash. Currently this can take one of the + two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash. + + + + + + + + ipc.client.idlethreshold + 4000 + Defines the threshold number of connections after which + connections will be inspected for idleness. + + + + + ipc.client.kill.max + 10 + Defines the maximum number of clients to disconnect in one go. + + + + + ipc.client.connection.maxidletime + 10000 + The maximum time in msec after which a client will bring down the + connection to the server. + + + + + ipc.client.connect.max.retries + 10 + Indicates the number of retries a client will make to establish + a server connection. + + + + + ipc.client.connect.retry.interval + 1000 + Indicates the number of milliseconds a client will wait for + before retrying to establish a server connection. + + + + + ipc.client.connect.timeout + 20000 + Indicates the number of milliseconds a client will wait for the + socket to establish a server connection. + + + + + ipc.client.connect.max.retries.on.timeouts + 45 + Indicates the number of retries a client will make on socket timeout + to establish a server connection. + + + + + ipc.client.tcpnodelay + true + Use TCP_NODELAY flag to bypass Nagle's algorithm transmission delays. + + + + + ipc.client.low-latency + false + Use low-latency QoS markers for IPC connections. + + + + + ipc.client.ping + true + Send a ping to the server when timeout on reading the response, + if set to true. If no failure is detected, the client retries until at least + a byte is read or the time given by ipc.client.rpc-timeout.ms is passed. + + + + + ipc.ping.interval + 60000 + Timeout on waiting response from server, in milliseconds. + The client will send ping when the interval is passed without receiving bytes, + if ipc.client.ping is set to true. + + + + + ipc.client.rpc-timeout.ms + 0 + Timeout on waiting response from server, in milliseconds. + If ipc.client.ping is set to true and this rpc-timeout is greater than + the value of ipc.ping.interval, the effective value of the rpc-timeout is + rounded up to multiple of ipc.ping.interval. + + + + + ipc.server.listen.queue.size + 128 + Indicates the length of the listen queue for servers accepting + client connections. + + + + + ipc.server.log.slow.rpc + false + This setting is useful to troubleshoot performance issues for + various services. If this value is set to true then we log requests that + fall into 99th percentile as well as increment RpcSlowCalls counter. + + + + + ipc.maximum.data.length + 67108864 + This indicates the maximum IPC message length (bytes) that can be + accepted by the server. Messages larger than this value are rejected by the + immediately to avoid possible OOMs. This setting should rarely need to be + changed. + + + + + ipc.maximum.response.length + 134217728 + This indicates the maximum IPC message length (bytes) that can be + accepted by the client. Messages larger than this value are rejected + immediately to avoid possible OOMs. This setting should rarely need to be + changed. Set to 0 to disable. + + + + + + + hadoop.security.impersonation.provider.class + + A class which implements ImpersonationProvider interface, used to + authorize whether one user can impersonate a specific user. + If not specified, the DefaultImpersonationProvider will be used. + If a class is specified, then that class will be used to determine + the impersonation capability. + + + + + hadoop.rpc.socket.factory.class.default + org.apache.hadoop.net.StandardSocketFactory + Default SocketFactory to use. This parameter is expected to be + formatted as "package.FactoryClassName". + + + + + hadoop.rpc.socket.factory.class.ClientProtocol + + SocketFactory to use to connect to a DFS. If null or empty, use + hadoop.rpc.socket.class.default. This socket factory is also used by + DFSClient to create sockets to DataNodes. + + + + + + + hadoop.socks.server + + Address (host:port) of the SOCKS server to be used by the + SocksSocketFactory. + + + + + + net.topology.node.switch.mapping.impl + org.apache.hadoop.net.ScriptBasedMapping + The default implementation of the DNSToSwitchMapping. It + invokes a script specified in net.topology.script.file.name to resolve + node names. If the value for net.topology.script.file.name is not set, the + default value of DEFAULT_RACK is returned for all node names. + + + + + net.topology.impl + org.apache.hadoop.net.NetworkTopology + The default implementation of NetworkTopology which is classic three layer one. + + + + + net.topology.script.file.name + + The script name that should be invoked to resolve DNS names to + NetworkTopology names. Example: the script would take host.foo.bar as an + argument, and return /rack1 as the output. + + + + + net.topology.script.number.args + 100 + The max number of args that the script configured with + net.topology.script.file.name should be run with. Each arg is an + IP address. + + + + + net.topology.table.file.name + + The file name for a topology file, which is used when the + net.topology.node.switch.mapping.impl property is set to + org.apache.hadoop.net.TableMapping. The file format is a two column text + file, with columns separated by whitespace. The first column is a DNS or + IP address and the second column specifies the rack where the address maps. + If no entry corresponding to a host in the cluster is found, then + /default-rack is assumed. + + + + + + file.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + file.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + file.stream-buffer-size + + + + file.client-write-packet-size + 65536 + Packet size for clients to write + + + + file.blocksize + 67108864 + Block size + + + + file.replication + 1 + Replication factor + + + + + + s3.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3.stream-buffer-size + + + + s3.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3.blocksize + 67108864 + Block size + + + + s3.replication + 3 + Replication factor + + + + + + s3native.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3native.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3native.stream-buffer-size + + + + s3native.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3native.blocksize + 67108864 + Block size + + + + s3native.replication + 3 + Replication factor + + + + + ftp.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + ftp.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + ftp.stream-buffer-size + + + + ftp.client-write-packet-size + 65536 + Packet size for clients to write + + + + ftp.blocksize + 67108864 + Block size + + + + ftp.replication + 3 + Replication factor + + + + + + tfile.io.chunk.size + 1048576 + + Value chunk size in bytes. Default to + 1MB. Values of the length less than the chunk size is + guaranteed to have known value length in read time (See also + TFile.Reader.Scanner.Entry.isValueLengthKnown()). + + + + + tfile.fs.output.buffer.size + 262144 + + Buffer size used for FSDataOutputStream in bytes. + + + + + tfile.fs.input.buffer.size + 262144 + + Buffer size used for FSDataInputStream in bytes. + + + + + + + hadoop.http.authentication.type + simple + + Defines authentication used for Oozie HTTP endpoint. + Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME# + + + + + hadoop.http.authentication.token.validity + 36000 + + Indicates how long (in seconds) an authentication token is valid before it has + to be renewed. + + + + + hadoop.http.authentication.signature.secret.file + ${user.home}/hadoop-http-auth-signature-secret + + The signature secret for signing the authentication tokens. + The same secret should be used for JT/NN/DN/TT configurations. + + + + + hadoop.http.authentication.cookie.domain + + + The domain to use for the HTTP cookie that stores the authentication token. + In order to authentiation to work correctly across all Hadoop nodes web-consoles + the domain must be correctly set. + IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings. + For this setting to work properly all nodes in the cluster must be configured + to generate URLs with hostname.domain names on it. + + + + + hadoop.http.authentication.simple.anonymous.allowed + true + + Indicates if anonymous requests are allowed when using 'simple' authentication. + + + + + hadoop.http.authentication.kerberos.principal + HTTP/_HOST@LOCALHOST + + Indicates the Kerberos principal to be used for HTTP endpoint. + The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification. + + + + + hadoop.http.authentication.kerberos.keytab + ${user.home}/hadoop.keytab + + Location of the keytab file with the credentials for the principal. + Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop. + + + + + + Enable/disable the cross-origin (CORS) filter. + hadoop.http.cross-origin.enabled + false + + + + Comma separated list of origins that are allowed for web + services needing cross-origin (CORS) support. Wildcards (*) and patterns + allowed + hadoop.http.cross-origin.allowed-origins + * + + + + Comma separated list of methods that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-methods + GET,POST,HEAD + + + + Comma separated list of headers that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-headers + X-Requested-With,Content-Type,Accept,Origin + + + + The number of seconds a pre-flighted request can be cached + for web services needing cross-origin (CORS) support. + hadoop.http.cross-origin.max-age + 1800 + + + + dfs.ha.fencing.methods + + + List of fencing methods to use for service fencing. May contain + builtin methods (eg shell and sshfence) or user-defined method. + + + + + dfs.ha.fencing.ssh.connect-timeout + 30000 + + SSH connection timeout, in milliseconds, to use with the builtin + sshfence fencer. + + + + + dfs.ha.fencing.ssh.private-key-files + + + The SSH private key files to use with the builtin sshfence fencer. + + + + + + + + The user name to filter as, on static web filters + while rendering content. An example use is the HDFS + web UI (user to be used for browsing files). + + hadoop.http.staticuser.user + dr.who + + + + ha.zookeeper.quorum + + A list of ZooKeeper server addresses, separated by commas, that are + to be used by the ZKFailoverController in automatic failover. + + + + + ha.zookeeper.session-timeout.ms + 5000 + + The session timeout to use when the ZKFC connects to ZooKeeper. + Setting this value to a lower value implies that server crashes + will be detected more quickly, but risks triggering failover too + aggressively in the case of a transient error or network blip. + + + + + ha.zookeeper.parent-znode + /hadoop-ha + + The ZooKeeper znode under which the ZK failover controller stores + its information. Note that the nameservice ID is automatically + appended to this znode, so it is not normally necessary to + configure this, even in a federated environment. + + + + + ha.zookeeper.acl + world:anyone:rwcda + + A comma-separated list of ZooKeeper ACLs to apply to the znodes + used by automatic failover. These ACLs are specified in the same + format as used by the ZooKeeper CLI. + + If the ACL itself contains secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + ha.zookeeper.auth + + + A comma-separated list of ZooKeeper authentications to add when + connecting to ZooKeeper. These are specified in the same format + as used by the "addauth" command in the ZK CLI. It is + important that the authentications specified here are sufficient + to access znodes with the ACL specified in ha.zookeeper.acl. + + If the auths contain secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + + + hadoop.ssl.keystores.factory.class + org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory + + The keystores factory to use for retrieving certificates. + + + + + hadoop.ssl.require.client.cert + false + Whether client certificates are required + + + + hadoop.ssl.hostname.verifier + DEFAULT + + The hostname verifier to provide for HttpsURLConnections. + Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and + ALLOW_ALL + + + + + hadoop.ssl.server.conf + ssl-server.xml + + Resource file from which ssl server keystore information will be extracted. + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.client.conf + ssl-client.xml + + Resource file from which ssl client keystore information will be extracted + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.enabled + false + + Deprecated. Use dfs.http.policy and yarn.http.policy instead. + + + + + hadoop.ssl.enabled.protocols + TLSv1 + + Protocols supported by the ssl. + + + + + hadoop.jetty.logs.serve.aliases + true + + Enable/Disable aliases serving from jetty + + + + + fs.permissions.umask-mode + 022 + + The umask used when creating files and directories. + Can be in octal or in symbolic. Examples are: + "022" (octal for u=rwx,g=r-x,o=r-x in symbolic), + or "u=rwx,g=rwx,o=" (symbolic for 007 in octal). + + + + + + + ha.health-monitor.connect-retry-interval.ms + 1000 + + How often to retry connecting to the service. + + + + + ha.health-monitor.check-interval.ms + 1000 + + How often to check the service. + + + + + ha.health-monitor.sleep-after-disconnect.ms + 1000 + + How long to sleep after an unexpected RPC error. + + + + + ha.health-monitor.rpc-timeout.ms + 45000 + + Timeout for the actual monitorHealth() calls. + + + + + ha.failover-controller.new-active.rpc-timeout.ms + 60000 + + Timeout that the FC waits for the new active to become active + + + + + ha.failover-controller.graceful-fence.rpc-timeout.ms + 5000 + + Timeout that the FC waits for the old active to go to standby + + + + + ha.failover-controller.graceful-fence.connection.retries + 1 + + FC connection retries for graceful fencing + + + + + ha.failover-controller.cli-check.rpc-timeout.ms + 20000 + + Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState + + + + + ipc.client.fallback-to-simple-auth-allowed + false + + When a client is configured to attempt a secure connection, but attempts to + connect to an insecure server, that server may instruct the client to + switch to SASL SIMPLE (unsecure) authentication. This setting controls + whether or not the client will accept this instruction from the server. + When false (the default), the client will not allow the fallback to SIMPLE + authentication, and will abort the connection. + + + + + fs.client.resolve.remote.symlinks + true + + Whether to resolve symlinks when accessing a remote Hadoop filesystem. + Setting this to false causes an exception to be thrown upon encountering + a symlink. This setting does not apply to local filesystems, which + automatically resolve local symlinks. + + + + + nfs.exports.allowed.hosts + * rw + + By default, the export can be mounted by any client. The value string + contains machine name and access privilege, separated by whitespace + characters. The machine name format can be a single host, a Java regular + expression, or an IPv4 address. The access privilege uses rw or ro to + specify read/write or read-only access of the machines to exports. If the + access privilege is not provided, the default is read-only. Entries are separated by ";". + For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;". + Only the NFS gateway needs to restart after this property is updated. + + + + + hadoop.user.group.static.mapping.overrides + dr.who=; + + Static mapping of user to groups. This will override the groups if + available in the system for the specified user. In otherwords, groups + look-up will not happen for these users, instead groups mapped in this + configuration will be used. + Mapping should be in this format. + user1=group1,group2;user2=;user3=group2; + Default, "dr.who=;" will consider "dr.who" as user without groups. + + + + + rpc.metrics.quantile.enable + false + + Setting this property to true and rpc.metrics.percentiles.intervals + to a comma-separated list of the granularity in seconds, the + 50/75/90/95/99th percentile latency for rpc queue/processing time in + milliseconds are added to rpc metrics. + + + + + rpc.metrics.percentiles.intervals + + + A comma-separated list of the granularity in seconds for the metrics which + describe the 50/75/90/95/99th percentile latency for rpc queue/processing + time. The metrics are outputted if rpc.metrics.quantile.enable is set to + true. + + + + + hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE + + + The prefix for a given crypto codec, contains a comma-separated + list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE). + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.codec.classes.aes.ctr.nopadding + org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec + + Comma-separated list of crypto codec implementations for AES/CTR/NoPadding. + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.cipher.suite + AES/CTR/NoPadding + + Cipher suite for crypto codec. + + + + + hadoop.security.crypto.jce.provider + + + The JCE provider name used in CryptoCodec. + + + + + hadoop.security.crypto.buffer.size + 8192 + + The buffer size used by CryptoInputStream and CryptoOutputStream. + + + + + hadoop.security.java.secure.random.algorithm + SHA1PRNG + + The java secure random algorithm. + + + + + hadoop.security.secure.random.impl + + + Implementation of secure random. + + + + + hadoop.security.random.device.file.path + /dev/urandom + + OS security random device file path. + + + + + hadoop.security.key.provider.path + + The KeyProvider to use when managing zone keys, and interacting with + encryption keys when reading and writing to an encryption zone. + + + + + fs.har.impl.disable.cache + true + Don't cache 'har' filesystem instances. + + + + + hadoop.security.kms.client.authentication.retry-count + 1 + + Number of time to retry connecting to KMS on authentication failure + + + + hadoop.security.kms.client.encrypted.key.cache.size + 500 + + Size of the EncryptedKeyVersion cache Queue for each key + + + + hadoop.security.kms.client.encrypted.key.cache.low-watermark + 0.3f + + If size of the EncryptedKeyVersion cache Queue falls below the + low watermark, this cache queue will be scheduled for a refill + + + + hadoop.security.kms.client.encrypted.key.cache.num.refill.threads + 2 + + Number of threads to use for refilling depleted EncryptedKeyVersion + cache Queues + + + + hadoop.security.kms.client.encrypted.key.cache.expiry + 43200000 + + Cache expiry time for a Key, after which the cache Queue for this + key will be dropped. Default = 12hrs + + + + + ipc.server.max.connections + 0 + The maximum number of concurrent connections a server is allowed + to accept. If this limit is exceeded, incoming connections will first fill + the listen queue and then may go to an OS-specific listen overflow queue. + The client may fail or timeout, but the server can avoid running out of file + descriptors using this feature. 0 means no limit. + + + + + + + + + Is the registry enabled in the YARN Resource Manager? + + If true, the YARN RM will, as needed. + create the user and system paths, and purge + service records when containers, application attempts + and applications complete. + + If false, the paths must be created by other means, + and no automatic cleanup of service records will take place. + + hadoop.registry.rm.enabled + false + + + + + The root zookeeper node for the registry + + hadoop.registry.zk.root + /registry + + + + + Zookeeper session timeout in milliseconds + + hadoop.registry.zk.session.timeout.ms + 60000 + + + + + Zookeeper connection timeout in milliseconds + + hadoop.registry.zk.connection.timeout.ms + 15000 + + + + + Zookeeper connection retry count before failing + + hadoop.registry.zk.retry.times + 5 + + + + + + hadoop.registry.zk.retry.interval.ms + 1000 + + + + + Zookeeper retry limit in milliseconds, during + exponential backoff. + + This places a limit even + if the retry times and interval limit, combined + with the backoff policy, result in a long retry + period + + hadoop.registry.zk.retry.ceiling.ms + 60000 + + + + + List of hostname:port pairs defining the + zookeeper quorum binding for the registry + + hadoop.registry.zk.quorum + localhost:2181 + + + + + Key to set if the registry is secure. Turning it on + changes the permissions policy from "open access" + to restrictions on kerberos with the option of + a user adding one or more auth key pairs down their + own tree. + + hadoop.registry.secure + false + + + + + A comma separated list of Zookeeper ACL identifiers with + system access to the registry in a secure cluster. + + These are given full access to all entries. + + If there is an "@" at the end of a SASL entry it + instructs the registry client to append the default kerberos domain. + + hadoop.registry.system.acls + sasl:yarn@, sasl:mapred@, sasl:hdfs@ + + + + + The kerberos realm: used to set the realm of + system principals which do not declare their realm, + and any other accounts that need the value. + + If empty, the default realm of the running process + is used. + + If neither are known and the realm is needed, then the registry + service/client will fail. + + hadoop.registry.kerberos.realm + + + + + + Key to define the JAAS context. Used in secure + mode + + hadoop.registry.jaas.context + Client + + + + + Enable hdfs shell commands to display warnings if (fs.defaultFS) property + is not set. + + hadoop.shell.missing.defaultFs.warning + false + + + + hadoop.shell.safely.delete.limit.num.files + 100 + Used by -safely option of hadoop fs shell -rm command to avoid + accidental deletion of large directories. When enabled, the -rm command + requires confirmation if the number of files to be deleted is greater than + this limit. The default limit is 100 files. The warning is disabled if + the limit is 0 or the -safely is not specified in -rm command. + + + + + fs.client.htrace.sampler.classes + + The class names of the HTrace Samplers to use for Hadoop + filesystem clients. + + + + + hadoop.htrace.span.receiver.classes + + The class names of the Span Receivers to use for Hadoop. + + + + + hadoop.caller.context.enabled + false + When the feature is enabled, additional fields are written into + name-node audit log records for auditing coarse granularity operations. + + + + hadoop.caller.context.max.size + 128 + The maximum bytes a caller context string can have. If the + passed caller context is longer than this maximum bytes, client will + truncate it before sending to server. Note that the server may have a + different maximum size, and will truncate the caller context to the + maximum size it allows. + + + + hadoop.caller.context.signature.max.size + 40 + + The caller's signature (optional) is for offline validation. If the + signature exceeds the maximum allowed bytes in server, the caller context + will be abandoned, in which case the caller context will not be recorded + in audit logs. + + + + diff --git a/flink-filesystems/flink-s3-fs-presto/README.md b/flink-filesystems/flink-s3-fs-presto/README.md new file mode 100644 index 0000000000000..bf3c1fe30fddf --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/README.md @@ -0,0 +1,28 @@ +This project is a wrapper around the S3 file system from the Presto project which shades all dependencies. +Initial simple tests seem to indicate that it responds slightly faster +and in a bit more lightweight manner to write/read/list requests, compared +to the Hadoop s3a FS, but it has some semantic differences. + +We also relocate the shaded Hadoop version to allow running in a different +setup. For this to work, however, we needed to adapt Hadoop's `Configuration` +class to load a (shaded) `core-default-shaded.xml` configuration with the +relocated class names of classes loaded via reflection +(in the future, we may need to extend this to `mapred-default.xml` and `hdfs-defaults.xml` and their respective configuration classes). + +# Changing the Hadoop Version + +If you want to change the Hadoop version this project depends on, the following +steps are required to keep the shading correct: + +1. copy `org/apache/hadoop/conf/Configuration.java` from the respective Hadoop jar file (from `com.facebook.presto.hadoop/hadoop-apache2`) to this project + - adapt the `Configuration` class by replacing `core-default.xml` with `core-default-shaded.xml`. +2. copy `core-default.xml` from the respective Hadoop jar (from `com.facebook.presto.hadoop/hadoop-apache2`) file to this project as + - `src/main/resources/core-default-shaded.xml` (replacing every occurence of `org.apache.hadoop` with `org.apache.flink.fs.s3presto.shaded.org.apache.hadoop`) + - `src/test/resources/core-site.xml` (as is) +3. verify the shaded jar: + - does not contain any unshaded classes except for `org.apache.flink.fs.s3presto.S3FileSystemFactory` + - all other classes should be under `org.apache.flink.fs.s3presto.shaded` + - there should be a `META-INF/services/org.apache.flink.fs.s3presto.S3FileSystemFactory` file pointing to the `org.apache.flink.fs.s3presto.S3FileSystemFactory` class + - other service files under `META-INF/services` should have their names and contents in the relocated `org.apache.flink.fs.s3presto.shaded` package + - contains a `core-default-shaded.xml` file + - does not contain a `core-default.xml` or `core-site.xml` file diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index 6c5ab49273a9e..6dfa2d8deb5b7 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -33,6 +33,7 @@ under the License. jar + 0.185 @@ -237,28 +238,71 @@ under the License. - org - org.apache.flink.fs.s3presto.shaded.org + com.amazonaws + org.apache.flink.fs.s3presto.shaded.com.amazonaws + + + com.facebook + org.apache.flink.fs.s3presto.shaded.com.facebook + + + com.fasterxml + org.apache.flink.fs.s3presto.shaded.com.fasterxml + + + com.google + org.apache.flink.fs.s3presto.shaded.com.google + + + com.google.code.findbugs.** + + + + io.airlift + org.apache.flink.fs.s3presto.shaded.io.airlift + + + + + org.apache.flink.runtime.fs.hdfs + org.apache.flink.fs.s3presto.shaded.org.apache.flink.runtime.fs.hdfs + + + org.apache.flink.runtime.util + org.apache.flink.fs.s3presto.shaded.org.apache.flink.runtime.util + + org.apache.flink.runtime.util.**Hadoop* + + + + + org.apache + org.apache.flink.fs.s3presto.shaded.org.apache - org.apache.flink.core.fs.FileSystemFactory - org.apache.flink.fs.s3presto.** + + org.apache.flink.** + org.apache.log4j.** - com - org.apache.flink.fs.s3presto.shaded.com + org.HdrHistogram + org.apache.flink.fs.s3presto.shaded.org.HdrHistogram + + + org.joda + org.apache.flink.fs.s3presto.shaded.org.joda - io - org.apache.flink.fs.s3presto.shaded.io + org.openjdk + org.apache.flink.fs.s3presto.shaded.org.openjdk - net - org.apache.flink.fs.s3presto.shaded.net + org.weakref + org.apache.flink.fs.s3presto.shaded.org.weakref - software - org.apache.flink.fs.s3presto.shaded.software + software.amazon + org.apache.flink.fs.s3presto.shaded.software.amazon @@ -284,10 +328,13 @@ under the License. META-INF/maven/org.apache.h*/** META-INF/maven/org.apache.flink/flink-hadoop-fs/** META-INF/maven/org.apache.flink/force-shading/** + + core-default.xml + + core-site.xml - true diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java new file mode 100644 index 0000000000000..ebf79489a5dc4 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -0,0 +1,2951 @@ +/** + * 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.hadoop.conf; + +import com.facebook.presto.hadoop.$internal.com.google.common.annotations.VisibleForTesting; + +import java.io.BufferedInputStream; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.Reader; +import java.io.Writer; +import java.lang.ref.WeakReference; +import java.net.InetSocketAddress; +import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.StringTokenizer; +import java.util.WeakHashMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerException; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; + +import com.facebook.presto.hadoop.$internal.com.google.common.base.Charsets; +import com.facebook.presto.hadoop.$internal.org.apache.commons.collections.map.UnmodifiableMap; +import com.facebook.presto.hadoop.$internal.org.apache.commons.logging.Log; +import com.facebook.presto.hadoop.$internal.org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.alias.CredentialProvider; +import org.apache.hadoop.security.alias.CredentialProvider.CredentialEntry; +import org.apache.hadoop.security.alias.CredentialProviderFactory; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.StringInterner; +import org.apache.hadoop.util.StringUtils; +import com.facebook.presto.hadoop.$internal.org.codehaus.jackson.JsonFactory; +import com.facebook.presto.hadoop.$internal.org.codehaus.jackson.JsonGenerator; +import org.w3c.dom.DOMException; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.w3c.dom.Text; +import org.xml.sax.SAXException; + +import com.facebook.presto.hadoop.$internal.com.google.common.base.Preconditions; + +/** + * Provides access to configuration parameters. + * + *

Resources

+ * + *

Configurations are specified by resources. A resource contains a set of + * name/value pairs as XML data. Each resource is named by either a + * String or by a {@link Path}. If named by a String, + * then the classpath is examined for a file with that name. If named by a + * Path, then the local filesystem is examined directly, without + * referring to the classpath. + * + *

Unless explicitly turned off, Hadoop by default specifies two + * resources, loaded in-order from the classpath:

    + *
  1. + * + * core-default-shaded.xml: Read-only defaults for hadoop.
  2. + *
  3. core-site.xml: Site-specific configuration for a given hadoop + * installation.
  4. + *
+ * Applications may add additional resources, which are loaded + * subsequent to these resources in the order they are added. + * + *

Final Parameters

+ * + *

Configuration parameters may be declared final. + * Once a resource declares a value final, no subsequently-loaded + * resource can alter that value. + * For example, one might define a final parameter with: + *

+ *  <property>
+ *    <name>dfs.hosts.include</name>
+ *    <value>/etc/hadoop/conf/hosts.include</value>
+ *    <final>true</final>
+ *  </property>
+ * + * Administrators typically define parameters as final in + * core-site.xml for values that user applications may not alter. + * + *

Variable Expansion

+ * + *

Value strings are first processed for variable expansion. The + * available properties are:

    + *
  1. Other properties defined in this Configuration; and, if a name is + * undefined here,
  2. + *
  3. Properties in {@link System#getProperties()}.
  4. + *
+ * + *

For example, if a configuration resource contains the following property + * definitions: + *

+ *  <property>
+ *    <name>basedir</name>
+ *    <value>/user/${user.name}</value>
+ *  </property>
+ *
+ *  <property>
+ *    <name>tempdir</name>
+ *    <value>${basedir}/tmp</value>
+ *  </property>
+ * + * When conf.get("tempdir") is called, then ${basedir} + * will be resolved to another property in this Configuration, while + * ${user.name} would then ordinarily be resolved to the value + * of the System property with that name. + * By default, warnings will be given to any deprecated configuration + * parameters and these are suppressible by configuring + * log4j.logger.org.apache.hadoop.conf.Configuration.deprecation in + * log4j.properties file. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class Configuration implements Iterable>, + Writable { + private static final Log LOG = + LogFactory.getLog(Configuration.class); + + private static final Log LOG_DEPRECATION = + LogFactory.getLog("org.apache.hadoop.conf.Configuration.deprecation"); + + private boolean quietmode = true; + + private static final String DEFAULT_STRING_CHECK = + "testingforemptydefaultvalue"; + + private boolean allowNullValueProperties = false; + + private static class Resource { + private final Object resource; + private final String name; + + public Resource(Object resource) { + this(resource, resource.toString()); + } + + public Resource(Object resource, String name) { + this.resource = resource; + this.name = name; + } + + public String getName(){ + return name; + } + + public Object getResource() { + return resource; + } + + @Override + public String toString() { + return name; + } + } + + /** + * List of configuration resources. + */ + private ArrayList resources = new ArrayList(); + + /** + * The value reported as the setting resource when a key is set + * by code rather than a file resource by dumpConfiguration. + */ + static final String UNKNOWN_RESOURCE = "Unknown"; + + + /** + * List of configuration parameters marked final. + */ + private Set finalParameters = Collections.newSetFromMap( + new ConcurrentHashMap()); + + private boolean loadDefaults = true; + + /** + * Configuration objects + */ + private static final WeakHashMap REGISTRY = + new WeakHashMap(); + + /** + * List of default Resources. Resources are loaded in the order of the list + * entries + */ + private static final CopyOnWriteArrayList defaultResources = + new CopyOnWriteArrayList(); + + private static final Map>>> + CACHE_CLASSES = new WeakHashMap>>>(); + + /** + * Sentinel value to store negative cache results in {@link #CACHE_CLASSES}. + */ + private static final Class NEGATIVE_CACHE_SENTINEL = + NegativeCacheSentinel.class; + + /** + * Stores the mapping of key to the resource which modifies or loads + * the key most recently + */ + private Map updatingResource; + + /** + * Class to keep the information about the keys which replace the deprecated + * ones. + * + * This class stores the new keys which replace the deprecated keys and also + * gives a provision to have a custom message for each of the deprecated key + * that is being replaced. It also provides method to get the appropriate + * warning message which can be logged whenever the deprecated key is used. + */ + private static class DeprecatedKeyInfo { + private final String[] newKeys; + private final String customMessage; + private final AtomicBoolean accessed = new AtomicBoolean(false); + + DeprecatedKeyInfo(String[] newKeys, String customMessage) { + this.newKeys = newKeys; + this.customMessage = customMessage; + } + + /** + * Method to provide the warning message. It gives the custom message if + * non-null, and default message otherwise. + * @param key the associated deprecated key. + * @return message that is to be logged when a deprecated key is used. + */ + private final String getWarningMessage(String key) { + String warningMessage; + if(customMessage == null) { + StringBuilder message = new StringBuilder(key); + String deprecatedKeySuffix = " is deprecated. Instead, use "; + message.append(deprecatedKeySuffix); + for (int i = 0; i < newKeys.length; i++) { + message.append(newKeys[i]); + if(i != newKeys.length-1) { + message.append(", "); + } + } + warningMessage = message.toString(); + } + else { + warningMessage = customMessage; + } + return warningMessage; + } + + boolean getAndSetAccessed() { + return accessed.getAndSet(true); + } + + public void clearAccessed() { + accessed.set(false); + } + } + + /** + * A pending addition to the global set of deprecated keys. + */ + public static class DeprecationDelta { + private final String key; + private final String[] newKeys; + private final String customMessage; + + DeprecationDelta(String key, String[] newKeys, String customMessage) { + Preconditions.checkNotNull(key); + Preconditions.checkNotNull(newKeys); + Preconditions.checkArgument(newKeys.length > 0); + this.key = key; + this.newKeys = newKeys; + this.customMessage = customMessage; + } + + public DeprecationDelta(String key, String newKey, String customMessage) { + this(key, new String[] { newKey }, customMessage); + } + + public DeprecationDelta(String key, String newKey) { + this(key, new String[] { newKey }, null); + } + + public String getKey() { + return key; + } + + public String[] getNewKeys() { + return newKeys; + } + + public String getCustomMessage() { + return customMessage; + } + } + + /** + * The set of all keys which are deprecated. + * + * DeprecationContext objects are immutable. + */ + private static class DeprecationContext { + /** + * Stores the deprecated keys, the new keys which replace the deprecated keys + * and custom message(if any provided). + */ + private final Map deprecatedKeyMap; + + /** + * Stores a mapping from superseding keys to the keys which they deprecate. + */ + private final Map reverseDeprecatedKeyMap; + + /** + * Create a new DeprecationContext by copying a previous DeprecationContext + * and adding some deltas. + * + * @param other The previous deprecation context to copy, or null to start + * from nothing. + * @param deltas The deltas to apply. + */ + @SuppressWarnings("unchecked") + DeprecationContext(DeprecationContext other, DeprecationDelta[] deltas) { + HashMap newDeprecatedKeyMap = + new HashMap(); + HashMap newReverseDeprecatedKeyMap = + new HashMap(); + if (other != null) { + for (Entry entry : + other.deprecatedKeyMap.entrySet()) { + newDeprecatedKeyMap.put(entry.getKey(), entry.getValue()); + } + for (Entry entry : + other.reverseDeprecatedKeyMap.entrySet()) { + newReverseDeprecatedKeyMap.put(entry.getKey(), entry.getValue()); + } + } + for (DeprecationDelta delta : deltas) { + if (!newDeprecatedKeyMap.containsKey(delta.getKey())) { + DeprecatedKeyInfo newKeyInfo = + new DeprecatedKeyInfo(delta.getNewKeys(), delta.getCustomMessage()); + newDeprecatedKeyMap.put(delta.key, newKeyInfo); + for (String newKey : delta.getNewKeys()) { + newReverseDeprecatedKeyMap.put(newKey, delta.key); + } + } + } + this.deprecatedKeyMap = + UnmodifiableMap.decorate(newDeprecatedKeyMap); + this.reverseDeprecatedKeyMap = + UnmodifiableMap.decorate(newReverseDeprecatedKeyMap); + } + + Map getDeprecatedKeyMap() { + return deprecatedKeyMap; + } + + Map getReverseDeprecatedKeyMap() { + return reverseDeprecatedKeyMap; + } + } + + private static DeprecationDelta[] defaultDeprecations = + new DeprecationDelta[] { + new DeprecationDelta("topology.script.file.name", + CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY), + new DeprecationDelta("topology.script.number.args", + CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY), + new DeprecationDelta("hadoop.configured.node.mapping", + CommonConfigurationKeys.NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY), + new DeprecationDelta("topology.node.switch.mapping.impl", + CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY), + new DeprecationDelta("dfs.df.interval", + CommonConfigurationKeys.FS_DF_INTERVAL_KEY), + new DeprecationDelta("hadoop.native.lib", + CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY), + new DeprecationDelta("fs.default.name", + CommonConfigurationKeys.FS_DEFAULT_NAME_KEY), + new DeprecationDelta("dfs.umaskmode", + CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY), + new DeprecationDelta("dfs.nfs.exports.allowed.hosts", + CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY) + }; + + /** + * The global DeprecationContext. + */ + private static AtomicReference deprecationContext = + new AtomicReference( + new DeprecationContext(null, defaultDeprecations)); + + /** + * Adds a set of deprecated keys to the global deprecations. + * + * This method is lockless. It works by means of creating a new + * DeprecationContext based on the old one, and then atomically swapping in + * the new context. If someone else updated the context in between us reading + * the old context and swapping in the new one, we try again until we win the + * race. + * + * @param deltas The deprecations to add. + */ + public static void addDeprecations(DeprecationDelta[] deltas) { + DeprecationContext prev, next; + do { + prev = deprecationContext.get(); + next = new DeprecationContext(prev, deltas); + } while (!deprecationContext.compareAndSet(prev, next)); + } + + /** + * Adds the deprecated key to the global deprecation map. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If a key is deprecated in favor of multiple keys, they are all treated as + * aliases of each other, and setting any one of them resets all the others + * to the new value. + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key + * @param newKeys + * @param customMessage + * @deprecated use {@link #addDeprecation(String key, String newKey, + String customMessage)} instead + */ + @Deprecated + public static void addDeprecation(String key, String[] newKeys, + String customMessage) { + addDeprecations(new DeprecationDelta[] { + new DeprecationDelta(key, newKeys, customMessage) + }); + } + + /** + * Adds the deprecated key to the global deprecation map. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key + * @param newKey + * @param customMessage + */ + public static void addDeprecation(String key, String newKey, + String customMessage) { + addDeprecation(key, new String[] {newKey}, customMessage); + } + + /** + * Adds the deprecated key to the global deprecation map when no custom + * message is provided. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If a key is deprecated in favor of multiple keys, they are all treated as + * aliases of each other, and setting any one of them resets all the others + * to the new value. + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key Key that is to be deprecated + * @param newKeys list of keys that take up the values of deprecated key + * @deprecated use {@link #addDeprecation(String key, String newKey)} instead + */ + @Deprecated + public static void addDeprecation(String key, String[] newKeys) { + addDeprecation(key, newKeys, null); + } + + /** + * Adds the deprecated key to the global deprecation map when no custom + * message is provided. + * It does not override any existing entries in the deprecation map. + * This is to be used only by the developers in order to add deprecation of + * keys, and attempts to call this method after loading resources once, + * would lead to UnsupportedOperationException + * + * If you have multiple deprecation entries to add, it is more efficient to + * use #addDeprecations(DeprecationDelta[] deltas) instead. + * + * @param key Key that is to be deprecated + * @param newKey key that takes up the value of deprecated key + */ + public static void addDeprecation(String key, String newKey) { + addDeprecation(key, new String[] {newKey}, null); + } + + /** + * checks whether the given key is deprecated. + * + * @param key the parameter which is to be checked for deprecation + * @return true if the key is deprecated and + * false otherwise. + */ + public static boolean isDeprecated(String key) { + return deprecationContext.get().getDeprecatedKeyMap().containsKey(key); + } + + /** + * Sets all deprecated properties that are not currently set but have a + * corresponding new property that is set. Useful for iterating the + * properties when all deprecated properties for currently set properties + * need to be present. + */ + public void setDeprecatedProperties() { + DeprecationContext deprecations = deprecationContext.get(); + Properties props = getProps(); + Properties overlay = getOverlay(); + for (Map.Entry entry : + deprecations.getDeprecatedKeyMap().entrySet()) { + String depKey = entry.getKey(); + if (!overlay.contains(depKey)) { + for (String newKey : entry.getValue().newKeys) { + String val = overlay.getProperty(newKey); + if (val != null) { + props.setProperty(depKey, val); + overlay.setProperty(depKey, val); + break; + } + } + } + } + } + + /** + * Checks for the presence of the property name in the + * deprecation map. Returns the first of the list of new keys if present + * in the deprecation map or the name itself. If the property + * is not presently set but the property map contains an entry for the + * deprecated key, the value of the deprecated key is set as the value for + * the provided property name. + * + * @param name the property name + * @return the first property in the list of properties mapping + * the name or the name itself. + */ + private String[] handleDeprecation(DeprecationContext deprecations, + String name) { + if (null != name) { + name = name.trim(); + } + ArrayList names = new ArrayList(); + if (isDeprecated(name)) { + DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name); + warnOnceIfDeprecated(deprecations, name); + for (String newKey : keyInfo.newKeys) { + if(newKey != null) { + names.add(newKey); + } + } + } + if(names.size() == 0) { + names.add(name); + } + for(String n : names) { + String deprecatedKey = deprecations.getReverseDeprecatedKeyMap().get(n); + if (deprecatedKey != null && !getOverlay().containsKey(n) && + getOverlay().containsKey(deprecatedKey)) { + getProps().setProperty(n, getOverlay().getProperty(deprecatedKey)); + getOverlay().setProperty(n, getOverlay().getProperty(deprecatedKey)); + } + } + return names.toArray(new String[names.size()]); + } + + private void handleDeprecation() { + LOG.debug("Handling deprecation for all properties in config..."); + DeprecationContext deprecations = deprecationContext.get(); + Set keys = new HashSet(); + keys.addAll(getProps().keySet()); + for (Object item: keys) { + LOG.debug("Handling deprecation for " + (String)item); + handleDeprecation(deprecations, (String)item); + } + } + + static{ + //print deprecation warning if hadoop-site.xml is found in classpath + ClassLoader cL = Thread.currentThread().getContextClassLoader(); + if (cL == null) { + cL = Configuration.class.getClassLoader(); + } + if(cL.getResource("hadoop-site.xml")!=null) { + LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " + + "Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, " + + "mapred-site.xml and hdfs-site.xml to override properties of " + + "core-default-shaded.xml, mapred-default.xml and hdfs-default.xml " + + "respectively"); + } + addDefaultResource("core-default-shaded.xml"); + addDefaultResource("core-site.xml"); + } + + private Properties properties; + private Properties overlay; + private ClassLoader classLoader; + { + classLoader = Thread.currentThread().getContextClassLoader(); + if (classLoader == null) { + classLoader = Configuration.class.getClassLoader(); + } + } + + /** A new configuration. */ + public Configuration() { + this(true); + } + + /** A new configuration where the behavior of reading from the default + * resources can be turned off. + * + * If the parameter {@code loadDefaults} is false, the new instance + * will not load resources from the default files. + * @param loadDefaults specifies whether to load from the default files + */ + public Configuration(boolean loadDefaults) { + this.loadDefaults = loadDefaults; + updatingResource = new ConcurrentHashMap(); + synchronized(Configuration.class) { + REGISTRY.put(this, null); + } + } + + /** + * A new configuration with the same settings cloned from another. + * + * @param other the configuration from which to clone settings. + */ + @SuppressWarnings("unchecked") + public Configuration(Configuration other) { + this.resources = (ArrayList) other.resources.clone(); + synchronized(other) { + if (other.properties != null) { + this.properties = (Properties)other.properties.clone(); + } + + if (other.overlay!=null) { + this.overlay = (Properties)other.overlay.clone(); + } + + this.updatingResource = new ConcurrentHashMap( + other.updatingResource); + this.finalParameters = Collections.newSetFromMap( + new ConcurrentHashMap()); + this.finalParameters.addAll(other.finalParameters); + } + + synchronized(Configuration.class) { + REGISTRY.put(this, null); + } + this.classLoader = other.classLoader; + this.loadDefaults = other.loadDefaults; + setQuietMode(other.getQuietMode()); + } + + /** + * Add a default resource. Resources are loaded in the order of the resources + * added. + * @param name file name. File should be present in the classpath. + */ + public static synchronized void addDefaultResource(String name) { + if(!defaultResources.contains(name)) { + defaultResources.add(name); + for(Configuration conf : REGISTRY.keySet()) { + if(conf.loadDefaults) { + conf.reloadConfiguration(); + } + } + } + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param name resource to be added, the classpath is examined for a file + * with that name. + */ + public void addResource(String name) { + addResourceObject(new Resource(name)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param url url of the resource to be added, the local filesystem is + * examined directly to find the resource, without referring to + * the classpath. + */ + public void addResource(URL url) { + addResourceObject(new Resource(url)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param file file-path of resource to be added, the local filesystem is + * examined directly to find the resource, without referring to + * the classpath. + */ + public void addResource(Path file) { + addResourceObject(new Resource(file)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * WARNING: The contents of the InputStream will be cached, by this method. + * So use this sparingly because it does increase the memory consumption. + * + * @param in InputStream to deserialize the object from. In will be read from + * when a get or set is called next. After it is read the stream will be + * closed. + */ + public void addResource(InputStream in) { + addResourceObject(new Resource(in)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param in InputStream to deserialize the object from. + * @param name the name of the resource because InputStream.toString is not + * very descriptive some times. + */ + public void addResource(InputStream in, String name) { + addResourceObject(new Resource(in, name)); + } + + /** + * Add a configuration resource. + * + * The properties of this resource will override properties of previously + * added resources, unless they were marked final. + * + * @param conf Configuration object from which to load properties + */ + public void addResource(Configuration conf) { + addResourceObject(new Resource(conf.getProps())); + } + + + + /** + * Reload configuration from previously added resources. + * + * This method will clear all the configuration read from the added + * resources, and final parameters. This will make the resources to + * be read again before accessing the values. Values that are added + * via set methods will overlay values read from the resources. + */ + public synchronized void reloadConfiguration() { + properties = null; // trigger reload + finalParameters.clear(); // clear site-limits + } + + private synchronized void addResourceObject(Resource resource) { + resources.add(resource); // add to resources + reloadConfiguration(); + } + + private static final int MAX_SUBST = 20; + + private static final int SUB_START_IDX = 0; + private static final int SUB_END_IDX = SUB_START_IDX + 1; + + /** + * This is a manual implementation of the following regex + * "\\$\\{[^\\}\\$\u0020]+\\}". It can be 15x more efficient than + * a regex matcher as demonstrated by HADOOP-11506. This is noticeable with + * Hadoop apps building on the assumption Configuration#get is an O(1) + * hash table lookup, especially when the eval is a long string. + * + * @param eval a string that may contain variables requiring expansion. + * @return a 2-element int array res such that + * eval.substring(res[0], res[1]) is "var" for the left-most occurrence of + * ${var} in eval. If no variable is found -1, -1 is returned. + */ + private static int[] findSubVariable(String eval) { + int[] result = {-1, -1}; + + int matchStart; + int leftBrace; + + // scanning for a brace first because it's less frequent than $ + // that can occur in nested class names + // + match_loop: + for (matchStart = 1, leftBrace = eval.indexOf('{', matchStart); + // minimum left brace position (follows '$') + leftBrace > 0 + // right brace of a smallest valid expression "${c}" + && leftBrace + "{c".length() < eval.length(); + leftBrace = eval.indexOf('{', matchStart)) { + int matchedLen = 0; + if (eval.charAt(leftBrace - 1) == '$') { + int subStart = leftBrace + 1; // after '{' + for (int i = subStart; i < eval.length(); i++) { + switch (eval.charAt(i)) { + case '}': + if (matchedLen > 0) { // match + result[SUB_START_IDX] = subStart; + result[SUB_END_IDX] = subStart + matchedLen; + break match_loop; + } + // fall through to skip 1 char + case ' ': + case '$': + matchStart = i + 1; + continue match_loop; + default: + matchedLen++; + } + } + // scanned from "${" to the end of eval, and no reset via ' ', '$': + // no match! + break match_loop; + } else { + // not a start of a variable + // + matchStart = leftBrace + 1; + } + } + return result; + } + + /** + * Attempts to repeatedly expand the value {@code expr} by replacing the + * left-most substring of the form "${var}" in the following precedence order + *
    + *
  1. by the value of the Java system property "var" if defined
  2. + *
  3. by the value of the configuration key "var" if defined
  4. + *
+ * + * If var is unbounded the current state of expansion "prefix${var}suffix" is + * returned. + * + * @param expr the literal value of a config key + * @return null if expr is null, otherwise the value resulting from expanding + * expr using the algorithm above. + * @throws IllegalArgumentException when more than + * {@link Configuration#MAX_SUBST} replacements are required + */ + private String substituteVars(String expr) { + if (expr == null) { + return null; + } + String eval = expr; + for (int s = 0; s < MAX_SUBST; s++) { + final int[] varBounds = findSubVariable(eval); + if (varBounds[SUB_START_IDX] == -1) { + return eval; + } + final String var = eval.substring(varBounds[SUB_START_IDX], + varBounds[SUB_END_IDX]); + String val = null; + try { + val = System.getProperty(var); + } catch(SecurityException se) { + LOG.warn("Unexpected SecurityException in Configuration", se); + } + if (val == null) { + val = getRaw(var); + } + if (val == null) { + return eval; // return literal ${var}: var is unbound + } + final int dollar = varBounds[SUB_START_IDX] - "${".length(); + final int afterRightBrace = varBounds[SUB_END_IDX] + "}".length(); + // substitute + eval = eval.substring(0, dollar) + + val + + eval.substring(afterRightBrace); + } + throw new IllegalStateException("Variable substitution depth too large: " + + MAX_SUBST + " " + expr); + } + + /** + * Get the value of the name property, null if + * no such property exists. If the key is deprecated, it returns the value of + * the first key which replaces the deprecated key and is not null. + * + * Values are processed for variable expansion + * before being returned. + * + * @param name the property name, will be trimmed before get value. + * @return the value of the name or its replacing property, + * or null if no such property exists. + */ + public String get(String name) { + String[] names = handleDeprecation(deprecationContext.get(), name); + String result = null; + for(String n : names) { + result = substituteVars(getProps().getProperty(n)); + } + return result; + } + + /** + * Set Configuration to allow keys without values during setup. Intended + * for use during testing. + * + * @param val If true, will allow Configuration to store keys without values + */ + @VisibleForTesting + public void setAllowNullValueProperties( boolean val ) { + this.allowNullValueProperties = val; + } + + /** + * Return existence of the name property, but only for + * names which have no valid value, usually non-existent or commented + * out in XML. + * + * @param name the property name + * @return true if the property name exists without value + */ + @VisibleForTesting + public boolean onlyKeyExists(String name) { + String[] names = handleDeprecation(deprecationContext.get(), name); + for(String n : names) { + if ( getProps().getProperty(n,DEFAULT_STRING_CHECK) + .equals(DEFAULT_STRING_CHECK) ) { + return true; + } + } + return false; + } + + /** + * Get the value of the name property as a trimmed String, + * null if no such property exists. + * If the key is deprecated, it returns the value of + * the first key which replaces the deprecated key and is not null + * + * Values are processed for variable expansion + * before being returned. + * + * @param name the property name. + * @return the value of the name or its replacing property, + * or null if no such property exists. + */ + public String getTrimmed(String name) { + String value = get(name); + + if (null == value) { + return null; + } else { + return value.trim(); + } + } + + /** + * Get the value of the name property as a trimmed String, + * defaultValue if no such property exists. + * See @{Configuration#getTrimmed} for more details. + * + * @param name the property name. + * @param defaultValue the property default value. + * @return the value of the name or defaultValue + * if it is not set. + */ + public String getTrimmed(String name, String defaultValue) { + String ret = getTrimmed(name); + return ret == null ? defaultValue : ret; + } + + /** + * Get the value of the name property, without doing + * variable expansion.If the key is + * deprecated, it returns the value of the first key which replaces + * the deprecated key and is not null. + * + * @param name the property name. + * @return the value of the name property or + * its replacing property and null if no such property exists. + */ + public String getRaw(String name) { + String[] names = handleDeprecation(deprecationContext.get(), name); + String result = null; + for(String n : names) { + result = getProps().getProperty(n); + } + return result; + } + + /** + * Returns alternative names (non-deprecated keys or previously-set deprecated keys) + * for a given non-deprecated key. + * If the given key is deprecated, return null. + * + * @param name property name. + * @return alternative names. + */ + private String[] getAlternativeNames(String name) { + String altNames[] = null; + DeprecatedKeyInfo keyInfo = null; + DeprecationContext cur = deprecationContext.get(); + String depKey = cur.getReverseDeprecatedKeyMap().get(name); + if(depKey != null) { + keyInfo = cur.getDeprecatedKeyMap().get(depKey); + if(keyInfo.newKeys.length > 0) { + if(getProps().containsKey(depKey)) { + //if deprecated key is previously set explicitly + List list = new ArrayList(); + list.addAll(Arrays.asList(keyInfo.newKeys)); + list.add(depKey); + altNames = list.toArray(new String[list.size()]); + } + else { + altNames = keyInfo.newKeys; + } + } + } + return altNames; + } + + /** + * Set the value of the name property. If + * name is deprecated or there is a deprecated name associated to it, + * it sets the value to both names. Name will be trimmed before put into + * configuration. + * + * @param name property name. + * @param value property value. + */ + public void set(String name, String value) { + set(name, value, null); + } + + /** + * Set the value of the name property. If + * name is deprecated, it also sets the value to + * the keys that replace the deprecated key. Name will be trimmed before put + * into configuration. + * + * @param name property name. + * @param value property value. + * @param source the place that this configuration value came from + * (For debugging). + * @throws IllegalArgumentException when the value or name is null. + */ + public void set(String name, String value, String source) { + Preconditions.checkArgument( + name != null, + "Property name must not be null"); + Preconditions.checkArgument( + value != null, + "The value of property " + name + " must not be null"); + name = name.trim(); + DeprecationContext deprecations = deprecationContext.get(); + if (deprecations.getDeprecatedKeyMap().isEmpty()) { + getProps(); + } + getOverlay().setProperty(name, value); + getProps().setProperty(name, value); + String newSource = (source == null ? "programatically" : source); + + if (!isDeprecated(name)) { + updatingResource.put(name, new String[] {newSource}); + String[] altNames = getAlternativeNames(name); + if(altNames != null) { + for(String n: altNames) { + if(!n.equals(name)) { + getOverlay().setProperty(n, value); + getProps().setProperty(n, value); + updatingResource.put(n, new String[] {newSource}); + } + } + } + } + else { + String[] names = handleDeprecation(deprecationContext.get(), name); + String altSource = "because " + name + " is deprecated"; + for(String n : names) { + getOverlay().setProperty(n, value); + getProps().setProperty(n, value); + updatingResource.put(n, new String[] {altSource}); + } + } + } + + private void warnOnceIfDeprecated(DeprecationContext deprecations, String name) { + DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name); + if (keyInfo != null && !keyInfo.getAndSetAccessed()) { + LOG_DEPRECATION.info(keyInfo.getWarningMessage(name)); + } + } + + /** + * Unset a previously set property. + */ + public synchronized void unset(String name) { + String[] names = null; + if (!isDeprecated(name)) { + names = getAlternativeNames(name); + if(names == null) { + names = new String[]{name}; + } + } + else { + names = handleDeprecation(deprecationContext.get(), name); + } + + for(String n: names) { + getOverlay().remove(n); + getProps().remove(n); + } + } + + /** + * Sets a property if it is currently unset. + * @param name the property name + * @param value the new value + */ + public synchronized void setIfUnset(String name, String value) { + if (get(name) == null) { + set(name, value); + } + } + + private synchronized Properties getOverlay() { + if (overlay==null){ + overlay=new Properties(); + } + return overlay; + } + + /** + * Get the value of the name. If the key is deprecated, + * it returns the value of the first key which replaces the deprecated key + * and is not null. + * If no such property exists, + * then defaultValue is returned. + * + * @param name property name, will be trimmed before get value. + * @param defaultValue default value. + * @return property value, or defaultValue if the property + * doesn't exist. + */ + public String get(String name, String defaultValue) { + String[] names = handleDeprecation(deprecationContext.get(), name); + String result = null; + for(String n : names) { + result = substituteVars(getProps().getProperty(n, defaultValue)); + } + return result; + } + + /** + * Get the value of the name property as an int. + * + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid int, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as an int, + * or defaultValue. + */ + public int getInt(String name, int defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + String hexString = getHexDigits(valueString); + if (hexString != null) { + return Integer.parseInt(hexString, 16); + } + return Integer.parseInt(valueString); + } + + /** + * Get the value of the name property as a set of comma-delimited + * int values. + * + * If no such property exists, an empty array is returned. + * + * @param name property name + * @return property value interpreted as an array of comma-delimited + * int values + */ + public int[] getInts(String name) { + String[] strings = getTrimmedStrings(name); + int[] ints = new int[strings.length]; + for (int i = 0; i < strings.length; i++) { + ints[i] = Integer.parseInt(strings[i]); + } + return ints; + } + + /** + * Set the value of the name property to an int. + * + * @param name property name. + * @param value int value of the property. + */ + public void setInt(String name, int value) { + set(name, Integer.toString(value)); + } + + + /** + * Get the value of the name property as a long. + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid long, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a long, + * or defaultValue. + */ + public long getLong(String name, long defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + String hexString = getHexDigits(valueString); + if (hexString != null) { + return Long.parseLong(hexString, 16); + } + return Long.parseLong(valueString); + } + + /** + * Get the value of the name property as a long or + * human readable format. If no such property exists, the provided default + * value is returned, or if the specified value is not a valid + * long or human readable format, then an error is thrown. You + * can use the following suffix (case insensitive): k(kilo), m(mega), g(giga), + * t(tera), p(peta), e(exa) + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a long, + * or defaultValue. + */ + public long getLongBytes(String name, long defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + return StringUtils.TraditionalBinaryPrefix.string2long(valueString); + } + + private String getHexDigits(String value) { + boolean negative = false; + String str = value; + String hexString = null; + if (value.startsWith("-")) { + negative = true; + str = value.substring(1); + } + if (str.startsWith("0x") || str.startsWith("0X")) { + hexString = str.substring(2); + if (negative) { + hexString = "-" + hexString; + } + return hexString; + } + return null; + } + + /** + * Set the value of the name property to a long. + * + * @param name property name. + * @param value long value of the property. + */ + public void setLong(String name, long value) { + set(name, Long.toString(value)); + } + + /** + * Get the value of the name property as a float. + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid float, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a float, + * or defaultValue. + */ + public float getFloat(String name, float defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + return Float.parseFloat(valueString); + } + + /** + * Set the value of the name property to a float. + * + * @param name property name. + * @param value property value. + */ + public void setFloat(String name, float value) { + set(name,Float.toString(value)); + } + + /** + * Get the value of the name property as a double. + * If no such property exists, the provided default value is returned, + * or if the specified value is not a valid double, + * then an error is thrown. + * + * @param name property name. + * @param defaultValue default value. + * @throws NumberFormatException when the value is invalid + * @return property value as a double, + * or defaultValue. + */ + public double getDouble(String name, double defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + return Double.parseDouble(valueString); + } + + /** + * Set the value of the name property to a double. + * + * @param name property name. + * @param value property value. + */ + public void setDouble(String name, double value) { + set(name,Double.toString(value)); + } + + /** + * Get the value of the name property as a boolean. + * If no such property is specified, or if the specified value is not a valid + * boolean, then defaultValue is returned. + * + * @param name property name. + * @param defaultValue default value. + * @return property value as a boolean, + * or defaultValue. + */ + public boolean getBoolean(String name, boolean defaultValue) { + String valueString = getTrimmed(name); + if (null == valueString || valueString.isEmpty()) { + return defaultValue; + } + + if (StringUtils.equalsIgnoreCase("true", valueString)) + return true; + else if (StringUtils.equalsIgnoreCase("false", valueString)) + return false; + else return defaultValue; + } + + /** + * Set the value of the name property to a boolean. + * + * @param name property name. + * @param value boolean value of the property. + */ + public void setBoolean(String name, boolean value) { + set(name, Boolean.toString(value)); + } + + /** + * Set the given property, if it is currently unset. + * @param name property name + * @param value new value + */ + public void setBooleanIfUnset(String name, boolean value) { + setIfUnset(name, Boolean.toString(value)); + } + + /** + * Set the value of the name property to the given type. This + * is equivalent to set(<name>, value.toString()). + * @param name property name + * @param value new value + */ + public > void setEnum(String name, T value) { + set(name, value.toString()); + } + + /** + * Return value matching this enumerated type. + * Note that the returned value is trimmed by this method. + * @param name Property name + * @param defaultValue Value returned if no mapping exists + * @throws IllegalArgumentException If mapping is illegal for the type + * provided + */ + public > T getEnum(String name, T defaultValue) { + final String val = getTrimmed(name); + return null == val + ? defaultValue + : Enum.valueOf(defaultValue.getDeclaringClass(), val); + } + + enum ParsedTimeDuration { + NS { + TimeUnit unit() { return TimeUnit.NANOSECONDS; } + String suffix() { return "ns"; } + }, + US { + TimeUnit unit() { return TimeUnit.MICROSECONDS; } + String suffix() { return "us"; } + }, + MS { + TimeUnit unit() { return TimeUnit.MILLISECONDS; } + String suffix() { return "ms"; } + }, + S { + TimeUnit unit() { return TimeUnit.SECONDS; } + String suffix() { return "s"; } + }, + M { + TimeUnit unit() { return TimeUnit.MINUTES; } + String suffix() { return "m"; } + }, + H { + TimeUnit unit() { return TimeUnit.HOURS; } + String suffix() { return "h"; } + }, + D { + TimeUnit unit() { return TimeUnit.DAYS; } + String suffix() { return "d"; } + }; + abstract TimeUnit unit(); + abstract String suffix(); + static ParsedTimeDuration unitFor(String s) { + for (ParsedTimeDuration ptd : values()) { + // iteration order is in decl order, so SECONDS matched last + if (s.endsWith(ptd.suffix())) { + return ptd; + } + } + return null; + } + static ParsedTimeDuration unitFor(TimeUnit unit) { + for (ParsedTimeDuration ptd : values()) { + if (ptd.unit() == unit) { + return ptd; + } + } + return null; + } + } + + /** + * Set the value of name to the given time duration. This + * is equivalent to set(<name>, value + <time suffix>). + * @param name Property name + * @param value Time duration + * @param unit Unit of time + */ + public void setTimeDuration(String name, long value, TimeUnit unit) { + set(name, value + ParsedTimeDuration.unitFor(unit).suffix()); + } + + /** + * Return time duration in the given time unit. Valid units are encoded in + * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds + * (ms), seconds (s), minutes (m), hours (h), and days (d). + * @param name Property name + * @param defaultValue Value returned if no mapping exists. + * @param unit Unit to convert the stored property, if it exists. + * @throws NumberFormatException If the property stripped of its unit is not + * a number + */ + public long getTimeDuration(String name, long defaultValue, TimeUnit unit) { + String vStr = get(name); + if (null == vStr) { + return defaultValue; + } + vStr = vStr.trim(); + ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr); + if (null == vUnit) { + LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit); + vUnit = ParsedTimeDuration.unitFor(unit); + } else { + vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix())); + } + return unit.convert(Long.parseLong(vStr), vUnit.unit()); + } + + /** + * Get the value of the name property as a Pattern. + * If no such property is specified, or if the specified value is not a valid + * Pattern, then DefaultValue is returned. + * Note that the returned value is NOT trimmed by this method. + * + * @param name property name + * @param defaultValue default value + * @return property value as a compiled Pattern, or defaultValue + */ + public Pattern getPattern(String name, Pattern defaultValue) { + String valString = get(name); + if (null == valString || valString.isEmpty()) { + return defaultValue; + } + try { + return Pattern.compile(valString); + } catch (PatternSyntaxException pse) { + LOG.warn("Regular expression '" + valString + "' for property '" + + name + "' not valid. Using default", pse); + return defaultValue; + } + } + + /** + * Set the given property to Pattern. + * If the pattern is passed as null, sets the empty pattern which results in + * further calls to getPattern(...) returning the default value. + * + * @param name property name + * @param pattern new value + */ + public void setPattern(String name, Pattern pattern) { + assert pattern != null : "Pattern cannot be null"; + set(name, pattern.pattern()); + } + + /** + * Gets information about why a property was set. Typically this is the + * path to the resource objects (file, URL, etc.) the property came from, but + * it can also indicate that it was set programatically, or because of the + * command line. + * + * @param name - The property name to get the source of. + * @return null - If the property or its source wasn't found. Otherwise, + * returns a list of the sources of the resource. The older sources are + * the first ones in the list. So for example if a configuration is set from + * the command line, and then written out to a file that is read back in the + * first entry would indicate that it was set from the command line, while + * the second one would indicate the file that the new configuration was read + * in from. + */ + @InterfaceStability.Unstable + public synchronized String[] getPropertySources(String name) { + if (properties == null) { + // If properties is null, it means a resource was newly added + // but the props were cleared so as to load it upon future + // requests. So lets force a load by asking a properties list. + getProps(); + } + // Return a null right away if our properties still + // haven't loaded or the resource mapping isn't defined + if (properties == null || updatingResource == null) { + return null; + } else { + String[] source = updatingResource.get(name); + if(source == null) { + return null; + } else { + return Arrays.copyOf(source, source.length); + } + } + } + + /** + * A class that represents a set of positive integer ranges. It parses + * strings of the form: "2-3,5,7-" where ranges are separated by comma and + * the lower/upper bounds are separated by dash. Either the lower or upper + * bound may be omitted meaning all values up to or over. So the string + * above means 2, 3, 5, and 7, 8, 9, ... + */ + public static class IntegerRanges implements Iterable{ + private static class Range { + int start; + int end; + } + + private static class RangeNumberIterator implements Iterator { + Iterator internal; + int at; + int end; + + public RangeNumberIterator(List ranges) { + if (ranges != null) { + internal = ranges.iterator(); + } + at = -1; + end = -2; + } + + @Override + public boolean hasNext() { + if (at <= end) { + return true; + } else if (internal != null){ + return internal.hasNext(); + } + return false; + } + + @Override + public Integer next() { + if (at <= end) { + at++; + return at - 1; + } else if (internal != null){ + Range found = internal.next(); + if (found != null) { + at = found.start; + end = found.end; + at++; + return at - 1; + } + } + return null; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + + List ranges = new ArrayList(); + + public IntegerRanges() { + } + + public IntegerRanges(String newValue) { + StringTokenizer itr = new StringTokenizer(newValue, ","); + while (itr.hasMoreTokens()) { + String rng = itr.nextToken().trim(); + String[] parts = rng.split("-", 3); + if (parts.length < 1 || parts.length > 2) { + throw new IllegalArgumentException("integer range badly formed: " + + rng); + } + Range r = new Range(); + r.start = convertToInt(parts[0], 0); + if (parts.length == 2) { + r.end = convertToInt(parts[1], Integer.MAX_VALUE); + } else { + r.end = r.start; + } + if (r.start > r.end) { + throw new IllegalArgumentException("IntegerRange from " + r.start + + " to " + r.end + " is invalid"); + } + ranges.add(r); + } + } + + /** + * Convert a string to an int treating empty strings as the default value. + * @param value the string value + * @param defaultValue the value for if the string is empty + * @return the desired integer + */ + private static int convertToInt(String value, int defaultValue) { + String trim = value.trim(); + if (trim.length() == 0) { + return defaultValue; + } + return Integer.parseInt(trim); + } + + /** + * Is the given value in the set of ranges + * @param value the value to check + * @return is the value in the ranges? + */ + public boolean isIncluded(int value) { + for(Range r: ranges) { + if (r.start <= value && value <= r.end) { + return true; + } + } + return false; + } + + /** + * @return true if there are no values in this range, else false. + */ + public boolean isEmpty() { + return ranges == null || ranges.isEmpty(); + } + + @Override + public String toString() { + StringBuilder result = new StringBuilder(); + boolean first = true; + for(Range r: ranges) { + if (first) { + first = false; + } else { + result.append(','); + } + result.append(r.start); + result.append('-'); + result.append(r.end); + } + return result.toString(); + } + + @Override + public Iterator iterator() { + return new RangeNumberIterator(ranges); + } + + } + + /** + * Parse the given attribute as a set of integer ranges + * @param name the attribute name + * @param defaultValue the default value if it is not set + * @return a new set of ranges from the configured value + */ + public IntegerRanges getRange(String name, String defaultValue) { + return new IntegerRanges(get(name, defaultValue)); + } + + /** + * Get the comma delimited values of the name property as + * a collection of Strings. + * If no such property is specified then empty collection is returned. + *

+ * This is an optimized version of {@link #getStrings(String)} + * + * @param name property name. + * @return property value as a collection of Strings. + */ + public Collection getStringCollection(String name) { + String valueString = get(name); + return StringUtils.getStringCollection(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings. + * If no such property is specified then null is returned. + * + * @param name property name. + * @return property value as an array of Strings, + * or null. + */ + public String[] getStrings(String name) { + String valueString = get(name); + return StringUtils.getStrings(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings. + * If no such property is specified then default value is returned. + * + * @param name property name. + * @param defaultValue The default value + * @return property value as an array of Strings, + * or default value. + */ + public String[] getStrings(String name, String... defaultValue) { + String valueString = get(name); + if (valueString == null) { + return defaultValue; + } else { + return StringUtils.getStrings(valueString); + } + } + + /** + * Get the comma delimited values of the name property as + * a collection of Strings, trimmed of the leading and trailing whitespace. + * If no such property is specified then empty Collection is returned. + * + * @param name property name. + * @return property value as a collection of Strings, or empty Collection + */ + public Collection getTrimmedStringCollection(String name) { + String valueString = get(name); + if (null == valueString) { + Collection empty = new ArrayList(); + return empty; + } + return StringUtils.getTrimmedStringCollection(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings, trimmed of the leading and trailing whitespace. + * If no such property is specified then an empty array is returned. + * + * @param name property name. + * @return property value as an array of trimmed Strings, + * or empty array. + */ + public String[] getTrimmedStrings(String name) { + String valueString = get(name); + return StringUtils.getTrimmedStrings(valueString); + } + + /** + * Get the comma delimited values of the name property as + * an array of Strings, trimmed of the leading and trailing whitespace. + * If no such property is specified then default value is returned. + * + * @param name property name. + * @param defaultValue The default value + * @return property value as an array of trimmed Strings, + * or default value. + */ + public String[] getTrimmedStrings(String name, String... defaultValue) { + String valueString = get(name); + if (null == valueString) { + return defaultValue; + } else { + return StringUtils.getTrimmedStrings(valueString); + } + } + + /** + * Set the array of string values for the name property as + * as comma delimited values. + * + * @param name property name. + * @param values The values + */ + public void setStrings(String name, String... values) { + set(name, StringUtils.arrayToString(values)); + } + + /** + * Get the value for a known password configuration element. + * In order to enable the elimination of clear text passwords in config, + * this method attempts to resolve the property name as an alias through + * the CredentialProvider API and conditionally fallsback to config. + * @param name property name + * @return password + */ + public char[] getPassword(String name) throws IOException { + char[] pass = null; + + pass = getPasswordFromCredentialProviders(name); + + if (pass == null) { + pass = getPasswordFromConfig(name); + } + + return pass; + } + + /** + * Try and resolve the provided element name as a credential provider + * alias. + * @param name alias of the provisioned credential + * @return password or null if not found + * @throws IOException + */ + protected char[] getPasswordFromCredentialProviders(String name) + throws IOException { + char[] pass = null; + try { + List providers = + CredentialProviderFactory.getProviders(this); + + if (providers != null) { + for (CredentialProvider provider : providers) { + try { + CredentialEntry entry = provider.getCredentialEntry(name); + if (entry != null) { + pass = entry.getCredential(); + break; + } + } + catch (IOException ioe) { + throw new IOException("Can't get key " + name + " from key provider" + + "of type: " + provider.getClass().getName() + ".", ioe); + } + } + } + } + catch (IOException ioe) { + throw new IOException("Configuration problem with provider path.", ioe); + } + + return pass; + } + + /** + * Fallback to clear text passwords in configuration. + * @param name + * @return clear text password or null + */ + protected char[] getPasswordFromConfig(String name) { + char[] pass = null; + if (getBoolean(CredentialProvider.CLEAR_TEXT_FALLBACK, true)) { + String passStr = get(name); + if (passStr != null) { + pass = passStr.toCharArray(); + } + } + return pass; + } + + /** + * Get the socket address for hostProperty as a + * InetSocketAddress. If hostProperty is + * null, addressProperty will be used. This + * is useful for cases where we want to differentiate between host + * bind address and address clients should use to establish connection. + * + * @param hostProperty bind host property name. + * @param addressProperty address property name. + * @param defaultAddressValue the default value + * @param defaultPort the default port + * @return InetSocketAddress + */ + public InetSocketAddress getSocketAddr( + String hostProperty, + String addressProperty, + String defaultAddressValue, + int defaultPort) { + + InetSocketAddress bindAddr = getSocketAddr( + addressProperty, defaultAddressValue, defaultPort); + + final String host = get(hostProperty); + + if (host == null || host.isEmpty()) { + return bindAddr; + } + + return NetUtils.createSocketAddr( + host, bindAddr.getPort(), hostProperty); + } + + /** + * Get the socket address for name property as a + * InetSocketAddress. + * @param name property name. + * @param defaultAddress the default value + * @param defaultPort the default port + * @return InetSocketAddress + */ + public InetSocketAddress getSocketAddr( + String name, String defaultAddress, int defaultPort) { + final String address = getTrimmed(name, defaultAddress); + return NetUtils.createSocketAddr(address, defaultPort, name); + } + + /** + * Set the socket address for the name property as + * a host:port. + */ + public void setSocketAddr(String name, InetSocketAddress addr) { + set(name, NetUtils.getHostPortString(addr)); + } + + /** + * Set the socket address a client can use to connect for the + * name property as a host:port. The wildcard + * address is replaced with the local host's address. If the host and address + * properties are configured the host component of the address will be combined + * with the port component of the addr to generate the address. This is to allow + * optional control over which host name is used in multi-home bind-host + * cases where a host can have multiple names + * @param hostProperty the bind-host configuration name + * @param addressProperty the service address configuration name + * @param defaultAddressValue the service default address configuration value + * @param addr InetSocketAddress of the service listener + * @return InetSocketAddress for clients to connect + */ + public InetSocketAddress updateConnectAddr( + String hostProperty, + String addressProperty, + String defaultAddressValue, + InetSocketAddress addr) { + + final String host = get(hostProperty); + final String connectHostPort = getTrimmed(addressProperty, defaultAddressValue); + + if (host == null || host.isEmpty() || connectHostPort == null || connectHostPort.isEmpty()) { + //not our case, fall back to original logic + return updateConnectAddr(addressProperty, addr); + } + + final String connectHost = connectHostPort.split(":")[0]; + // Create connect address using client address hostname and server port. + return updateConnectAddr(addressProperty, NetUtils.createSocketAddrForHost( + connectHost, addr.getPort())); + } + + /** + * Set the socket address a client can use to connect for the + * name property as a host:port. The wildcard + * address is replaced with the local host's address. + * @param name property name. + * @param addr InetSocketAddress of a listener to store in the given property + * @return InetSocketAddress for clients to connect + */ + public InetSocketAddress updateConnectAddr(String name, + InetSocketAddress addr) { + final InetSocketAddress connectAddr = NetUtils.getConnectAddress(addr); + setSocketAddr(name, connectAddr); + return connectAddr; + } + + /** + * Load a class by name. + * + * @param name the class name. + * @return the class object. + * @throws ClassNotFoundException if the class is not found. + */ + public Class getClassByName(String name) throws ClassNotFoundException { + Class ret = getClassByNameOrNull(name); + if (ret == null) { + throw new ClassNotFoundException("Class " + name + " not found"); + } + return ret; + } + + /** + * Load a class by name, returning null rather than throwing an exception + * if it couldn't be loaded. This is to avoid the overhead of creating + * an exception. + * + * @param name the class name + * @return the class object, or null if it could not be found. + */ + public Class getClassByNameOrNull(String name) { + Map>> map; + + synchronized (CACHE_CLASSES) { + map = CACHE_CLASSES.get(classLoader); + if (map == null) { + map = Collections.synchronizedMap( + new WeakHashMap>>()); + CACHE_CLASSES.put(classLoader, map); + } + } + + Class clazz = null; + WeakReference> ref = map.get(name); + if (ref != null) { + clazz = ref.get(); + } + + if (clazz == null) { + try { + clazz = Class.forName(name, true, classLoader); + } catch (ClassNotFoundException e) { + // Leave a marker that the class isn't found + map.put(name, new WeakReference>(NEGATIVE_CACHE_SENTINEL)); + return null; + } + // two putters can race here, but they'll put the same class + map.put(name, new WeakReference>(clazz)); + return clazz; + } else if (clazz == NEGATIVE_CACHE_SENTINEL) { + return null; // not found + } else { + // cache hit + return clazz; + } + } + + /** + * Get the value of the name property + * as an array of Class. + * The value of the property specifies a list of comma separated class names. + * If no such property is specified, then defaultValue is + * returned. + * + * @param name the property name. + * @param defaultValue default value. + * @return property value as a Class[], + * or defaultValue. + */ + public Class[] getClasses(String name, Class ... defaultValue) { + String[] classnames = getTrimmedStrings(name); + if (classnames == null) + return defaultValue; + try { + Class[] classes = new Class[classnames.length]; + for(int i = 0; i < classnames.length; i++) { + classes[i] = getClassByName(classnames[i]); + } + return classes; + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + /** + * Get the value of the name property as a Class. + * If no such property is specified, then defaultValue is + * returned. + * + * @param name the class name. + * @param defaultValue default value. + * @return property value as a Class, + * or defaultValue. + */ + public Class getClass(String name, Class defaultValue) { + String valueString = getTrimmed(name); + if (valueString == null) + return defaultValue; + try { + return getClassByName(valueString); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + /** + * Get the value of the name property as a Class + * implementing the interface specified by xface. + * + * If no such property is specified, then defaultValue is + * returned. + * + * An exception is thrown if the returned class does not implement the named + * interface. + * + * @param name the class name. + * @param defaultValue default value. + * @param xface the interface implemented by the named class. + * @return property value as a Class, + * or defaultValue. + */ + public Class getClass(String name, + Class defaultValue, + Class xface) { + try { + Class theClass = getClass(name, defaultValue); + if (theClass != null && !xface.isAssignableFrom(theClass)) + throw new RuntimeException(theClass+" not "+xface.getName()); + else if (theClass != null) + return theClass.asSubclass(xface); + else + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Get the value of the name property as a List + * of objects implementing the interface specified by xface. + * + * An exception is thrown if any of the classes does not exist, or if it does + * not implement the named interface. + * + * @param name the property name. + * @param xface the interface implemented by the classes named by + * name. + * @return a List of objects implementing xface. + */ + @SuppressWarnings("unchecked") + public List getInstances(String name, Class xface) { + List ret = new ArrayList(); + Class[] classes = getClasses(name); + for (Class cl: classes) { + if (!xface.isAssignableFrom(cl)) { + throw new RuntimeException(cl + " does not implement " + xface); + } + ret.add((U)ReflectionUtils.newInstance(cl, this)); + } + return ret; + } + + /** + * Set the value of the name property to the name of a + * theClass implementing the given interface xface. + * + * An exception is thrown if theClass does not implement the + * interface xface. + * + * @param name property name. + * @param theClass property value. + * @param xface the interface implemented by the named class. + */ + public void setClass(String name, Class theClass, Class xface) { + if (!xface.isAssignableFrom(theClass)) + throw new RuntimeException(theClass+" not "+xface.getName()); + set(name, theClass.getName()); + } + + /** + * Get a local file under a directory named by dirsProp with + * the given path. If dirsProp contains multiple directories, + * then one is chosen based on path's hash code. If the selected + * directory does not exist, an attempt is made to create it. + * + * @param dirsProp directory in which to locate the file. + * @param path file-path. + * @return local file under the directory with the given path. + */ + public Path getLocalPath(String dirsProp, String path) + throws IOException { + String[] dirs = getTrimmedStrings(dirsProp); + int hashCode = path.hashCode(); + FileSystem fs = FileSystem.getLocal(this); + for (int i = 0; i < dirs.length; i++) { // try each local dir + int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length; + Path file = new Path(dirs[index], path); + Path dir = file.getParent(); + if (fs.mkdirs(dir) || fs.exists(dir)) { + return file; + } + } + LOG.warn("Could not make " + path + + " in local directories from " + dirsProp); + for(int i=0; i < dirs.length; i++) { + int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length; + LOG.warn(dirsProp + "[" + index + "]=" + dirs[index]); + } + throw new IOException("No valid local directories in property: "+dirsProp); + } + + /** + * Get a local file name under a directory named in dirsProp with + * the given path. If dirsProp contains multiple directories, + * then one is chosen based on path's hash code. If the selected + * directory does not exist, an attempt is made to create it. + * + * @param dirsProp directory in which to locate the file. + * @param path file-path. + * @return local file under the directory with the given path. + */ + public File getFile(String dirsProp, String path) + throws IOException { + String[] dirs = getTrimmedStrings(dirsProp); + int hashCode = path.hashCode(); + for (int i = 0; i < dirs.length; i++) { // try each local dir + int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length; + File file = new File(dirs[index], path); + File dir = file.getParentFile(); + if (dir.exists() || dir.mkdirs()) { + return file; + } + } + throw new IOException("No valid local directories in property: "+dirsProp); + } + + /** + * Get the {@link URL} for the named resource. + * + * @param name resource name. + * @return the url for the named resource. + */ + public URL getResource(String name) { + return classLoader.getResource(name); + } + + /** + * Get an input stream attached to the configuration resource with the + * given name. + * + * @param name configuration resource name. + * @return an input stream attached to the resource. + */ + public InputStream getConfResourceAsInputStream(String name) { + try { + URL url= getResource(name); + + if (url == null) { + LOG.info(name + " not found"); + return null; + } else { + LOG.info("found resource " + name + " at " + url); + } + + return url.openStream(); + } catch (Exception e) { + return null; + } + } + + /** + * Get a {@link Reader} attached to the configuration resource with the + * given name. + * + * @param name configuration resource name. + * @return a reader attached to the resource. + */ + public Reader getConfResourceAsReader(String name) { + try { + URL url= getResource(name); + + if (url == null) { + LOG.info(name + " not found"); + return null; + } else { + LOG.info("found resource " + name + " at " + url); + } + + return new InputStreamReader(url.openStream(), Charsets.UTF_8); + } catch (Exception e) { + return null; + } + } + + /** + * Get the set of parameters marked final. + * + * @return final parameter set. + */ + public Set getFinalParameters() { + Set setFinalParams = Collections.newSetFromMap( + new ConcurrentHashMap()); + setFinalParams.addAll(finalParameters); + return setFinalParams; + } + + protected synchronized Properties getProps() { + if (properties == null) { + properties = new Properties(); + Map backup = + new ConcurrentHashMap(updatingResource); + loadResources(properties, resources, quietmode); + + if (overlay != null) { + properties.putAll(overlay); + for (Map.Entry item: overlay.entrySet()) { + String key = (String)item.getKey(); + String[] source = backup.get(key); + if(source != null) { + updatingResource.put(key, source); + } + } + } + } + return properties; + } + + /** + * Return the number of keys in the configuration. + * + * @return number of keys in the configuration. + */ + public int size() { + return getProps().size(); + } + + /** + * Clears all keys from the configuration. + */ + public void clear() { + getProps().clear(); + getOverlay().clear(); + } + + /** + * Get an {@link Iterator} to go through the list of String + * key-value pairs in the configuration. + * + * @return an iterator over the entries. + */ + @Override + public Iterator> iterator() { + // Get a copy of just the string to string pairs. After the old object + // methods that allow non-strings to be put into configurations are removed, + // we could replace properties with a Map and get rid of this + // code. + Map result = new HashMap(); + for(Map.Entry item: getProps().entrySet()) { + if (item.getKey() instanceof String && + item.getValue() instanceof String) { + result.put((String) item.getKey(), (String) item.getValue()); + } + } + return result.entrySet().iterator(); + } + + private Document parse(DocumentBuilder builder, URL url) + throws IOException, SAXException { + if (!quietmode) { + LOG.debug("parsing URL " + url); + } + if (url == null) { + return null; + } + return parse(builder, url.openStream(), url.toString()); + } + + private Document parse(DocumentBuilder builder, InputStream is, + String systemId) throws IOException, SAXException { + if (!quietmode) { + LOG.debug("parsing input stream " + is); + } + if (is == null) { + return null; + } + try { + return (systemId == null) ? builder.parse(is) : builder.parse(is, + systemId); + } finally { + is.close(); + } + } + + private void loadResources(Properties properties, + ArrayList resources, + boolean quiet) { + if(loadDefaults) { + for (String resource : defaultResources) { + loadResource(properties, new Resource(resource), quiet); + } + + //support the hadoop-site.xml as a deprecated case + if(getResource("hadoop-site.xml")!=null) { + loadResource(properties, new Resource("hadoop-site.xml"), quiet); + } + } + + for (int i = 0; i < resources.size(); i++) { + Resource ret = loadResource(properties, resources.get(i), quiet); + if (ret != null) { + resources.set(i, ret); + } + } + } + + private Resource loadResource(Properties properties, Resource wrapper, boolean quiet) { + String name = UNKNOWN_RESOURCE; + try { + Object resource = wrapper.getResource(); + name = wrapper.getName(); + + DocumentBuilderFactory docBuilderFactory + = DocumentBuilderFactory.newInstance(); + //ignore all comments inside the xml file + docBuilderFactory.setIgnoringComments(true); + + //allow includes in the xml file + docBuilderFactory.setNamespaceAware(true); + try { + docBuilderFactory.setXIncludeAware(true); + } catch (UnsupportedOperationException e) { + LOG.error("Failed to set setXIncludeAware(true) for parser " + + docBuilderFactory + + ":" + e, + e); + } + DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); + Document doc = null; + Element root = null; + boolean returnCachedProperties = false; + + if (resource instanceof URL) { // an URL resource + doc = parse(builder, (URL)resource); + } else if (resource instanceof String) { // a CLASSPATH resource + URL url = getResource((String)resource); + doc = parse(builder, url); + } else if (resource instanceof Path) { // a file resource + // Can't use FileSystem API or we get an infinite loop + // since FileSystem uses Configuration API. Use java.io.File instead. + File file = new File(((Path)resource).toUri().getPath()) + .getAbsoluteFile(); + if (file.exists()) { + if (!quiet) { + LOG.debug("parsing File " + file); + } + doc = parse(builder, new BufferedInputStream( + new FileInputStream(file)), ((Path)resource).toString()); + } + } else if (resource instanceof InputStream) { + doc = parse(builder, (InputStream) resource, null); + returnCachedProperties = true; + } else if (resource instanceof Properties) { + overlay(properties, (Properties)resource); + } else if (resource instanceof Element) { + root = (Element)resource; + } + + if (root == null) { + if (doc == null) { + if (quiet) { + return null; + } + throw new RuntimeException(resource + " not found"); + } + root = doc.getDocumentElement(); + } + Properties toAddTo = properties; + if(returnCachedProperties) { + toAddTo = new Properties(); + } + if (!"configuration".equals(root.getTagName())) + LOG.fatal("bad conf file: top-level element not "); + NodeList props = root.getChildNodes(); + DeprecationContext deprecations = deprecationContext.get(); + for (int i = 0; i < props.getLength(); i++) { + Node propNode = props.item(i); + if (!(propNode instanceof Element)) + continue; + Element prop = (Element)propNode; + if ("configuration".equals(prop.getTagName())) { + loadResource(toAddTo, new Resource(prop, name), quiet); + continue; + } + if (!"property".equals(prop.getTagName())) + LOG.warn("bad conf file: element not "); + NodeList fields = prop.getChildNodes(); + String attr = null; + String value = null; + boolean finalParameter = false; + LinkedList source = new LinkedList(); + for (int j = 0; j < fields.getLength(); j++) { + Node fieldNode = fields.item(j); + if (!(fieldNode instanceof Element)) + continue; + Element field = (Element)fieldNode; + if ("name".equals(field.getTagName()) && field.hasChildNodes()) + attr = StringInterner.weakIntern( + ((Text)field.getFirstChild()).getData().trim()); + if ("value".equals(field.getTagName()) && field.hasChildNodes()) + value = StringInterner.weakIntern( + ((Text)field.getFirstChild()).getData()); + if ("final".equals(field.getTagName()) && field.hasChildNodes()) + finalParameter = "true".equals(((Text)field.getFirstChild()).getData()); + if ("source".equals(field.getTagName()) && field.hasChildNodes()) + source.add(StringInterner.weakIntern( + ((Text)field.getFirstChild()).getData())); + } + source.add(name); + + // Ignore this parameter if it has already been marked as 'final' + if (attr != null) { + if (deprecations.getDeprecatedKeyMap().containsKey(attr)) { + DeprecatedKeyInfo keyInfo = + deprecations.getDeprecatedKeyMap().get(attr); + keyInfo.clearAccessed(); + for (String key:keyInfo.newKeys) { + // update new keys with deprecated key's value + loadProperty(toAddTo, name, key, value, finalParameter, + source.toArray(new String[source.size()])); + } + } + else { + loadProperty(toAddTo, name, attr, value, finalParameter, + source.toArray(new String[source.size()])); + } + } + } + + if (returnCachedProperties) { + overlay(properties, toAddTo); + return new Resource(toAddTo, name); + } + return null; + } catch (IOException e) { + LOG.fatal("error parsing conf " + name, e); + throw new RuntimeException(e); + } catch (DOMException e) { + LOG.fatal("error parsing conf " + name, e); + throw new RuntimeException(e); + } catch (SAXException e) { + LOG.fatal("error parsing conf " + name, e); + throw new RuntimeException(e); + } catch (ParserConfigurationException e) { + LOG.fatal("error parsing conf " + name , e); + throw new RuntimeException(e); + } + } + + private void overlay(Properties to, Properties from) { + for (Entry entry: from.entrySet()) { + to.put(entry.getKey(), entry.getValue()); + } + } + + private void loadProperty(Properties properties, String name, String attr, + String value, boolean finalParameter, String[] source) { + if (value != null || allowNullValueProperties) { + if (!finalParameters.contains(attr)) { + if (value==null && allowNullValueProperties) { + value = DEFAULT_STRING_CHECK; + } + properties.setProperty(attr, value); + if(source != null) { + updatingResource.put(attr, source); + } + } else if (!value.equals(properties.getProperty(attr))) { + LOG.warn(name+":an attempt to override final parameter: "+attr + +"; Ignoring."); + } + } + if (finalParameter && attr != null) { + finalParameters.add(attr); + } + } + + /** + * Write out the non-default properties in this configuration to the given + * {@link OutputStream} using UTF-8 encoding. + * + * @param out the output stream to write to. + */ + public void writeXml(OutputStream out) throws IOException { + writeXml(new OutputStreamWriter(out, "UTF-8")); + } + + /** + * Write out the non-default properties in this configuration to the given + * {@link Writer}. + * + * @param out the writer to write to. + */ + public void writeXml(Writer out) throws IOException { + Document doc = asXmlDocument(); + + try { + DOMSource source = new DOMSource(doc); + StreamResult result = new StreamResult(out); + TransformerFactory transFactory = TransformerFactory.newInstance(); + Transformer transformer = transFactory.newTransformer(); + + // Important to not hold Configuration log while writing result, since + // 'out' may be an HDFS stream which needs to lock this configuration + // from another thread. + transformer.transform(source, result); + } catch (TransformerException te) { + throw new IOException(te); + } + } + + /** + * Return the XML DOM corresponding to this Configuration. + */ + private synchronized Document asXmlDocument() throws IOException { + Document doc; + try { + doc = + DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument(); + } catch (ParserConfigurationException pe) { + throw new IOException(pe); + } + Element conf = doc.createElement("configuration"); + doc.appendChild(conf); + conf.appendChild(doc.createTextNode("\n")); + handleDeprecation(); //ensure properties is set and deprecation is handled + for (Enumeration e = properties.keys(); e.hasMoreElements();) { + String name = (String)e.nextElement(); + Object object = properties.get(name); + String value = null; + if (object instanceof String) { + value = (String) object; + }else { + continue; + } + Element propNode = doc.createElement("property"); + conf.appendChild(propNode); + + Element nameNode = doc.createElement("name"); + nameNode.appendChild(doc.createTextNode(name)); + propNode.appendChild(nameNode); + + Element valueNode = doc.createElement("value"); + valueNode.appendChild(doc.createTextNode(value)); + propNode.appendChild(valueNode); + + if (updatingResource != null) { + String[] sources = updatingResource.get(name); + if(sources != null) { + for(String s : sources) { + Element sourceNode = doc.createElement("source"); + sourceNode.appendChild(doc.createTextNode(s)); + propNode.appendChild(sourceNode); + } + } + } + + conf.appendChild(doc.createTextNode("\n")); + } + return doc; + } + + /** + * Writes out all the parameters and their properties (final and resource) to + * the given {@link Writer} + * The format of the output would be + * { "properties" : [ {key1,value1,key1.isFinal,key1.resource}, {key2,value2, + * key2.isFinal,key2.resource}... ] } + * It does not output the parameters of the configuration object which is + * loaded from an input stream. + * @param out the Writer to write to + * @throws IOException + */ + public static void dumpConfiguration(Configuration config, + Writer out) throws IOException { + JsonFactory dumpFactory = new JsonFactory(); + JsonGenerator dumpGenerator = dumpFactory.createJsonGenerator(out); + dumpGenerator.writeStartObject(); + dumpGenerator.writeFieldName("properties"); + dumpGenerator.writeStartArray(); + dumpGenerator.flush(); + synchronized (config) { + for (Map.Entry item: config.getProps().entrySet()) { + dumpGenerator.writeStartObject(); + dumpGenerator.writeStringField("key", (String) item.getKey()); + dumpGenerator.writeStringField("value", + config.get((String) item.getKey())); + dumpGenerator.writeBooleanField("isFinal", + config.finalParameters.contains(item.getKey())); + String[] resources = config.updatingResource.get(item.getKey()); + String resource = UNKNOWN_RESOURCE; + if(resources != null && resources.length > 0) { + resource = resources[0]; + } + dumpGenerator.writeStringField("resource", resource); + dumpGenerator.writeEndObject(); + } + } + dumpGenerator.writeEndArray(); + dumpGenerator.writeEndObject(); + dumpGenerator.flush(); + } + + /** + * Get the {@link ClassLoader} for this job. + * + * @return the correct class loader. + */ + public ClassLoader getClassLoader() { + return classLoader; + } + + /** + * Set the class loader that will be used to load the various objects. + * + * @param classLoader the new class loader. + */ + public void setClassLoader(ClassLoader classLoader) { + this.classLoader = classLoader; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("Configuration: "); + if(loadDefaults) { + toString(defaultResources, sb); + if(resources.size()>0) { + sb.append(", "); + } + } + toString(resources, sb); + return sb.toString(); + } + + private void toString(List resources, StringBuilder sb) { + ListIterator i = resources.listIterator(); + while (i.hasNext()) { + if (i.nextIndex() != 0) { + sb.append(", "); + } + sb.append(i.next()); + } + } + + /** + * Set the quietness-mode. + * + * In the quiet-mode, error and informational messages might not be logged. + * + * @param quietmode true to set quiet-mode on, false + * to turn it off. + */ + public synchronized void setQuietMode(boolean quietmode) { + this.quietmode = quietmode; + } + + synchronized boolean getQuietMode() { + return this.quietmode; + } + + /** For debugging. List non-default properties to the terminal and exit. */ + public static void main(String[] args) throws Exception { + new Configuration().writeXml(System.out); + } + + @Override + public void readFields(DataInput in) throws IOException { + clear(); + int size = WritableUtils.readVInt(in); + for(int i=0; i < size; ++i) { + String key = org.apache.hadoop.io.Text.readString(in); + String value = org.apache.hadoop.io.Text.readString(in); + set(key, value); + String sources[] = WritableUtils.readCompressedStringArray(in); + if(sources != null) { + updatingResource.put(key, sources); + } + } + } + + //@Override + @Override + public void write(DataOutput out) throws IOException { + Properties props = getProps(); + WritableUtils.writeVInt(out, props.size()); + for(Map.Entry item: props.entrySet()) { + org.apache.hadoop.io.Text.writeString(out, (String) item.getKey()); + org.apache.hadoop.io.Text.writeString(out, (String) item.getValue()); + WritableUtils.writeCompressedStringArray(out, + updatingResource.get(item.getKey())); + } + } + + /** + * get keys matching the the regex + * @param regex + * @return Map with matching keys + */ + public Map getValByRegex(String regex) { + Pattern p = Pattern.compile(regex); + + Map result = new HashMap(); + Matcher m; + + for(Map.Entry item: getProps().entrySet()) { + if (item.getKey() instanceof String && + item.getValue() instanceof String) { + m = p.matcher((String)item.getKey()); + if(m.find()) { // match + result.put((String) item.getKey(), + substituteVars(getProps().getProperty((String) item.getKey()))); + } + } + } + return result; + } + + /** + * A unique class which is used as a sentinel value in the caching + * for getClassByName. {@see Configuration#getClassByNameOrNull(String)} + */ + private static abstract class NegativeCacheSentinel {} + + public static void dumpDeprecatedKeys() { + DeprecationContext deprecations = deprecationContext.get(); + for (Map.Entry entry : + deprecations.getDeprecatedKeyMap().entrySet()) { + StringBuilder newKeys = new StringBuilder(); + for (String newKey : entry.getValue().newKeys) { + newKeys.append(newKey).append("\t"); + } + System.out.println(entry.getKey() + "\t" + newKeys.toString()); + } + } + + /** + * Returns whether or not a deprecated name has been warned. If the name is not + * deprecated then always return false + */ + public static boolean hasWarnedDeprecation(String name) { + DeprecationContext deprecations = deprecationContext.get(); + if(deprecations.getDeprecatedKeyMap().containsKey(name)) { + if(deprecations.getDeprecatedKeyMap().get(name).accessed.get()) { + return true; + } + } + return false; + } +} diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/core-default-shaded.xml b/flink-filesystems/flink-s3-fs-presto/src/main/resources/core-default-shaded.xml new file mode 100644 index 0000000000000..cfb2ab29f0ae6 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/core-default-shaded.xml @@ -0,0 +1,1978 @@ + + + + + + + + + + + + + + + hadoop.common.configuration.version + 0.23.0 + version of this configuration file + + + + hadoop.tmp.dir + /tmp/hadoop-${user.name} + A base for other temporary directories. + + + + io.native.lib.available + true + Controls whether to use native libraries for bz2 and zlib + compression codecs or not. The property does not control any other native + libraries. + + + + + hadoop.http.filter.initializers + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.http.lib.StaticUserWebFilter + A comma separated list of class names. Each class in the list + must extend org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.http.FilterInitializer. The corresponding + Filter will be initialized. Then, the Filter will be applied to all user + facing jsp and servlet web pages. The ordering of the list defines the + ordering of the filters. + + + + + + hadoop.security.authorization + false + Is service-level authorization enabled? + + + + hadoop.security.instrumentation.requires.admin + false + + Indicates if administrator ACLs are required to access + instrumentation servlets (JMX, METRICS, CONF, STACKS). + + + + + hadoop.security.authentication + simple + Possible values are simple (no authentication), and kerberos + + + + + hadoop.security.group.mapping + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback + + Class for user to group mapping (get groups for a given user) for ACL. + The default implementation, + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback, + will determine if the Java Native Interface (JNI) is available. If JNI is + available the implementation will use the API within hadoop to resolve a + list of groups for a user. If JNI is not available then the shell + implementation, ShellBasedUnixGroupsMapping, is used. This implementation + shells out to the Linux/Unix environment with the + bash -c groups command to resolve a list of groups for a user. + + + + + + + hadoop.security.groups.cache.secs + 300 + + This is the config controlling the validity of the entries in the cache + containing the user->group mapping. When this duration has expired, + then the implementation of the group mapping provider is invoked to get + the groups of the user and then cached back. + + + + + hadoop.security.groups.negative-cache.secs + 30 + + Expiration time for entries in the the negative user-to-group mapping + caching, in seconds. This is useful when invalid users are retrying + frequently. It is suggested to set a small value for this expiration, since + a transient error in group lookup could temporarily lock out a legitimate + user. + + Set this to zero or negative value to disable negative user-to-group caching. + + + + + hadoop.security.groups.cache.warn.after.ms + 5000 + + If looking up a single user to group takes longer than this amount of + milliseconds, we will log a warning message. + + + + + hadoop.security.group.mapping.ldap.url + + + The URL of the LDAP server to use for resolving user groups when using + the LdapGroupsMapping user to group mapping. + + + + + hadoop.security.group.mapping.ldap.ssl + false + + Whether or not to use SSL when connecting to the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore + + + File path to the SSL keystore that contains the SSL certificate required + by the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore.password.file + + + The path to a file containing the password of the LDAP SSL keystore. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.bind.user + + + The distinguished name of the user to bind as when connecting to the LDAP + server. This may be left blank if the LDAP server supports anonymous binds. + + + + + hadoop.security.group.mapping.ldap.bind.password.file + + + The path to a file containing the password of the bind user. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.base + + + The search base for the LDAP connection. This is a distinguished name, + and will typically be the root of the LDAP directory. + + + + + hadoop.security.group.mapping.ldap.search.filter.user + (&(objectClass=user)(sAMAccountName={0})) + + An additional filter to use when searching for LDAP users. The default will + usually be appropriate for Active Directory installations. If connecting to + an LDAP server with a non-AD schema, this should be replaced with + (&(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to + denote where the username fits into the filter. + + + + + hadoop.security.group.mapping.ldap.search.filter.group + (objectClass=group) + + An additional filter to use when searching for LDAP groups. This should be + changed when resolving groups against a non-Active Directory installation. + posixGroups are currently not a supported group class. + + + + + hadoop.security.group.mapping.ldap.search.attr.member + member + + The attribute of the group object that identifies the users that are + members of the group. The default will usually be appropriate for + any LDAP installation. + + + + + hadoop.security.group.mapping.ldap.search.attr.group.name + cn + + The attribute of the group object that identifies the group name. The + default will usually be appropriate for all LDAP systems. + + + + + hadoop.security.group.mapping.ldap.directory.search.timeout + 10000 + + The attribute applied to the LDAP SearchControl properties to set a + maximum time limit when searching and awaiting a result. + Set to 0 if infinite wait period is desired. + Default is 10 seconds. Units in milliseconds. + + + + + hadoop.security.service.user.name.key + + + For those cases where the same RPC protocol is implemented by multiple + servers, this configuration is required for specifying the principal + name to use for the service when the client wishes to make an RPC call. + + + + + + hadoop.security.uid.cache.secs + 14400 + + This is the config controlling the validity of the entries in the cache + containing the userId to userName and groupId to groupName used by + NativeIO getFstat(). + + + + + hadoop.rpc.protection + authentication + A comma-separated list of protection values for secured sasl + connections. Possible values are authentication, integrity and privacy. + authentication means authentication only and no integrity or privacy; + integrity implies authentication and integrity are enabled; and privacy + implies all of authentication, integrity and privacy are enabled. + hadoop.security.saslproperties.resolver.class can be used to override + the hadoop.rpc.protection for a connection at the server side. + + + + + hadoop.security.saslproperties.resolver.class + + SaslPropertiesResolver used to resolve the QOP used for a + connection. If not specified, the full set of values specified in + hadoop.rpc.protection is used while determining the QOP used for the + connection. If a class is specified, then the QOP values returned by + the class will be used while determining the QOP used for the connection. + + + + + hadoop.work.around.non.threadsafe.getpwuid + false + Some operating systems or authentication modules are known to + have broken implementations of getpwuid_r and getpwgid_r, such that these + calls are not thread-safe. Symptoms of this problem include JVM crashes + with a stack trace inside these functions. If your system exhibits this + issue, enable this configuration parameter to include a lock around the + calls as a workaround. + + An incomplete list of some systems known to have this issue is available + at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations + + + + + hadoop.kerberos.kinit.command + kinit + Used to periodically renew Kerberos credentials when provided + to Hadoop. The default setting assumes that kinit is in the PATH of users + running the Hadoop client. Change this to the absolute path to kinit if this + is not the case. + + + + + hadoop.security.auth_to_local + + Maps kerberos principals to local user names + + + + + io.file.buffer.size + 4096 + The size of buffer for use in sequence files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + io.bytes.per.checksum + 512 + The number of bytes per checksum. Must not be larger than + io.file.buffer.size. + + + + io.skip.checksum.errors + false + If true, when a checksum error is encountered while + reading a sequence file, entries are skipped, instead of throwing an + exception. + + + + io.compression.codecs + + A comma-separated list of the compression codec classes that can + be used for compression/decompression. In addition to any classes specified + with this property (which take precedence), codec classes on the classpath + are discovered using a Java ServiceLoader. + + + + io.compression.codec.bzip2.library + system-native + The native-code library to be used for compression and + decompression by the bzip2 codec. This library could be specified + either by by name or the full pathname. In the former case, the + library is located by the dynamic linker, usually searching the + directories specified in the environment variable LD_LIBRARY_PATH. + + The value of "system-native" indicates that the default system + library should be used. To indicate that the algorithm should + operate entirely in Java, specify "java-builtin". + + + + io.serializations + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.io.serializer.WritableSerialization,org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.io.serializer.avro.AvroReflectSerialization + A list of serialization classes that can be used for + obtaining serializers and deserializers. + + + + io.seqfile.local.dir + ${hadoop.tmp.dir}/io/local + The local directory where sequence file stores intermediate + data files during merge. May be a comma-separated list of + directories on different devices in order to spread disk i/o. + Directories that do not exist are ignored. + + + + + io.map.index.skip + 0 + Number of index entries to skip between each entry. + Zero by default. Setting this to values larger than zero can + facilitate opening large MapFiles using less memory. + + + + io.map.index.interval + 128 + + MapFile consist of two files - data file (tuples) and index file + (keys). For every io.map.index.interval records written in the + data file, an entry (record-key, data-file-position) is written + in the index file. This is to allow for doing binary search later + within the index file to look up records by their keys and get their + closest positions in the data file. + + + + + + + fs.defaultFS + file:/// + The name of the default file system. A URI whose + scheme and authority determine the FileSystem implementation. The + uri's scheme determines the config property (fs.SCHEME.impl) naming + the FileSystem implementation class. The uri's authority is used to + determine the host, port, etc. for a filesystem. + + + + fs.default.name + file:/// + Deprecated. Use (fs.defaultFS) property + instead + + + + fs.trash.interval + 0 + Number of minutes after which the checkpoint + gets deleted. If zero, the trash feature is disabled. + This option may be configured both on the server and the + client. If trash is disabled server side then the client + side configuration is checked. If trash is enabled on the + server side then the value configured on the server is + used and the client configuration value is ignored. + + + + + fs.trash.checkpoint.interval + 0 + Number of minutes between trash checkpoints. + Should be smaller or equal to fs.trash.interval. If zero, + the value is set to the value of fs.trash.interval. + Every time the checkpointer runs it creates a new checkpoint + out of current and removes checkpoints created more than + fs.trash.interval minutes ago. + + + + + fs.AbstractFileSystem.file.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.local.LocalFs + The AbstractFileSystem for file: uris. + + + + fs.AbstractFileSystem.har.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.HarFs + The AbstractFileSystem for har: uris. + + + + fs.AbstractFileSystem.hdfs.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.Hdfs + The FileSystem for hdfs: uris. + + + + fs.AbstractFileSystem.viewfs.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.viewfs.ViewFs + The AbstractFileSystem for view file system for viewfs: uris + (ie client side mount table:). + + + + fs.AbstractFileSystem.ftp.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.ftp.FtpFs + The FileSystem for Ftp: uris. + + + + fs.ftp.host + 0.0.0.0 + FTP filesystem connects to this server + + + + fs.ftp.host.port + 21 + + FTP filesystem connects to fs.ftp.host on this port + + + + + fs.df.interval + 60000 + Disk usage statistics refresh interval in msec. + + + + fs.du.interval + 600000 + File space usage statistics refresh interval in msec. + + + + fs.s3.block.size + 67108864 + Block size to use when writing files to S3. + + + + fs.s3.buffer.dir + ${hadoop.tmp.dir}/s3 + Determines where on the local filesystem the S3 filesystem + should store files before sending them to S3 + (or after retrieving them from S3). + + + + + fs.s3.maxRetries + 4 + The maximum number of retries for reading or writing files to S3, + before we signal failure to the application. + + + + + fs.s3.sleepTimeSeconds + 10 + The number of seconds to sleep between each S3 retry. + + + + + fs.swift.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem + The implementation class of the OpenStack Swift Filesystem + + + + fs.automatic.close + true + By default, FileSystem instances are automatically closed at program + exit using a JVM shutdown hook. Setting this property to false disables this + behavior. This is an advanced option that should only be used by server applications + requiring a more carefully orchestrated shutdown sequence. + + + + + fs.s3n.block.size + 67108864 + Block size to use when reading files using the native S3 + filesystem (s3n: URIs). + + + + fs.s3n.multipart.uploads.enabled + false + Setting this property to true enables multiple uploads to + native S3 filesystem. When uploading a file, it is split into blocks + if the size is larger than fs.s3n.multipart.uploads.block.size. + + + + + fs.s3n.multipart.uploads.block.size + 67108864 + The block size for multipart uploads to native S3 filesystem. + Default size is 64MB. + + + + + fs.s3n.multipart.copy.block.size + 5368709120 + The block size for multipart copy in native S3 filesystem. + Default size is 5GB. + + + + + fs.s3n.server-side-encryption-algorithm + + Specify a server-side encryption algorithm for S3. + The default is NULL, and the only other currently allowable value is AES256. + + + + + fs.s3a.awsAccessKeyId + AWS access key ID. Omit for Role-based authentication. + + + + fs.s3a.awsSecretAccessKey + AWS secret key. Omit for Role-based authentication. + + + + fs.s3a.connection.maximum + 15 + Controls the maximum number of simultaneous connections to S3. + + + + fs.s3a.connection.ssl.enabled + true + Enables or disables SSL connections to S3. + + + + fs.s3a.endpoint + AWS S3 endpoint to connect to. An up-to-date list is + provided in the AWS Documentation: regions and endpoints. Without this + property, the standard region (s3.amazonaws.com) is assumed. + + + + + fs.s3a.proxy.host + Hostname of the (optional) proxy server for S3 connections. + + + + fs.s3a.proxy.port + Proxy server port. If this property is not set + but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with + the value of fs.s3a.connection.ssl.enabled). + + + + fs.s3a.proxy.username + Username for authenticating with proxy server. + + + + fs.s3a.proxy.password + Password for authenticating with proxy server. + + + + fs.s3a.proxy.domain + Domain for authenticating with proxy server. + + + + fs.s3a.proxy.workstation + Workstation for authenticating with proxy server. + + + + fs.s3a.attempts.maximum + 10 + How many times we should retry commands on transient errors. + + + + fs.s3a.connection.establish.timeout + 5000 + Socket connection setup timeout in milliseconds. + + + + fs.s3a.connection.timeout + 50000 + Socket connection timeout in milliseconds. + + + + fs.s3a.paging.maximum + 5000 + How many keys to request from S3 when doing + directory listings at a time. + + + + fs.s3a.threads.max + 256 + Maximum number of concurrent active (part)uploads, + which each use a thread from the threadpool. + + + + fs.s3a.threads.core + 15 + Number of core threads in the threadpool. + + + + fs.s3a.threads.keepalivetime + 60 + Number of seconds a thread can be idle before being + terminated. + + + + fs.s3a.max.total.tasks + 1000 + Number of (part)uploads allowed to the queue before + blocking additional uploads. + + + + fs.s3a.multipart.size + 104857600 + How big (in bytes) to split upload or copy operations up into. + + + + fs.s3a.multipart.threshold + 2147483647 + Threshold before uploads or copies use parallel multipart operations. + + + + fs.s3a.acl.default + Set a canned ACL for newly created and copied objects. Value may be private, + public-read, public-read-write, authenticated-read, log-delivery-write, + bucket-owner-read, or bucket-owner-full-control. + + + + fs.s3a.multipart.purge + false + True if you want to purge existing multipart uploads that may not have been + completed/aborted correctly + + + + fs.s3a.multipart.purge.age + 86400 + Minimum age in seconds of multipart uploads to purge + + + + fs.s3a.buffer.dir + ${hadoop.tmp.dir}/s3a + Comma separated list of directories that will be used to buffer file + uploads to. + + + + fs.s3a.fast.upload + false + Upload directly from memory instead of buffering to + disk first. Memory usage and parallelism can be controlled as up to + fs.s3a.multipart.size memory is consumed for each (part)upload actively + uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks) + + + + fs.s3a.fast.buffer.size + 1048576 + Size of initial memory buffer in bytes allocated for an + upload. No effect if fs.s3a.fast.upload is false. + + + + fs.s3a.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.fs.s3a.S3AFileSystem + The implementation class of the S3A Filesystem + + + + io.seqfile.compress.blocksize + 1000000 + The minimum block size for compression in block compressed + SequenceFiles. + + + + + io.seqfile.lazydecompress + true + Should values of block-compressed SequenceFiles be decompressed + only when necessary. + + + + + io.seqfile.sorter.recordlimit + 1000000 + The limit on number of records to be kept in memory in a spill + in SequenceFiles.Sorter + + + + + io.mapfile.bloom.size + 1048576 + The size of BloomFilter-s used in BloomMapFile. Each time this many + keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter). + Larger values minimize the number of filters, which slightly increases the performance, + but may waste too much space if the total number of keys is usually much smaller + than this number. + + + + + io.mapfile.bloom.error.rate + 0.005 + The rate of false positives in BloomFilter-s used in BloomMapFile. + As this value decreases, the size of BloomFilter-s increases exponentially. This + value is the probability of encountering false positives (default is 0.5%). + + + + + hadoop.util.hash.type + murmur + The default implementation of Hash. Currently this can take one of the + two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash. + + + + + + + + ipc.client.idlethreshold + 4000 + Defines the threshold number of connections after which + connections will be inspected for idleness. + + + + + ipc.client.kill.max + 10 + Defines the maximum number of clients to disconnect in one go. + + + + + ipc.client.connection.maxidletime + 10000 + The maximum time in msec after which a client will bring down the + connection to the server. + + + + + ipc.client.connect.max.retries + 10 + Indicates the number of retries a client will make to establish + a server connection. + + + + + ipc.client.connect.retry.interval + 1000 + Indicates the number of milliseconds a client will wait for + before retrying to establish a server connection. + + + + + ipc.client.connect.timeout + 20000 + Indicates the number of milliseconds a client will wait for the + socket to establish a server connection. + + + + + ipc.client.connect.max.retries.on.timeouts + 45 + Indicates the number of retries a client will make on socket timeout + to establish a server connection. + + + + + ipc.client.ping + true + Send a ping to the server when timeout on reading the response, + if set to true. If no failure is detected, the client retries until at least + a byte is read. + + + + + ipc.ping.interval + 60000 + Timeout on waiting response from server, in milliseconds. + The client will send ping when the interval is passed without receiving bytes, + if ipc.client.ping is set to true. + + + + + ipc.client.rpc-timeout.ms + 0 + Timeout on waiting response from server, in milliseconds. + Currently this timeout works only when ipc.client.ping is set to true + because it uses the same facilities with IPC ping. + The timeout overrides the ipc.ping.interval and client will throw exception + instead of sending ping when the interval is passed. + + + + + ipc.server.listen.queue.size + 128 + Indicates the length of the listen queue for servers accepting + client connections. + + + + + ipc.maximum.data.length + 67108864 + This indicates the maximum IPC message length (bytes) that can be + accepted by the server. Messages larger than this value are rejected by + server immediately. This setting should rarely need to be changed. It merits + investigating whether the cause of long RPC messages can be fixed instead, + e.g. by splitting into smaller messages. + + + + + + + hadoop.security.impersonation.provider.class + + A class which implements ImpersonationProvider interface, used to + authorize whether one user can impersonate a specific user. + If not specified, the DefaultImpersonationProvider will be used. + If a class is specified, then that class will be used to determine + the impersonation capability. + + + + + hadoop.rpc.socket.factory.class.default + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.net.StandardSocketFactory + Default SocketFactory to use. This parameter is expected to be + formatted as "package.FactoryClassName". + + + + + hadoop.rpc.socket.factory.class.ClientProtocol + + SocketFactory to use to connect to a DFS. If null or empty, use + hadoop.rpc.socket.class.default. This socket factory is also used by + DFSClient to create sockets to DataNodes. + + + + + + + hadoop.socks.server + + Address (host:port) of the SOCKS server to be used by the + SocksSocketFactory. + + + + + + net.topology.node.switch.mapping.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.net.ScriptBasedMapping + The default implementation of the DNSToSwitchMapping. It + invokes a script specified in net.topology.script.file.name to resolve + node names. If the value for net.topology.script.file.name is not set, the + default value of DEFAULT_RACK is returned for all node names. + + + + + net.topology.impl + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.net.NetworkTopology + The default implementation of NetworkTopology which is classic three layer one. + + + + + net.topology.script.file.name + + The script name that should be invoked to resolve DNS names to + NetworkTopology names. Example: the script would take host.foo.bar as an + argument, and return /rack1 as the output. + + + + + net.topology.script.number.args + 100 + The max number of args that the script configured with + net.topology.script.file.name should be run with. Each arg is an + IP address. + + + + + net.topology.table.file.name + + The file name for a topology file, which is used when the + net.topology.node.switch.mapping.impl property is set to + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.net.TableMapping. The file format is a two column text + file, with columns separated by whitespace. The first column is a DNS or + IP address and the second column specifies the rack where the address maps. + If no entry corresponding to a host in the cluster is found, then + /default-rack is assumed. + + + + + + file.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + file.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + file.stream-buffer-size + + + + file.client-write-packet-size + 65536 + Packet size for clients to write + + + + file.blocksize + 67108864 + Block size + + + + file.replication + 1 + Replication factor + + + + + + s3.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3.stream-buffer-size + + + + s3.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3.blocksize + 67108864 + Block size + + + + s3.replication + 3 + Replication factor + + + + + + s3native.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3native.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3native.stream-buffer-size + + + + s3native.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3native.blocksize + 67108864 + Block size + + + + s3native.replication + 3 + Replication factor + + + + + ftp.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + ftp.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + ftp.stream-buffer-size + + + + ftp.client-write-packet-size + 65536 + Packet size for clients to write + + + + ftp.blocksize + 67108864 + Block size + + + + ftp.replication + 3 + Replication factor + + + + + + tfile.io.chunk.size + 1048576 + + Value chunk size in bytes. Default to + 1MB. Values of the length less than the chunk size is + guaranteed to have known value length in read time (See also + TFile.Reader.Scanner.Entry.isValueLengthKnown()). + + + + + tfile.fs.output.buffer.size + 262144 + + Buffer size used for FSDataOutputStream in bytes. + + + + + tfile.fs.input.buffer.size + 262144 + + Buffer size used for FSDataInputStream in bytes. + + + + + + + hadoop.http.authentication.type + simple + + Defines authentication used for Oozie HTTP endpoint. + Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME# + + + + + hadoop.http.authentication.token.validity + 36000 + + Indicates how long (in seconds) an authentication token is valid before it has + to be renewed. + + + + + hadoop.http.authentication.signature.secret.file + ${user.home}/hadoop-http-auth-signature-secret + + The signature secret for signing the authentication tokens. + The same secret should be used for JT/NN/DN/TT configurations. + + + + + hadoop.http.authentication.cookie.domain + + + The domain to use for the HTTP cookie that stores the authentication token. + In order to authentiation to work correctly across all Hadoop nodes web-consoles + the domain must be correctly set. + IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings. + For this setting to work properly all nodes in the cluster must be configured + to generate URLs with hostname.domain names on it. + + + + + hadoop.http.authentication.simple.anonymous.allowed + true + + Indicates if anonymous requests are allowed when using 'simple' authentication. + + + + + hadoop.http.authentication.kerberos.principal + HTTP/_HOST@LOCALHOST + + Indicates the Kerberos principal to be used for HTTP endpoint. + The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification. + + + + + hadoop.http.authentication.kerberos.keytab + ${user.home}/hadoop.keytab + + Location of the keytab file with the credentials for the principal. + Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop. + + + + + + Enable/disable the cross-origin (CORS) filter. + hadoop.http.cross-origin.enabled + false + + + + Comma separated list of origins that are allowed for web + services needing cross-origin (CORS) support. Wildcards (*) and patterns + allowed + hadoop.http.cross-origin.allowed-origins + * + + + + Comma separated list of methods that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-methods + GET,POST,HEAD + + + + Comma separated list of headers that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-headers + X-Requested-With,Content-Type,Accept,Origin + + + + The number of seconds a pre-flighted request can be cached + for web services needing cross-origin (CORS) support. + hadoop.http.cross-origin.max-age + 1800 + + + + dfs.ha.fencing.methods + + + List of fencing methods to use for service fencing. May contain + builtin methods (eg shell and sshfence) or user-defined method. + + + + + dfs.ha.fencing.ssh.connect-timeout + 30000 + + SSH connection timeout, in milliseconds, to use with the builtin + sshfence fencer. + + + + + dfs.ha.fencing.ssh.private-key-files + + + The SSH private key files to use with the builtin sshfence fencer. + + + + + + + + The user name to filter as, on static web filters + while rendering content. An example use is the HDFS + web UI (user to be used for browsing files). + + hadoop.http.staticuser.user + dr.who + + + + ha.zookeeper.quorum + + A list of ZooKeeper server addresses, separated by commas, that are + to be used by the ZKFailoverController in automatic failover. + + + + + ha.zookeeper.session-timeout.ms + 5000 + + The session timeout to use when the ZKFC connects to ZooKeeper. + Setting this value to a lower value implies that server crashes + will be detected more quickly, but risks triggering failover too + aggressively in the case of a transient error or network blip. + + + + + ha.zookeeper.parent-znode + /hadoop-ha + + The ZooKeeper znode under which the ZK failover controller stores + its information. Note that the nameservice ID is automatically + appended to this znode, so it is not normally necessary to + configure this, even in a federated environment. + + + + + ha.zookeeper.acl + world:anyone:rwcda + + A comma-separated list of ZooKeeper ACLs to apply to the znodes + used by automatic failover. These ACLs are specified in the same + format as used by the ZooKeeper CLI. + + If the ACL itself contains secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + ha.zookeeper.auth + + + A comma-separated list of ZooKeeper authentications to add when + connecting to ZooKeeper. These are specified in the same format + as used by the "addauth" command in the ZK CLI. It is + important that the authentications specified here are sufficient + to access znodes with the ACL specified in ha.zookeeper.acl. + + If the auths contain secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + + + hadoop.ssl.keystores.factory.class + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory + + The keystores factory to use for retrieving certificates. + + + + + hadoop.ssl.require.client.cert + false + Whether client certificates are required + + + + hadoop.ssl.hostname.verifier + DEFAULT + + The hostname verifier to provide for HttpsURLConnections. + Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and + ALLOW_ALL + + + + + hadoop.ssl.server.conf + ssl-server.xml + + Resource file from which ssl server keystore information will be extracted. + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.client.conf + ssl-client.xml + + Resource file from which ssl client keystore information will be extracted + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.enabled + false + + Deprecated. Use dfs.http.policy and yarn.http.policy instead. + + + + + hadoop.ssl.enabled.protocols + TLSv1 + + Protocols supported by the ssl. + + + + + hadoop.jetty.logs.serve.aliases + true + + Enable/Disable aliases serving from jetty + + + + + fs.permissions.umask-mode + 022 + + The umask used when creating files and directories. + Can be in octal or in symbolic. Examples are: + "022" (octal for u=rwx,g=r-x,o=r-x in symbolic), + or "u=rwx,g=rwx,o=" (symbolic for 007 in octal). + + + + + + + ha.health-monitor.connect-retry-interval.ms + 1000 + + How often to retry connecting to the service. + + + + + ha.health-monitor.check-interval.ms + 1000 + + How often to check the service. + + + + + ha.health-monitor.sleep-after-disconnect.ms + 1000 + + How long to sleep after an unexpected RPC error. + + + + + ha.health-monitor.rpc-timeout.ms + 45000 + + Timeout for the actual monitorHealth() calls. + + + + + ha.failover-controller.new-active.rpc-timeout.ms + 60000 + + Timeout that the FC waits for the new active to become active + + + + + ha.failover-controller.graceful-fence.rpc-timeout.ms + 5000 + + Timeout that the FC waits for the old active to go to standby + + + + + ha.failover-controller.graceful-fence.connection.retries + 1 + + FC connection retries for graceful fencing + + + + + ha.failover-controller.cli-check.rpc-timeout.ms + 20000 + + Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState + + + + + ipc.client.fallback-to-simple-auth-allowed + false + + When a client is configured to attempt a secure connection, but attempts to + connect to an insecure server, that server may instruct the client to + switch to SASL SIMPLE (unsecure) authentication. This setting controls + whether or not the client will accept this instruction from the server. + When false (the default), the client will not allow the fallback to SIMPLE + authentication, and will abort the connection. + + + + + fs.client.resolve.remote.symlinks + true + + Whether to resolve symlinks when accessing a remote Hadoop filesystem. + Setting this to false causes an exception to be thrown upon encountering + a symlink. This setting does not apply to local filesystems, which + automatically resolve local symlinks. + + + + + nfs.exports.allowed.hosts + * rw + + By default, the export can be mounted by any client. The value string + contains machine name and access privilege, separated by whitespace + characters. The machine name format can be a single host, a Java regular + expression, or an IPv4 address. The access privilege uses rw or ro to + specify read/write or read-only access of the machines to exports. If the + access privilege is not provided, the default is read-only. Entries are separated by ";". + For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;". + Only the NFS gateway needs to restart after this property is updated. + + + + + hadoop.user.group.static.mapping.overrides + dr.who=; + + Static mapping of user to groups. This will override the groups if + available in the system for the specified user. In otherwords, groups + look-up will not happen for these users, instead groups mapped in this + configuration will be used. + Mapping should be in this format. + user1=group1,group2;user2=;user3=group2; + Default, "dr.who=;" will consider "dr.who" as user without groups. + + + + + rpc.metrics.quantile.enable + false + + Setting this property to true and rpc.metrics.percentiles.intervals + to a comma-separated list of the granularity in seconds, the + 50/75/90/95/99th percentile latency for rpc queue/processing time in + milliseconds are added to rpc metrics. + + + + + rpc.metrics.percentiles.intervals + + + A comma-separated list of the granularity in seconds for the metrics which + describe the 50/75/90/95/99th percentile latency for rpc queue/processing + time. The metrics are outputted if rpc.metrics.quantile.enable is set to + true. + + + + + hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE + + + The prefix for a given crypto codec, contains a comma-separated + list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE). + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.codec.classes.aes.ctr.nopadding + org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec,org.apache.flink.fs.s3presto.shaded.org.apache.hadoop.crypto.JceAesCtrCryptoCodec + + Comma-separated list of crypto codec implementations for AES/CTR/NoPadding. + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.cipher.suite + AES/CTR/NoPadding + + Cipher suite for crypto codec. + + + + + hadoop.security.crypto.jce.provider + + + The JCE provider name used in CryptoCodec. + + + + + hadoop.security.crypto.buffer.size + 8192 + + The buffer size used by CryptoInputStream and CryptoOutputStream. + + + + + hadoop.security.java.secure.random.algorithm + SHA1PRNG + + The java secure random algorithm. + + + + + hadoop.security.secure.random.impl + + + Implementation of secure random. + + + + + hadoop.security.random.device.file.path + /dev/urandom + + OS security random device file path. + + + + + fs.har.impl.disable.cache + true + Don't cache 'har' filesystem instances. + + + + + hadoop.security.kms.client.authentication.retry-count + 1 + + Number of time to retry connecting to KMS on authentication failure + + + + hadoop.security.kms.client.encrypted.key.cache.size + 500 + + Size of the EncryptedKeyVersion cache Queue for each key + + + + hadoop.security.kms.client.encrypted.key.cache.low-watermark + 0.3f + + If size of the EncryptedKeyVersion cache Queue falls below the + low watermark, this cache queue will be scheduled for a refill + + + + hadoop.security.kms.client.encrypted.key.cache.num.refill.threads + 2 + + Number of threads to use for refilling depleted EncryptedKeyVersion + cache Queues + + + + hadoop.security.kms.client.encrypted.key.cache.expiry + 43200000 + + Cache expiry time for a Key, after which the cache Queue for this + key will be dropped. Default = 12hrs + + + + + hadoop.htrace.spanreceiver.classes + + + A comma separated list of the fully-qualified class name of classes + implementing SpanReceiver. The tracing system works by collecting + information in structs called 'Spans'. It is up to you to choose + how you want to receive this information by implementing the + SpanReceiver interface. + + + + + ipc.server.max.connections + 0 + The maximum number of concurrent connections a server is allowed + to accept. If this limit is exceeded, incoming connections will first fill + the listen queue and then may go to an OS-specific listen overflow queue. + The client may fail or timeout, but the server can avoid running out of file + descriptors using this feature. 0 means no limit. + + + + + + + + + Is the registry enabled in the YARN Resource Manager? + + If true, the YARN RM will, as needed. + create the user and system paths, and purge + service records when containers, application attempts + and applications complete. + + If false, the paths must be created by other means, + and no automatic cleanup of service records will take place. + + hadoop.registry.rm.enabled + false + + + + + The root zookeeper node for the registry + + hadoop.registry.zk.root + /registry + + + + + Zookeeper session timeout in milliseconds + + hadoop.registry.zk.session.timeout.ms + 60000 + + + + + Zookeeper connection timeout in milliseconds + + hadoop.registry.zk.connection.timeout.ms + 15000 + + + + + Zookeeper connection retry count before failing + + hadoop.registry.zk.retry.times + 5 + + + + + + hadoop.registry.zk.retry.interval.ms + 1000 + + + + + Zookeeper retry limit in milliseconds, during + exponential backoff. + + This places a limit even + if the retry times and interval limit, combined + with the backoff policy, result in a long retry + period + + hadoop.registry.zk.retry.ceiling.ms + 60000 + + + + + List of hostname:port pairs defining the + zookeeper quorum binding for the registry + + hadoop.registry.zk.quorum + localhost:2181 + + + + + Key to set if the registry is secure. Turning it on + changes the permissions policy from "open access" + to restrictions on kerberos with the option of + a user adding one or more auth key pairs down their + own tree. + + hadoop.registry.secure + false + + + + + A comma separated list of Zookeeper ACL identifiers with + system access to the registry in a secure cluster. + + These are given full access to all entries. + + If there is an "@" at the end of a SASL entry it + instructs the registry client to append the default kerberos domain. + + hadoop.registry.system.acls + sasl:yarn@, sasl:mapred@, sasl:hdfs@ + + + + + The kerberos realm: used to set the realm of + system principals which do not declare their realm, + and any other accounts that need the value. + + If empty, the default realm of the running process + is used. + + If neither are known and the realm is needed, then the registry + service/client will fail. + + hadoop.registry.kerberos.realm + + + + + + Key to define the JAAS context. Used in secure + mode + + hadoop.registry.jaas.context + Client + + + diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml b/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml new file mode 100644 index 0000000000000..85c1f0d76f40f --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml @@ -0,0 +1,1978 @@ + + + + + + + + + + + + + + + hadoop.common.configuration.version + 0.23.0 + version of this configuration file + + + + hadoop.tmp.dir + /tmp/hadoop-${user.name} + A base for other temporary directories. + + + + io.native.lib.available + true + Controls whether to use native libraries for bz2 and zlib + compression codecs or not. The property does not control any other native + libraries. + + + + + hadoop.http.filter.initializers + org.apache.hadoop.http.lib.StaticUserWebFilter + A comma separated list of class names. Each class in the list + must extend org.apache.hadoop.http.FilterInitializer. The corresponding + Filter will be initialized. Then, the Filter will be applied to all user + facing jsp and servlet web pages. The ordering of the list defines the + ordering of the filters. + + + + + + hadoop.security.authorization + false + Is service-level authorization enabled? + + + + hadoop.security.instrumentation.requires.admin + false + + Indicates if administrator ACLs are required to access + instrumentation servlets (JMX, METRICS, CONF, STACKS). + + + + + hadoop.security.authentication + simple + Possible values are simple (no authentication), and kerberos + + + + + hadoop.security.group.mapping + org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback + + Class for user to group mapping (get groups for a given user) for ACL. + The default implementation, + org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback, + will determine if the Java Native Interface (JNI) is available. If JNI is + available the implementation will use the API within hadoop to resolve a + list of groups for a user. If JNI is not available then the shell + implementation, ShellBasedUnixGroupsMapping, is used. This implementation + shells out to the Linux/Unix environment with the + bash -c groups command to resolve a list of groups for a user. + + + + + + + hadoop.security.groups.cache.secs + 300 + + This is the config controlling the validity of the entries in the cache + containing the user->group mapping. When this duration has expired, + then the implementation of the group mapping provider is invoked to get + the groups of the user and then cached back. + + + + + hadoop.security.groups.negative-cache.secs + 30 + + Expiration time for entries in the the negative user-to-group mapping + caching, in seconds. This is useful when invalid users are retrying + frequently. It is suggested to set a small value for this expiration, since + a transient error in group lookup could temporarily lock out a legitimate + user. + + Set this to zero or negative value to disable negative user-to-group caching. + + + + + hadoop.security.groups.cache.warn.after.ms + 5000 + + If looking up a single user to group takes longer than this amount of + milliseconds, we will log a warning message. + + + + + hadoop.security.group.mapping.ldap.url + + + The URL of the LDAP server to use for resolving user groups when using + the LdapGroupsMapping user to group mapping. + + + + + hadoop.security.group.mapping.ldap.ssl + false + + Whether or not to use SSL when connecting to the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore + + + File path to the SSL keystore that contains the SSL certificate required + by the LDAP server. + + + + + hadoop.security.group.mapping.ldap.ssl.keystore.password.file + + + The path to a file containing the password of the LDAP SSL keystore. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.bind.user + + + The distinguished name of the user to bind as when connecting to the LDAP + server. This may be left blank if the LDAP server supports anonymous binds. + + + + + hadoop.security.group.mapping.ldap.bind.password.file + + + The path to a file containing the password of the bind user. + + IMPORTANT: This file should be readable only by the Unix user running + the daemons. + + + + + hadoop.security.group.mapping.ldap.base + + + The search base for the LDAP connection. This is a distinguished name, + and will typically be the root of the LDAP directory. + + + + + hadoop.security.group.mapping.ldap.search.filter.user + (&(objectClass=user)(sAMAccountName={0})) + + An additional filter to use when searching for LDAP users. The default will + usually be appropriate for Active Directory installations. If connecting to + an LDAP server with a non-AD schema, this should be replaced with + (&(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to + denote where the username fits into the filter. + + + + + hadoop.security.group.mapping.ldap.search.filter.group + (objectClass=group) + + An additional filter to use when searching for LDAP groups. This should be + changed when resolving groups against a non-Active Directory installation. + posixGroups are currently not a supported group class. + + + + + hadoop.security.group.mapping.ldap.search.attr.member + member + + The attribute of the group object that identifies the users that are + members of the group. The default will usually be appropriate for + any LDAP installation. + + + + + hadoop.security.group.mapping.ldap.search.attr.group.name + cn + + The attribute of the group object that identifies the group name. The + default will usually be appropriate for all LDAP systems. + + + + + hadoop.security.group.mapping.ldap.directory.search.timeout + 10000 + + The attribute applied to the LDAP SearchControl properties to set a + maximum time limit when searching and awaiting a result. + Set to 0 if infinite wait period is desired. + Default is 10 seconds. Units in milliseconds. + + + + + hadoop.security.service.user.name.key + + + For those cases where the same RPC protocol is implemented by multiple + servers, this configuration is required for specifying the principal + name to use for the service when the client wishes to make an RPC call. + + + + + + hadoop.security.uid.cache.secs + 14400 + + This is the config controlling the validity of the entries in the cache + containing the userId to userName and groupId to groupName used by + NativeIO getFstat(). + + + + + hadoop.rpc.protection + authentication + A comma-separated list of protection values for secured sasl + connections. Possible values are authentication, integrity and privacy. + authentication means authentication only and no integrity or privacy; + integrity implies authentication and integrity are enabled; and privacy + implies all of authentication, integrity and privacy are enabled. + hadoop.security.saslproperties.resolver.class can be used to override + the hadoop.rpc.protection for a connection at the server side. + + + + + hadoop.security.saslproperties.resolver.class + + SaslPropertiesResolver used to resolve the QOP used for a + connection. If not specified, the full set of values specified in + hadoop.rpc.protection is used while determining the QOP used for the + connection. If a class is specified, then the QOP values returned by + the class will be used while determining the QOP used for the connection. + + + + + hadoop.work.around.non.threadsafe.getpwuid + false + Some operating systems or authentication modules are known to + have broken implementations of getpwuid_r and getpwgid_r, such that these + calls are not thread-safe. Symptoms of this problem include JVM crashes + with a stack trace inside these functions. If your system exhibits this + issue, enable this configuration parameter to include a lock around the + calls as a workaround. + + An incomplete list of some systems known to have this issue is available + at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations + + + + + hadoop.kerberos.kinit.command + kinit + Used to periodically renew Kerberos credentials when provided + to Hadoop. The default setting assumes that kinit is in the PATH of users + running the Hadoop client. Change this to the absolute path to kinit if this + is not the case. + + + + + hadoop.security.auth_to_local + + Maps kerberos principals to local user names + + + + + io.file.buffer.size + 4096 + The size of buffer for use in sequence files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + io.bytes.per.checksum + 512 + The number of bytes per checksum. Must not be larger than + io.file.buffer.size. + + + + io.skip.checksum.errors + false + If true, when a checksum error is encountered while + reading a sequence file, entries are skipped, instead of throwing an + exception. + + + + io.compression.codecs + + A comma-separated list of the compression codec classes that can + be used for compression/decompression. In addition to any classes specified + with this property (which take precedence), codec classes on the classpath + are discovered using a Java ServiceLoader. + + + + io.compression.codec.bzip2.library + system-native + The native-code library to be used for compression and + decompression by the bzip2 codec. This library could be specified + either by by name or the full pathname. In the former case, the + library is located by the dynamic linker, usually searching the + directories specified in the environment variable LD_LIBRARY_PATH. + + The value of "system-native" indicates that the default system + library should be used. To indicate that the algorithm should + operate entirely in Java, specify "java-builtin". + + + + io.serializations + org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization + A list of serialization classes that can be used for + obtaining serializers and deserializers. + + + + io.seqfile.local.dir + ${hadoop.tmp.dir}/io/local + The local directory where sequence file stores intermediate + data files during merge. May be a comma-separated list of + directories on different devices in order to spread disk i/o. + Directories that do not exist are ignored. + + + + + io.map.index.skip + 0 + Number of index entries to skip between each entry. + Zero by default. Setting this to values larger than zero can + facilitate opening large MapFiles using less memory. + + + + io.map.index.interval + 128 + + MapFile consist of two files - data file (tuples) and index file + (keys). For every io.map.index.interval records written in the + data file, an entry (record-key, data-file-position) is written + in the index file. This is to allow for doing binary search later + within the index file to look up records by their keys and get their + closest positions in the data file. + + + + + + + fs.defaultFS + file:/// + The name of the default file system. A URI whose + scheme and authority determine the FileSystem implementation. The + uri's scheme determines the config property (fs.SCHEME.impl) naming + the FileSystem implementation class. The uri's authority is used to + determine the host, port, etc. for a filesystem. + + + + fs.default.name + file:/// + Deprecated. Use (fs.defaultFS) property + instead + + + + fs.trash.interval + 0 + Number of minutes after which the checkpoint + gets deleted. If zero, the trash feature is disabled. + This option may be configured both on the server and the + client. If trash is disabled server side then the client + side configuration is checked. If trash is enabled on the + server side then the value configured on the server is + used and the client configuration value is ignored. + + + + + fs.trash.checkpoint.interval + 0 + Number of minutes between trash checkpoints. + Should be smaller or equal to fs.trash.interval. If zero, + the value is set to the value of fs.trash.interval. + Every time the checkpointer runs it creates a new checkpoint + out of current and removes checkpoints created more than + fs.trash.interval minutes ago. + + + + + fs.AbstractFileSystem.file.impl + org.apache.hadoop.fs.local.LocalFs + The AbstractFileSystem for file: uris. + + + + fs.AbstractFileSystem.har.impl + org.apache.hadoop.fs.HarFs + The AbstractFileSystem for har: uris. + + + + fs.AbstractFileSystem.hdfs.impl + org.apache.hadoop.fs.Hdfs + The FileSystem for hdfs: uris. + + + + fs.AbstractFileSystem.viewfs.impl + org.apache.hadoop.fs.viewfs.ViewFs + The AbstractFileSystem for view file system for viewfs: uris + (ie client side mount table:). + + + + fs.AbstractFileSystem.ftp.impl + org.apache.hadoop.fs.ftp.FtpFs + The FileSystem for Ftp: uris. + + + + fs.ftp.host + 0.0.0.0 + FTP filesystem connects to this server + + + + fs.ftp.host.port + 21 + + FTP filesystem connects to fs.ftp.host on this port + + + + + fs.df.interval + 60000 + Disk usage statistics refresh interval in msec. + + + + fs.du.interval + 600000 + File space usage statistics refresh interval in msec. + + + + fs.s3.block.size + 67108864 + Block size to use when writing files to S3. + + + + fs.s3.buffer.dir + ${hadoop.tmp.dir}/s3 + Determines where on the local filesystem the S3 filesystem + should store files before sending them to S3 + (or after retrieving them from S3). + + + + + fs.s3.maxRetries + 4 + The maximum number of retries for reading or writing files to S3, + before we signal failure to the application. + + + + + fs.s3.sleepTimeSeconds + 10 + The number of seconds to sleep between each S3 retry. + + + + + fs.swift.impl + org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem + The implementation class of the OpenStack Swift Filesystem + + + + fs.automatic.close + true + By default, FileSystem instances are automatically closed at program + exit using a JVM shutdown hook. Setting this property to false disables this + behavior. This is an advanced option that should only be used by server applications + requiring a more carefully orchestrated shutdown sequence. + + + + + fs.s3n.block.size + 67108864 + Block size to use when reading files using the native S3 + filesystem (s3n: URIs). + + + + fs.s3n.multipart.uploads.enabled + false + Setting this property to true enables multiple uploads to + native S3 filesystem. When uploading a file, it is split into blocks + if the size is larger than fs.s3n.multipart.uploads.block.size. + + + + + fs.s3n.multipart.uploads.block.size + 67108864 + The block size for multipart uploads to native S3 filesystem. + Default size is 64MB. + + + + + fs.s3n.multipart.copy.block.size + 5368709120 + The block size for multipart copy in native S3 filesystem. + Default size is 5GB. + + + + + fs.s3n.server-side-encryption-algorithm + + Specify a server-side encryption algorithm for S3. + The default is NULL, and the only other currently allowable value is AES256. + + + + + fs.s3a.awsAccessKeyId + AWS access key ID. Omit for Role-based authentication. + + + + fs.s3a.awsSecretAccessKey + AWS secret key. Omit for Role-based authentication. + + + + fs.s3a.connection.maximum + 15 + Controls the maximum number of simultaneous connections to S3. + + + + fs.s3a.connection.ssl.enabled + true + Enables or disables SSL connections to S3. + + + + fs.s3a.endpoint + AWS S3 endpoint to connect to. An up-to-date list is + provided in the AWS Documentation: regions and endpoints. Without this + property, the standard region (s3.amazonaws.com) is assumed. + + + + + fs.s3a.proxy.host + Hostname of the (optional) proxy server for S3 connections. + + + + fs.s3a.proxy.port + Proxy server port. If this property is not set + but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with + the value of fs.s3a.connection.ssl.enabled). + + + + fs.s3a.proxy.username + Username for authenticating with proxy server. + + + + fs.s3a.proxy.password + Password for authenticating with proxy server. + + + + fs.s3a.proxy.domain + Domain for authenticating with proxy server. + + + + fs.s3a.proxy.workstation + Workstation for authenticating with proxy server. + + + + fs.s3a.attempts.maximum + 10 + How many times we should retry commands on transient errors. + + + + fs.s3a.connection.establish.timeout + 5000 + Socket connection setup timeout in milliseconds. + + + + fs.s3a.connection.timeout + 50000 + Socket connection timeout in milliseconds. + + + + fs.s3a.paging.maximum + 5000 + How many keys to request from S3 when doing + directory listings at a time. + + + + fs.s3a.threads.max + 256 + Maximum number of concurrent active (part)uploads, + which each use a thread from the threadpool. + + + + fs.s3a.threads.core + 15 + Number of core threads in the threadpool. + + + + fs.s3a.threads.keepalivetime + 60 + Number of seconds a thread can be idle before being + terminated. + + + + fs.s3a.max.total.tasks + 1000 + Number of (part)uploads allowed to the queue before + blocking additional uploads. + + + + fs.s3a.multipart.size + 104857600 + How big (in bytes) to split upload or copy operations up into. + + + + fs.s3a.multipart.threshold + 2147483647 + Threshold before uploads or copies use parallel multipart operations. + + + + fs.s3a.acl.default + Set a canned ACL for newly created and copied objects. Value may be private, + public-read, public-read-write, authenticated-read, log-delivery-write, + bucket-owner-read, or bucket-owner-full-control. + + + + fs.s3a.multipart.purge + false + True if you want to purge existing multipart uploads that may not have been + completed/aborted correctly + + + + fs.s3a.multipart.purge.age + 86400 + Minimum age in seconds of multipart uploads to purge + + + + fs.s3a.buffer.dir + ${hadoop.tmp.dir}/s3a + Comma separated list of directories that will be used to buffer file + uploads to. + + + + fs.s3a.fast.upload + false + Upload directly from memory instead of buffering to + disk first. Memory usage and parallelism can be controlled as up to + fs.s3a.multipart.size memory is consumed for each (part)upload actively + uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks) + + + + fs.s3a.fast.buffer.size + 1048576 + Size of initial memory buffer in bytes allocated for an + upload. No effect if fs.s3a.fast.upload is false. + + + + fs.s3a.impl + org.apache.hadoop.fs.s3a.S3AFileSystem + The implementation class of the S3A Filesystem + + + + io.seqfile.compress.blocksize + 1000000 + The minimum block size for compression in block compressed + SequenceFiles. + + + + + io.seqfile.lazydecompress + true + Should values of block-compressed SequenceFiles be decompressed + only when necessary. + + + + + io.seqfile.sorter.recordlimit + 1000000 + The limit on number of records to be kept in memory in a spill + in SequenceFiles.Sorter + + + + + io.mapfile.bloom.size + 1048576 + The size of BloomFilter-s used in BloomMapFile. Each time this many + keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter). + Larger values minimize the number of filters, which slightly increases the performance, + but may waste too much space if the total number of keys is usually much smaller + than this number. + + + + + io.mapfile.bloom.error.rate + 0.005 + The rate of false positives in BloomFilter-s used in BloomMapFile. + As this value decreases, the size of BloomFilter-s increases exponentially. This + value is the probability of encountering false positives (default is 0.5%). + + + + + hadoop.util.hash.type + murmur + The default implementation of Hash. Currently this can take one of the + two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash. + + + + + + + + ipc.client.idlethreshold + 4000 + Defines the threshold number of connections after which + connections will be inspected for idleness. + + + + + ipc.client.kill.max + 10 + Defines the maximum number of clients to disconnect in one go. + + + + + ipc.client.connection.maxidletime + 10000 + The maximum time in msec after which a client will bring down the + connection to the server. + + + + + ipc.client.connect.max.retries + 10 + Indicates the number of retries a client will make to establish + a server connection. + + + + + ipc.client.connect.retry.interval + 1000 + Indicates the number of milliseconds a client will wait for + before retrying to establish a server connection. + + + + + ipc.client.connect.timeout + 20000 + Indicates the number of milliseconds a client will wait for the + socket to establish a server connection. + + + + + ipc.client.connect.max.retries.on.timeouts + 45 + Indicates the number of retries a client will make on socket timeout + to establish a server connection. + + + + + ipc.client.ping + true + Send a ping to the server when timeout on reading the response, + if set to true. If no failure is detected, the client retries until at least + a byte is read. + + + + + ipc.ping.interval + 60000 + Timeout on waiting response from server, in milliseconds. + The client will send ping when the interval is passed without receiving bytes, + if ipc.client.ping is set to true. + + + + + ipc.client.rpc-timeout.ms + 0 + Timeout on waiting response from server, in milliseconds. + Currently this timeout works only when ipc.client.ping is set to true + because it uses the same facilities with IPC ping. + The timeout overrides the ipc.ping.interval and client will throw exception + instead of sending ping when the interval is passed. + + + + + ipc.server.listen.queue.size + 128 + Indicates the length of the listen queue for servers accepting + client connections. + + + + + ipc.maximum.data.length + 67108864 + This indicates the maximum IPC message length (bytes) that can be + accepted by the server. Messages larger than this value are rejected by + server immediately. This setting should rarely need to be changed. It merits + investigating whether the cause of long RPC messages can be fixed instead, + e.g. by splitting into smaller messages. + + + + + + + hadoop.security.impersonation.provider.class + + A class which implements ImpersonationProvider interface, used to + authorize whether one user can impersonate a specific user. + If not specified, the DefaultImpersonationProvider will be used. + If a class is specified, then that class will be used to determine + the impersonation capability. + + + + + hadoop.rpc.socket.factory.class.default + org.apache.hadoop.net.StandardSocketFactory + Default SocketFactory to use. This parameter is expected to be + formatted as "package.FactoryClassName". + + + + + hadoop.rpc.socket.factory.class.ClientProtocol + + SocketFactory to use to connect to a DFS. If null or empty, use + hadoop.rpc.socket.class.default. This socket factory is also used by + DFSClient to create sockets to DataNodes. + + + + + + + hadoop.socks.server + + Address (host:port) of the SOCKS server to be used by the + SocksSocketFactory. + + + + + + net.topology.node.switch.mapping.impl + org.apache.hadoop.net.ScriptBasedMapping + The default implementation of the DNSToSwitchMapping. It + invokes a script specified in net.topology.script.file.name to resolve + node names. If the value for net.topology.script.file.name is not set, the + default value of DEFAULT_RACK is returned for all node names. + + + + + net.topology.impl + org.apache.hadoop.net.NetworkTopology + The default implementation of NetworkTopology which is classic three layer one. + + + + + net.topology.script.file.name + + The script name that should be invoked to resolve DNS names to + NetworkTopology names. Example: the script would take host.foo.bar as an + argument, and return /rack1 as the output. + + + + + net.topology.script.number.args + 100 + The max number of args that the script configured with + net.topology.script.file.name should be run with. Each arg is an + IP address. + + + + + net.topology.table.file.name + + The file name for a topology file, which is used when the + net.topology.node.switch.mapping.impl property is set to + org.apache.hadoop.net.TableMapping. The file format is a two column text + file, with columns separated by whitespace. The first column is a DNS or + IP address and the second column specifies the rack where the address maps. + If no entry corresponding to a host in the cluster is found, then + /default-rack is assumed. + + + + + + file.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + file.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + file.stream-buffer-size + + + + file.client-write-packet-size + 65536 + Packet size for clients to write + + + + file.blocksize + 67108864 + Block size + + + + file.replication + 1 + Replication factor + + + + + + s3.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3.stream-buffer-size + + + + s3.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3.blocksize + 67108864 + Block size + + + + s3.replication + 3 + Replication factor + + + + + + s3native.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + s3native.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + s3native.stream-buffer-size + + + + s3native.client-write-packet-size + 65536 + Packet size for clients to write + + + + s3native.blocksize + 67108864 + Block size + + + + s3native.replication + 3 + Replication factor + + + + + ftp.stream-buffer-size + 4096 + The size of buffer to stream files. + The size of this buffer should probably be a multiple of hardware + page size (4096 on Intel x86), and it determines how much data is + buffered during read and write operations. + + + + ftp.bytes-per-checksum + 512 + The number of bytes per checksum. Must not be larger than + ftp.stream-buffer-size + + + + ftp.client-write-packet-size + 65536 + Packet size for clients to write + + + + ftp.blocksize + 67108864 + Block size + + + + ftp.replication + 3 + Replication factor + + + + + + tfile.io.chunk.size + 1048576 + + Value chunk size in bytes. Default to + 1MB. Values of the length less than the chunk size is + guaranteed to have known value length in read time (See also + TFile.Reader.Scanner.Entry.isValueLengthKnown()). + + + + + tfile.fs.output.buffer.size + 262144 + + Buffer size used for FSDataOutputStream in bytes. + + + + + tfile.fs.input.buffer.size + 262144 + + Buffer size used for FSDataInputStream in bytes. + + + + + + + hadoop.http.authentication.type + simple + + Defines authentication used for Oozie HTTP endpoint. + Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME# + + + + + hadoop.http.authentication.token.validity + 36000 + + Indicates how long (in seconds) an authentication token is valid before it has + to be renewed. + + + + + hadoop.http.authentication.signature.secret.file + ${user.home}/hadoop-http-auth-signature-secret + + The signature secret for signing the authentication tokens. + The same secret should be used for JT/NN/DN/TT configurations. + + + + + hadoop.http.authentication.cookie.domain + + + The domain to use for the HTTP cookie that stores the authentication token. + In order to authentiation to work correctly across all Hadoop nodes web-consoles + the domain must be correctly set. + IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings. + For this setting to work properly all nodes in the cluster must be configured + to generate URLs with hostname.domain names on it. + + + + + hadoop.http.authentication.simple.anonymous.allowed + true + + Indicates if anonymous requests are allowed when using 'simple' authentication. + + + + + hadoop.http.authentication.kerberos.principal + HTTP/_HOST@LOCALHOST + + Indicates the Kerberos principal to be used for HTTP endpoint. + The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification. + + + + + hadoop.http.authentication.kerberos.keytab + ${user.home}/hadoop.keytab + + Location of the keytab file with the credentials for the principal. + Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop. + + + + + + Enable/disable the cross-origin (CORS) filter. + hadoop.http.cross-origin.enabled + false + + + + Comma separated list of origins that are allowed for web + services needing cross-origin (CORS) support. Wildcards (*) and patterns + allowed + hadoop.http.cross-origin.allowed-origins + * + + + + Comma separated list of methods that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-methods + GET,POST,HEAD + + + + Comma separated list of headers that are allowed for web + services needing cross-origin (CORS) support. + hadoop.http.cross-origin.allowed-headers + X-Requested-With,Content-Type,Accept,Origin + + + + The number of seconds a pre-flighted request can be cached + for web services needing cross-origin (CORS) support. + hadoop.http.cross-origin.max-age + 1800 + + + + dfs.ha.fencing.methods + + + List of fencing methods to use for service fencing. May contain + builtin methods (eg shell and sshfence) or user-defined method. + + + + + dfs.ha.fencing.ssh.connect-timeout + 30000 + + SSH connection timeout, in milliseconds, to use with the builtin + sshfence fencer. + + + + + dfs.ha.fencing.ssh.private-key-files + + + The SSH private key files to use with the builtin sshfence fencer. + + + + + + + + The user name to filter as, on static web filters + while rendering content. An example use is the HDFS + web UI (user to be used for browsing files). + + hadoop.http.staticuser.user + dr.who + + + + ha.zookeeper.quorum + + A list of ZooKeeper server addresses, separated by commas, that are + to be used by the ZKFailoverController in automatic failover. + + + + + ha.zookeeper.session-timeout.ms + 5000 + + The session timeout to use when the ZKFC connects to ZooKeeper. + Setting this value to a lower value implies that server crashes + will be detected more quickly, but risks triggering failover too + aggressively in the case of a transient error or network blip. + + + + + ha.zookeeper.parent-znode + /hadoop-ha + + The ZooKeeper znode under which the ZK failover controller stores + its information. Note that the nameservice ID is automatically + appended to this znode, so it is not normally necessary to + configure this, even in a federated environment. + + + + + ha.zookeeper.acl + world:anyone:rwcda + + A comma-separated list of ZooKeeper ACLs to apply to the znodes + used by automatic failover. These ACLs are specified in the same + format as used by the ZooKeeper CLI. + + If the ACL itself contains secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + ha.zookeeper.auth + + + A comma-separated list of ZooKeeper authentications to add when + connecting to ZooKeeper. These are specified in the same format + as used by the "addauth" command in the ZK CLI. It is + important that the authentications specified here are sufficient + to access znodes with the ACL specified in ha.zookeeper.acl. + + If the auths contain secrets, you may instead specify a + path to a file, prefixed with the '@' symbol, and the value of + this configuration will be loaded from within. + + + + + + + hadoop.ssl.keystores.factory.class + org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory + + The keystores factory to use for retrieving certificates. + + + + + hadoop.ssl.require.client.cert + false + Whether client certificates are required + + + + hadoop.ssl.hostname.verifier + DEFAULT + + The hostname verifier to provide for HttpsURLConnections. + Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and + ALLOW_ALL + + + + + hadoop.ssl.server.conf + ssl-server.xml + + Resource file from which ssl server keystore information will be extracted. + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.client.conf + ssl-client.xml + + Resource file from which ssl client keystore information will be extracted + This file is looked up in the classpath, typically it should be in Hadoop + conf/ directory. + + + + + hadoop.ssl.enabled + false + + Deprecated. Use dfs.http.policy and yarn.http.policy instead. + + + + + hadoop.ssl.enabled.protocols + TLSv1 + + Protocols supported by the ssl. + + + + + hadoop.jetty.logs.serve.aliases + true + + Enable/Disable aliases serving from jetty + + + + + fs.permissions.umask-mode + 022 + + The umask used when creating files and directories. + Can be in octal or in symbolic. Examples are: + "022" (octal for u=rwx,g=r-x,o=r-x in symbolic), + or "u=rwx,g=rwx,o=" (symbolic for 007 in octal). + + + + + + + ha.health-monitor.connect-retry-interval.ms + 1000 + + How often to retry connecting to the service. + + + + + ha.health-monitor.check-interval.ms + 1000 + + How often to check the service. + + + + + ha.health-monitor.sleep-after-disconnect.ms + 1000 + + How long to sleep after an unexpected RPC error. + + + + + ha.health-monitor.rpc-timeout.ms + 45000 + + Timeout for the actual monitorHealth() calls. + + + + + ha.failover-controller.new-active.rpc-timeout.ms + 60000 + + Timeout that the FC waits for the new active to become active + + + + + ha.failover-controller.graceful-fence.rpc-timeout.ms + 5000 + + Timeout that the FC waits for the old active to go to standby + + + + + ha.failover-controller.graceful-fence.connection.retries + 1 + + FC connection retries for graceful fencing + + + + + ha.failover-controller.cli-check.rpc-timeout.ms + 20000 + + Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState + + + + + ipc.client.fallback-to-simple-auth-allowed + false + + When a client is configured to attempt a secure connection, but attempts to + connect to an insecure server, that server may instruct the client to + switch to SASL SIMPLE (unsecure) authentication. This setting controls + whether or not the client will accept this instruction from the server. + When false (the default), the client will not allow the fallback to SIMPLE + authentication, and will abort the connection. + + + + + fs.client.resolve.remote.symlinks + true + + Whether to resolve symlinks when accessing a remote Hadoop filesystem. + Setting this to false causes an exception to be thrown upon encountering + a symlink. This setting does not apply to local filesystems, which + automatically resolve local symlinks. + + + + + nfs.exports.allowed.hosts + * rw + + By default, the export can be mounted by any client. The value string + contains machine name and access privilege, separated by whitespace + characters. The machine name format can be a single host, a Java regular + expression, or an IPv4 address. The access privilege uses rw or ro to + specify read/write or read-only access of the machines to exports. If the + access privilege is not provided, the default is read-only. Entries are separated by ";". + For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;". + Only the NFS gateway needs to restart after this property is updated. + + + + + hadoop.user.group.static.mapping.overrides + dr.who=; + + Static mapping of user to groups. This will override the groups if + available in the system for the specified user. In otherwords, groups + look-up will not happen for these users, instead groups mapped in this + configuration will be used. + Mapping should be in this format. + user1=group1,group2;user2=;user3=group2; + Default, "dr.who=;" will consider "dr.who" as user without groups. + + + + + rpc.metrics.quantile.enable + false + + Setting this property to true and rpc.metrics.percentiles.intervals + to a comma-separated list of the granularity in seconds, the + 50/75/90/95/99th percentile latency for rpc queue/processing time in + milliseconds are added to rpc metrics. + + + + + rpc.metrics.percentiles.intervals + + + A comma-separated list of the granularity in seconds for the metrics which + describe the 50/75/90/95/99th percentile latency for rpc queue/processing + time. The metrics are outputted if rpc.metrics.quantile.enable is set to + true. + + + + + hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE + + + The prefix for a given crypto codec, contains a comma-separated + list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE). + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.codec.classes.aes.ctr.nopadding + org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec,org.apache.hadoop.crypto.JceAesCtrCryptoCodec + + Comma-separated list of crypto codec implementations for AES/CTR/NoPadding. + The first implementation will be used if available, others are fallbacks. + + + + + hadoop.security.crypto.cipher.suite + AES/CTR/NoPadding + + Cipher suite for crypto codec. + + + + + hadoop.security.crypto.jce.provider + + + The JCE provider name used in CryptoCodec. + + + + + hadoop.security.crypto.buffer.size + 8192 + + The buffer size used by CryptoInputStream and CryptoOutputStream. + + + + + hadoop.security.java.secure.random.algorithm + SHA1PRNG + + The java secure random algorithm. + + + + + hadoop.security.secure.random.impl + + + Implementation of secure random. + + + + + hadoop.security.random.device.file.path + /dev/urandom + + OS security random device file path. + + + + + fs.har.impl.disable.cache + true + Don't cache 'har' filesystem instances. + + + + + hadoop.security.kms.client.authentication.retry-count + 1 + + Number of time to retry connecting to KMS on authentication failure + + + + hadoop.security.kms.client.encrypted.key.cache.size + 500 + + Size of the EncryptedKeyVersion cache Queue for each key + + + + hadoop.security.kms.client.encrypted.key.cache.low-watermark + 0.3f + + If size of the EncryptedKeyVersion cache Queue falls below the + low watermark, this cache queue will be scheduled for a refill + + + + hadoop.security.kms.client.encrypted.key.cache.num.refill.threads + 2 + + Number of threads to use for refilling depleted EncryptedKeyVersion + cache Queues + + + + hadoop.security.kms.client.encrypted.key.cache.expiry + 43200000 + + Cache expiry time for a Key, after which the cache Queue for this + key will be dropped. Default = 12hrs + + + + + hadoop.htrace.spanreceiver.classes + + + A comma separated list of the fully-qualified class name of classes + implementing SpanReceiver. The tracing system works by collecting + information in structs called 'Spans'. It is up to you to choose + how you want to receive this information by implementing the + SpanReceiver interface. + + + + + ipc.server.max.connections + 0 + The maximum number of concurrent connections a server is allowed + to accept. If this limit is exceeded, incoming connections will first fill + the listen queue and then may go to an OS-specific listen overflow queue. + The client may fail or timeout, but the server can avoid running out of file + descriptors using this feature. 0 means no limit. + + + + + + + + + Is the registry enabled in the YARN Resource Manager? + + If true, the YARN RM will, as needed. + create the user and system paths, and purge + service records when containers, application attempts + and applications complete. + + If false, the paths must be created by other means, + and no automatic cleanup of service records will take place. + + hadoop.registry.rm.enabled + false + + + + + The root zookeeper node for the registry + + hadoop.registry.zk.root + /registry + + + + + Zookeeper session timeout in milliseconds + + hadoop.registry.zk.session.timeout.ms + 60000 + + + + + Zookeeper connection timeout in milliseconds + + hadoop.registry.zk.connection.timeout.ms + 15000 + + + + + Zookeeper connection retry count before failing + + hadoop.registry.zk.retry.times + 5 + + + + + + hadoop.registry.zk.retry.interval.ms + 1000 + + + + + Zookeeper retry limit in milliseconds, during + exponential backoff. + + This places a limit even + if the retry times and interval limit, combined + with the backoff policy, result in a long retry + period + + hadoop.registry.zk.retry.ceiling.ms + 60000 + + + + + List of hostname:port pairs defining the + zookeeper quorum binding for the registry + + hadoop.registry.zk.quorum + localhost:2181 + + + + + Key to set if the registry is secure. Turning it on + changes the permissions policy from "open access" + to restrictions on kerberos with the option of + a user adding one or more auth key pairs down their + own tree. + + hadoop.registry.secure + false + + + + + A comma separated list of Zookeeper ACL identifiers with + system access to the registry in a secure cluster. + + These are given full access to all entries. + + If there is an "@" at the end of a SASL entry it + instructs the registry client to append the default kerberos domain. + + hadoop.registry.system.acls + sasl:yarn@, sasl:mapred@, sasl:hdfs@ + + + + + The kerberos realm: used to set the realm of + system principals which do not declare their realm, + and any other accounts that need the value. + + If empty, the default realm of the running process + is used. + + If neither are known and the realm is needed, then the registry + service/client will fail. + + hadoop.registry.kerberos.realm + + + + + + Key to define the JAAS context. Used in secure + mode + + hadoop.registry.jaas.context + Client + + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index 19ec17a9f3df2..90a1eea7d9265 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -35,4 +35,8 @@ under the License. + + diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index fda602305170e..9e2c27afc6303 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -368,6 +368,47 @@ check_shaded_artifacts() { return 0 } +# Check the S3 fs implementations' fat jars for illegal or missing artifacts +check_shaded_artifacts_s3_fs() { + VARIANT=$1 + jar tf flink-filesystems/flink-s3-fs-${VARIANT}/target/flink-s3-fs-${VARIANT}*.jar > allClasses + + UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/fs/s3${VARIANT}/" | grep '\.class$'` + if [ "$?" == "0" ]; then + echo "==============================================================================" + echo "Detected unshaded dependencies in fat jar:" + echo "${UNSHADED_CLASSES}" + echo "==============================================================================" + return 1 + fi + + if [ ! `cat allClasses | grep '^META-INF/services/org\.apache\.flink\.core\.fs\.FileSystemFactory$'` ]; then + echo "==============================================================================" + echo "File does not exist: services/org.apache.flink.core.fs.FileSystemFactory" + echo "==============================================================================" + fi + + UNSHADED_SERVICES=`cat allClasses | grep '^META-INF/services/' | grep -v -e '^META-INF/services/org\.apache\.flink\.core\.fs\.FileSystemFactory$' -e "^META-INF/services/org\.apache\.flink\.fs\.s3${VARIANT}\.shaded" -e '^META-INF/services/'` + if [ "$?" == "0" ]; then + echo "==============================================================================" + echo "Detected unshaded service files in fat jar:" + echo "${UNSHADED_SERVICES}" + echo "==============================================================================" + return 1 + fi + + FS_SERVICE_FILE_CLASS=`unzip -q -c flink-filesystems/flink-s3-fs-${VARIANT}/target/flink-s3-fs-${VARIANT}*.jar META-INF/services/org.apache.flink.core.fs.FileSystemFactory | grep -v -e '^#' -e '^$'` + if [ "${FS_SERVICE_FILE_CLASS}" != "org.apache.flink.fs.s3${VARIANT}.S3FileSystemFactory" ]; then + echo "==============================================================================" + echo "Detected wrong content in services/org.apache.flink.core.fs.FileSystemFactory:" + echo "${FS_SERVICE_FILE_CLASS}" + echo "==============================================================================" + return 1 + fi + + return 0 +} + # ============================================================================= # WATCHDOG # ============================================================================= @@ -458,7 +499,17 @@ case $TEST in echo "Compilation/test failure detected, skipping shaded dependency check." echo "==============================================================================" fi - + ;; + (connectors) + if [ $EXIT_CODE == 0 ]; then + check_shaded_artifacts_s3_fs hadoop + check_shaded_artifacts_s3_fs presto + EXIT_CODE=$? + else + echo "==============================================================================" + echo "Compilation/test failure detected, skipping shaded dependency check." + echo "==============================================================================" + fi ;; esac From 9f68212603e3601e2f7a67ff93be9b15844c14da Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 8 Nov 2017 17:49:17 +0100 Subject: [PATCH 034/367] [FLINK-7973] Add shaded S3 FileSystem end-to-end tests --- test-infra/end-to-end-test/common.sh | 53 ++++++++++++--- .../end-to-end-test/test_shaded_hadoop_s3a.sh | 64 +++++++++++++++++++ .../end-to-end-test/test_shaded_presto_s3.sh | 64 +++++++++++++++++++ tools/travis_mvn_watchdog.sh | 12 ++++ 4 files changed, 185 insertions(+), 8 deletions(-) create mode 100755 test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh create mode 100755 test-infra/end-to-end-test/test_shaded_presto_s3.sh diff --git a/test-infra/end-to-end-test/common.sh b/test-infra/end-to-end-test/common.sh index 9a1c79c9fc073..cc31c90a15914 100644 --- a/test-infra/end-to-end-test/common.sh +++ b/test-infra/end-to-end-test/common.sh @@ -92,14 +92,14 @@ function stop_cluster { PASS="" fi - for f in `ls $FLINK_DIR/log/*.out` - do - if [[ -s $f ]]; then - echo "Found non-empty file $f" - cat $f - PASS="" - fi - done + if grep -rv "NativeCodeLoader" $FLINK_DIR/log/*.out \ + | grep -v "Unable to load native-hadoop" \ + | grep -v "amazonaws" \ + | grep -i "."; then + echo "Found non-empty .out files:" + cat $FLINK_DIR/log/*.out + PASS="" + fi rm $FLINK_DIR/log/* } @@ -135,3 +135,40 @@ function check_all_pass { function clean_data_dir { rm -r $TEST_DATA_DIR } + +function s3_put { + local_file=$1 + bucket=$2 + s3_file=$3 + resource="/${bucket}/${s3_file}" + contentType="application/octet-stream" + dateValue=`date -R` + stringToSign="PUT\n\n${contentType}\n${dateValue}\n${resource}" + s3Key=$ARTIFACTS_AWS_ACCESS_KEY + s3Secret=$ARTIFACTS_AWS_SECRET_KEY + signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` + curl -X PUT -T "${local_file}" \ + -H "Host: ${bucket}.s3.amazonaws.com" \ + -H "Date: ${dateValue}" \ + -H "Content-Type: ${contentType}" \ + -H "Authorization: AWS ${s3Key}:${signature}" \ + https://${bucket}.s3.amazonaws.com/${s3_file} +} + +function s3_delete { + bucket=$1 + s3_file=$2 + resource="/${bucket}/${s3_file}" + contentType="application/octet-stream" + dateValue=`date -R` + stringToSign="DELETE\n\n${contentType}\n${dateValue}\n${resource}" + s3Key=$ARTIFACTS_AWS_ACCESS_KEY + s3Secret=$ARTIFACTS_AWS_SECRET_KEY + signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` + curl -X DELETE \ + -H "Host: ${bucket}.s3.amazonaws.com" \ + -H "Date: ${dateValue}" \ + -H "Content-Type: ${contentType}" \ + -H "Authorization: AWS ${s3Key}:${signature}" \ + https://${bucket}.s3.amazonaws.com/${s3_file} +} diff --git a/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh b/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh new file mode 100755 index 0000000000000..90bf73b6ed599 --- /dev/null +++ b/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env 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. +################################################################################ + +if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then + echo "Did not find AWS environment variables, NOT running Shaded Hadoop S3A e2e tests." + exit 0 +else + echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running Shaded Hadoop S3A e2e tests." +fi + +# Tests for our shaded/bundled Hadoop S3A file system. + +set -e +set -o pipefail + +# Convert relative path to absolute path +TEST_ROOT=`pwd` +TEST_INFRA_DIR="$0" +TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` +cd $TEST_INFRA_DIR +TEST_INFRA_DIR=`pwd` +cd $TEST_ROOT + +. "$TEST_INFRA_DIR"/common.sh + +s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + +cp $FLINK_DIR/opt/flink-s3-fs-hadoop-*.jar $FLINK_DIR/lib/ +echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" +echo "s3.secret-key: $ARTIFACTS_AWS_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" + +start_cluster + +$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out + +check_result_hash "WordCountWithShadedS3A" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" + +# remove any leftover settings +sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" +sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" + +rm $FLINK_DIR/lib/flink-s3-fs*.jar + +s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + +stop_cluster +clean_data_dir +check_all_pass diff --git a/test-infra/end-to-end-test/test_shaded_presto_s3.sh b/test-infra/end-to-end-test/test_shaded_presto_s3.sh new file mode 100755 index 0000000000000..6ded115cfcb16 --- /dev/null +++ b/test-infra/end-to-end-test/test_shaded_presto_s3.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env 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. +################################################################################ + +if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then + echo "Did not find AWS environment variables, NOT running Shaded Presto S3 e2e tests." + exit 0 +else + echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running Shaded Presto S3 e2e tests." +fi + +# Tests for our shaded/bundled Hadoop S3A file system. + +set -e +set -o pipefail + +# Convert relative path to absolute path +TEST_ROOT=`pwd` +TEST_INFRA_DIR="$0" +TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` +cd $TEST_INFRA_DIR +TEST_INFRA_DIR=`pwd` +cd $TEST_ROOT + +. "$TEST_INFRA_DIR"/common.sh + +s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 + +cp $FLINK_DIR/opt/flink-s3-fs-presto-*.jar $FLINK_DIR/lib/ +echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" +echo "s3.secret-key: $ARTIFACTS_AWS_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" + +start_cluster + +$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out + +check_result_hash "WordCountWithShadedPrestoS3" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" + +# remove any leftover settings +sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" +sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" + +rm $FLINK_DIR/lib/flink-s3-fs*.jar + +s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 + +stop_cluster +clean_data_dir +check_all_pass diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 9e2c27afc6303..0417cd35e9346 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -544,6 +544,18 @@ case $TEST in printf "==============================================================================\n" test-infra/end-to-end-test/test_streaming_classloader.sh build-target cluster EXIT_CODE=$(($EXIT_CODE+$?)) + + printf "\n==============================================================================\n" + printf "Running Shaded Hadoop S3A end-to-end test\n" + printf "==============================================================================\n" + test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh build-target cluster + EXIT_CODE=$(($EXIT_CODE+$?)) + + printf "\n==============================================================================\n" + printf "Running Shaded Presto S3 end-to-end test\n" + printf "==============================================================================\n" + test-infra/end-to-end-test/test_shaded_presto_s3.sh build-target cluster + EXIT_CODE=$(($EXIT_CODE+$?)) else printf "\n==============================================================================\n" printf "Previous build failure detected, skipping end-to-end tests.\n" From e666e6279e602d83ad37f264b2cf8a5345287bb1 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 8 Nov 2017 18:57:22 +0100 Subject: [PATCH 035/367] [hotfix] Make end-to-end test scripts more robust This uses traps to ensure that we properly do cleanups, remove config values and shutdown things. --- test-infra/end-to-end-test/common.sh | 31 ++++++++++---- .../end-to-end-test/test_batch_wordcount.sh | 20 +-------- .../end-to-end-test/test_shaded_hadoop_s3a.sh | 42 +++++++------------ .../end-to-end-test/test_shaded_presto_s3.sh | 36 ++++++---------- .../test_streaming_classloader.sh | 22 +--------- .../test_streaming_kafka010.sh | 34 ++++++--------- tools/travis_mvn_watchdog.sh | 10 ++--- 7 files changed, 75 insertions(+), 120 deletions(-) diff --git a/test-infra/end-to-end-test/common.sh b/test-infra/end-to-end-test/common.sh index cc31c90a15914..35d1d7e7639ed 100644 --- a/test-infra/end-to-end-test/common.sh +++ b/test-infra/end-to-end-test/common.sh @@ -17,16 +17,28 @@ # limitations under the License. ################################################################################ -set -e set -o pipefail -export FLINK_DIR="$1" -export CLUSTER_MODE="$2" +if [[ -z $FLINK_DIR ]]; then + echo "FLINK_DIR needs to point to a Flink distribution directory" + exit 1 +fi +if [[ -z $CLUSTER_MODE ]]; then + echo "CLUSTER_MODE needs to be one of local or cluster." + exit 1 +fi export PASS=1 echo "Flink dist directory: $FLINK_DIR" +TEST_ROOT=`pwd` +TEST_INFRA_DIR="$0" +TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` +cd $TEST_INFRA_DIR +TEST_INFRA_DIR=`pwd` +cd $TEST_ROOT + # used to randomize created directories export TEST_DATA_DIR=$TEST_INFRA_DIR/temp-test-directory-$(date +%S%N) echo "TEST_DATA_DIR: $TEST_DATA_DIR" @@ -73,6 +85,7 @@ function stop_cluster { | grep -v "AskTimeoutException" \ | grep -v "WARN akka.remote.transport.netty.NettyTransport" \ | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ + | grep -v "jvm-exit-on-fatal-error" \ | grep -iq "error"; then echo "Found error in log files:" cat $FLINK_DIR/log/* @@ -132,10 +145,6 @@ function check_all_pass { echo "All tests PASS" } -function clean_data_dir { - rm -r $TEST_DATA_DIR -} - function s3_put { local_file=$1 bucket=$2 @@ -172,3 +181,11 @@ function s3_delete { -H "Authorization: AWS ${s3Key}:${signature}" \ https://${bucket}.s3.amazonaws.com/${s3_file} } + +# make sure to clean up even in case of failures +function cleanup { + stop_cluster + rm -r $TEST_DATA_DIR + check_all_pass +} +trap cleanup EXIT diff --git a/test-infra/end-to-end-test/test_batch_wordcount.sh b/test-infra/end-to-end-test/test_batch_wordcount.sh index dfde5c6b692cc..2c9a17597ca65 100755 --- a/test-infra/end-to-end-test/test_batch_wordcount.sh +++ b/test-infra/end-to-end-test/test_batch_wordcount.sh @@ -17,25 +17,9 @@ # limitations under the License. ################################################################################ - -set -e -set -o pipefail - -# Convert relative path to absolute path -TEST_ROOT=`pwd` -TEST_INFRA_DIR="$0" -TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` -cd $TEST_INFRA_DIR -TEST_INFRA_DIR=`pwd` -cd $TEST_ROOT - -. "$TEST_INFRA_DIR"/common.sh +source "$(dirname "$0")"/common.sh start_cluster $FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input $TEST_INFRA_DIR/test-data/words --output $TEST_DATA_DIR/out/wc_out -check_result_hash "WordCount" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" - -stop_cluster -clean_data_dir -check_all_pass \ No newline at end of file +check_result_hash "WordCount" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" \ No newline at end of file diff --git a/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh b/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh index 90bf73b6ed599..a989488faccaa 100755 --- a/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh +++ b/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +# Tests for our shaded/bundled Hadoop S3A file system. + if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then echo "Did not find AWS environment variables, NOT running Shaded Hadoop S3A e2e tests." exit 0 @@ -24,22 +26,22 @@ else echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running Shaded Hadoop S3A e2e tests." fi -# Tests for our shaded/bundled Hadoop S3A file system. +source "$(dirname "$0")"/common.sh -set -e -set -o pipefail - -# Convert relative path to absolute path -TEST_ROOT=`pwd` -TEST_INFRA_DIR="$0" -TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` -cd $TEST_INFRA_DIR -TEST_INFRA_DIR=`pwd` -cd $TEST_ROOT +s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a +# make sure we delete the file at the end +function s3_cleanup { + s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + rm $FLINK_DIR/lib/flink-s3-fs*.jar -. "$TEST_INFRA_DIR"/common.sh + # remove any leftover settings + sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" + sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" -s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + # make sure to run regular cleanup as well + cleanup +} +trap s3_cleanup EXIT cp $FLINK_DIR/opt/flink-s3-fs-hadoop-*.jar $FLINK_DIR/lib/ echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" @@ -49,16 +51,4 @@ start_cluster $FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out -check_result_hash "WordCountWithShadedS3A" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" - -# remove any leftover settings -sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" -sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" - -rm $FLINK_DIR/lib/flink-s3-fs*.jar - -s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a - -stop_cluster -clean_data_dir -check_all_pass +check_result_hash "WordCountWithShadedS3A" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" \ No newline at end of file diff --git a/test-infra/end-to-end-test/test_shaded_presto_s3.sh b/test-infra/end-to-end-test/test_shaded_presto_s3.sh index 6ded115cfcb16..bca7649b07048 100755 --- a/test-infra/end-to-end-test/test_shaded_presto_s3.sh +++ b/test-infra/end-to-end-test/test_shaded_presto_s3.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +# Tests for our shaded/bundled Hadoop S3A file system. + if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then echo "Did not find AWS environment variables, NOT running Shaded Presto S3 e2e tests." exit 0 @@ -24,22 +26,18 @@ else echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running Shaded Presto S3 e2e tests." fi -# Tests for our shaded/bundled Hadoop S3A file system. - -set -e -set -o pipefail - -# Convert relative path to absolute path -TEST_ROOT=`pwd` -TEST_INFRA_DIR="$0" -TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` -cd $TEST_INFRA_DIR -TEST_INFRA_DIR=`pwd` -cd $TEST_ROOT - -. "$TEST_INFRA_DIR"/common.sh +source "$(dirname "$0")"/common.sh s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 +# make sure we delete the file at the end +function s3_cleanup { + s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + rm $FLINK_DIR/lib/flink-s3-fs*.jar + + # make sure to run regular cleanup as well + cleanup +} +trap s3_cleanup EXIT cp $FLINK_DIR/opt/flink-s3-fs-presto-*.jar $FLINK_DIR/lib/ echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" @@ -53,12 +51,4 @@ check_result_hash "WordCountWithShadedPrestoS3" $TEST_DATA_DIR/out/wc_out "72a69 # remove any leftover settings sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" -sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" - -rm $FLINK_DIR/lib/flink-s3-fs*.jar - -s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 - -stop_cluster -clean_data_dir -check_all_pass +sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" \ No newline at end of file diff --git a/test-infra/end-to-end-test/test_streaming_classloader.sh b/test-infra/end-to-end-test/test_streaming_classloader.sh index 8bc6858517d20..95c58f8a1f584 100755 --- a/test-infra/end-to-end-test/test_streaming_classloader.sh +++ b/test-infra/end-to-end-test/test_streaming_classloader.sh @@ -17,25 +17,10 @@ # limitations under the License. ################################################################################ - -set -e -set -o pipefail - -# Convert relative path to absolute path -TEST_ROOT=`pwd` -TEST_INFRA_DIR="$0" -TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` -cd $TEST_INFRA_DIR -TEST_INFRA_DIR=`pwd` -cd $TEST_ROOT - -. "$TEST_INFRA_DIR"/common.sh +source "$(dirname "$0")"/common.sh TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/target/ClassLoaderTestProgram.jar -# kill any remaining JobManagers/TaskManagers at the end -trap 'pkill -f "JobManager|TaskManager"' EXIT - echo "Testing parent-first class loading" # retrieve git.remote.origin.url from .version.properties @@ -126,7 +111,4 @@ if [[ "$OUTPUT" != "$EXPECTED" ]]; then echo -e "EXPECTED: $EXPECTED" echo -e "ACTUAL: $OUTPUT" PASS="" -fi - -clean_data_dir -check_all_pass +fi \ No newline at end of file diff --git a/test-infra/end-to-end-test/test_streaming_kafka010.sh b/test-infra/end-to-end-test/test_streaming_kafka010.sh index cce8db4399592..1324e5aead3d0 100755 --- a/test-infra/end-to-end-test/test_streaming_kafka010.sh +++ b/test-infra/end-to-end-test/test_streaming_kafka010.sh @@ -17,19 +17,7 @@ # limitations under the License. ################################################################################ -set -e -set -o pipefail - -# Convert relative path to absolute path -TEST_ROOT=`pwd` -TEST_INFRA_DIR="$0" -TEST_INFRA_DIR=`dirname "$TEST_INFRA_DIR"` -cd $TEST_INFRA_DIR -TEST_INFRA_DIR=`pwd` -cd $TEST_ROOT - -. "$TEST_INFRA_DIR"/common.sh - +source "$(dirname "$0")"/common.sh start_cluster @@ -54,6 +42,17 @@ sed -i -e "s+^\(log\.dirs\s*=\s*\).*$+\1$TEST_DATA_DIR/kafka+" $KAFKA_DIR/config $KAFKA_DIR/bin/zookeeper-server-start.sh -daemon $KAFKA_DIR/config/zookeeper.properties $KAFKA_DIR/bin/kafka-server-start.sh -daemon $KAFKA_DIR/config/server.properties +# make sure to stop Kafka and ZooKeeper at the end + +function kafka_cleanup { + $KAFKA_DIR/bin/kafka-server-stop.sh + $KAFKA_DIR/bin/zookeeper-server-stop.sh + + # make sure to run regular cleanup as well + cleanup +} +trap kafka_cleanup EXIT + # zookeeper outputs the "Node does not exist" bit to stderr while [[ $($KAFKA_DIR/bin/zookeeper-shell.sh localhost:2181 get /brokers/ids/0 2>&1) =~ .*Node\ does\ not\ exist.* ]]; do echo "Waiting for broker..." @@ -82,11 +81,4 @@ if [[ "$DATA_FROM_KAFKA" != "$EXPECTED" ]]; then echo -e "EXPECTED: --$EXPECTED--" echo -e "ACTUAL: --$DATA_FROM_KAFKA--" PASS="" -fi - -$KAFKA_DIR/bin/kafka-server-stop.sh -$KAFKA_DIR/bin/zookeeper-server-stop.sh - -stop_cluster -clean_data_dir -check_all_pass +fi \ No newline at end of file diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 0417cd35e9346..8e315fbf6f341 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -530,31 +530,31 @@ case $TEST in printf "\n==============================================================================\n" printf "Running Wordcount end-to-end test\n" printf "==============================================================================\n" - test-infra/end-to-end-test/test_batch_wordcount.sh build-target cluster + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_batch_wordcount.sh EXIT_CODE=$(($EXIT_CODE+$?)) printf "\n==============================================================================\n" printf "Running Kafka end-to-end test\n" printf "==============================================================================\n" - test-infra/end-to-end-test/test_streaming_kafka010.sh build-target cluster + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_streaming_kafka010.sh EXIT_CODE=$(($EXIT_CODE+$?)) printf "\n==============================================================================\n" printf "Running class loading end-to-end test\n" printf "==============================================================================\n" - test-infra/end-to-end-test/test_streaming_classloader.sh build-target cluster + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_streaming_classloader.sh EXIT_CODE=$(($EXIT_CODE+$?)) printf "\n==============================================================================\n" printf "Running Shaded Hadoop S3A end-to-end test\n" printf "==============================================================================\n" - test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh build-target cluster + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh EXIT_CODE=$(($EXIT_CODE+$?)) printf "\n==============================================================================\n" printf "Running Shaded Presto S3 end-to-end test\n" printf "==============================================================================\n" - test-infra/end-to-end-test/test_shaded_presto_s3.sh build-target cluster + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_shaded_presto_s3.sh EXIT_CODE=$(($EXIT_CODE+$?)) else printf "\n==============================================================================\n" From 3574f8b430c19b21735995defa35a604db960046 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 10 Nov 2017 13:16:40 +0100 Subject: [PATCH 036/367] [hotfix] fix presto end-to-end test not cleaning up --- test-infra/end-to-end-test/test_shaded_presto_s3.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test-infra/end-to-end-test/test_shaded_presto_s3.sh b/test-infra/end-to-end-test/test_shaded_presto_s3.sh index bca7649b07048..559f1fe6f1101 100755 --- a/test-infra/end-to-end-test/test_shaded_presto_s3.sh +++ b/test-infra/end-to-end-test/test_shaded_presto_s3.sh @@ -31,7 +31,7 @@ source "$(dirname "$0")"/common.sh s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 # make sure we delete the file at the end function s3_cleanup { - s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 rm $FLINK_DIR/lib/flink-s3-fs*.jar # make sure to run regular cleanup as well @@ -51,4 +51,4 @@ check_result_hash "WordCountWithShadedPrestoS3" $TEST_DATA_DIR/out/wc_out "72a69 # remove any leftover settings sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" -sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" \ No newline at end of file +sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" From 97a3491fbde9992507d6d247c82d560dd8753a0a Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 10 Nov 2017 14:04:29 +0100 Subject: [PATCH 037/367] [hotfix] ignore a warning from the error check of the S3 e2e tests --- test-infra/end-to-end-test/common.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test-infra/end-to-end-test/common.sh b/test-infra/end-to-end-test/common.sh index 35d1d7e7639ed..8dbf0a063ab77 100644 --- a/test-infra/end-to-end-test/common.sh +++ b/test-infra/end-to-end-test/common.sh @@ -86,6 +86,7 @@ function stop_cluster { | grep -v "WARN akka.remote.transport.netty.NettyTransport" \ | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ | grep -v "jvm-exit-on-fatal-error" \ + | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' \ | grep -iq "error"; then echo "Found error in log files:" cat $FLINK_DIR/log/* @@ -99,6 +100,7 @@ function stop_cluster { | grep -v "AskTimeoutException" \ | grep -v "WARN akka.remote.transport.netty.NettyTransport" \ | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ + | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' \ | grep -iq "exception"; then echo "Found exception in log files:" cat $FLINK_DIR/log/* From 8f2d0fa8c9331d856c635dffa02840bd690b684e Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 10 Nov 2017 16:42:10 +0100 Subject: [PATCH 038/367] [hotfix] let end-to-end tests check for empty .out files again --- test-infra/end-to-end-test/common.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/test-infra/end-to-end-test/common.sh b/test-infra/end-to-end-test/common.sh index 8dbf0a063ab77..5cbfb3f75a3cb 100644 --- a/test-infra/end-to-end-test/common.sh +++ b/test-infra/end-to-end-test/common.sh @@ -107,10 +107,7 @@ function stop_cluster { PASS="" fi - if grep -rv "NativeCodeLoader" $FLINK_DIR/log/*.out \ - | grep -v "Unable to load native-hadoop" \ - | grep -v "amazonaws" \ - | grep -i "."; then + if grep -ri "." $FLINK_DIR/log/*.out > /dev/null; then echo "Found non-empty .out files:" cat $FLINK_DIR/log/*.out PASS="" From 8b7698dbc8c82d91872a61a34035663725762ff5 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 13 Nov 2017 14:31:45 +0100 Subject: [PATCH 039/367] [FLINK-8053] [checkpoints] Default to asynchronous snapshots for FsStateBackend and MemoryStateBackend. (cherry picked from commit 2906698) --- docs/ops/state/large_state_tuning.md | 6 +- docs/ops/state/state_backends.md | 5 +- .../state/filesystem/FsStateBackend.java | 4 +- .../state/memory/MemoryStateBackend.java | 2 +- ...apKeyedStateBackendAsyncByDefaultTest.java | 86 +++++++++++++++++++ 5 files changed, 93 insertions(+), 10 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/HeapKeyedStateBackendAsyncByDefaultTest.java diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md index dd3e404845d60..85ffd99c5b519 100644 --- a/docs/ops/state/large_state_tuning.md +++ b/docs/ops/state/large_state_tuning.md @@ -113,11 +113,9 @@ To get state to be snapshotted asynchronously, applications have to do two thing interfaces like `ValueState`, `ListState`, `ReducingState`, ... 2. Use a state backend that supports asynchronous snapshots. In Flink 1.2, only the RocksDB state backend uses - fully asynchronous snapshots. - -The above two points imply that (in Flink 1.2) large state should generally be kept as keyed state, not as operator state. -This is subject to change with the planned introduction of *managed operator state*. + fully asynchronous snapshots. Starting from Flink 1.3, heap-based state backends also support asynchronous snapshots. +The above two points imply that large state should generally be kept as keyed state, not as operator state. ## Tuning RocksDB diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md index 422df3ec509b0..b32ad9f1c8a27 100644 --- a/docs/ops/state/state_backends.md +++ b/docs/ops/state/state_backends.md @@ -81,11 +81,10 @@ The *FsStateBackend* is configured with a file system URL (type, address, path), The FsStateBackend holds in-flight data in the TaskManager's memory. Upon checkpointing, it writes state snapshots into files in the configured file system and directory. Minimal metadata is stored in the JobManager's memory (or, in high-availability mode, in the metadata checkpoint). -The FsStateBackend can be configured to use asynchronous snapshots. While we strongly encourage the use of asynchronous snapshots to avoid blocking pipelines, please note that this is a new feature and currently not enabled -by default. To enable this feature, users can instantiate a `FsStateBackend` with the corresponding boolean flag in the constructor set to `true`, e.g.: +The FsStateBackend uses *asynchronous snapshots by default* to avoid blocking the processing pipeline while writing state checkpoints. To disable this feature, users can instantiate a `FsStateBackend` with the corresponding boolean flag in the constructor set to `false`, e.g.: {% highlight java %} - new FsStateBackend(path, true); + new FsStateBackend(path, false); {% endhighlight %} The FsStateBackend is encouraged for: diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java index ddfa85c97ed63..952988f72d5d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java @@ -163,7 +163,7 @@ public FsStateBackend(Path checkpointDataUri, boolean asynchronousSnapshots) thr * @throws IOException Thrown, if no file system can be found for the scheme in the URI. */ public FsStateBackend(URI checkpointDataUri) throws IOException { - this(checkpointDataUri, DEFAULT_FILE_STATE_THRESHOLD, false); + this(checkpointDataUri, DEFAULT_FILE_STATE_THRESHOLD, true); } /** @@ -208,7 +208,7 @@ public FsStateBackend(URI checkpointDataUri, boolean asynchronousSnapshots) thro */ public FsStateBackend(URI checkpointDataUri, int fileStateSizeThreshold) throws IOException { - this(checkpointDataUri, fileStateSizeThreshold, false); + this(checkpointDataUri, fileStateSizeThreshold, true); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java index 7ed1dea17a13f..b8ebedfe9285b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java @@ -65,7 +65,7 @@ public MemoryStateBackend() { * @param maxStateSize The maximal size of the serialized state */ public MemoryStateBackend(int maxStateSize) { - this(maxStateSize, false); + this(maxStateSize, true); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/HeapKeyedStateBackendAsyncByDefaultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/HeapKeyedStateBackendAsyncByDefaultTest.java new file mode 100644 index 0000000000000..ac4cbebe5329d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/HeapKeyedStateBackendAsyncByDefaultTest.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.flink.runtime.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.util.IOUtils; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; + +/** + * This tests that all heap-based {@link StateBackend}s create {@link KeyedStateBackend}s that use asynchronous + * snapshots by default. + */ +public class HeapKeyedStateBackendAsyncByDefaultTest { + + @Test + public void testFsStateBackendDefaultsToAsync() throws Exception { + TemporaryFolder temporaryFolder = new TemporaryFolder(); + temporaryFolder.create(); + + File folder = temporaryFolder.newFolder(); + + try { + // This backend has two constructors that use a default value for async snapshots. + FsStateBackend fsStateBackend = new FsStateBackend(folder.toURI()); + validateSupportForAsyncSnapshots(fsStateBackend); + + fsStateBackend = new FsStateBackend(folder.toURI(), 1024); + validateSupportForAsyncSnapshots(fsStateBackend); + } finally { + folder.delete(); + temporaryFolder.delete(); + } + } + + @Test + public void testMemoryStateBackendDefaultsToAsync() throws Exception { + MemoryStateBackend memoryStateBackend = new MemoryStateBackend(); + validateSupportForAsyncSnapshots(memoryStateBackend); + } + + private void validateSupportForAsyncSnapshots(AbstractStateBackend backend) throws IOException { + + AbstractKeyedStateBackend keyedStateBackend = backend.createKeyedStateBackend( + new DummyEnvironment("Test", 1, 0), + new JobID(), + "testOperator", + IntSerializer.INSTANCE, + 1, + new KeyGroupRange(0, 0), + null + ); + + try { + Assert.assertTrue(keyedStateBackend.supportsAsynchronousSnapshots()); + } finally { + IOUtils.closeQuietly(keyedStateBackend); + keyedStateBackend.dispose(); + } + } +} From 6f9ab7217266f5458263fdd976214c1b4c552576 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 13 Nov 2017 16:51:18 +0100 Subject: [PATCH 040/367] [FLINK-7845][runtime] Make NettyMessage public This a walkaround strange javaassist bug. The issue should go away once we upgrade netty dependency. Please check the ticket for more information. This closes #5007. --- .../apache/flink/runtime/io/network/netty/NettyMessage.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java index e73f61df07f86..89fb9e85e3301 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java @@ -52,8 +52,11 @@ /** * A simple and generic interface to serialize messages to Netty's buffer space. + * + *

This class must be public as long as we are using a Netty version prior to 4.0.45. Please check FLINK-7845 for + * more information. */ -abstract class NettyMessage { +public abstract class NettyMessage { // ------------------------------------------------------------------------ // Note: Every NettyMessage subtype needs to have a public 0-argument From cf099f1d5c3234efa058351c221089c4e124e82d Mon Sep 17 00:00:00 2001 From: yew1eb Date: Sat, 11 Nov 2017 20:45:42 +0800 Subject: [PATCH 041/367] [hotfix][docs][javadocs] Remove double "of" This closes #4999. --- docs/ops/config.md | 2 +- .../java/org/apache/flink/configuration/ConfigConstants.java | 4 ++-- .../scala/org/apache/flink/table/plan/schema/RowSchema.scala | 2 +- .../apache/flink/runtime/clusterframework/BootstrapTools.java | 2 +- .../flink/runtime/instance/SlotSharingGroupAssignment.java | 2 +- .../apache/flink/runtime/io/network/netty/NettyClient.java | 2 +- .../apache/flink/yarn/configuration/YarnConfigOptions.java | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/ops/config.md b/docs/ops/config.md index 9ee710645bd5e..55dec4cd3fc01 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -316,7 +316,7 @@ The following parameters configure Flink's JobManager and TaskManagers. - `taskmanager.exit-on-fatal-akka-error`: Whether the TaskManager shall be terminated in case of a fatal Akka error (quarantining event). (DEFAULT: **false**) -- `jobmanager.tdd.offload.minsize`: Maximum size of of the `TaskDeploymentDescriptor`'s serialized task and job information to still transmit them via RPC. Larger blobs may be offloaded to the BLOB server. (DEFAULT: **1 KiB**). +- `jobmanager.tdd.offload.minsize`: Maximum size of the `TaskDeploymentDescriptor`'s serialized task and job information to still transmit them via RPC. Larger blobs may be offloaded to the BLOB server. (DEFAULT: **1 KiB**). ### Distributed Coordination (via Akka) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 4153e456e3160..f9e473578c536 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -420,7 +420,7 @@ public final class ConfigConstants { * The maximum number of failed YARN containers before entirely stopping * the YARN session / job on YARN. * - * By default, we take the number of of initially requested containers. + * By default, we take the number of initially requested containers. * @deprecated in favor of {@code YarnConfigOptions#MAX_FAILED_CONTAINERS}. */ @Deprecated @@ -523,7 +523,7 @@ public final class ConfigConstants { * The maximum number of failed Mesos tasks before entirely stopping * the Mesos session / job on Mesos. * - * By default, we take the number of of initially requested tasks. + * By default, we take the number of initially requested tasks. * @deprecated in favor of {@code MesosOptions#MAX_FAILED_TASKS}. */ @Deprecated diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala index cfe6683d276dc..f5f463cb502cf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala @@ -48,7 +48,7 @@ class RowSchema(private val logicalRowType: RelDataType) { def relDataType: RelDataType = logicalRowType /** - * Returns the [[TypeInformation]] of of the schema + * Returns the [[TypeInformation]] of the schema */ def typeInfo: TypeInformation[Row] = physicalRowTypeInfo diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 5849c21b72e6b..d1efd77aa5564 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -288,7 +288,7 @@ public static void substituteDeprecatedConfigKey(Configuration config, String de } /** - * Sets the value of of a new config key to the value of a deprecated config key. Taking into + * Sets the value of a new config key to the value of a deprecated config key. Taking into * account the changed prefix. * @param config Config to write * @param deprecatedPrefix Old prefix of key diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java index a071e5068fba3..7618b188a69c0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java @@ -53,7 +53,7 @@ * the {@link SimpleSlot}s.

* *

An exception are the co-location-constraints, that define that the i-th subtask of one - * vertex needs to be scheduled strictly together with the i-th subtasks of of the vertices + * vertex needs to be scheduled strictly together with the i-th subtasks of the vertices * that share the co-location-constraint. To manage that, a co-location-constraint gets its * own shared slot inside the shared slots of a sharing group.

* diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java index 485af2dbc216a..c845cb3604dc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java @@ -208,7 +208,7 @@ public void initChannel(SocketChannel channel) throws Exception { { throw new ChannelException( "The operating system does not offer enough file handles to open the network connection. " + - "Please increase the number of of available file handles.", e.getCause()); + "Please increase the number of available file handles.", e.getCause()); } else { throw e; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index 37733523a0a25..02ab6a02af41f 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -62,7 +62,7 @@ public class YarnConfigOptions { /** * The maximum number of failed YARN containers before entirely stopping * the YARN session / job on YARN. - * By default, we take the number of of initially requested containers. + * By default, we take the number of initially requested containers. * *

Note: This option returns a String since Integer options must have a static default value. */ From c2d3d6aec42170a9962c47556f0e88f4e6815633 Mon Sep 17 00:00:00 2001 From: yew1eb Date: Sat, 11 Nov 2017 21:08:21 +0800 Subject: [PATCH 042/367] [hotfix][docs] Fix typos in deployment AWS documentation This closes #5000. --- docs/ops/deployment/aws.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ops/deployment/aws.md b/docs/ops/deployment/aws.md index 709ecd1929026..bd11bec05a9ef 100644 --- a/docs/ops/deployment/aws.md +++ b/docs/ops/deployment/aws.md @@ -338,7 +338,7 @@ Caused by: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native ### IOException: `400: Bad Request` -If you you have configured everything properly, but get a `Bad Request` Exception **and** your S3 bucket is located in region `eu-central-1`, you might be running an S3 client, which does not support [Amazon's signature version 4](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html). +If you have configured everything properly, but get a `Bad Request` Exception **and** your S3 bucket is located in region `eu-central-1`, you might be running an S3 client, which does not support [Amazon's signature version 4](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html). Currently, this includes all Hadoop versions up to 2.7.2 running `NativeS3FileSystem`, which depend on `JetS3t 0.9.0` instead of a version [>= 0.9.4](http://www.jets3t.org/RELEASE_NOTES.html). From 24970a9f258c36c3406500ab8f383ded38c25216 Mon Sep 17 00:00:00 2001 From: Michael Fong Date: Mon, 13 Nov 2017 10:17:02 +0800 Subject: [PATCH 043/367] [FLINK-4500][docs] Update cassandra documentation regarding data loss As of FLINK-4500 the Cassandra connector will wait for pending updates to finish upon checkpoint. This closes #5002. --- docs/dev/connectors/cassandra.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/dev/connectors/cassandra.md b/docs/dev/connectors/cassandra.md index dd8a775278688..64d81b5c96752 100644 --- a/docs/dev/connectors/cassandra.md +++ b/docs/dev/connectors/cassandra.md @@ -101,8 +101,6 @@ Note that that enabling this feature will have an adverse impact on latency. ### Checkpointing and Fault Tolerance With checkpointing enabled, Cassandra Sink guarantees at-least-once delivery of action requests to C* instance. -

Note: However, current Cassandra Sink implementation does not flush the pending mutations before the checkpoint was triggered. Thus, some in-flight mutations might not be replayed when the job recovered.

- More details on [checkpoints docs]({{ site.baseurl }}/dev/stream/state/checkpointing.html) and [fault tolerance guarantee docs]({{ site.baseurl }}/dev/connectors/guarantees.html) ## Examples From a4b99967cf83084bfa037a507b503b43846431d3 Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 10 Nov 2017 14:54:32 +0100 Subject: [PATCH 044/367] [hotfix][docs] Fix broken link to FLINK-7811 This closes #4995. --- docs/start/building.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/start/building.md b/docs/start/building.md index 455ebbfe6eb9f..cbf1d1fa274a9 100644 --- a/docs/start/building.md +++ b/docs/start/building.md @@ -109,7 +109,7 @@ Flink has APIs, libraries, and runtime modules written in [Scala](http://scala-l Flink 1.4 currently builds only with Scala version 2.11. -We are working on supporting Scala 2.12, but certain breaking changes in Scala 2.12 make this a more involved effort. Please check out (this JIRA issue)[https://issues.apache.org/jira/browse/FLINK-7811] for updates. +We are working on supporting Scala 2.12, but certain breaking changes in Scala 2.12 make this a more involved effort. Please check out [this JIRA issue](https://issues.apache.org/jira/browse/FLINK-7811) for updates. {% top %} From 8d8c52f876b79524393d2604b5bedc54c0764ae7 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 7 Nov 2017 17:13:57 +0100 Subject: [PATCH 045/367] [FLINK-8011][dist] Set flink-python to provided This closes #4973. --- flink-dist/pom.xml | 15 ++++++++------- flink-dist/src/main/assemblies/bin.xml | 11 ++++++++++- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index d009ead5ce25a..dad3b95660e00 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -97,12 +97,6 @@ under the License. ${project.version} - - org.apache.flink - flink-python_${scala.binary.version} - ${project.version} - - org.apache.flink flink-scala-shell_${scala.binary.version} @@ -263,6 +257,14 @@ under the License. + + + org.apache.flink + flink-python_${scala.binary.version} + ${project.version} + provided + + org.apache.flink flink-cep_${scala.binary.version} @@ -485,7 +487,6 @@ under the License. - org.apache.flink:flink-python_${scala.binary.version} org.slf4j:slf4j-log4j12 log4j:log4j diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 8b4526d066f05..eb6867dd3e3d9 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -39,7 +39,6 @@ under the License. true - org.apache.flink:flink-python_${scala.binary.version} org.slf4j:slf4j-log4j12 log4j:log4j @@ -184,6 +183,16 @@ under the License. + + + ../flink-libraries/flink-python/target + lib + 0644 + + flink-python_${scala.binary.version}-${project.version}.jar + + + ../flink-libraries/flink-python/src/main/python/org/apache/flink/python/api From fcc79c0ed76818e147381117ea735f5796be6066 Mon Sep 17 00:00:00 2001 From: Alejandro Alcalde Date: Tue, 7 Nov 2017 13:09:46 +0100 Subject: [PATCH 046/367] [FLINK-8006] [Startup Shell Scripts] - Fixing $pid This closes #4968. --- .../src/main/flink-bin/bin/flink-daemon.sh | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/flink-dist/src/main/flink-bin/bin/flink-daemon.sh b/flink-dist/src/main/flink-bin/bin/flink-daemon.sh index 6985776f9eb24..b337a1770c1fa 100644 --- a/flink-dist/src/main/flink-bin/bin/flink-daemon.sh +++ b/flink-dist/src/main/flink-bin/bin/flink-daemon.sh @@ -84,7 +84,7 @@ fi # Ascending ID depending on number of lines in pid file. # This allows us to start multiple daemon of each type. -id=$([ -f "$pid" ] && echo $(wc -l < $pid) || echo "0") +id=$([ -f "$pid" ] && echo $(wc -l < "$pid") || echo "0") FLINK_LOG_PREFIX="${FLINK_LOG_DIR}/flink-${FLINK_IDENT_STRING}-${DAEMON}-${id}-${HOSTNAME}" log="${FLINK_LOG_PREFIX}.log" @@ -108,7 +108,7 @@ case $STARTSTOP in rotateLogFilesWithPrefix "$FLINK_LOG_DIR" "$FLINK_LOG_PREFIX" # Print a warning if daemons are already running on host - if [ -f $pid ]; then + if [ -f "$pid" ]; then active=() while IFS='' read -r p || [[ -n "$p" ]]; do kill -0 $p >/dev/null 2>&1 @@ -134,7 +134,7 @@ case $STARTSTOP in # Add to pid file if successful start if [[ ${mypid} =~ ${IS_NUMBER} ]] && kill -0 $mypid > /dev/null 2>&1 ; then - echo $mypid >> $pid + echo $mypid >> "$pid" else echo "Error starting $DAEMON daemon." exit 1 @@ -142,18 +142,18 @@ case $STARTSTOP in ;; (stop) - if [ -f $pid ]; then + if [ -f "$pid" ]; then # Remove last in pid file - to_stop=$(tail -n 1 $pid) + to_stop=$(tail -n 1 "$pid") if [ -z $to_stop ]; then - rm $pid # If all stopped, clean up pid file + rm "$pid" # If all stopped, clean up pid file echo "No $DAEMON daemon to stop on host $HOSTNAME." else - sed \$d $pid > $pid.tmp # all but last line + sed \$d "$pid" > "$pid.tmp" # all but last line # If all stopped, clean up pid file - [ $(wc -l < $pid.tmp) -eq 0 ] && rm $pid $pid.tmp || mv $pid.tmp $pid + [ $(wc -l < "$pid.tmp") -eq 0 ] && rm "$pid" "$pid.tmp" || mv "$pid.tmp" "$pid" if kill -0 $to_stop > /dev/null 2>&1; then echo "Stopping $DAEMON daemon (pid: $to_stop) on host $HOSTNAME." @@ -168,8 +168,8 @@ case $STARTSTOP in ;; (stop-all) - if [ -f $pid ]; then - mv $pid ${pid}.tmp + if [ -f "$pid" ]; then + mv "$pid" "${pid}.tmp" while read to_stop; do if kill -0 $to_stop > /dev/null 2>&1; then @@ -178,8 +178,8 @@ case $STARTSTOP in else echo "Skipping $DAEMON daemon (pid: $to_stop), because it is not running anymore on $HOSTNAME." fi - done < ${pid}.tmp - rm ${pid}.tmp + done < "${pid}.tmp" + rm "${pid}.tmp" fi ;; From 7c7f24ec84dc73b59c8523b4c7af8d89f1a17eb5 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 14 Nov 2017 10:58:43 +0100 Subject: [PATCH 047/367] [FLINK-8056][dist] Use 'web.port' instead of 'jobmanager.web.port' This closes #5010. --- flink-dist/src/main/resources/flink-conf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index 2fb155926c721..ddff9c8229686 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -74,7 +74,7 @@ parallelism.default: 1 # The port under which the web-based runtime monitor listens. # A value of -1 deactivates the web server. -jobmanager.web.port: 8081 +web.port: 8081 # Flag to specify whether job submission is enabled from the web-based # runtime monitor. Uncomment to disable. From 2774335983bf83b68ff061388d0bd69d4346bb07 Mon Sep 17 00:00:00 2001 From: Vetriselvan1187 Date: Mon, 6 Nov 2017 22:41:41 +0530 Subject: [PATCH 048/367] [FLINK-7998][examples] Fix TPCHQuery3 examples This closes #4959. --- .../examples/java/relational/TPCHQuery3.java | 19 ++++++++++++++----- .../scala/relational/TPCHQuery3.scala | 2 +- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java index f416f3038c408..66213e3326247 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java @@ -101,8 +101,8 @@ public static void main(String[] args) throws Exception { // get input data DataSet lineitems = getLineitemDataSet(env, params.get("lineitem")); - DataSet orders = getOrdersDataSet(env, params.get("customer")); - DataSet customers = getCustomerDataSet(env, params.get("orders")); + DataSet customers = getCustomerDataSet(env, params.get("customer")); + DataSet orders = getOrdersDataSet(env, params.get("orders")); // Filter market segment "AUTOMOBILE" customers = customers.filter( @@ -180,7 +180,10 @@ public ShippingPriorityItem join(ShippingPriorityItem i, Lineitem l) { // DATA TYPES // ************************************************************************* - private static class Lineitem extends Tuple4 { + /** + * Lineitem. + */ + public static class Lineitem extends Tuple4 { public Long getOrderkey() { return this.f0; @@ -199,7 +202,10 @@ public String getShipdate() { } } - private static class Customer extends Tuple2 { + /** + * Customer. + */ + public static class Customer extends Tuple2 { public Long getCustKey() { return this.f0; @@ -210,7 +216,10 @@ public String getMktsegment() { } } - private static class Order extends Tuple4 { + /** + * Order. + */ + public static class Order extends Tuple4 { public Long getOrderKey() { return this.f0; diff --git a/flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/relational/TPCHQuery3.scala b/flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/relational/TPCHQuery3.scala index 3d6e9ab143df2..2db1fe7f7bbbd 100644 --- a/flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/relational/TPCHQuery3.scala +++ b/flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/relational/TPCHQuery3.scala @@ -102,7 +102,7 @@ object TPCHQuery3 { getCustomerDataSet(env, params.get("customer")). filter( c => c.mktSegment.equals("AUTOMOBILE")) // read orders - val orders = getOrdersDataSet(env, params.get("order")) + val orders = getOrdersDataSet(env, params.get("orders")) // filter orders by order date val items = orders.filter( o => dateFormat.parse(o.orderDate).before(date) ) From 195e3da8a78b80116ef00e65d771daa5a0a1d7a3 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 14 Nov 2017 14:47:00 +0100 Subject: [PATCH 049/367] [FLINK-8071][build] Bump shade-plugin asm version to 5.1 This closes #5014. --- pom.xml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pom.xml b/pom.xml index bfb6f7c365bd0..890e7f690e68e 100644 --- a/pom.xml +++ b/pom.xml @@ -1317,6 +1317,24 @@ under the License. org.apache.maven.plugins maven-shade-plugin 2.4.1 + + + + org.ow2.asm + asm + 5.1 + + + org.ow2.asm + asm-commons + 5.1 + + + org.ow2.asm + asm-tree + 5.1 + + From 5c6eaabfcd541ffb35d911c6d9d2f6f12f207cd8 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 8 Nov 2017 11:58:38 +0100 Subject: [PATCH 050/367] [FLINK-7419][build][avro] Shade jackson dependency in flink-dist This closes #4981. --- flink-dist/pom.xml | 6 ++++++ flink-formats/flink-avro/pom.xml | 29 ----------------------------- tools/travis_mvn_watchdog.sh | 16 ++++++++++++++++ 3 files changed, 22 insertions(+), 29 deletions(-) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index dad3b95660e00..6696294f405b9 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -491,6 +491,12 @@ under the License. log4j:log4j + + + org.codehaus.jackson + org.apache.flink.formats.avro.shaded.org.codehaus.jackson + + reference.conf diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml index 0313b051593f9..3359c40a15b22 100644 --- a/flink-formats/flink-avro/pom.xml +++ b/flink-formats/flink-avro/pom.xml @@ -171,35 +171,6 @@ under the License. - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - ${project.basedir}/target/dependency-reduced-pom.xml - true - - - org.codehaus.jackson:* - - - - - org.codehaus.jackson - org.apache.flink.avro.shaded.org.codehaus.jackson - - - - - - diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 8e315fbf6f341..42f7ae3355dd3 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -301,6 +301,22 @@ check_shaded_artifacts() { return 1 fi + CODEHAUS_JACKSON=`cat allClasses | grep '^org/codehaus/jackson' | wc -l` + if [ "$CODEHAUS_JACKSON" != "0" ]; then + echo "==============================================================================" + echo "Detected '$CODEHAUS_JACKSON' unshaded org.codehaus.jackson classes in fat jar" + echo "==============================================================================" + return 1 + fi + + FASTERXML_JACKSON=`cat allClasses | grep '^com/fasterxml/jackson' | wc -l` + if [ "$FASTERXML_JACKSON" != "0" ]; then + echo "==============================================================================" + echo "Detected '$FASTERXML_JACKSON' unshaded com.fasterxml.jackson classes in fat jar" + echo "==============================================================================" + return 1 + fi + SNAPPY=`cat allClasses | grep '^org/xerial/snappy' | wc -l` if [ "$SNAPPY" == "0" ]; then echo "==============================================================================" From d6d35fa145c122f469d72bf73433afde5d778aac Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 14 Nov 2017 14:36:22 +0100 Subject: [PATCH 051/367] [FLINK-7973] disable JNI bridge for relocated hadoop classes in s3-fs-* --- .../flink-s3-fs-hadoop/README.md | 21 +++-- .../apache/hadoop/util/NativeCodeLoader.java | 94 +++++++++++++++++++ .../flink-s3-fs-presto/README.md | 21 +++-- .../apache/hadoop/util/NativeCodeLoader.java | 94 +++++++++++++++++++ tools/maven/suppressions.xml | 5 +- 5 files changed, 222 insertions(+), 13 deletions(-) create mode 100644 flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java create mode 100644 flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java diff --git a/flink-filesystems/flink-s3-fs-hadoop/README.md b/flink-filesystems/flink-s3-fs-hadoop/README.md index 3ad90e36e2c3a..ca8fdab029fd8 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/README.md +++ b/flink-filesystems/flink-s3-fs-hadoop/README.md @@ -13,12 +13,21 @@ relocated class names of classes loaded via reflection If you want to change the Hadoop version this project depends on, the following steps are required to keep the shading correct: -1. copy `org/apache/hadoop/conf/Configuration.java` from the respective Hadoop jar file to this project - - adapt the `Configuration` class by replacing `core-default.xml` with `core-default-shaded.xml`. -2. copy `core-default.xml` from the respective Hadoop jar file to this project as - - `src/main/resources/core-default-shaded.xml` (replacing every occurence of `org.apache.hadoop` with `org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop`) - - `src/test/resources/core-site.xml` (as is) -3. verify the shaded jar: +1. from the respective Hadoop jar (currently 2.8.1 as of the `s3hadoop.hadoop.version` property our `pom.xml`), + - copy `org/apache/hadoop/conf/Configuration.java` to `src/main/java/org/apache/hadoop/conf/` and + - replace `core-default.xml` with `core-default-shaded.xml`. + - copy `org/apache/hadoop/util/NativeCodeLoader.java` to `src/main/java/org/apache/hadoop/util/` and + - replace the static initializer with + ``` + static { + LOG.info("Skipping native-hadoop library for flink-s3-fs-hadoop's relocated Hadoop... " + + "using builtin-java classes where applicable"); + } +``` + - copy `core-default.xml` to `src/main/resources/core-default-shaded.xml` and + - change every occurence of `org.apache.hadoop` into `org.apache.flink.fs.s3hadoop.shaded.org.apache.hadoop` + - copy `core-site.xml` to `src/test/resources/core-site.xml` (as is) +2. verify the shaded jar: - does not contain any unshaded classes except for `org.apache.flink.fs.s3hadoop.S3FileSystemFactory` - all other classes should be under `org.apache.flink.fs.s3hadoop.shaded` - there should be a `META-INF/services/org.apache.flink.fs.s3hadoop.S3FileSystemFactory` file pointing to the `org.apache.flink.fs.s3hadoop.S3FileSystemFactory` class diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java new file mode 100644 index 0000000000000..822da5b519e34 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java @@ -0,0 +1,94 @@ +/** + * 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.hadoop.util; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; + +/** + * A helper to load the native hadoop code i.e. libhadoop.so. + * This handles the fallback to either the bundled libhadoop-Linux-i386-32.so + * or the default java implementations where appropriate. + * + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class NativeCodeLoader { + + private static final Log LOG = + LogFactory.getLog(NativeCodeLoader.class); + + private static boolean nativeCodeLoaded = false; + + static { + LOG.info("Skipping native-hadoop library for flink-s3-fs-hadoop's relocated Hadoop... " + + "using builtin-java classes where applicable"); + } + + /** + * Check if native-hadoop code is loaded for this platform. + * + * @return true if native-hadoop is loaded, + * else false + */ + public static boolean isNativeCodeLoaded() { + return nativeCodeLoaded; + } + + /** + * Returns true only if this build was compiled with support for snappy. + */ + public static native boolean buildSupportsSnappy(); + + /** + * Returns true only if this build was compiled with support for openssl. + */ + public static native boolean buildSupportsOpenssl(); + + public static native String getLibraryName(); + + /** + * Return if native hadoop libraries, if present, can be used for this job. + * @param conf configuration + * + * @return true if native hadoop libraries, if present, can be + * used for this job; false otherwise. + */ + public boolean getLoadNativeLibraries(Configuration conf) { + return conf.getBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, + CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_DEFAULT); + } + + /** + * Set if native hadoop libraries, if present, can be used for this job. + * + * @param conf configuration + * @param loadNativeLibraries can native hadoop libraries be loaded + */ + public void setLoadNativeLibraries(Configuration conf, + boolean loadNativeLibraries) { + conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, + loadNativeLibraries); + } + +} diff --git a/flink-filesystems/flink-s3-fs-presto/README.md b/flink-filesystems/flink-s3-fs-presto/README.md index bf3c1fe30fddf..cae39c98e6d78 100644 --- a/flink-filesystems/flink-s3-fs-presto/README.md +++ b/flink-filesystems/flink-s3-fs-presto/README.md @@ -14,12 +14,21 @@ relocated class names of classes loaded via reflection If you want to change the Hadoop version this project depends on, the following steps are required to keep the shading correct: -1. copy `org/apache/hadoop/conf/Configuration.java` from the respective Hadoop jar file (from `com.facebook.presto.hadoop/hadoop-apache2`) to this project - - adapt the `Configuration` class by replacing `core-default.xml` with `core-default-shaded.xml`. -2. copy `core-default.xml` from the respective Hadoop jar (from `com.facebook.presto.hadoop/hadoop-apache2`) file to this project as - - `src/main/resources/core-default-shaded.xml` (replacing every occurence of `org.apache.hadoop` with `org.apache.flink.fs.s3presto.shaded.org.apache.hadoop`) - - `src/test/resources/core-site.xml` (as is) -3. verify the shaded jar: +1. from the respective Hadoop jar (from the `com.facebook.presto.hadoop/hadoop-apache2` resource, currently version 2.7.3-1 as of our `pom.xml`), + - copy `org/apache/hadoop/conf/Configuration.java` to `src/main/java/org/apache/hadoop/conf/` and + - replace `core-default.xml` with `core-default-shaded.xml`. + - copy `org/apache/hadoop/util/NativeCodeLoader.java` to `src/main/java/org/apache/hadoop/util/` and + - replace the static initializer with + ``` + static { + LOG.info("Skipping native-hadoop library for flink-s3-fs-presto's relocated Hadoop... " + + "using builtin-java classes where applicable"); + } +``` + - copy `core-default.xml` to `src/main/resources/core-default-shaded.xml` and + - change every occurence of `org.apache.hadoop` into `org.apache.flink.fs.s3presto.shaded.org.apache.hadoop` + - copy `core-site.xml` to `src/test/resources/core-site.xml` (as is) +2. verify the shaded jar: - does not contain any unshaded classes except for `org.apache.flink.fs.s3presto.S3FileSystemFactory` - all other classes should be under `org.apache.flink.fs.s3presto.shaded` - there should be a `META-INF/services/org.apache.flink.fs.s3presto.S3FileSystemFactory` file pointing to the `org.apache.flink.fs.s3presto.S3FileSystemFactory` class diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java new file mode 100644 index 0000000000000..cbcb756bf218d --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java @@ -0,0 +1,94 @@ +/** + * 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.hadoop.util; + +import com.facebook.presto.hadoop.$internal.org.apache.commons.logging.Log; +import com.facebook.presto.hadoop.$internal.org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; + +/** + * A helper to load the native hadoop code i.e. libhadoop.so. + * This handles the fallback to either the bundled libhadoop-Linux-i386-32.so + * or the default java implementations where appropriate. + * + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class NativeCodeLoader { + + private static final Log LOG = + LogFactory.getLog(NativeCodeLoader.class); + + private static boolean nativeCodeLoaded = false; + + static { + LOG.info("Skipping native-hadoop library for flink-s3-fs-presto's relocated Hadoop... " + + "using builtin-java classes where applicable"); + } + + /** + * Check if native-hadoop code is loaded for this platform. + * + * @return true if native-hadoop is loaded, + * else false + */ + public static boolean isNativeCodeLoaded() { + return nativeCodeLoaded; + } + + /** + * Returns true only if this build was compiled with support for snappy. + */ + public static native boolean buildSupportsSnappy(); + + /** + * Returns true only if this build was compiled with support for openssl. + */ + public static native boolean buildSupportsOpenssl(); + + public static native String getLibraryName(); + + /** + * Return if native hadoop libraries, if present, can be used for this job. + * @param conf configuration + * + * @return true if native hadoop libraries, if present, can be + * used for this job; false otherwise. + */ + public boolean getLoadNativeLibraries(Configuration conf) { + return conf.getBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, + CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_DEFAULT); + } + + /** + * Set if native hadoop libraries, if present, can be used for this job. + * + * @param conf configuration + * @param loadNativeLibraries can native hadoop libraries be loaded + */ + public void setLoadNativeLibraries(Configuration conf, + boolean loadNativeLibraries) { + conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, + loadNativeLibraries); + } + +} diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index 90a1eea7d9265..a2de9e8877d82 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -35,8 +35,11 @@ under the License. - + + From 81e3a889116466037c59514b28312257005529c0 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Aug 2017 19:24:57 +0800 Subject: [PATCH 052/367] [FLINK-7451] [table] Support non-ascii character literals in Table API and SQL This closes #4544. --- .../table/calcite/FlinkTypeFactory.scala | 6 +++ ...eralPrefixTest.scala => LiteralTest.scala} | 46 +++++++++++++++++-- .../UserDefinedScalarFunctionTest.scala | 10 ---- 3 files changed, 47 insertions(+), 15 deletions(-) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/{LiteralPrefixTest.scala => LiteralTest.scala} (62%) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala index 04fab76ba181e..7bcdc0f40ea39 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.calcite import java.util +import java.nio.charset.Charset import org.apache.calcite.avatica.util.TimeUnit import org.apache.calcite.jdbc.JavaTypeFactoryImpl @@ -27,6 +28,7 @@ import org.apache.calcite.sql.SqlIntervalQualifier import org.apache.calcite.sql.`type`.SqlTypeName._ import org.apache.calcite.sql.`type`.{BasicSqlType, SqlTypeName} import org.apache.calcite.sql.parser.SqlParserPos +import org.apache.calcite.util.ConversionUtil import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ import org.apache.flink.api.common.typeinfo._ import org.apache.flink.api.common.typeutils.CompositeType @@ -301,6 +303,10 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp } } } + + override def getDefaultCharset: Charset = { + Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME) + } } object FlinkTypeFactory { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala similarity index 62% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala index 0f46c4e8d46a2..0a60eae0e5f56 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala @@ -22,11 +22,12 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.Types -import org.apache.flink.table.expressions.utils.ExpressionTestBase +import org.apache.flink.table.expressions.utils.{ExpressionTestBase, Func3} +import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.types.Row import org.junit.Test -class LiteralPrefixTest extends ExpressionTestBase { +class LiteralTest extends ExpressionTestBase { @Test def testFieldWithBooleanPrefix(): Unit = { @@ -62,18 +63,53 @@ class LiteralPrefixTest extends ExpressionTestBase { ) } + @Test + def testNonAsciiLiteral(): Unit = { + testAllApis( + 'f4.like("%测试%"), + "f4.like('%测试%')", + "f4 LIKE '%测试%'", + "true") + + testAllApis( + "Абвгде" + "谢谢", + "'Абвгде' + '谢谢'", + "'Абвгде' || '谢谢'", + "Абвгде谢谢") + } + + @Test + def testDoubleQuote(): Unit = { + val hello = "\"\"" + testAllApis( + Func3(42, hello), + s"Func3(42, '$hello')", + s"Func3(42, '$hello')", + s"42 and $hello") + } + def testData: Any = { - val testData = new Row(3) + val testData = new Row(4) testData.setField(0, "trUeX_value") testData.setField(1, "FALSE_A_value") testData.setField(2, "FALSE_AB_value") + testData.setField(3, "这是个测试字符串") testData } def typeInfo: TypeInformation[Any] = { new RowTypeInfo( - Array(Types.STRING, Types.STRING, Types.STRING).asInstanceOf[Array[TypeInformation[_]]], - Array("trUeX", "FALSE_A", "FALSE_AB") + Array( + Types.STRING, + Types.STRING, + Types.STRING, + Types.STRING + ).asInstanceOf[Array[TypeInformation[_]]], + Array("trUeX", "FALSE_A", "FALSE_AB", "f4") ).asInstanceOf[TypeInformation[Any]] } + + override def functions: Map[String, ScalarFunction] = Map( + "Func3" -> Func3 + ) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala index a3b2f07b3b584..a01f2aef76144 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala @@ -136,16 +136,6 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { "-1") } - @Test - def testDoubleQuoteParameters(): Unit = { - val hello = "\"\"" - testAllApis( - Func3(42, hello), - s"Func3(42, '$hello')", - s"Func3(42, '$hello')", - s"42 and $hello") - } - @Test def testResults(): Unit = { testAllApis( From 49aeb8f1d98fe99eaf97c3009ba23008652964e7 Mon Sep 17 00:00:00 2001 From: twalthr Date: Wed, 15 Nov 2017 14:39:21 +0100 Subject: [PATCH 053/367] [FLINK-7451] [table] Disable testing of the charset in TableTestBase --- .../flink/table/utils/TableTestBase.scala | 33 +++++++++---------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index 5f8f5d6a40d87..382931423fb4b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.utils import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.{DataSet => JDataSet, ExecutionEnvironment => JExecutionEnvironment} import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment} @@ -85,6 +86,16 @@ abstract class TableTestUtil { def printTable(resultTable: Table): Unit def printSql(query: String): Unit + + protected def verifyString(expected: String, optimized: RelNode) { + val actual = RelOptUtil.toString(optimized) + // we remove the charset for testing because it + // depends on the native machine (Little/Big Endian) + val actualNoCharset = actual.replace("_UTF-16LE'", "'") + assertEquals( + expected.split("\n").map(_.trim).mkString("\n"), + actualNoCharset.split("\n").map(_.trim).mkString("\n")) + } } object TableTestUtil { @@ -125,7 +136,6 @@ object TableTestUtil { def streamTableNode(idx: Int): String = { s"DataStreamScan(table=[[_DataStreamTable_$idx]])" } - } case class BatchTableTestUtil() extends TableTestUtil { @@ -184,10 +194,7 @@ case class BatchTableTestUtil() extends TableTestUtil { def verifyTable(resultTable: Table, expected: String): Unit = { val relNode = resultTable.getRelNode val optimized = tableEnv.optimize(relNode) - val actual = RelOptUtil.toString(optimized) - assertEquals( - expected.split("\n").map(_.trim).mkString("\n"), - actual.split("\n").map(_.trim).mkString("\n")) + verifyString(expected, optimized) } def verifyJavaSql(query: String, expected: String): Unit = { @@ -197,10 +204,7 @@ case class BatchTableTestUtil() extends TableTestUtil { def verifyJavaTable(resultTable: Table, expected: String): Unit = { val relNode = resultTable.getRelNode val optimized = javaTableEnv.optimize(relNode) - val actual = RelOptUtil.toString(optimized) - assertEquals( - expected.split("\n").map(_.trim).mkString("\n"), - actual.split("\n").map(_.trim).mkString("\n")) + verifyString(expected, optimized) } def printTable(resultTable: Table): Unit = { @@ -273,10 +277,7 @@ case class StreamTableTestUtil() extends TableTestUtil { def verifyTable(resultTable: Table, expected: String): Unit = { val relNode = resultTable.getRelNode val optimized = tableEnv.optimize(relNode, updatesAsRetraction = false) - val actual = RelOptUtil.toString(optimized) - assertEquals( - expected.split("\n").map(_.trim).mkString("\n"), - actual.split("\n").map(_.trim).mkString("\n")) + verifyString(expected, optimized) } def verifyJavaSql(query: String, expected: String): Unit = { @@ -286,10 +287,7 @@ case class StreamTableTestUtil() extends TableTestUtil { def verifyJavaTable(resultTable: Table, expected: String): Unit = { val relNode = resultTable.getRelNode val optimized = javaTableEnv.optimize(relNode, updatesAsRetraction = false) - val actual = RelOptUtil.toString(optimized) - assertEquals( - expected.split("\n").map(_.trim).mkString("\n"), - actual.split("\n").map(_.trim).mkString("\n")) + verifyString(expected, optimized) } // the print methods are for debugging purposes only @@ -302,5 +300,4 @@ case class StreamTableTestUtil() extends TableTestUtil { def printSql(query: String): Unit = { printTable(tableEnv.sqlQuery(query)) } - } From 32bfc38875faa5baea0dc63d91e1273d7df7b2ef Mon Sep 17 00:00:00 2001 From: twalthr Date: Tue, 14 Nov 2017 11:06:54 +0100 Subject: [PATCH 054/367] [FLINK-8013] [table] Support aggregate functions with generic arrays This closes #5011. --- .../java/typeutils/TypeExtractionUtils.java | 16 +++ .../codegen/AggregationCodeGenerator.scala | 20 ++- .../table/expressions/aggregations.scala | 6 +- .../utils/UserDefinedFunctionUtils.scala | 9 +- .../runtime/batch/table/AggregateITCase.scala | 25 +++- .../flink/table/utils/TableTestBase.scala | 2 +- .../table/utils/UserDefinedAggFunctions.scala | 126 ++++++++++++++++++ .../api/scala/util/CollectionDataSets.scala | 1 - 8 files changed, 185 insertions(+), 20 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java index c5c25652ab629..56fcf82296ba7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java @@ -18,7 +18,9 @@ package org.apache.flink.api.java.typeutils; +import java.lang.reflect.Array; import java.lang.reflect.Constructor; +import java.lang.reflect.GenericArrayType; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.lang.reflect.ParameterizedType; @@ -322,4 +324,18 @@ public static boolean hasSuperclass(Class clazz, String superClassName) { } return false; } + + /** + * Returns the raw class of both parameterized types and generic arrays. + * Returns java.lang.Object for all other types. + */ + public static Class getRawClass(Type t) { + if (isClassType(t)) { + return typeToClass(t); + } else if (t instanceof GenericArrayType) { + Type component = ((GenericArrayType)t).getGenericComponentType(); + return Array.newInstance(getRawClass(component), 0).getClass(); + } + return Object.class; + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala index c85b111852c49..32cbde2390c66 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala @@ -17,17 +17,18 @@ */ package org.apache.flink.table.codegen -import java.lang.reflect.{Modifier, ParameterizedType} +import java.lang.reflect.Modifier import java.lang.{Iterable => JIterable} import org.apache.calcite.rex.RexLiteral import org.apache.commons.codec.binary.Base64 import org.apache.flink.api.common.state.{State, StateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.TypeExtractionUtils.{extractTypeArgument, getRawClass} import org.apache.flink.table.api.TableConfig import org.apache.flink.table.api.dataview._ -import org.apache.flink.table.codegen.Indenter.toISC import org.apache.flink.table.codegen.CodeGenUtils.{newName, reflectiveFieldWriteAccess} +import org.apache.flink.table.codegen.Indenter.toISC import org.apache.flink.table.functions.AggregateFunction import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getUserDefinedMethod, signatureToString} @@ -175,7 +176,7 @@ class AggregationCodeGenerator( } if (needMerge) { - val methods = + val method = getUserDefinedMethod(a, "merge", Array(accTypeClasses(i), classOf[JIterable[Any]])) .getOrElse( throw new CodeGenException( @@ -183,17 +184,14 @@ class AggregationCodeGenerator( s"${a.getClass.getCanonicalName}'.") ) - var iterableTypeClass = methods.getGenericParameterTypes.apply(1) - .asInstanceOf[ParameterizedType].getActualTypeArguments.apply(0) - // further extract iterableTypeClass if the accumulator has generic type - iterableTypeClass match { - case impl: ParameterizedType => iterableTypeClass = impl.getRawType - case _ => - } + // use the TypeExtractionUtils here to support nested GenericArrayTypes and + // other complex types + val iterableGenericType = extractTypeArgument(method.getGenericParameterTypes()(1), 0) + val iterableTypeClass = getRawClass(iterableGenericType) if (iterableTypeClass != accTypeClasses(i)) { throw new CodeGenException( - s"merge method in AggregateFunction ${a.getClass.getCanonicalName} does not have " + + s"Merge method in AggregateFunction ${a.getClass.getCanonicalName} does not have " + s"the correct Iterable type. Actually: ${iterableTypeClass.toString}. " + s"Expected: ${accTypeClasses(i).toString}") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala index 1ffcb12c3e619..3adaaa90e2a7b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala @@ -245,8 +245,10 @@ case class AggFunctionCall( ValidationFailure(s"Given parameters do not match any signature. \n" + s"Actual: ${signatureToString(signature)} \n" + s"Expected: ${ - getMethodSignatures(aggregateFunction, "accumulate").drop(1) - .map(signatureToString).mkString(", ")}") + getMethodSignatures(aggregateFunction, "accumulate") + .map(_.drop(1)) + .map(signatureToString) + .mkString(", ")}") } else { ValidationSuccess } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala index 3cd694a8dd08c..4a34732801525 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala @@ -607,13 +607,14 @@ object UserDefinedFunctionUtils { candidate == expected || expected == classOf[Object] || expected.isPrimitive && Primitives.wrap(expected) == candidate || + // time types candidate == classOf[Date] && (expected == classOf[Int] || expected == classOf[JInt]) || candidate == classOf[Time] && (expected == classOf[Int] || expected == classOf[JInt]) || candidate == classOf[Timestamp] && (expected == classOf[Long] || expected == classOf[JLong]) || - (candidate.isArray && - expected.isArray && - candidate.getComponentType.isInstanceOf[Object] && - expected.getComponentType == classOf[Object]) + // arrays + (candidate.isArray && expected.isArray && + (candidate.getComponentType == expected.getComponentType || + expected.getComponentType == classOf[Object])) @throws[Exception] def serialize(function: UserDefinedFunction): String = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala index cf96d19966b6a..e1348f600b52f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala @@ -22,12 +22,13 @@ import java.math.BigDecimal import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.{TableEnvironment, Types} import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinctWithMergeAndReset, WeightedAvgWithMergeAndReset} import org.apache.flink.table.api.scala._ import org.apache.flink.table.functions.aggfunctions.CountAggFunction import org.apache.flink.table.runtime.utils.TableProgramsCollectionTestBase import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode +import org.apache.flink.table.utils.Top10 import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row import org.junit._ @@ -392,6 +393,28 @@ class AggregationsITCase( val results = res.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } + + @Test + def testComplexAggregate(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + val top10Fun = new Top10 + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .groupBy('b) + .select('b, top10Fun('b.cast(Types.INT), 'a.cast(Types.FLOAT))) + + val expected = + "1,[(1,1.0), null, null, null, null, null, null, null, null, null]\n" + + "2,[(2,3.0), (2,2.0), null, null, null, null, null, null, null, null]\n" + + "3,[(3,6.0), (3,5.0), (3,4.0), null, null, null, null, null, null, null]\n" + + "4,[(4,10.0), (4,9.0), (4,8.0), (4,7.0), null, null, null, null, null, null]\n" + + "5,[(5,15.0), (5,14.0), (5,13.0), (5,12.0), (5,11.0), null, null, null, null, null]\n" + + "6,[(6,21.0), (6,20.0), (6,19.0), (6,18.0), (6,17.0), (6,16.0), null, null, null, null]" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } } case class WC(word: String, frequency: Long) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index 382931423fb4b..804fad8cf1bb9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -91,7 +91,7 @@ abstract class TableTestUtil { val actual = RelOptUtil.toString(optimized) // we remove the charset for testing because it // depends on the native machine (Little/Big Endian) - val actualNoCharset = actual.replace("_UTF-16LE'", "'") + val actualNoCharset = actual.replace("_UTF-16LE'", "'").replace("_UTF-16BE'", "'") assertEquals( expected.split("\n").map(_.trim).mkString("\n"), actualNoCharset.split("\n").map(_.trim).mkString("\n")) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala new file mode 100644 index 0000000000000..7d4393c72be86 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala @@ -0,0 +1,126 @@ +/* + * 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.flink.table.utils + +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import java.lang.{Integer => JInt} +import java.lang.{Float => JFloat} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.{ObjectArrayTypeInfo, TupleTypeInfo} +import org.apache.flink.table.api.Types + +/** + * User-defined aggregation function to compute the top 10 most visited Int IDs + * with the highest Float values. We use an Array[Tuple2[Int, Float]] as accumulator to + * store the top 10 entries. + * + * The result is emitted as Array as well. + */ +class Top10 extends AggregateFunction[Array[JTuple2[JInt, JFloat]], Array[JTuple2[JInt, JFloat]]] { + + @Override + def createAccumulator(): Array[JTuple2[JInt, JFloat]] = { + new Array[JTuple2[JInt, JFloat]](10) + } + + /** + * Adds a new entry and count to the top 10 entries if necessary. + * + * @param acc The current top 10 + * @param id The ID + * @param value The value for the ID + */ + def accumulate(acc: Array[JTuple2[JInt, JFloat]], id: Int, value: Float) { + + var i = 9 + var skipped = 0 + + // skip positions without records + while (i >= 0 && acc(i) == null) { + if (acc(i) == null) { + // continue until first entry in the top10 list + i -= 1 + } + } + // backward linear search for insert position + while (i >= 0 && value > acc(i).f1) { + // check next entry + skipped += 1 + i -= 1 + } + + // set if necessary + if (i < 9) { + // move entries with lower count by one position + if (i < 8 && skipped > 0) { + System.arraycopy(acc, i + 1, acc, i + 2, skipped) + } + + // add ID to top10 list + acc(i + 1) = JTuple2.of(id, value) + } + } + + override def getValue(acc: Array[JTuple2[JInt, JFloat]]): Array[JTuple2[JInt, JFloat]] = acc + + def resetAccumulator(acc: Array[JTuple2[JInt, JFloat]]): Unit = { + util.Arrays.fill(acc.asInstanceOf[Array[Object]], null) + } + + def merge( + acc: Array[JTuple2[JInt, JFloat]], + its: java.lang.Iterable[Array[JTuple2[JInt, JFloat]]]): Unit = { + + val it = its.iterator() + while(it.hasNext) { + val acc2 = it.next() + + var i = 0 + var i2 = 0 + while (i < 10 && i2 < 10 && acc2(i2) != null) { + if (acc(i) == null) { + // copy to empty place + acc(i) = acc2(i2) + i += 1 + i2 += 1 + } else if (acc(i).f1.asInstanceOf[Float] >= acc2(i2).f1.asInstanceOf[Float]) { + // forward to next + i += 1 + } else { + // shift and copy + System.arraycopy(acc, i, acc, i + 1, 9 - i) + acc(i) = acc2(i2) + i += 1 + i2 += 1 + } + } + } + } + + override def getAccumulatorType: TypeInformation[Array[JTuple2[JInt, JFloat]]] = { + ObjectArrayTypeInfo.getInfoFor(new TupleTypeInfo[JTuple2[JInt, JFloat]](Types.INT, Types.FLOAT)) + } + + override def getResultType: TypeInformation[Array[JTuple2[JInt, JFloat]]] = { + ObjectArrayTypeInfo.getInfoFor(new TupleTypeInfo[JTuple2[JInt, JFloat]](Types.INT, Types.FLOAT)) + } +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/util/CollectionDataSets.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/util/CollectionDataSets.scala index ec1a810f903fd..1cb5b5273dded 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/util/CollectionDataSets.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/util/CollectionDataSets.scala @@ -55,7 +55,6 @@ object CollectionDataSets { data.+=((19, 6L, "Comment#13")) data.+=((20, 6L, "Comment#14")) data.+=((21, 6L, "Comment#15")) - Random.shuffle(data) env.fromCollection(Random.shuffle(data)) } From 06a922e8113709b6768d1b25b3bb6f59a360e428 Mon Sep 17 00:00:00 2001 From: twalthr Date: Tue, 26 Sep 2017 12:10:33 +0200 Subject: [PATCH 055/367] [FLINK-7678] [table] Support composite inputs for user-defined functions This closes #4726. --- .../table/api/stream/sql/CorrelateTest.scala | 35 +++++++++++++++++++ .../UserDefinedScalarFunctionTest.scala | 20 +++++++++-- .../utils/userDefinedScalarFunctions.scala | 16 ++++++++- .../runtime/batch/table/CorrelateITCase.scala | 9 +++-- .../stream/table/CorrelateITCase.scala | 33 ++++++++++++++--- .../utils/UserDefinedTableFunctions.scala | 27 ++++++++++---- 6 files changed, 121 insertions(+), 19 deletions(-) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala index f2431585e9529..efb83b456eded 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala @@ -19,10 +19,12 @@ package org.apache.flink.table.api.stream.sql import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.JavaVarsArgTableFunc0 import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.{HierarchyTableFunction, PojoTableFunc, TableFunc2, _} +import org.apache.flink.types.Row import org.junit.Test class CorrelateTest extends TableTestBase { @@ -226,6 +228,39 @@ class CorrelateTest extends TableTestBase { util.verifySql(sqlQuery, expected) } + @Test + def testRowType(): Unit = { + val util = streamTestUtil() + val rowType = Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)) + util.addTable[Row]("MyTable", 'a, 'b, 'c)(rowType) + val function = new TableFunc5 + util.addFunction("tableFunc5", function) + + val sqlQuery = "SELECT c, tf.f2 FROM MyTable, LATERAL TABLE(tableFunc5(c)) AS tf" + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamCorrelate", + streamTableNode(0), + term("invocation", "tableFunc5($cor0.c)"), + term("correlate", "table(tableFunc5($cor0.c))"), + term("select", "a", "b", "c", "f0", "f1", "f2"), + term("rowType", "RecordType(" + + "INTEGER a, " + + "BOOLEAN b, " + + "COMPOSITE(Row(f0: Integer, f1: Integer, f2: Integer)) c, " + + "INTEGER f0, " + + "INTEGER f1, " + + "INTEGER f2)"), + term("joinType", "INNER") + ), + term("select", "c", "f2") + ) + + util.verifySql(sqlQuery, expected) + } + @Test def testFilter(): Unit = { val util = streamTestUtil() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala index a01f2aef76144..dbfe5f6c02a74 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala @@ -30,6 +30,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.utils.{ExpressionTestBase, _} import org.apache.flink.table.functions.ScalarFunction import org.junit.Test +import java.lang.{Boolean => JBoolean} class UserDefinedScalarFunctionTest extends ExpressionTestBase { @@ -107,6 +108,14 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { "Nullable(f0)", "Nullable(f0)", "42") + + // test row type input + testAllApis( + Func19('f14), + "Func19(f14)", + "Func19(f14)", + "12,true,1,2,3" + ) } @Test @@ -368,7 +377,7 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { // ---------------------------------------------------------------------------------------------- override def testData: Any = { - val testData = new Row(14) + val testData = new Row(15) testData.setField(0, 42) testData.setField(1, "Test") testData.setField(2, null) @@ -383,6 +392,11 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { testData.setField(11, 3.toByte) testData.setField(12, 3.toShort) testData.setField(13, 3.toFloat) + testData.setField(14, Row.of( + 12.asInstanceOf[Integer], + true.asInstanceOf[JBoolean], + Row.of(1.asInstanceOf[Integer], 2.asInstanceOf[Integer], 3.asInstanceOf[Integer])) + ) testData } @@ -401,7 +415,8 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO, Types.BYTE, Types.SHORT, - Types.FLOAT + Types.FLOAT, + Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)) ).asInstanceOf[TypeInformation[Any]] } @@ -427,6 +442,7 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { "Func15" -> Func15, "Func16" -> Func16, "Func17" -> Func17, + "Func19" -> Func19, "JavaFunc0" -> new JavaFunc0, "JavaFunc1" -> new JavaFunc1, "JavaFunc2" -> new JavaFunc2, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala index 9535cdf18d65d..3f6ebbd5f3749 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala @@ -22,7 +22,8 @@ import java.sql.{Date, Time, Timestamp} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.Types -import org.apache.flink.table.functions.{ScalarFunction, FunctionContext} +import org.apache.flink.table.functions.{FunctionContext, ScalarFunction} +import org.apache.flink.types.Row import org.junit.Assert import scala.annotation.varargs @@ -274,3 +275,16 @@ object Func18 extends ScalarFunction { str.startsWith(prefix) } } + +object Func19 extends ScalarFunction { + def eval(row: Row): Row = { + row + } + + override def getParameterTypes(signature: Array[Class[_]]): Array[TypeInformation[_]] = + Array(Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT))) + + override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = + Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)) + +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala index 79243dd0dddca..828a9e2654a62 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala @@ -22,13 +22,12 @@ import java.sql.{Date, Timestamp} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.table.api.{TableEnvironment, TableException, Types, ValidationException} -import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.JavaTableFunc0 import org.apache.flink.table.api.scala._ -import org.apache.flink.table.expressions.utils.{Func1, Func13, Func18, RichFunc2} -import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase +import org.apache.flink.table.api.{TableEnvironment, Types, ValidationException} +import org.apache.flink.table.expressions.utils.{Func1, Func18, RichFunc2} +import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.JavaTableFunc0 import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.table.runtime.utils._ +import org.apache.flink.table.runtime.utils.{TableProgramsClusterTestBase, _} import org.apache.flink.table.utils._ import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala index 79f3f58f5ce77..215526d820694 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala @@ -17,14 +17,15 @@ */ package org.apache.flink.table.runtime.stream.table +import java.lang.{Boolean => JBoolean} + import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase -import org.apache.flink.table.api.{TableEnvironment, ValidationException} import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.{TableEnvironment, Types, ValidationException} import org.apache.flink.table.expressions.utils.{Func18, RichFunc2} -import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData} -import org.apache.flink.table.runtime.utils._ +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData, _} import org.apache.flink.table.utils._ import org.apache.flink.types.Row import org.junit.Assert._ @@ -231,6 +232,31 @@ class CorrelateITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + @Test + def testRowType(): Unit = { + val row = Row.of( + 12.asInstanceOf[Integer], + true.asInstanceOf[JBoolean], + Row.of(1.asInstanceOf[Integer], 2.asInstanceOf[Integer], 3.asInstanceOf[Integer]) + ) + + val rowType = Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)) + val in = env.fromElements(row, row)(rowType).toTable(tEnv).as('a, 'b, 'c) + + val tableFunc5 = new TableFunc5() + val result = in + .join(tableFunc5('c) as ('f0, 'f1, 'f2)) + .select('c, 'f2) + + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = mutable.MutableList( + "1,2,3,3", + "1,2,3,3") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + private def testData( env: StreamExecutionEnvironment) : DataStream[(Int, Long, String)] = { @@ -242,5 +268,4 @@ class CorrelateITCase extends StreamingMultipleProgramsTestBase { data.+=((4, 3L, "nosharp")) env.fromCollection(data) } - } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala index e1af23b71ecd1..9060db5300e0a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala @@ -119,6 +119,26 @@ class TableFunc4 extends TableFunction[Row] { } } +class TableFunc5 extends TableFunction[Row] { + def eval(row: Row): Unit = { + collect(row) + } + + override def getParameterTypes(signature: Array[Class[_]]): Array[TypeInformation[_]] = + Array(Types.ROW(Types.INT, Types.INT, Types.INT)) + + override def getResultType: TypeInformation[Row] = + Types.ROW(Types.INT, Types.INT, Types.INT) + +} + +class VarArgsFunc0 extends TableFunction[String] { + @varargs + def eval(str: String*): Unit = { + str.foreach(collect) + } +} + class HierarchyTableFunction extends SplittableTableFunction[Boolean, Integer] { def eval(user: String) { if (user.contains("#")) { @@ -215,10 +235,3 @@ class RichTableFunc1 extends TableFunction[String] { separator = None } } - -class VarArgsFunc0 extends TableFunction[String] { - @varargs - def eval(str: String*): Unit = { - str.foreach(collect) - } -} From 084ff68d5434805d9fc4208fd52f04c2201e362c Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 15 Nov 2017 10:12:55 +0100 Subject: [PATCH 056/367] [FLINK-7490] [table] Use correct classloader to compile generated code that calls UDAGGs. This closes #5018. --- .../operators/translation/RichCombineToGroupCombineWrapper.java | 1 + .../flink/table/runtime/aggregate/AggregateAggFunction.scala | 2 +- .../flink/table/runtime/aggregate/DataSetAggFunction.scala | 2 +- .../flink/table/runtime/aggregate/DataSetFinalAggFunction.scala | 2 +- .../flink/table/runtime/aggregate/DataSetPreAggFunction.scala | 2 +- .../aggregate/DataSetSessionWindowAggReduceGroupFunction.scala | 2 +- .../aggregate/DataSetSessionWindowAggregatePreProcessor.scala | 2 +- .../DataSetSlideTimeWindowAggReduceGroupFunction.scala | 2 +- .../aggregate/DataSetSlideWindowAggReduceCombineFunction.scala | 2 +- .../aggregate/DataSetSlideWindowAggReduceGroupFunction.scala | 2 +- .../DataSetTumbleCountWindowAggReduceGroupFunction.scala | 2 +- .../DataSetTumbleTimeWindowAggReduceCombineFunction.scala | 2 +- .../DataSetTumbleTimeWindowAggReduceGroupFunction.scala | 2 +- 13 files changed, 13 insertions(+), 12 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java index 3f6463a07aed5..9cbda50efc5b3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java @@ -42,6 +42,7 @@ public RichCombineToGroupCombineWrapper(F wrappedFunction) { @Override public void open(Configuration config) throws Exception { + wrappedFunction.setRuntimeContext(getRuntimeContext()); wrappedFunction.open(config); } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala index 330386b1e93e0..4dbaeea46bab0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala @@ -70,7 +70,7 @@ class AggregateAggFunction(genAggregations: GeneratedAggregationsFunction) LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + Thread.currentThread().getContextClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala index bc0c16385a04b..ced14504e8e37 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala @@ -46,7 +46,7 @@ class DataSetAggFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala index 3b3be708c3c9e..f2eb3d943dbfc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala @@ -47,7 +47,7 @@ class DataSetFinalAggFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala index fc3366bd31691..744a739a394d2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala @@ -48,7 +48,7 @@ class DataSetPreAggFunction(genAggregations: GeneratedAggregationsFunction) LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala index 372fc0d27c11e..0d54de62335a7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala @@ -72,7 +72,7 @@ class DataSetSessionWindowAggReduceGroupFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala index 666bfee19348e..35e814264a7f0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala @@ -59,7 +59,7 @@ class DataSetSessionWindowAggregatePreProcessor( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala index 3af7969a37c06..f2987a74b5e39 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala @@ -68,7 +68,7 @@ class DataSetSlideTimeWindowAggReduceGroupFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceCombineFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceCombineFunction.scala index 2da838fe8abf7..6a9d63122746e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceCombineFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceCombineFunction.scala @@ -67,7 +67,7 @@ class DataSetSlideWindowAggReduceCombineFunction( LOG.debug(s"Compiling AggregateHelper: $genPreAggregations.name \n\n " + s"Code:\n$genPreAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genPreAggregations.name, genPreAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala index 474a09b5d45a4..f96e8416bf147 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala @@ -63,7 +63,7 @@ class DataSetSlideWindowAggReduceGroupFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala index 22fe389a5b250..f4d347a9ad7d0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala @@ -50,7 +50,7 @@ class DataSetTumbleCountWindowAggReduceGroupFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceCombineFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceCombineFunction.scala index 9eeab33c45b1d..a3a72ae47677b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceCombineFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceCombineFunction.scala @@ -66,7 +66,7 @@ class DataSetTumbleTimeWindowAggReduceCombineFunction( LOG.debug(s"Compiling AggregateHelper: $genPreAggregations.name \n\n " + s"Code:\n$genPreAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genPreAggregations.name, genPreAggregations.code) LOG.debug("Instantiating AggregateHelper.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala index 4e9214850a485..14e89adca33da 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala @@ -62,7 +62,7 @@ class DataSetTumbleTimeWindowAggReduceGroupFunction( LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + s"Code:\n$genAggregations.code") val clazz = compile( - getClass.getClassLoader, + getRuntimeContext.getUserCodeClassLoader, genAggregations.name, genAggregations.code) LOG.debug("Instantiating AggregateHelper.") From 13962e1ffda62218031bf426ee9c06146c7c5573 Mon Sep 17 00:00:00 2001 From: twalthr Date: Wed, 15 Nov 2017 12:07:16 +0100 Subject: [PATCH 057/367] [FLINK-7942] [table] Reduce aliasing in RexNodes This closes #5019. --- .../org/apache/flink/table/api/table.scala | 12 ++++- .../flink/table/plan/logical/operators.scala | 26 ++++++++-- .../table/api/batch/table/JoinTest.scala | 47 +++++++++++++++++++ .../table/plan/RetractionRulesTest.scala | 40 ++++++---------- 4 files changed, 93 insertions(+), 32 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala index 7349a0e1ae5ad..071cc69b7a240 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala @@ -1106,7 +1106,13 @@ class OverWindowedTable( new Table( table.tableEnv, - Project(expandedOverFields.map(UnresolvedAlias), table.logicalPlan).validate(table.tableEnv)) + Project( + expandedOverFields.map(UnresolvedAlias), + table.logicalPlan, + // required for proper projection push down + explicitAlias = true) + .validate(table.tableEnv) + ) } def select(fields: String): Table = { @@ -1150,7 +1156,9 @@ class WindowGroupedTable( propNames.map(a => Alias(a._1, a._2)).toSeq, aggNames.map(a => Alias(a._1, a._2)).toSeq, Project(projectFields, table.logicalPlan).validate(table.tableEnv) - ).validate(table.tableEnv) + ).validate(table.tableEnv), + // required for proper resolution of the time attribute in multi-windows + explicitAlias = true ).validate(table.tableEnv)) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala index fe2bfe53f4452..a2bd1e45124fe 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala @@ -42,7 +42,12 @@ import org.apache.flink.table.validate.{ValidationFailure, ValidationSuccess} import scala.collection.JavaConverters._ import scala.collection.mutable -case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extends UnaryNode { +case class Project( + projectList: Seq[NamedExpression], + child: LogicalNode, + explicitAlias: Boolean = false) + extends UnaryNode { + override def output: Seq[Attribute] = projectList.map(_.toAttribute) override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = { @@ -61,7 +66,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extend throw new RuntimeException("This should never be called and probably points to a bug.") } } - Project(newProjectList, child) + Project(newProjectList, child, explicitAlias) } override def validate(tableEnv: TableEnvironment): LogicalNode = { @@ -90,8 +95,19 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extend override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { child.construct(relBuilder) + + val exprs = if (explicitAlias) { + projectList + } else { + // remove AS expressions, according to Calcite they should not be in a final RexNode + projectList.map { + case Alias(e: Expression, _, _) => e + case e: Expression => e + } + } + relBuilder.project( - projectList.map(_.toRexNode(relBuilder)).asJava, + exprs.map(_.toRexNode(relBuilder)).asJava, projectList.map(_.name).asJava, true) } @@ -116,7 +132,9 @@ case class AliasNode(aliasList: Seq[Expression], child: LogicalNode) extends Una val input = child.output Project( names.zip(input).map { case (name, attr) => - Alias(attr, name)} ++ input.drop(names.length), child) + Alias(attr, name)} ++ input.drop(names.length), + child, + explicitAlias = true) } } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/JoinTest.scala index 9ee7fc20138ae..ce6225280839c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/JoinTest.scala @@ -19,7 +19,9 @@ package org.apache.flink.table.api.batch.table import org.apache.flink.api.scala._ +import org.apache.flink.table.api.batch.table.JoinTest.Merger import org.apache.flink.table.api.scala._ +import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ import org.junit.Test @@ -301,4 +303,49 @@ class JoinTest extends TableTestBase { util.verifyTable(joined, expected) } + + @Test + def testFilterJoinRule(): Unit = { + val util = batchTestUtil() + val t1 = util.addTable[(String, Int, Int)]('a, 'b, 'c) + val t2 = util.addTable[(String, Int, Int)]('d, 'e, 'f) + val results = t1 + .leftOuterJoin(t2, 'b === 'e) + .select('c, Merger('c, 'f) as 'c0) + .select(Merger('c, 'c0) as 'c1) + .where('c1 >= 0) + + val expected = unaryNode( + "DataSetCalc", + binaryNode( + "DataSetJoin", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "b", "c") + ), + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "e", "f") + ), + term("where", "=(b, e)"), + term("join", "b", "c", "e", "f"), + term("joinType", "LeftOuterJoin") + ), + term("select", "Merger$(c, Merger$(c, f)) AS c1"), + term("where", ">=(Merger$(c, Merger$(c, f)), 0)") + ) + + util.verifyTable(results, expected) + } +} + +object JoinTest { + + object Merger extends ScalarFunction { + def eval(f0: Int, f1: Int): Int = { + f0 + f1 + } + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RetractionRulesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RetractionRulesTest.scala index 999a808a77e7c..ba3c3142ecece 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RetractionRulesTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RetractionRulesTest.scala @@ -86,22 +86,18 @@ class RetractionRulesTest extends TableTestBase { val expected = unaryNode( - "DataStreamCalc", + "DataStreamGroupAggregate", unaryNode( - "DataStreamGroupAggregate", + "DataStreamCalc", unaryNode( - "DataStreamCalc", - unaryNode( - "DataStreamGroupAggregate", - "DataStreamScan(true, Acc)", - "true, AccRetract" - ), + "DataStreamGroupAggregate", + "DataStreamScan(true, Acc)", "true, AccRetract" ), - s"$defaultStatus" + "true, AccRetract" ), s"$defaultStatus" - ) + ) util.verifyTableTrait(resultTable, expected) } @@ -253,28 +249,20 @@ class RetractionRulesTest extends TableTestBase { val expected = unaryNode( - "DataStreamCalc", + "DataStreamGroupAggregate", unaryNode( - "DataStreamGroupAggregate", - unaryNode( - "DataStreamCalc", - binaryNode( - "DataStreamUnion", - unaryNode( - "DataStreamCalc", - unaryNode( - "DataStreamGroupAggregate", - "DataStreamScan(true, Acc)", - "true, AccRetract" - ), - "true, AccRetract" - ), + "DataStreamCalc", + binaryNode( + "DataStreamUnion", + unaryNode( + "DataStreamGroupAggregate", "DataStreamScan(true, Acc)", "true, AccRetract" ), + "DataStreamScan(true, Acc)", "true, AccRetract" ), - s"$defaultStatus" + "true, AccRetract" ), s"$defaultStatus" ) From 397f0d150255ea24fd5fd4cc38848441bb0fff58 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 16 Nov 2017 11:00:30 +0100 Subject: [PATCH 058/367] [FLINK-7698] [table] Tests joins with null literals --- .../flink/table/api/stream/sql/JoinTest.scala | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index 8c1865c69643b..c14b6985ef814 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -19,7 +19,9 @@ package org.apache.flink.table.api.stream.sql import org.apache.calcite.rel.logical.LogicalJoin import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.Null import org.apache.flink.table.plan.logical.TumblingGroupWindow import org.apache.flink.table.runtime.join.WindowJoinUtil import org.apache.flink.table.utils.TableTestUtil.{term, _} @@ -244,6 +246,49 @@ class JoinTest extends TableTestBase { streamUtil.verifySql(sqlQuery, expected) } + @Test + def testJoinWithNullLiteral(): Unit = { + val streamUtil: StreamTableTestUtil = streamTestUtil() + + val t1 = streamUtil.addTable[(Int, Long, String)]("Table1", 'a, 'b, 'c, 'proctime.proctime) + .select('a, 'b, 'c, 'proctime, Null(Types.LONG) as 'nullField) + + val t2 = streamUtil.addTable[(Int, Long, String)]("Table2", 'a, 'b, 'c, 'proctime.proctime) + .select('a, 'b, 'c, 'proctime, 12L as 'nullField) + + streamUtil.tableEnv.registerTable("T1", t1) + streamUtil.tableEnv.registerTable("T2", t2) + + val sqlQuery = + """ + |SELECT t2.a, t2.c, t1.c + |FROM T1 AS t1 + |JOIN T2 AS t2 ON t1.a = t2.a AND t1.nullField = t2.nullField AND + | t1.proctime BETWEEN t2.proctime - INTERVAL '5' SECOND AND + | t2.proctime + INTERVAL '5' SECOND + |""".stripMargin + + val expected = + unaryNode("DataStreamCalc", + binaryNode("DataStreamWindowJoin", + unaryNode("DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "proctime", "null AS nullField") + ), + unaryNode("DataStreamCalc", + streamTableNode(1), + term("select", "a", "c", "proctime", "12 AS nullField") + ), + term("where", "AND(=(a, a0), =(nullField, nullField0), >=(proctime, " + + "-(proctime0, 5000)), <=(proctime, DATETIME_PLUS(proctime0, 5000)))"), + term("join", "a", "c", "proctime", "nullField", "a0", "c0", "proctime0", "nullField0"), + term("joinType", "InnerJoin") + ), + term("select", "a0 AS a", "c0 AS c", "c AS c0") + ) + streamUtil.verifySql(sqlQuery, expected) + } + @Test def testRowTimeInnerJoinAndWindowAggregationOnFirst(): Unit = { From b59fae3f51b22c299c3d4c51ff0da74fd245072b Mon Sep 17 00:00:00 2001 From: Shuyi Chen Date: Thu, 9 Nov 2017 00:05:20 -0800 Subject: [PATCH 059/367] [FLINK-7003] [table] Fix 'SELECT *' for tables with nested schema. This closes #4989. --- .../plan/schema/CompositeRelDataType.scala | 4 +++- .../table/runtime/batch/sql/CalcITCase.scala | 19 +++++++++++++++++ .../runtime/batch/table/CalcITCase.scala | 10 +++------ .../table/runtime/stream/sql/SqlITCase.scala | 21 +++++++++++++++++++ .../runtime/stream/table/CalcITCase.scala | 16 ++++++++++++++ .../table/runtime/utils/StreamTestData.scala | 9 ++++++++ 6 files changed, 71 insertions(+), 8 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala index e0c6b6ffcc460..f8c61fbfc88ab 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala @@ -38,7 +38,9 @@ class CompositeRelDataType( val compositeType: CompositeType[_], val nullable: Boolean, typeFactory: FlinkTypeFactory) - extends RelRecordType(StructKind.PEEK_FIELDS, createFieldList(compositeType, typeFactory)) { + extends RelRecordType( + StructKind.PEEK_FIELDS_NO_EXPAND, + createFieldList(compositeType, typeFactory)) { override def toString = s"COMPOSITE($compositeType)" diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala index b891a7d3cf1b4..7ca3e9c0f6308 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala @@ -66,6 +66,25 @@ class CalcITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test + def testSelectStarFromNestedTable(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT * FROM MyTable" + + val ds = CollectionDataSets.getSmallNestedTupleDataSet(env).toTable(tEnv).as('a, 'b) + tEnv.registerTable("MyTable", ds) + + val result = tEnv.sqlQuery(sqlQuery) + + val expected = "(1,1),one\n" + "(2,2),two\n" + "(3,3),three\n" + + val results = result.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + @Test def testSelectStarFromDataSet(): Unit = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala index e947c3f3bed55..22373d2ac001d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala @@ -118,14 +118,10 @@ class CalcITCase( val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*) + val t = CollectionDataSets.getSmallNestedTupleDataSet(env).toTable(tEnv, 'a, 'b).select('*) - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val expected = + "(1,1),one\n" + "(2,2),two\n" + "(3,3),three\n" val results = t.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index c49af5cc626e3..3acdd58ee5f39 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -151,6 +151,27 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) } + /** test select star **/ + @Test + def testSelectStar(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val sqlQuery = "SELECT * FROM MyTable" + + val t = StreamTestData.getSmallNestedTupleDataStream(env).toTable(tEnv).as('a, 'b) + tEnv.registerTable("MyTable", t) + + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = List("(1,1),one", "(2,2),two", "(3,3),three") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + /** test selection **/ @Test def testSelectExpressionFromTable(): Unit = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala index 480d817d4a3cf..ca6da809d1fa4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala @@ -54,6 +54,22 @@ class CalcITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + @Test + def testSelectStar(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.testResults = mutable.MutableList() + val ds = StreamTestData.getSmallNestedTupleDataStream(env).toTable(tEnv).select('*) + + val results = ds.toAppendStream[Row] + results.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = mutable.MutableList("(1,1),one", "(2,2),two", "(3,3),three") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + @Test def testSelectFirst(): Unit = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala index 94ced19385aa3..58d3c635a66bd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala @@ -80,4 +80,13 @@ object StreamTestData { data.+=((5, 15L, 14, "KLM", 2L)) env.fromCollection(data) } + + def getSmallNestedTupleDataStream(env: StreamExecutionEnvironment): + DataStream[((Int, Int), String)] = { + val data = new mutable.MutableList[((Int, Int), String)] + data.+=(((1, 1), "one")) + data.+=(((2, 2), "two")) + data.+=(((3, 3), "three")) + env.fromCollection(data) + } } From 8883fa2dd910909d4aaf5c57cf4b15a41c8cf7e1 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 16 Nov 2017 10:20:16 +0100 Subject: [PATCH 060/367] [FLINK-7389] [table] Remove Calcite PushProjector This closes #5022. --- .../calcite/rel/rules/PushProjector.java | 868 ------------------ .../table/runtime/batch/sql/CalcITCase.scala | 2 +- .../table/runtime/batch/sql/JoinITCase.scala | 4 +- 3 files changed, 4 insertions(+), 870 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java deleted file mode 100644 index 0955aebed4017..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java +++ /dev/null @@ -1,868 +0,0 @@ -/* - * 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.calcite.rel.rules; - -import org.apache.calcite.linq4j.Ord; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.plan.Strong; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.Join; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.core.SemiJoin; -import org.apache.calcite.rel.core.SetOp; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.rex.RexVisitorImpl; -import org.apache.calcite.runtime.PredicateImpl; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.tools.RelBuilder; -import org.apache.calcite.util.BitSets; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.Pair; - -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; - -import java.util.ArrayList; -import java.util.BitSet; -import java.util.List; -import java.util.Set; - -// This class is copied from Apache Calcite except that it does not -// automatically name the field using the name of the operators -// as the Table API rejects special characters like '-' in the field names. - -/** - * PushProjector is a utility class used to perform operations used in push - * projection rules. - * - *

Pushing is particularly interesting in the case of join, because there - * are multiple inputs. Generally an expression can be pushed down to a - * particular input if it depends upon no other inputs. If it can be pushed - * down to both sides, it is pushed down to the left. - * - *

Sometimes an expression needs to be split before it can be pushed down. - * To flag that an expression cannot be split, specify a rule that it must be - * preserved. Such an expression will be pushed down intact to one - * of the inputs, or not pushed down at all.

- */ -public class PushProjector { - //~ Instance fields -------------------------------------------------------- - - private final Project origProj; - private final RexNode origFilter; - private final RelNode childRel; - private final ExprCondition preserveExprCondition; - private final RelBuilder relBuilder; - - /** - * Original projection expressions - */ - final List origProjExprs; - - /** - * Fields from the RelNode that the projection is being pushed past - */ - final List childFields; - - /** - * Number of fields in the RelNode that the projection is being pushed past - */ - final int nChildFields; - - /** - * Bitmap containing the references in the original projection - */ - final BitSet projRefs; - - /** - * Bitmap containing the fields in the RelNode that the projection is being - * pushed past, if the RelNode is not a join. If the RelNode is a join, then - * the fields correspond to the left hand side of the join. - */ - final ImmutableBitSet childBitmap; - - /** - * Bitmap containing the fields in the right hand side of a join, in the - * case where the projection is being pushed past a join. Not used - * otherwise. - */ - final ImmutableBitSet rightBitmap; - - /** - * Bitmap containing the fields that should be strong, i.e. when preserving expressions - * we can only preserve them if the expressions if it is null when these fields are null. - */ - final ImmutableBitSet strongBitmap; - - /** - * Number of fields in the RelNode that the projection is being pushed past, - * if the RelNode is not a join. If the RelNode is a join, then this is the - * number of fields in the left hand side of the join. - * - *

The identity - * {@code nChildFields == nSysFields + nFields + nFieldsRight} - * holds. {@code nFields} does not include {@code nSysFields}. - * The output of a join looks like this: - * - *

-   * | nSysFields | nFields | nFieldsRight |
-   * 
- * - *

The output of a single-input rel looks like this: - * - *

-   * | nSysFields | nFields |
-   * 
- */ - final int nFields; - - /** - * Number of fields in the right hand side of a join, in the case where the - * projection is being pushed past a join. Always 0 otherwise. - */ - final int nFieldsRight; - - /** - * Number of system fields. System fields appear at the start of a join, - * before the first field from the left input. - */ - private final int nSysFields; - - /** - * Expressions referenced in the projection/filter that should be preserved. - * In the case where the projection is being pushed past a join, then the - * list only contains the expressions corresponding to the left hand side of - * the join. - */ - final List childPreserveExprs; - - /** - * Expressions referenced in the projection/filter that should be preserved, - * corresponding to expressions on the right hand side of the join, if the - * projection is being pushed past a join. Empty list otherwise. - */ - final List rightPreserveExprs; - - /** - * Number of system fields being projected. - */ - int nSystemProject; - - /** - * Number of fields being projected. In the case where the projection is - * being pushed past a join, the number of fields being projected from the - * left hand side of the join. - */ - int nProject; - - /** - * Number of fields being projected from the right hand side of a join, in - * the case where the projection is being pushed past a join. 0 otherwise. - */ - int nRightProject; - - /** - * Rex builder used to create new expressions. - */ - final RexBuilder rexBuilder; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a PushProjector object for pushing projects past a RelNode. - * - * @param origProj the original projection that is being pushed; - * may be null if the projection is implied as a - * result of a projection having been trivially - * removed - * @param origFilter the filter that the projection must also be - * pushed past, if applicable - * @param childRel the RelNode that the projection is being - * pushed past - * @param preserveExprCondition condition for whether an expression should - * be preserved in the projection - */ - public PushProjector( - Project origProj, - RexNode origFilter, - RelNode childRel, - ExprCondition preserveExprCondition, - RelBuilder relBuilder) { - this.origProj = origProj; - this.origFilter = origFilter; - this.childRel = childRel; - this.preserveExprCondition = preserveExprCondition; - this.relBuilder = Preconditions.checkNotNull(relBuilder); - if (origProj == null) { - origProjExprs = ImmutableList.of(); - } else { - origProjExprs = origProj.getProjects(); - } - - childFields = childRel.getRowType().getFieldList(); - nChildFields = childFields.size(); - - projRefs = new BitSet(nChildFields); - if (childRel instanceof Join) { - Join joinRel = (Join) childRel; - List leftFields = - joinRel.getLeft().getRowType().getFieldList(); - List rightFields = - joinRel.getRight().getRowType().getFieldList(); - nFields = leftFields.size(); - nFieldsRight = childRel instanceof SemiJoin ? 0 : rightFields.size(); - nSysFields = joinRel.getSystemFieldList().size(); - childBitmap = - ImmutableBitSet.range(nSysFields, nFields + nSysFields); - rightBitmap = - ImmutableBitSet.range(nFields + nSysFields, nChildFields); - - switch (joinRel.getJoinType()) { - case INNER: - strongBitmap = ImmutableBitSet.of(); - break; - case RIGHT: // All the left-input's columns must be strong - strongBitmap = ImmutableBitSet.range(nSysFields, nFields + nSysFields); - break; - case LEFT: // All the right-input's columns must be strong - strongBitmap = ImmutableBitSet.range(nFields + nSysFields, nChildFields); - break; - case FULL: - default: - strongBitmap = ImmutableBitSet.range(nSysFields, nChildFields); - } - - } else { - nFields = nChildFields; - nFieldsRight = 0; - childBitmap = ImmutableBitSet.range(nChildFields); - rightBitmap = null; - nSysFields = 0; - strongBitmap = ImmutableBitSet.of(); - } - assert nChildFields == nSysFields + nFields + nFieldsRight; - - childPreserveExprs = new ArrayList(); - rightPreserveExprs = new ArrayList(); - - rexBuilder = childRel.getCluster().getRexBuilder(); - } - - //~ Methods ---------------------------------------------------------------- - - /** - * Decomposes a projection to the input references referenced by a - * projection and a filter, either of which is optional. If both are - * provided, the filter is underneath the project. - * - *

Creates a projection containing all input references as well as - * preserving any special expressions. Converts the original projection - * and/or filter to reference the new projection. Then, finally puts on top, - * a final projection corresponding to the original projection. - * - * @param defaultExpr expression to be used in the projection if no fields - * or special columns are selected - * @return the converted projection if it makes sense to push elements of - * the projection; otherwise returns null - */ - public RelNode convertProject(RexNode defaultExpr) { - // locate all fields referenced in the projection and filter - locateAllRefs(); - - // if all columns are being selected (either explicitly in the - // projection) or via a "select *", then there needs to be some - // special expressions to preserve in the projection; otherwise, - // there's no point in proceeding any further - if (origProj == null) { - if (childPreserveExprs.size() == 0) { - return null; - } - - // even though there is no projection, this is the same as - // selecting all fields - if (nChildFields > 0) { - // Calling with nChildFields == 0 should be safe but hits - // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6222207 - projRefs.set(0, nChildFields); - } - nProject = nChildFields; - } else if ( - (projRefs.cardinality() == nChildFields) - && (childPreserveExprs.size() == 0)) { - return null; - } - - // if nothing is being selected from the underlying rel, just - // project the default expression passed in as a parameter or the - // first column if there is no default expression - if ((projRefs.cardinality() == 0) && (childPreserveExprs.size() == 0)) { - if (defaultExpr != null) { - childPreserveExprs.add(defaultExpr); - } else if (nChildFields == 1) { - return null; - } else { - projRefs.set(0); - nProject = 1; - } - } - - // create a new projection referencing all fields referenced in - // either the project or the filter - RelNode newProject = createProjectRefsAndExprs(childRel, false, false); - - int[] adjustments = getAdjustments(); - - // if a filter was passed in, convert it to reference the projected - // columns, placing it on top of the project just created - RelNode projChild; - if (origFilter != null) { - RexNode newFilter = - convertRefsAndExprs( - origFilter, - newProject.getRowType().getFieldList(), - adjustments); - relBuilder.push(newProject); - relBuilder.filter(newFilter); - projChild = relBuilder.build(); - } else { - projChild = newProject; - } - - // put the original project on top of the filter/project, converting - // it to reference the modified projection list; otherwise, create - // a projection that essentially selects all fields - return createNewProject(projChild, adjustments); - } - - /** - * Locates all references found in either the projection expressions a - * filter, as well as references to expressions that should be preserved. - * Based on that, determines whether pushing the projection makes sense. - * - * @return true if all inputs from the child that the projection is being - * pushed past are referenced in the projection/filter and no special - * preserve expressions are referenced; in that case, it does not make sense - * to push the projection - */ - public boolean locateAllRefs() { - RexUtil.apply( - new InputSpecialOpFinder( - projRefs, - childBitmap, - rightBitmap, - strongBitmap, - preserveExprCondition, - childPreserveExprs, - rightPreserveExprs), - origProjExprs, - origFilter); - - // The system fields of each child are always used by the join, even if - // they are not projected out of it. - projRefs.set( - nSysFields, - nSysFields + nSysFields, - true); - projRefs.set( - nSysFields + nFields, - nSysFields + nFields + nSysFields, - true); - - // Count how many fields are projected. - nSystemProject = 0; - nProject = 0; - nRightProject = 0; - for (int bit : BitSets.toIter(projRefs)) { - if (bit < nSysFields) { - nSystemProject++; - } else if (bit < nSysFields + nFields) { - nProject++; - } else { - nRightProject++; - } - } - - assert nSystemProject + nProject + nRightProject - == projRefs.cardinality(); - - if ((childRel instanceof Join) - || (childRel instanceof SetOp)) { - // if nothing is projected from the children, arbitrarily project - // the first columns; this is necessary since Fennel doesn't - // handle 0-column projections - if ((nProject == 0) && (childPreserveExprs.size() == 0)) { - projRefs.set(0); - nProject = 1; - } - if (childRel instanceof Join) { - if ((nRightProject == 0) && (rightPreserveExprs.size() == 0)) { - projRefs.set(nFields); - nRightProject = 1; - } - } - } - - // no need to push projections if all children fields are being - // referenced and there are no special preserve expressions; note - // that we need to do this check after we've handled the 0-column - // project cases - if (projRefs.cardinality() == nChildFields - && childPreserveExprs.size() == 0 - && rightPreserveExprs.size() == 0) { - return true; - } - - return false; - } - - /** - * Creates a projection based on the inputs specified in a bitmap and the - * expressions that need to be preserved. The expressions are appended after - * the input references. - * - * @param projChild child that the projection will be created on top of - * @param adjust if true, need to create new projection expressions; - * otherwise, the existing ones are reused - * @param rightSide if true, creating a projection for the right hand side - * of a join - * @return created projection - */ - public Project createProjectRefsAndExprs( - RelNode projChild, - boolean adjust, - boolean rightSide) { - List preserveExprs; - int nInputRefs; - int offset; - - if (rightSide) { - preserveExprs = rightPreserveExprs; - nInputRefs = nRightProject; - offset = nSysFields + nFields; - } else { - preserveExprs = childPreserveExprs; - nInputRefs = nProject; - offset = nSysFields; - } - int refIdx = offset - 1; - List> newProjects = - new ArrayList>(); - List destFields = - projChild.getRowType().getFieldList(); - - // add on the input references - for (int i = 0; i < nInputRefs; i++) { - refIdx = projRefs.nextSetBit(refIdx + 1); - assert refIdx >= 0; - final RelDataTypeField destField = destFields.get(refIdx - offset); - newProjects.add( - Pair.of( - (RexNode) rexBuilder.makeInputRef( - destField.getType(), refIdx - offset), - destField.getName())); - } - - // add on the expressions that need to be preserved, converting the - // arguments to reference the projected columns (if necessary) - int[] adjustments = {}; - if ((preserveExprs.size() > 0) && adjust) { - adjustments = new int[childFields.size()]; - for (int idx = offset; idx < childFields.size(); idx++) { - adjustments[idx] = -offset; - } - } - for (RexNode projExpr : preserveExprs) { - RexNode newExpr; - if (adjust) { - newExpr = - projExpr.accept( - new RelOptUtil.RexInputConverter( - rexBuilder, - childFields, - destFields, - adjustments)); - } else { - newExpr = projExpr; - } - newProjects.add( - Pair.of( - newExpr, - null)); - } - - return (Project) RelOptUtil.createProject( - projChild, - Pair.left(newProjects), - Pair.right(newProjects), - false, - relBuilder); - } - - /** - * Determines how much each input reference needs to be adjusted as a result - * of projection - * - * @return array indicating how much each input needs to be adjusted by - */ - public int[] getAdjustments() { - int[] adjustments = new int[nChildFields]; - int newIdx = 0; - int rightOffset = childPreserveExprs.size(); - for (int pos : BitSets.toIter(projRefs)) { - adjustments[pos] = -(pos - newIdx); - if (pos >= nSysFields + nFields) { - adjustments[pos] += rightOffset; - } - newIdx++; - } - return adjustments; - } - - /** - * Clones an expression tree and walks through it, adjusting each - * RexInputRef index by some amount, and converting expressions that need to - * be preserved to field references. - * - * @param rex the expression - * @param destFields fields that the new expressions will be referencing - * @param adjustments the amount each input reference index needs to be - * adjusted by - * @return modified expression tree - */ - public RexNode convertRefsAndExprs( - RexNode rex, - List destFields, - int[] adjustments) { - return rex.accept( - new RefAndExprConverter( - rexBuilder, - childFields, - destFields, - adjustments, - childPreserveExprs, - nProject, - rightPreserveExprs, - nProject + childPreserveExprs.size() + nRightProject)); - } - - /** - * Creates a new projection based on the original projection, adjusting all - * input refs using an adjustment array passed in. If there was no original - * projection, create a new one that selects every field from the underlying - * rel. - * - *

If the resulting projection would be trivial, return the child. - * - * @param projChild child of the new project - * @param adjustments array indicating how much each input reference should - * be adjusted by - * @return the created projection - */ - public RelNode createNewProject(RelNode projChild, int[] adjustments) { - final List> projects = Lists.newArrayList(); - - if (origProj != null) { - for (Pair p : origProj.getNamedProjects()) { - projects.add( - Pair.of( - convertRefsAndExprs( - p.left, - projChild.getRowType().getFieldList(), - adjustments), - p.right)); - } - } else { - for (Ord field : Ord.zip(childFields)) { - projects.add( - Pair.of( - (RexNode) rexBuilder.makeInputRef( - field.e.getType(), field.i), field.e.getName())); - } - } - return RelOptUtil.createProject( - projChild, - Pair.left(projects), - Pair.right(projects), - true /* optimize to avoid trivial projections, as per javadoc */, - relBuilder); - } - - //~ Inner Classes ---------------------------------------------------------- - - /** - * Visitor which builds a bitmap of the inputs used by an expressions, as - * well as locating expressions corresponding to special operators. - */ - private class InputSpecialOpFinder extends RexVisitorImpl { - private final BitSet rexRefs; - private final ImmutableBitSet leftFields; - private final ImmutableBitSet rightFields; - private final ImmutableBitSet strongFields; - private final ExprCondition preserveExprCondition; - private final List preserveLeft; - private final List preserveRight; - private final Strong strong; - - public InputSpecialOpFinder( - BitSet rexRefs, - ImmutableBitSet leftFields, - ImmutableBitSet rightFields, - final ImmutableBitSet strongFields, - ExprCondition preserveExprCondition, - List preserveLeft, - List preserveRight) { - super(true); - this.rexRefs = rexRefs; - this.leftFields = leftFields; - this.rightFields = rightFields; - this.preserveExprCondition = preserveExprCondition; - this.preserveLeft = preserveLeft; - this.preserveRight = preserveRight; - - this.strongFields = strongFields; - this.strong = Strong.of(strongFields); - } - - public Void visitCall(RexCall call) { - if (preserve(call)) { - return null; - } - super.visitCall(call); - return null; - } - - private boolean isStrong(final ImmutableBitSet exprArgs, final RexNode call) { - // If the expressions do not use any of the inputs that require output to be null, - // no need to check. Otherwise, check that the expression is null. - // For example, in an "left outer join", we don't require that expressions - // pushed down into the left input to be strong. On the other hand, - // expressions pushed into the right input must be. In that case, - // strongFields == right input fields. - return !strongFields.intersects(exprArgs) || strong.isNull(call); - } - - private boolean preserve(RexNode call) { - if (preserveExprCondition.test(call)) { - // if the arguments of the expression only reference the - // left hand side, preserve it on the left; similarly, if - // it only references expressions on the right - final ImmutableBitSet exprArgs = RelOptUtil.InputFinder.bits(call); - if (exprArgs.cardinality() > 0) { - if (leftFields.contains(exprArgs) && isStrong(exprArgs, call)) { - addExpr(preserveLeft, call); - return true; - } else if (rightFields.contains(exprArgs) && isStrong(exprArgs, call)) { - assert preserveRight != null; - addExpr(preserveRight, call); - return true; - } - } - // if the expression arguments reference both the left and - // right, fall through and don't attempt to preserve the - // expression, but instead locate references and special - // ops in the call operands - } - return false; - } - - public Void visitInputRef(RexInputRef inputRef) { - rexRefs.set(inputRef.getIndex()); - return null; - } - - /** - * Adds an expression to a list if the same expression isn't already in - * the list. Expressions are identical if their digests are the same. - * - * @param exprList current list of expressions - * @param newExpr new expression to be added - */ - private void addExpr(List exprList, RexNode newExpr) { - String newExprString = newExpr.toString(); - for (RexNode expr : exprList) { - if (newExprString.compareTo(expr.toString()) == 0) { - return; - } - } - exprList.add(newExpr); - } - } - - /** - * Walks an expression tree, replacing input refs with new values to reflect - * projection and converting special expressions to field references. - */ - private class RefAndExprConverter extends RelOptUtil.RexInputConverter { - private final List preserveLeft; - private final int firstLeftRef; - private final List preserveRight; - private final int firstRightRef; - - public RefAndExprConverter( - RexBuilder rexBuilder, - List srcFields, - List destFields, - int[] adjustments, - List preserveLeft, - int firstLeftRef, - List preserveRight, - int firstRightRef) { - super(rexBuilder, srcFields, destFields, adjustments); - this.preserveLeft = preserveLeft; - this.firstLeftRef = firstLeftRef; - this.preserveRight = preserveRight; - this.firstRightRef = firstRightRef; - } - - public RexNode visitCall(RexCall call) { - // if the expression corresponds to one that needs to be preserved, - // convert it to a field reference; otherwise, convert the entire - // expression - int match = - findExprInLists( - call, - preserveLeft, - firstLeftRef, - preserveRight, - firstRightRef); - if (match >= 0) { - return rexBuilder.makeInputRef( - destFields.get(match).getType(), - match); - } - return super.visitCall(call); - } - - /** - * Looks for a matching RexNode from among two lists of RexNodes and - * returns the offset into the list corresponding to the match, adjusted - * by an amount, depending on whether the match was from the first or - * second list. - * - * @param rex RexNode that is being matched against - * @param rexList1 first list of RexNodes - * @param adjust1 adjustment if match occurred in first list - * @param rexList2 second list of RexNodes - * @param adjust2 adjustment if match occurred in the second list - * @return index in the list corresponding to the matching RexNode; -1 - * if no match - */ - private int findExprInLists( - RexNode rex, - List rexList1, - int adjust1, - List rexList2, - int adjust2) { - int match = findExprInList(rex, rexList1); - if (match >= 0) { - return match + adjust1; - } - - if (rexList2 != null) { - match = findExprInList(rex, rexList2); - if (match >= 0) { - return match + adjust2; - } - } - - return -1; - } - - private int findExprInList(RexNode rex, List rexList) { - int match = 0; - for (RexNode rexElement : rexList) { - if (rexElement.toString().compareTo(rex.toString()) == 0) { - return match; - } - match++; - } - return -1; - } - } - - /** - * A functor that replies true or false for a given expression. - * - * @see org.apache.calcite.rel.rules.PushProjector.OperatorExprCondition - */ - public interface ExprCondition extends Predicate { - /** - * Evaluates a condition for a given expression. - * - * @param expr Expression - * @return result of evaluating the condition - */ - boolean test(RexNode expr); - - /** - * Constant condition that replies {@code false} for all expressions. - */ - ExprCondition FALSE = - new ExprConditionImpl() { - @Override public boolean test(RexNode expr) { - return false; - } - }; - - /** - * Constant condition that replies {@code true} for all expressions. - */ - ExprCondition TRUE = - new ExprConditionImpl() { - @Override public boolean test(RexNode expr) { - return true; - } - }; - } - - /** Implementation of {@link ExprCondition}. */ - abstract static class ExprConditionImpl extends PredicateImpl - implements ExprCondition { - } - - /** - * An expression condition that evaluates to true if the expression is - * a call to one of a set of operators. - */ - class OperatorExprCondition extends ExprConditionImpl { - private final Set operatorSet; - - /** - * Creates an OperatorExprCondition. - * - * @param operatorSet Set of operators - */ - public OperatorExprCondition(Iterable operatorSet) { - this.operatorSet = ImmutableSet.copyOf(operatorSet); - } - - public boolean test(RexNode expr) { - return expr instanceof RexCall - && operatorSet.contains(((RexCall) expr).getOperator()); - } - } -} - -// End PushProjector.java diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala index 7ca3e9c0f6308..71df4e6ace3a6 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala @@ -139,7 +139,7 @@ class CalcITCase( val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) - val sqlQuery = "SELECT _1 as a, _2 as b FROM MyTable" + val sqlQuery = "SELECT `1-_./Ü`, b FROM (SELECT _1 as `1-_./Ü`, _2 as b FROM MyTable)" val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) tEnv.registerTable("MyTable", ds) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala index 6a17cb4dab85e..14c98594087bd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala @@ -123,7 +123,9 @@ class JoinITCase( val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) - val sqlQuery = "SELECT Table5.c, Table3.c FROM Table3, Table5 WHERE a = d AND a < 4" + val sqlQuery = + "SELECT Table5.c, T.`1-_./Ü` FROM (SELECT a, b, c AS `1-_./Ü` FROM Table3) AS T, Table5 " + + "WHERE a = d AND a < 4" val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'c) From 2fd53112289ec9002b2c26920dd56ea77be73cd5 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Tue, 7 Nov 2017 17:59:43 +0100 Subject: [PATCH 061/367] [FLINK-8014] [table] Add Kafka010JsonTableSink. - Refactor KafkaTableSink tests. --- .../kafka/Kafka010JsonTableSink.java | 73 +++++++++++++++++++ .../kafka/Kafka010JsonTableSinkTest.java | 53 ++++++++++++++ .../kafka/Kafka08JsonTableSink.java | 26 ++++++- .../kafka/Kafka08JsonTableSinkTest.java | 27 +++---- .../kafka/Kafka09JsonTableSink.java | 26 ++++++- .../kafka/Kafka09JsonTableSinkTest.java | 27 +++---- .../connectors/kafka/KafkaJsonTableSink.java | 5 +- .../connectors/kafka/KafkaTableSink.java | 10 ++- .../JsonRowSerializationSchema.java | 22 +++++- .../kafka/JsonRowSerializationSchemaTest.java | 46 ++++++++---- .../kafka/KafkaTableSinkTestBase.java | 30 ++++---- 11 files changed, 269 insertions(+), 76 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java new file mode 100644 index 0000000000000..431ace0bbd676 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java @@ -0,0 +1,73 @@ +/* + * 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.flink.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Kafka 0.10 {@link KafkaTableSink} that serializes data in JSON format. + */ +public class Kafka010JsonTableSink extends KafkaJsonTableSink { + + /** + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.10 + * topic with fixed partition assignment. + * + *

Each parallel TableSink instance will write its rows to a single Kafka partition.

+ *
    + *
  • If the number of Kafka partitions is less than the number of sink instances, different + * sink instances will write to the same partition.
  • + *
  • If the number of Kafka partitions is higher than the number of sink instance, some + * Kafka partitions won't receive data.
  • + *
+ * + * @param topic topic in Kafka to which table is written + * @param properties properties to connect to Kafka + */ + public Kafka010JsonTableSink(String topic, Properties properties) { + super(topic, properties, new FlinkFixedPartitioner<>()); + } + + /** + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.10 + * topic with custom partition assignment. + * + * @param topic topic in Kafka to which table is written + * @param properties properties to connect to Kafka + * @param partitioner Kafka partitioner + */ + public Kafka010JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner) { + super(topic, properties, partitioner); + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer010<>(topic, serializationSchema, properties, partitioner); + } + + @Override + protected Kafka010JsonTableSink createCopy() { + return new Kafka010JsonTableSink(topic, properties, partitioner); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java new file mode 100644 index 0000000000000..4d805d5f3ce07 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.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.flink.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Tests for the {@link Kafka010JsonTableSink}. + */ +public class Kafka010JsonTableSinkTest extends KafkaTableSinkTestBase { + + @Override + protected KafkaTableSink createTableSink( + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner) { + + return new Kafka010JsonTableSink(topic, properties, partitioner); + } + + @Override + protected Class> getSerializationSchemaClass() { + return JsonRowSerializationSchema.class; + } + + @Override + protected Class getProducerClass() { + return FlinkKafkaProducer010.class; + } + +} + diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java index a887048a5d12f..39d5cb2c9e5d1 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; @@ -32,7 +33,27 @@ public class Kafka08JsonTableSink extends KafkaJsonTableSink { /** - * Creates {@link KafkaTableSink} for Kafka 0.8. + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.8 + * topic with fixed partition assignment. + * + *

Each parallel TableSink instance will write its rows to a single Kafka partition.

+ *
    + *
  • If the number of Kafka partitions is less than the number of sink instances, different + * sink instances will write to the same partition.
  • + *
  • If the number of Kafka partitions is higher than the number of sink instance, some + * Kafka partitions won't receive data.
  • + *
+ * + * @param topic topic in Kafka to which table is written + * @param properties properties to connect to Kafka + */ + public Kafka08JsonTableSink(String topic, Properties properties) { + super(topic, properties, new FlinkFixedPartitioner<>()); + } + + /** + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.8 + * topic with custom partition assignment. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka @@ -43,7 +64,8 @@ public Kafka08JsonTableSink(String topic, Properties properties, FlinkKafkaParti } /** - * Creates {@link KafkaTableSink} for Kafka 0.8. + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.8 + * topic with custom partition assignment. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java index 890fc3abd7cce..d7bb683b10e2a 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java @@ -34,26 +34,19 @@ public class Kafka08JsonTableSinkTest extends KafkaTableSinkTestBase { protected KafkaTableSink createTableSink( String topic, Properties properties, - FlinkKafkaPartitioner partitioner, - final FlinkKafkaProducerBase kafkaProducer) { - - return new Kafka08JsonTableSink(topic, properties, partitioner) { - @Override - protected FlinkKafkaProducerBase createKafkaProducer( - String topic, - Properties properties, - SerializationSchema serializationSchema, - FlinkKafkaPartitioner partitioner) { - - return kafkaProducer; - } - }; + FlinkKafkaPartitioner partitioner) { + + return new Kafka08JsonTableSink(topic, properties, partitioner); + } + + @Override + protected Class> getSerializationSchemaClass() { + return JsonRowSerializationSchema.class; } @Override - @SuppressWarnings("unchecked") - protected SerializationSchema getSerializationSchema() { - return new JsonRowSerializationSchema(FIELD_NAMES); + protected Class getProducerClass() { + return FlinkKafkaProducer08.class; } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java index f65a02d717d1b..a4d266184ce40 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; @@ -32,7 +33,27 @@ public class Kafka09JsonTableSink extends KafkaJsonTableSink { /** - * Creates {@link KafkaTableSink} for Kafka 0.9 . + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.9 + * topic with fixed partition assignment. + * + *

Each parallel TableSink instance will write its rows to a single Kafka partition.

+ *
    + *
  • If the number of Kafka partitions is less than the number of sink instances, different + * sink instances will write to the same partition.
  • + *
  • If the number of Kafka partitions is higher than the number of sink instance, some + * Kafka partitions won't receive data.
  • + *
+ * + * @param topic topic in Kafka to which table is written + * @param properties properties to connect to Kafka + */ + public Kafka09JsonTableSink(String topic, Properties properties) { + super(topic, properties, new FlinkFixedPartitioner<>()); + } + + /** + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.9 + * topic with custom partition assignment. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka @@ -43,7 +64,8 @@ public Kafka09JsonTableSink(String topic, Properties properties, FlinkKafkaParti } /** - * Creates {@link KafkaTableSink} for Kafka 0.9 . + * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.9 + * topic with custom partition assignment. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java index c52b4ca9cc427..58f2b0555eb90 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java @@ -34,26 +34,19 @@ public class Kafka09JsonTableSinkTest extends KafkaTableSinkTestBase { protected KafkaTableSink createTableSink( String topic, Properties properties, - FlinkKafkaPartitioner partitioner, - final FlinkKafkaProducerBase kafkaProducer) { - - return new Kafka09JsonTableSink(topic, properties, partitioner) { - @Override - protected FlinkKafkaProducerBase createKafkaProducer( - String topic, - Properties properties, - SerializationSchema serializationSchema, - FlinkKafkaPartitioner partitioner) { - - return kafkaProducer; - } - }; + FlinkKafkaPartitioner partitioner) { + + return new Kafka09JsonTableSink(topic, properties, partitioner); + } + + @Override + protected Class> getSerializationSchemaClass() { + return JsonRowSerializationSchema.class; } @Override - @SuppressWarnings("unchecked") - protected SerializationSchema getSerializationSchema() { - return new JsonRowSerializationSchema(FIELD_NAMES); + protected Class getProducerClass() { + return FlinkKafkaProducer09.class; } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java index f354dadb19bee..6665dbd9bed73 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.types.Row; @@ -42,7 +43,7 @@ public KafkaJsonTableSink(String topic, Properties properties, FlinkKafkaPartiti } @Override - protected SerializationSchema createSerializationSchema(String[] fieldNames) { - return new JsonRowSerializationSchema(fieldNames); + protected SerializationSchema createSerializationSchema(RowTypeInfo rowSchema) { + return new JsonRowSerializationSchema(rowSchema); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java index cac71dc708af6..f42827e5f35ce 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java @@ -77,10 +77,10 @@ protected abstract FlinkKafkaProducerBase createKafkaProducer( /** * Create serialization schema for converting table rows into bytes. * - * @param fieldNames Field names in table rows. + * @param rowSchema the schema of the row to serialize. * @return Instance of serialization schema */ - protected abstract SerializationSchema createSerializationSchema(String[] fieldNames); + protected abstract SerializationSchema createSerializationSchema(RowTypeInfo rowSchema); /** * Create a deep copy of this sink. @@ -92,6 +92,8 @@ protected abstract FlinkKafkaProducerBase createKafkaProducer( @Override public void emitDataStream(DataStream dataStream) { FlinkKafkaProducerBase kafkaProducer = createKafkaProducer(topic, properties, serializationSchema, partitioner); + // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. + kafkaProducer.setFlushOnCheckpoint(true); dataStream.addSink(kafkaProducer); } @@ -116,7 +118,9 @@ public KafkaTableSink configure(String[] fieldNames, TypeInformation[] fieldT copy.fieldTypes = Preconditions.checkNotNull(fieldTypes, "fieldTypes"); Preconditions.checkArgument(fieldNames.length == fieldTypes.length, "Number of provided field names and types does not match."); - copy.serializationSchema = createSerializationSchema(fieldNames); + + RowTypeInfo rowSchema = new RowTypeInfo(fieldTypes, fieldNames); + copy.serializationSchema = createSerializationSchema(rowSchema); return copy; } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java index 5ece193a838b4..36d3137112a6f 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java @@ -18,6 +18,9 @@ package org.apache.flink.streaming.util.serialization; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.CompositeType; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; @@ -43,10 +46,23 @@ public class JsonRowSerializationSchema implements SerializationSchema { /** * Creates a JSON serialization schema for the given fields and types. * - * @param fieldNames Names of JSON fields to parse. + * @param rowSchema The schema of the rows to encode. */ - public JsonRowSerializationSchema(String[] fieldNames) { - this.fieldNames = Preconditions.checkNotNull(fieldNames); + public JsonRowSerializationSchema(RowTypeInfo rowSchema) { + + Preconditions.checkNotNull(rowSchema); + String[] fieldNames = rowSchema.getFieldNames(); + TypeInformation[] fieldTypes = rowSchema.getFieldTypes(); + + // check that no field is composite + for (int i = 0; i < fieldTypes.length; i++) { + if (fieldTypes[i] instanceof CompositeType) { + throw new IllegalArgumentException("JsonRowSerializationSchema cannot encode rows with nested schema, " + + "but field '" + fieldNames[i] + "' is nested: " + fieldTypes[i].toString()); + } + } + + this.fieldNames = fieldNames; } @Override diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java index 43bde35bb6a05..70140a6c584e6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.Types; @@ -36,31 +37,34 @@ public class JsonRowSerializationSchemaTest { @Test public void testRowSerialization() throws IOException { - String[] fieldNames = new String[] {"f1", "f2", "f3"}; - TypeInformation[] fieldTypes = new TypeInformation[] { Types.INT(), Types.BOOLEAN(), Types.STRING() }; + RowTypeInfo rowSchema = new RowTypeInfo( + new TypeInformation[]{Types.INT(), Types.BOOLEAN(), Types.STRING()}, + new String[] {"f1", "f2", "f3"} + ); + Row row = new Row(3); row.setField(0, 1); row.setField(1, true); row.setField(2, "str"); - Row resultRow = serializeAndDeserialize(fieldNames, fieldTypes, row); + Row resultRow = serializeAndDeserialize(rowSchema, row); assertEqualRows(row, resultRow); } @Test public void testSerializationOfTwoRows() throws IOException { - String[] fieldNames = new String[] {"f1", "f2", "f3"}; - TypeInformation row = Types.ROW( - fieldNames, - new TypeInformation[] { Types.INT(), Types.BOOLEAN(), Types.STRING() } + RowTypeInfo rowSchema = new RowTypeInfo( + new TypeInformation[]{Types.INT(), Types.BOOLEAN(), Types.STRING()}, + new String[] {"f1", "f2", "f3"} ); + Row row1 = new Row(3); row1.setField(0, 1); row1.setField(1, true); row1.setField(2, "str"); - JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames); - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(row); + JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema); byte[] bytes = serializationSchema.serialize(row1); assertEqualRows(row1, deserializationSchema.deserialize(bytes)); @@ -79,19 +83,33 @@ public void testInputValidation() { new JsonRowSerializationSchema(null); } + @Test(expected = IllegalArgumentException.class) + public void testRejectNestedSchema() { + RowTypeInfo rowSchema = new RowTypeInfo( + new TypeInformation[]{Types.INT(), Types.BOOLEAN(), Types.ROW(Types.INT(), Types.DOUBLE())}, + new String[] {"f1", "f2", "f3"} + ); + + new JsonRowSerializationSchema(rowSchema); + } + @Test(expected = IllegalStateException.class) public void testSerializeRowWithInvalidNumberOfFields() { - String[] fieldNames = new String[] {"f1", "f2", "f3"}; + RowTypeInfo rowSchema = new RowTypeInfo( + new TypeInformation[]{Types.INT(), Types.BOOLEAN(), Types.STRING()}, + new String[] {"f1", "f2", "f3"} + ); + Row row = new Row(1); row.setField(0, 1); - JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames); + JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); serializationSchema.serialize(row); } - private Row serializeAndDeserialize(String[] fieldNames, TypeInformation[] fieldTypes, Row row) throws IOException { - JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames); - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(Types.ROW(fieldNames, fieldTypes)); + private Row serializeAndDeserialize(RowTypeInfo rowSchema, Row row) throws IOException { + JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema); byte[] bytes = serializationSchema.serialize(row); return deserializationSchema.deserialize(bytes); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java index 313815245f428..ac5259e6919a5 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.table.api.Types; import org.apache.flink.types.Row; @@ -46,32 +45,27 @@ public abstract class KafkaTableSinkTestBase { private static final String TOPIC = "testTopic"; - protected static final String[] FIELD_NAMES = new String[] {"field1", "field2"}; + private static final String[] FIELD_NAMES = new String[] {"field1", "field2"}; private static final TypeInformation[] FIELD_TYPES = new TypeInformation[] { Types.INT(), Types.STRING() }; private static final FlinkKafkaPartitioner PARTITIONER = new CustomPartitioner(); private static final Properties PROPERTIES = createSinkProperties(); - @SuppressWarnings("unchecked") - private final FlinkKafkaProducerBase producer = new FlinkKafkaProducerBase( - TOPIC, new KeyedSerializationSchemaWrapper(getSerializationSchema()), PROPERTIES, PARTITIONER) { - - @Override - protected void flush() {} - }; - @Test @SuppressWarnings("unchecked") + @Test public void testKafkaTableSink() throws Exception { DataStream dataStream = mock(DataStream.class); KafkaTableSink kafkaTableSink = spy(createTableSink()); kafkaTableSink.emitDataStream(dataStream); - verify(dataStream).addSink(eq(producer)); + // verify correct producer class + verify(dataStream).addSink(any(getProducerClass())); + // verify correctly configured producer verify(kafkaTableSink).createKafkaProducer( eq(TOPIC), eq(PROPERTIES), - any(getSerializationSchema().getClass()), + any(getSerializationSchemaClass()), eq(PARTITIONER)); } @@ -86,13 +80,17 @@ public void testConfiguration() { assertEquals(new RowTypeInfo(FIELD_TYPES), newKafkaTableSink.getOutputType()); } - protected abstract KafkaTableSink createTableSink(String topic, Properties properties, - FlinkKafkaPartitioner partitioner, FlinkKafkaProducerBase kafkaProducer); + protected abstract KafkaTableSink createTableSink( + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner); + + protected abstract Class> getSerializationSchemaClass(); - protected abstract SerializationSchema getSerializationSchema(); + protected abstract Class getProducerClass(); private KafkaTableSink createTableSink() { - return createTableSink(TOPIC, PROPERTIES, PARTITIONER, producer); + return createTableSink(TOPIC, PROPERTIES, PARTITIONER); } private static Properties createSinkProperties() { From f14fcefbe1b30eb8178a5caa053c84e505c288b2 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 9 Nov 2017 15:07:17 +0100 Subject: [PATCH 062/367] [FLINK-8016] [docs] Add documentation for KafkaJsonTableSinks. This closes #4990. --- docs/dev/table/sourceSinks.md | 45 +++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index dfa7954e2541b..0b4bdbede0d3d 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -496,6 +496,7 @@ The following table lists the `TableSink`s which are provided with Flink. | `CassandraAppendTableSink` | `flink-connector-cassandra` | N | Append | Writes a Table to a Cassandra table. | `Kafka08JsonTableSink` | `flink-connector-kafka-0.8` | N | Append | A Kafka 0.8 sink with JSON encoding. | `Kafka09JsonTableSink` | `flink-connector-kafka-0.9` | N | Append | A Kafka 0.9 sink with JSON encoding. +| `Kafka010JsonTableSink` | `flink-connector-kafka-0.10` | N | Append | A Kafka 0.10 sink with JSON encoding. All sinks that come with the `flink-table` dependency can be directly used by your Table programs. For all other table sinks, you have to add the respective dependency in addition to the `flink-table` dependency. @@ -503,6 +504,50 @@ A custom `TableSink` can be defined by implementing the `BatchTableSink`, `Appen {% top %} +### KafkaJsonTableSink + +A `KafkaJsonTableSink` emits a [streaming append `Table`](./streaming.html#table-to-stream-conversion) to an Apache Kafka topic. The rows of the table are encoded as JSON records. Currently, only tables with flat schema, i.e., non-nested fields, are supported. + +A `KafkaJsonTableSink` produces with at-least-once guarantees into a Kafka topic if the query is executed with [checkpointing enabled]({{ site.baseurl }}/dev/stream/state/checkpointing.html#enabling-and-configuring-checkpointing). + +By default, a `KafkaJsonTableSink` writes to at most as many partitions as its own parallelism (each parallel instance of the sink writes to exactly one partition). In order to distribute the writes to more partitions or control the routing of rows into partitions, a custom `FlinkKafkaPartitioner` can be provided. + +The following example shows how to create a `KafkaJsonTableSink` for Kafka 0.10. Sinks for Kafka 0.8 and 0.9 are instantiated analogously. + +
+
+{% highlight java %} + +Table table = ... + +Properties props = new Properties(); +props.setProperty("bootstrap.servers", "localhost:9092"); + +table.writeToSink( + new Kafka010JsonTableSink( + "myTopic", // Kafka topic to write to + props)); // Properties to configure the producer + +{% endhighlight %} +
+ +
+{% highlight scala %} + +val table: Table = ??? + +val props = new Properties() +props.setProperty("bootstrap.servers", "localhost:9092") + +table.writeToSink( + new Kafka010JsonTableSink( + "myTopic", // Kafka topic to write to + props)) // Properties to configure the producer + +{% endhighlight %} +
+
+ ### CsvTableSink The `CsvTableSink` emits a `Table` to one or more CSV files. From e7f7d0c9333f3f9db488c5f968a1627401485067 Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Wed, 15 Nov 2017 11:01:13 +0800 Subject: [PATCH 063/367] [FLINK-8069] [table] Add preserving WatermarkStrategy. This closes #5016. --- .../datastream/StreamTableSourceScan.scala | 5 +- .../wmstrategies/watermarkStrategies.scala | 6 +++ .../stream/table/TableSourceITCase.scala | 51 +++++++++++++++++++ .../flink/table/utils/testTableSources.scala | 28 +++++++++- 4 files changed, 88 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala index 5d305b44134e0..9179d4b52e2b6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala @@ -32,7 +32,7 @@ import org.apache.flink.table.plan.nodes.PhysicalTableSourceScan import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.sources._ -import org.apache.flink.table.sources.wmstrategies.{PeriodicWatermarkAssigner, PunctuatedWatermarkAssigner} +import org.apache.flink.table.sources.wmstrategies.{PeriodicWatermarkAssigner, PunctuatedWatermarkAssigner, PreserveWatermarks} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo /** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */ @@ -134,6 +134,9 @@ class StreamTableSourceScan( case p: PunctuatedWatermarkAssigner => val watermarkGenerator = new PunctuatedWatermarkAssignerWrapper(rowtimeFieldIdx, p) ingestedTable.assignTimestampsAndWatermarks(watermarkGenerator) + case _: PreserveWatermarks => + // The watermarks have already been provided by the underlying DataStream. + ingestedTable } } else { // No need to generate watermarks if no rowtime attribute is specified. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/watermarkStrategies.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/watermarkStrategies.scala index 0dd82f148a7f5..4c7f4e4b6da8a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/watermarkStrategies.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/watermarkStrategies.scala @@ -60,3 +60,9 @@ abstract class PunctuatedWatermarkAssigner extends WatermarkStrategy { */ def getWatermark(row: Row, timestamp: Long): Watermark } + +/** A strategy which indicates the watermarks should be preserved from the underlying datastream.*/ +class PreserveWatermarks extends WatermarkStrategy +object PreserveWatermarks { + val INSTANCE: PreserveWatermarks = new PreserveWatermarks +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala index 77c1e081686d5..c9ea30a1aa3f5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSourceITCase.scala @@ -27,6 +27,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JExecEnv} +import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.apache.flink.table.api.{TableEnvironment, TableException, TableSchema, Types} @@ -35,6 +36,7 @@ import org.apache.flink.table.runtime.utils.{CommonTestData, StreamITCase} import org.apache.flink.table.sources.StreamTableSource import org.apache.flink.table.utils._ import org.apache.flink.types.Row +import org.apache.flink.util.Collector import org.junit.Assert._ import org.junit.Test @@ -690,4 +692,53 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase { "3,Mike,30000,true,3000") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + @Test + def testRowtimeTableSourcePreserveWatermarks(): Unit = { + StreamITCase.testResults = mutable.MutableList() + val tableName = "MyTable" + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + + // rows with timestamps and watermarks + val data = Seq( + Right(1L), + Left(5L, Row.of(new JInt(1), new JLong(5), "A")), + Left(2L, Row.of(new JInt(2), new JLong(1), "B")), + Right(10L), + Left(8L, Row.of(new JInt(6), new JLong(8), "C")), + Right(20L), + Left(21L, Row.of(new JInt(6), new JLong(21), "D")), + Right(30L) + ) + + val fieldNames = Array("id", "rtime", "name") + val schema = new TableSchema(fieldNames, Array(Types.INT, Types.SQL_TIMESTAMP, Types.STRING)) + val rowType = new RowTypeInfo( + Array(Types.INT, Types.LONG, Types.STRING).asInstanceOf[Array[TypeInformation[_]]], + fieldNames) + + val tableSource = new TestPreserveWMTableSource(schema, rowType, data, "rtime") + tEnv.registerTableSource(tableName, tableSource) + + tEnv.scan(tableName) + .where('rtime.cast(Types.LONG) > 3L) + .select('id, 'name) + .toAppendStream[Row] + // append current watermark to each row to verify that original watermarks were preserved + .process(new ProcessFunction[Row, (Row, Long)] { + override def processElement( + value: Row, + ctx: ProcessFunction[Row, (Row, Long)]#Context, + out: Collector[(Row, Long)]): Unit = { + out.collect(value, ctx.timerService().currentWatermark()) + } + }) + .addSink(new StreamITCase.StringSink[(Row, Long)]) + env.execute() + + val expected = Seq("(1,A,1)", "(6,C,10)", "(6,D,20)") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala index a546919189e2e..f7263c8d4aab3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/testTableSources.scala @@ -27,9 +27,10 @@ import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api.TableSchema +import org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeSourceFunction import org.apache.flink.table.sources._ import org.apache.flink.table.sources.tsextractors.ExistingField -import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps +import org.apache.flink.table.sources.wmstrategies.{AscendingTimestamps, PreserveWatermarks} import org.apache.flink.types.Row import scala.collection.JavaConverters._ @@ -199,3 +200,28 @@ class TestNestedProjectableTableSource( s"read nested fields: ${readNestedFields.mkString(", ")})" } } + +class TestPreserveWMTableSource[T]( + tableSchema: TableSchema, + returnType: TypeInformation[T], + values: Seq[Either[(Long, T), Long]], + rowtime: String) + extends StreamTableSource[T] + with DefinedRowtimeAttributes { + + override def getRowtimeAttributeDescriptors: util.List[RowtimeAttributeDescriptor] = { + Collections.singletonList(new RowtimeAttributeDescriptor( + rowtime, + new ExistingField(rowtime), + PreserveWatermarks.INSTANCE)) + } + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[T] = { + execEnv.addSource(new EventTimeSourceFunction[T](values)).setParallelism(1).returns(returnType) + } + + override def getReturnType: TypeInformation[T] = returnType + + override def getTableSchema: TableSchema = tableSchema + +} From 42e24413b5a47928e06f2a61086f7559370c65d8 Mon Sep 17 00:00:00 2001 From: Xpray Date: Mon, 6 Nov 2017 23:47:33 +0800 Subject: [PATCH 064/367] [FLINK-7986] [table] Introduce FilterSetOpTransposeRule This closes #4956. --- .../table/plan/rules/FlinkRuleSets.scala | 2 + .../api/batch/table/SetOperatorsTest.scala | 80 +++++++++++++++++++ .../api/stream/table/SetOperatorsTest.scala | 68 ++++++++++++++++ 3 files changed, 150 insertions(+) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index dcc735dca4a26..a20d14fe5baa0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -52,6 +52,8 @@ object FlinkRuleSets { FilterJoinRule.JOIN, // push filter through an aggregation FilterAggregateTransposeRule.INSTANCE, + // push filter through set operation + FilterSetOpTransposeRule.INSTANCE, // aggregation and projection rules AggregateProjectMergeRule.INSTANCE, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala index 2d4e2056518d9..35f4429662383 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala @@ -135,4 +135,84 @@ class SetOperatorsTest extends TableTestBase { util.verifyJavaTable(in, expected) } + + @Test + def testFilterUnionTranspose(): Unit = { + val util = batchTestUtil() + val left = util.addTable[(Int, Long, String)]("left", 'a, 'b, 'c) + val right = util.addTable[(Int, Long, String)]("right", 'a, 'b, 'c) + + val result = left.unionAll(right) + .where('a > 0) + .groupBy('b) + .select('a.sum as 'a, 'b as 'b, 'c.count as 'c) + + val expected = unaryNode( + "DataSetCalc", + unaryNode( + "DataSetAggregate", + binaryNode( + "DataSetUnion", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a", "b", "c"), + term("where", ">(a, 0)") + ), + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "a", "b", "c"), + term("where", ">(a, 0)") + ), + term("union", "a", "b", "c") + ), + term("groupBy", "b"), + term("select", "b", "SUM(a) AS TMP_0", "COUNT(c) AS TMP_1") + ), + term("select", "TMP_0 AS a", "b", "TMP_1 AS c") + ) + + util.verifyTable(result, expected) + } + + @Test + def testFilterMinusTranspose(): Unit = { + val util = batchTestUtil() + val left = util.addTable[(Int, Long, String)]("left", 'a, 'b, 'c) + val right = util.addTable[(Int, Long, String)]("right", 'a, 'b, 'c) + + val result = left.minusAll(right) + .where('a > 0) + .groupBy('b) + .select('a.sum as 'a, 'b as 'b, 'c.count as 'c) + + val expected = unaryNode( + "DataSetCalc", + unaryNode( + "DataSetAggregate", + binaryNode( + "DataSetMinus", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a", "b", "c"), + term("where", ">(a, 0)") + ), + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "a", "b", "c"), + term("where", ">(a, 0)") + ), + term("minus", "a", "b", "c") + ), + term("groupBy", "b"), + term("select", "b", "SUM(a) AS TMP_0", "COUNT(c) AS TMP_1") + ), + term("select", "TMP_0 AS a", "b", "TMP_1 AS c") + ) + + util.verifyTable(result, expected) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala new file mode 100644 index 0000000000000..b1b700bb522d6 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala @@ -0,0 +1,68 @@ +/* + * 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.flink.table.api.stream.table + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.utils.TableTestBase +import org.apache.flink.table.utils.TableTestUtil.{binaryNode, streamTableNode, term, unaryNode} +import org.junit.Test + +class SetOperatorsTest extends TableTestBase { + + @Test + def testFilterUnionTranspose(): Unit = { + val util = streamTestUtil() + val left = util.addTable[(Int, Long, String)]("left", 'a, 'b, 'c) + val right = util.addTable[(Int, Long, String)]("right", 'a, 'b, 'c) + + val result = left.unionAll(right) + .where('a > 0) + .groupBy('b) + .select('a.sum as 'a, 'b as 'b, 'c.count as 'c) + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamGroupAggregate", + binaryNode( + "DataStreamUnion", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "b", "c"), + term("where", ">(a, 0)") + ), + unaryNode( + "DataStreamCalc", + streamTableNode(1), + term("select", "a", "b", "c"), + term("where", ">(a, 0)") + ), + term("union all", "a", "b", "c") + ), + term("groupBy", "b"), + term("select", "b", "SUM(a) AS TMP_0", "COUNT(c) AS TMP_1") + ), + term("select", "TMP_0 AS a", "b", "TMP_1 AS c") + ) + + util.verifyTable(result, expected) + } +} From d0324e34a06e7374179d1627a4a3653d07f1c614 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Tue, 14 Nov 2017 15:05:45 +0100 Subject: [PATCH 065/367] [FLINK-8063][QS] QS client does not retry when an UnknownKvStateLocation is thrown. --- .../network/AbstractServerHandler.java | 2 +- .../proxy/KvStateClientProxyHandler.java | 11 +- .../AbstractQueryableStateTestBase.java | 230 +++++++++++------- 3 files changed, 150 insertions(+), 93 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java index 9e02291b52895..7e71a11c3e860 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java @@ -262,7 +262,7 @@ public void run() { try { stats.reportFailedRequest(); - final String errMsg = "Failed request " + requestId + ". Caused by: " + ExceptionUtils.stringifyException(t); + final String errMsg = "Failed request " + requestId + "." + System.lineSeparator() + " Caused by: " + ExceptionUtils.stringifyException(t); final ByteBuf err = MessageSerializer.serializeRequestFailure(ctx.alloc(), requestId, new RuntimeException(errMsg)); ctx.writeAndFlush(err); } catch (IOException io) { diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java index 73ef7f3d0c850..af3370198e1df 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java @@ -37,7 +37,6 @@ import org.apache.flink.runtime.query.KvStateClientProxy; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.KvStateMessage; -import org.apache.flink.runtime.query.UnknownKvStateLocation; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.util.Preconditions; @@ -48,7 +47,6 @@ import java.net.ConnectException; import java.net.InetSocketAddress; -import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -133,12 +131,11 @@ private void executeActionAsync( operationFuture.whenCompleteAsync( (t, throwable) -> { if (throwable != null) { - if (throwable instanceof CancellationException) { - result.completeExceptionally(throwable); - } else if (throwable.getCause() instanceof UnknownKvStateIdException || + if ( + throwable.getCause() instanceof UnknownKvStateIdException || throwable.getCause() instanceof UnknownKvStateKeyGroupLocationException || - throwable.getCause() instanceof UnknownKvStateLocation || - throwable.getCause() instanceof ConnectException) { + throwable.getCause() instanceof ConnectException + ) { // These failures are likely to be caused by out-of-sync // KvStateLocation. Therefore we retry this query and diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index b4bae9ce1d03b..c1cbb61e6276a 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -37,7 +37,6 @@ import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.base.StringSerializer; @@ -89,12 +88,10 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongArray; -import java.util.function.Supplier; import scala.concurrent.Await; import scala.concurrent.duration.Deadline; @@ -103,15 +100,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Base class for queryable state integration tests with a configurable state backend. */ public abstract class AbstractQueryableStateTestBase extends TestLogger { - private static final int NO_OF_RETRIES = 100; private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10000L, TimeUnit.SECONDS); - private static final Time QUERY_RETRY_DELAY = Time.milliseconds(100L); private final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(4); private final ScheduledExecutor executor = new ScheduledExecutorServiceAdapter(executorService); @@ -229,14 +225,14 @@ public Integer getKey(Tuple2 value) throws Exception { allNonZero = false; } - CompletableFuture>> result = getKvStateWithRetries( + CompletableFuture>> result = getKvState( + deadline, client, jobId, queryName, key, BasicTypeInfo.INT_TYPE_INFO, reducingState, - QUERY_RETRY_DELAY, false, executor); @@ -284,7 +280,7 @@ public Integer getKey(Tuple2 value) throws Exception { * * NOTE: This test is only in the non-HA variant of the tests because * in the HA mode we use the actual JM code which does not recognize the - * {@code NotifyWhenJobStatus} message. * + * {@code NotifyWhenJobStatus} message. */ @Test public void testDuplicateRegistrationFailsJob() throws Exception { @@ -438,6 +434,92 @@ public Integer getKey(Tuple2 value) throws Exception { } } + /** + * Tests that the correct exception is thrown if the query + * contains a wrong queryable state name. + */ + @Test + public void testWrongQueryableStateName() throws Exception { + // Config + final Deadline deadline = TEST_TIMEOUT.fromNow(); + + final long numElements = 1024L; + + JobID jobId = null; + try { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); + + DataStream> source = env + .addSource(new TestAscendingValueSource(numElements)); + + // Value state + ValueStateDescriptor> valueState = + new ValueStateDescriptor<>("any", source.getType()); + + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 7662520075515707428L; + + @Override + public Integer getKey(Tuple2 value) throws Exception { + return value.f0; + } + }).asQueryableState("hakuna", valueState); + + // Submit the job graph + JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + jobId = jobGraph.getJobID(); + + CompletableFuture runningFuture = FutureUtils.toJava( + cluster.getLeaderGateway(deadline.timeLeft()) + .ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.RUNNING), deadline.timeLeft()) + .mapTo(ClassTag$.MODULE$.apply(TestingJobManagerMessages.JobStatusIs.class))); + + cluster.submitJobDetached(jobGraph); + + // expect for the job to be running + TestingJobManagerMessages.JobStatusIs jobStatus = + runningFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertEquals(JobStatus.RUNNING, jobStatus.state()); + + CompletableFuture>> future = client.getKvState( + jobId, + "wrong-hankuna", // this is the wrong name. + 0, + VoidNamespace.INSTANCE, + BasicTypeInfo.INT_TYPE_INFO, + VoidNamespaceTypeInfo.INSTANCE, + valueState); + + try { + future.get(); + fail(); // by now the job must have failed. + } catch (ExecutionException e) { + Assert.assertTrue(e.getCause() instanceof RuntimeException); + Assert.assertTrue(e.getCause().getMessage().contains( + "UnknownKvStateLocation: No KvStateLocation found for KvState instance with name 'wrong-hankuna'.")); + } catch (Exception ignored) { + fail("Unexpected type of exception."); + } + + } finally { + // Free cluster resources + if (jobId != null) { + CompletableFuture cancellation = FutureUtils.toJava(cluster + .getLeaderGateway(deadline.timeLeft()) + .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) + .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); + + cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + } + } + } + /** * Similar tests as {@link #testValueState()} but before submitting the * job, we already issue one request which fails. @@ -572,14 +654,14 @@ public Integer getKey( // Now query int key = 0; - CompletableFuture>> future = getKvStateWithRetries( + CompletableFuture>> future = getKvState( + deadline, client, jobId, queryableState.getQueryableStateName(), key, BasicTypeInfo.INT_TYPE_INFO, valueState, - QUERY_RETRY_DELAY, true, executor); @@ -723,14 +805,14 @@ public Integer getKey(Tuple2 value) throws Exception { for (int key = 0; key < maxParallelism; key++) { boolean success = false; while (deadline.hasTimeLeft() && !success) { - CompletableFuture, String>> future = getKvStateWithRetries( + CompletableFuture, String>> future = getKvState( + deadline, client, jobId, "pumba", key, BasicTypeInfo.INT_TYPE_INFO, foldingState, - QUERY_RETRY_DELAY, false, executor); @@ -814,14 +896,14 @@ public Integer getKey(Tuple2 value) throws Exception { for (int key = 0; key < maxParallelism; key++) { boolean success = false; while (deadline.hasTimeLeft() && !success) { - CompletableFuture>> future = getKvStateWithRetries( + CompletableFuture>> future = getKvState( + deadline, client, jobId, "jungle", key, BasicTypeInfo.INT_TYPE_INFO, reducingState, - QUERY_RETRY_DELAY, false, executor); @@ -923,14 +1005,14 @@ public void processElement(Tuple2 value, Context ctx, Collector>> future = getKvStateWithRetries( + CompletableFuture>> future = getKvState( + deadline, client, jobId, "timon-queryable", key, BasicTypeInfo.INT_TYPE_INFO, mapStateDescriptor, - QUERY_RETRY_DELAY, false, executor); @@ -1028,14 +1110,14 @@ public void processElement(Tuple2 value, Context ctx, Collector> future = getKvStateWithRetries( + final CompletableFuture> future = getKvState( + deadline, client, jobId, "list-queryable", key, BasicTypeInfo.INT_TYPE_INFO, listStateDescriptor, - QUERY_RETRY_DELAY, false, executor); @@ -1130,14 +1212,14 @@ public Integer getKey(Tuple2 value) throws Exception { for (int key = 0; key < maxParallelism; key++) { boolean success = false; while (deadline.hasTimeLeft() && !success) { - CompletableFuture, String>> future = getKvStateWithRetries( + CompletableFuture, String>> future = getKvState( + deadline, client, jobId, "aggr-queryable", key, BasicTypeInfo.INT_TYPE_INFO, aggrStateDescriptor, - QUERY_RETRY_DELAY, false, executor); @@ -1372,84 +1454,62 @@ public Tuple2 reduce(Tuple2 value1, Tuple2 CompletableFuture getKvStateWithRetries( + private static CompletableFuture getKvState( + final Deadline deadline, final QueryableStateClient client, final JobID jobId, final String queryName, final K key, final TypeInformation keyTypeInfo, final StateDescriptor stateDescriptor, - final Time retryDelay, final boolean failForUnknownKeyOrNamespace, - final ScheduledExecutor executor) { - return retryWithDelay( - () -> client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor), - NO_OF_RETRIES, - retryDelay, - executor, - failForUnknownKeyOrNamespace); - } - - private static CompletableFuture retryWithDelay( - final Supplier> operation, - final int retries, - final Time retryDelay, - final ScheduledExecutor scheduledExecutor, - final boolean failIfUnknownKeyOrNamespace) { - - final CompletableFuture resultFuture = new CompletableFuture<>(); - - retryWithDelay( - resultFuture, - operation, - retries, - retryDelay, - scheduledExecutor, - failIfUnknownKeyOrNamespace); + final ScheduledExecutor executor) throws InterruptedException { + final CompletableFuture resultFuture = new CompletableFuture<>(); + getKvStateIgnoringCertainExceptions( + deadline, resultFuture, client, jobId, queryName, key, keyTypeInfo, + stateDescriptor, failForUnknownKeyOrNamespace, executor); return resultFuture; } - public static void retryWithDelay( - final CompletableFuture resultFuture, - final Supplier> operation, - final int retries, - final Time retryDelay, - final ScheduledExecutor scheduledExecutor, - final boolean failIfUnknownKeyOrNamespace) { + private static void getKvStateIgnoringCertainExceptions( + final Deadline deadline, + final CompletableFuture resultFuture, + final QueryableStateClient client, + final JobID jobId, + final String queryName, + final K key, + final TypeInformation keyTypeInfo, + final StateDescriptor stateDescriptor, + final boolean failForUnknownKeyOrNamespace, + final ScheduledExecutor executor) throws InterruptedException { if (!resultFuture.isDone()) { - final CompletableFuture operationResultFuture = operation.get(); - operationResultFuture.whenCompleteAsync( - (t, throwable) -> { - if (throwable != null) { - if (throwable.getCause() instanceof CancellationException) { - resultFuture.completeExceptionally(new FutureUtils.RetryException("Operation future was cancelled.", throwable.getCause())); - } else if (throwable.getCause() instanceof AssertionError || - (failIfUnknownKeyOrNamespace && throwable.getCause() instanceof UnknownKeyOrNamespaceException)) { - resultFuture.completeExceptionally(throwable.getCause()); - } else { - if (retries > 0) { - final ScheduledFuture scheduledFuture = scheduledExecutor.schedule( - () -> retryWithDelay(resultFuture, operation, retries - 1, retryDelay, scheduledExecutor, failIfUnknownKeyOrNamespace), - retryDelay.toMilliseconds(), - TimeUnit.MILLISECONDS); - - resultFuture.whenComplete( - (innerT, innerThrowable) -> scheduledFuture.cancel(false)); - } else { - resultFuture.completeExceptionally(new FutureUtils.RetryException("Could not complete the operation. Number of retries " + - "has been exhausted.", throwable)); - } - } - } else { - resultFuture.complete(t); + Thread.sleep(100L); + CompletableFuture expected = client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor); + expected.whenCompleteAsync((result, throwable) -> { + if (throwable != null) { + if ( + throwable.getCause() instanceof CancellationException || + throwable.getCause() instanceof AssertionError || + (failForUnknownKeyOrNamespace && throwable.getCause() instanceof UnknownKeyOrNamespaceException) + ) { + resultFuture.completeExceptionally(throwable.getCause()); + } else if (deadline.hasTimeLeft()) { + try { + getKvStateIgnoringCertainExceptions( + deadline, resultFuture, client, jobId, queryName, key, keyTypeInfo, + stateDescriptor, failForUnknownKeyOrNamespace, executor); + } catch (InterruptedException e) { + e.printStackTrace(); } - }, - scheduledExecutor); + } + } else { + resultFuture.complete(result); + } + }, executor); - resultFuture.whenComplete( - (t, throwable) -> operationResultFuture.cancel(false)); + resultFuture.whenComplete((result, throwable) -> expected.cancel(false)); } } @@ -1468,14 +1528,14 @@ private void executeValueQuery( for (int key = 0; key < maxParallelism; key++) { boolean success = false; while (deadline.hasTimeLeft() && !success) { - CompletableFuture>> future = getKvStateWithRetries( + CompletableFuture>> future = getKvState( + deadline, client, jobId, queryableStateName, key, BasicTypeInfo.INT_TYPE_INFO, stateDescriptor, - QUERY_RETRY_DELAY, false, executor); From 96b350ad91a1f248d0a3c616d1e59638013892be Mon Sep 17 00:00:00 2001 From: kkloudas Date: Wed, 15 Nov 2017 15:32:42 +0100 Subject: [PATCH 066/367] [FLINK-8062][QS] Make getKvState() with namespace private. --- .../flink/queryablestate/client/QueryableStateClient.java | 3 +-- .../itcases/AbstractQueryableStateTestBase.java | 7 +------ 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java index 304505ac29dde..03e02e1c10d9d 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java @@ -186,8 +186,7 @@ public CompletableFuture getKvState( * @param stateDescriptor The {@link StateDescriptor} of the state we want to query. * @return Future holding the immutable {@link State} object containing the result. */ - @PublicEvolving - public CompletableFuture getKvState( + private CompletableFuture getKvState( final JobID jobId, final String queryableStateName, final K key, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index c1cbb61e6276a..a789dbda250dc 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -47,7 +47,6 @@ import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.queryablestate.client.VoidNamespace; import org.apache.flink.queryablestate.client.VoidNamespaceSerializer; -import org.apache.flink.queryablestate.client.VoidNamespaceTypeInfo; import org.apache.flink.queryablestate.exceptions.UnknownKeyOrNamespaceException; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -491,9 +490,7 @@ public Integer getKey(Tuple2 value) throws Exception { jobId, "wrong-hankuna", // this is the wrong name. 0, - VoidNamespace.INSTANCE, BasicTypeInfo.INT_TYPE_INFO, - VoidNamespaceTypeInfo.INSTANCE, valueState); try { @@ -572,9 +569,7 @@ public Integer getKey(Tuple2 value) throws Exception { jobId, queryableState.getQueryableStateName(), 0, - VoidNamespace.INSTANCE, BasicTypeInfo.INT_TYPE_INFO, - VoidNamespaceTypeInfo.INSTANCE, valueState); cluster.submitJobDetached(jobGraph); @@ -1486,7 +1481,7 @@ private static void getKvStateIgnoringCertainExceptions( if (!resultFuture.isDone()) { Thread.sleep(100L); - CompletableFuture expected = client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor); + CompletableFuture expected = client.getKvState(jobId, queryName, key, keyTypeInfo, stateDescriptor); expected.whenCompleteAsync((result, throwable) -> { if (throwable != null) { if ( From 6314e4861df3100e8edd666f00e062c128f6e09f Mon Sep 17 00:00:00 2001 From: kkloudas Date: Wed, 15 Nov 2017 15:38:36 +0100 Subject: [PATCH 067/367] [FLINK-8065][QS] Improve error message when client already shut down. --- .../java/org/apache/flink/queryablestate/network/Client.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java index 13d34fb48fadb..e21145ba8638b 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java @@ -133,7 +133,7 @@ public String getClientName() { public CompletableFuture sendRequest(final InetSocketAddress serverAddress, final REQ request) { if (shutDown.get()) { - return FutureUtils.getFailedFuture(new IllegalStateException("Shut down")); + return FutureUtils.getFailedFuture(new IllegalStateException(clientName + " is already shut down.")); } EstablishedConnection connection = establishedConnections.get(serverAddress); From 12b0c58f6780376ac2da0f02c6e7eb8a24ab8a13 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Thu, 16 Nov 2017 17:02:16 +0100 Subject: [PATCH 068/367] [FLINK-8055][QS] Deduplicate logging messages about QS start. --- .../network/AbstractServerBase.java | 20 +++++++++---------- .../flink/queryablestate/network/Client.java | 20 +++++++++++++------ .../server/KvStateServerImpl.java | 5 ----- .../HAAbstractQueryableStateTestBase.java | 2 +- .../network/AbstractServerTest.java | 2 +- .../network/KvStateServerHandlerTest.java | 2 +- .../io/network/NetworkEnvironment.java | 2 -- 7 files changed, 27 insertions(+), 26 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java index 07ca26ddce60a..82a05f2a09a30 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java @@ -60,7 +60,7 @@ @Internal public abstract class AbstractServerBase { - private static final Logger LOG = LoggerFactory.getLogger(AbstractServerBase.class); + protected final Logger log = LoggerFactory.getLogger(getClass()); /** AbstractServerBase config: low water mark. */ private static final int LOW_WATER_MARK = 8 * 1024; @@ -180,16 +180,16 @@ public InetSocketAddress getServerAddress() { */ public void start() throws Throwable { Preconditions.checkState(serverAddress == null, - "The " + serverName + " already running @ " + serverAddress + '.'); + serverName + " is already running @ " + serverAddress + '.'); Iterator portIterator = bindPortRange.iterator(); while (portIterator.hasNext() && !attemptToBind(portIterator.next())) {} if (serverAddress != null) { - LOG.info("Started the {} @ {}.", serverName, serverAddress); + log.info("Started {} @ {}.", serverName, serverAddress); } else { - LOG.info("Unable to start the {}. All ports in provided range are occupied.", serverName); - throw new FlinkRuntimeException("Unable to start the " + serverName + ". All ports in provided range are occupied."); + log.info("Unable to start {}. All ports in provided range are occupied.", serverName); + throw new FlinkRuntimeException("Unable to start " + serverName + ". All ports in provided range are occupied."); } } @@ -203,7 +203,7 @@ public void start() throws Throwable { * @throws Exception If something goes wrong during the bind operation. */ private boolean attemptToBind(final int port) throws Throwable { - LOG.debug("Attempting to start server {} on port {}.", serverName, port); + log.debug("Attempting to start {} on port {}.", serverName, port); this.queryExecutor = createQueryExecutor(); this.handler = initializeHandler(); @@ -250,7 +250,7 @@ private boolean attemptToBind(final int port) throws Throwable { throw future.cause(); } catch (BindException e) { - LOG.debug("Failed to start server {} on port {}: {}.", serverName, port, e.getMessage()); + log.debug("Failed to start {} on port {}: {}.", serverName, port, e.getMessage()); shutdown(); } // any other type of exception we let it bubble up. @@ -261,7 +261,7 @@ private boolean attemptToBind(final int port) throws Throwable { * Shuts down the server and all related thread pools. */ public void shutdown() { - LOG.info("Shutting down server {} @ {}", serverName, serverAddress); + log.info("Shutting down {} @ {}", serverName, serverAddress); if (handler != null) { handler.shutdown(); @@ -311,7 +311,7 @@ protected void initChannel(SocketChannel channel) throws Exception { } @VisibleForTesting - public boolean isExecutorShutdown() { - return queryExecutor.isShutdown(); + public boolean isEventGroupShutdown() { + return bootstrap == null || bootstrap.group().isTerminated(); } } diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java index e21145ba8638b..12286faa02505 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java @@ -19,6 +19,7 @@ package org.apache.flink.queryablestate.network; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.queryablestate.FutureUtils; import org.apache.flink.queryablestate.network.messages.MessageBody; import org.apache.flink.queryablestate.network.messages.MessageSerializer; @@ -282,12 +283,14 @@ private void handInChannel(Channel channel) { while (!queuedRequests.isEmpty()) { final PendingRequest pending = queuedRequests.poll(); - established.sendRequest(pending.request) - .thenAccept(resp -> pending.complete(resp)) - .exceptionally(throwable -> { - pending.completeExceptionally(throwable); - return null; - }); + established.sendRequest(pending.request).whenComplete( + (response, throwable) -> { + if (throwable != null) { + pending.completeExceptionally(throwable); + } else { + pending.complete(response); + } + }); } // Publish the channel for the general public @@ -533,4 +536,9 @@ public long getTimestamp() { } } } + + @VisibleForTesting + public boolean isEventGroupShutdown() { + return bootstrap == null || bootstrap.group().isTerminated(); + } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java index fe0768748862a..3a37a3a620c3a 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java @@ -29,9 +29,6 @@ import org.apache.flink.runtime.query.KvStateServer; import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.Iterator; @@ -42,8 +39,6 @@ @Internal public class KvStateServerImpl extends AbstractServerBase implements KvStateServer { - private static final Logger LOG = LoggerFactory.getLogger(KvStateServerImpl.class); - /** The {@link KvStateRegistry} to query for state instances. */ private final KvStateRegistry kvStateRegistry; diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java index fc4b2bc103be4..79809b3004781 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java @@ -65,7 +65,7 @@ public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); cluster = new TestingCluster(config, false); - cluster.start(); + cluster.start(true); client = new QueryableStateClient("localhost", proxyPortRangeStart); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java index 2775cd42c790d..3d2ed4066ca69 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java @@ -58,7 +58,7 @@ public void testServerInitializationFailure() throws Throwable { // the expected exception along with the adequate message expectedEx.expect(FlinkRuntimeException.class); - expectedEx.expectMessage("Unable to start the Test Server 2. All ports in provided range are occupied."); + expectedEx.expectMessage("Unable to start Test Server 2. All ports in provided range are occupied."); TestServer server1 = null; TestServer server2 = null; diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java index 041544d845bf3..7b301ede8848c 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java @@ -391,7 +391,7 @@ public void testQueryExecutorShutDown() throws Throwable { localTestServer.start(); localTestServer.shutdown(); - assertTrue(localTestServer.isExecutorShutdown()); + assertTrue(localTestServer.getQueryExecutor().isTerminated()); MessageSerializer serializer = new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java index 4fffacd42449b..71d038615837a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java @@ -310,7 +310,6 @@ public void start() throws IOException { if (kvStateServer != null) { try { kvStateServer.start(); - LOG.info("Started the Queryable State Data Server @ {}", kvStateServer.getServerAddress()); } catch (Throwable ie) { kvStateServer.shutdown(); kvStateServer = null; @@ -321,7 +320,6 @@ public void start() throws IOException { if (kvStateProxy != null) { try { kvStateProxy.start(); - LOG.info("Started the Queryable State Client Proxy @ {}", kvStateProxy.getServerAddress()); } catch (Throwable ie) { kvStateProxy.shutdown(); kvStateProxy = null; From 1a68d7527932b12bd2cb392c7c7781023756bf0c Mon Sep 17 00:00:00 2001 From: kkloudas Date: Thu, 16 Nov 2017 17:45:49 +0100 Subject: [PATCH 069/367] [FLINK-8059][QS] QS client throws FlinkJobNotFoundException for queries with unknown jobIds. --- .../AbstractQueryableStateTestBase.java | 32 ++++++++++++++----- .../flink/runtime/jobmanager/JobManager.scala | 4 +-- .../runtime/jobmanager/JobManagerTest.java | 5 +-- 3 files changed, 29 insertions(+), 12 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index a789dbda250dc..65e9bb5183a7e 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -276,10 +276,6 @@ public Integer getKey(Tuple2 value) throws Exception { /** * Tests that duplicate query registrations fail the job at the JobManager. - * - * NOTE: This test is only in the non-HA variant of the tests because - * in the HA mode we use the actual JM code which does not recognize the - * {@code NotifyWhenJobStatus} message. */ @Test public void testDuplicateRegistrationFailsJob() throws Exception { @@ -435,10 +431,10 @@ public Integer getKey(Tuple2 value) throws Exception { /** * Tests that the correct exception is thrown if the query - * contains a wrong queryable state name. + * contains a wrong jobId or wrong queryable state name. */ @Test - public void testWrongQueryableStateName() throws Exception { + public void testWrongJobIdAndWrongQueryableStateName() throws Exception { // Config final Deadline deadline = TEST_TIMEOUT.fromNow(); @@ -486,7 +482,27 @@ public Integer getKey(Tuple2 value) throws Exception { runningFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); assertEquals(JobStatus.RUNNING, jobStatus.state()); - CompletableFuture>> future = client.getKvState( + final JobID wrongJobId = new JobID(); + + CompletableFuture>> unknownJobFuture = client.getKvState( + wrongJobId, // this is the wrong job id + "hankuna", + 0, + BasicTypeInfo.INT_TYPE_INFO, + valueState); + + try { + unknownJobFuture.get(); + fail(); // by now the job must have failed. + } catch (ExecutionException e) { + Assert.assertTrue(e.getCause() instanceof RuntimeException); + Assert.assertTrue(e.getCause().getMessage().contains( + "FlinkJobNotFoundException: Could not find Flink job (" + wrongJobId + ")")); + } catch (Exception ignored) { + fail("Unexpected type of exception."); + } + + CompletableFuture>> unknownQSName = client.getKvState( jobId, "wrong-hankuna", // this is the wrong name. 0, @@ -494,7 +510,7 @@ public Integer getKey(Tuple2 value) throws Exception { valueState); try { - future.get(); + unknownQSName.get(); fail(); // by now the job must have failed. } catch (ExecutionException e) { Assert.assertTrue(e.getCause() instanceof RuntimeException); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 4fb11963015b1..f57637a5d71e7 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -70,7 +70,7 @@ import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState import org.apache.flink.runtime.messages.accumulators._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint, DeclineCheckpoint} import org.apache.flink.runtime.messages.webmonitor.{InfoMessage, _} -import org.apache.flink.runtime.messages.{Acknowledge, StackTrace} +import org.apache.flink.runtime.messages.{Acknowledge, FlinkJobNotFoundException, StackTrace} import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup import org.apache.flink.runtime.metrics.util.MetricUtils import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistryImpl, MetricRegistry => FlinkMetricRegistry} @@ -1503,7 +1503,7 @@ class JobManager( } case None => - sender() ! Status.Failure(new IllegalStateException(s"Job ${msg.getJobId} not found")) + sender() ! Status.Failure(new FlinkJobNotFoundException(msg.getJobId)) } // TaskManager KvState registration diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java index a697aae2a693a..6a02d1f024482 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java @@ -63,6 +63,7 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.jobmanager.JobManagerHARecoveryTest.BlockingStatefulInvokable; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure; import org.apache.flink.runtime.messages.JobManagerMessages.CancellationResponse; @@ -672,7 +673,7 @@ public void testKvStateMessages() throws Exception { try { Await.result(lookupFuture, deadline.timeLeft()); fail("Did not throw expected Exception"); - } catch (IllegalStateException ignored) { + } catch (FlinkJobNotFoundException ignored) { // Expected } @@ -735,7 +736,7 @@ public void testKvStateMessages() throws Exception { try { Await.result(lookupFuture, deadline.timeLeft()); fail("Did not throw expected Exception"); - } catch (IllegalStateException ignored) { + } catch (FlinkJobNotFoundException ignored) { // Expected } From 3753ae2517fbc940c05ea54e3eb0a960fecdf879 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Fri, 17 Nov 2017 09:26:10 +0100 Subject: [PATCH 070/367] [FLINK-8057][QS] Change error message in KvStateRegistry.registerKvState(). --- .../flink/runtime/query/KvStateRegistry.java | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java index af19d8161ab74..ed1f92ec6aa2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java @@ -45,7 +45,7 @@ public class KvStateRegistry { new ConcurrentHashMap<>(); /** Registry listener to be notified on registration/unregistration. */ - private final AtomicReference listener = new AtomicReference<>(); + private final AtomicReference listenerRef = new AtomicReference<>(); /** * Registers a listener with the registry. @@ -54,7 +54,7 @@ public class KvStateRegistry { * @throws IllegalStateException If there is a registered listener */ public void registerListener(KvStateRegistryListener listener) { - if (!this.listener.compareAndSet(null, listener)) { + if (!listenerRef.compareAndSet(null, listener)) { throw new IllegalStateException("Listener already registered."); } } @@ -63,19 +63,9 @@ public void registerListener(KvStateRegistryListener listener) { * Unregisters the listener with the registry. */ public void unregisterListener() { - listener.set(null); + listenerRef.set(null); } - /** - * Registers the KvState instance identified by the given 4-tuple of JobID, - * JobVertexID, key group index, and registration name. - * - * @param kvStateId KvStateID to identify the KvState instance - * @param kvState KvState instance to register - * @throws IllegalStateException If there is a KvState instance registered - * with the same ID. - */ - /** * Registers the KvState instance and returns the assigned ID. * @@ -96,7 +86,7 @@ public KvStateID registerKvState( KvStateID kvStateId = new KvStateID(); if (registeredKvStates.putIfAbsent(kvStateId, kvState) == null) { - KvStateRegistryListener listener = this.listener.get(); + final KvStateRegistryListener listener = listenerRef.get(); if (listener != null) { listener.notifyKvStateRegistered( jobId, @@ -108,7 +98,8 @@ public KvStateID registerKvState( return kvStateId; } else { - throw new IllegalStateException(kvStateId + " is already registered."); + throw new IllegalStateException( + "State \"" + registrationName + " \"(id=" + kvStateId + ") appears registered although it should not."); } } @@ -127,7 +118,7 @@ public void unregisterKvState( KvStateID kvStateId) { if (registeredKvStates.remove(kvStateId) != null) { - KvStateRegistryListener listener = this.listener.get(); + final KvStateRegistryListener listener = listenerRef.get(); if (listener != null) { listener.notifyKvStateUnregistered( jobId, From e784f3a184ecc35ea26fdfa6bcb4fa74520fa9df Mon Sep 17 00:00:00 2001 From: Vetriselvan1187 Date: Mon, 13 Nov 2017 17:54:43 +0100 Subject: [PATCH 071/367] [FLINK-8061][QS] Remove trailing * in QSClient javadocs. --- .../flink/queryablestate/client/QueryableStateClient.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java index 03e02e1c10d9d..7abf6bc398725 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java @@ -132,7 +132,7 @@ public ExecutionConfig setExecutionConfig(ExecutionConfig config) { } /** - * Returns a future holding the request result. * + * Returns a future holding the request result. * @param jobId JobID of the job the queryable state belongs to. * @param queryableStateName Name under which the state is queryable. * @param key The key we are interested in. @@ -155,7 +155,7 @@ public CompletableFuture getKvState( } /** - * Returns a future holding the request result. * + * Returns a future holding the request result. * @param jobId JobID of the job the queryable state belongs to. * @param queryableStateName Name under which the state is queryable. * @param key The key we are interested in. From a0dbe182fa677a87f601cbedc4115e63fff9fe4f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 25 Jul 2017 17:19:25 +0200 Subject: [PATCH 072/367] [FLINK-7265] [core] Introduce FileSystemKind to differentiate between FileSystem and ObjectStore --- .../org/apache/flink/core/fs/FileSystem.java | 5 + .../apache/flink/core/fs/FileSystemKind.java | 40 +++++++ .../core/fs/SafetyNetWrapperFileSystem.java | 5 + .../flink/core/fs/local/LocalFileSystem.java | 10 +- .../core/fs/local/LocalFileSystemTest.java | 7 ++ .../runtime/fs/hdfs/HadoopFileSystem.java | 47 ++++++++ .../runtime/fs/maprfs/MapRFileSystem.java | 6 ++ .../flink/runtime/fs/hdfs/HdfsKindTest.java | 101 ++++++++++++++++++ 8 files changed, 219 insertions(+), 2 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java create mode 100644 flink-fs-tests/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index d66a893ff4ed4..982e496d26253 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -633,6 +633,11 @@ public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { */ public abstract boolean isDistributedFS(); + /** + * Gets a description of the characteristics of this file system. + */ + public abstract FileSystemKind getKind(); + // ------------------------------------------------------------------------ // output directory initialization // ------------------------------------------------------------------------ diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java new file mode 100644 index 0000000000000..52f58ac9d2775 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java @@ -0,0 +1,40 @@ +/* + * 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.flink.core.fs; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * An enumeration defining the kind and characteristics of a {@link FileSystem}. + */ +@PublicEvolving +public enum FileSystemKind { + + /** + * An actual file system, with files and directories. + */ + FILE_SYSTEM, + + /** + * An Object store. Files correspond to objects. + * There are not really directories, but a directory-like structure may be mimicked + * by hierarchical naming of files. + */ + OBJECT_STORE +} diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java index a1167dd2835a8..e7f43a464d063 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java @@ -140,6 +140,11 @@ public boolean isDistributedFS() { return unsafeFileSystem.isDistributedFS(); } + @Override + public FileSystemKind getKind() { + return unsafeFileSystem.getKind(); + } + @Override public FileSystem getWrappedDelegate() { return unsafeFileSystem; diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java index ecfd21c3dbb99..a96f2212d046c 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java @@ -31,6 +31,7 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileStatus; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; import org.apache.flink.util.OperatingSystem; @@ -283,13 +284,18 @@ public boolean isDistributedFS() { return false; } + @Override + public FileSystemKind getKind() { + return FileSystemKind.FILE_SYSTEM; + } + // ------------------------------------------------------------------------ /** * Gets the URI that represents the local file system. * That URI is {@code "file:/"} on Windows platforms and {@code "file:///"} on other * UNIX family platforms. - * + * * @return The URI that represents the local file system. */ public static URI getLocalFsURI() { @@ -298,7 +304,7 @@ public static URI getLocalFsURI() { /** * Gets the shared instance of this file system. - * + * * @return The shared instance of this file system. */ public static LocalFileSystem getSharedInstance() { diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java index 2312ee9856ef3..96c526944fa93 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java @@ -34,6 +34,7 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileStatus; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.util.FileUtils; @@ -312,4 +313,10 @@ public void testRenameToNonEmptyTargetDir() throws IOException { assertTrue(fs.rename(new Path(srcFolder.toURI()), new Path(dstFolder.toURI()))); assertTrue(new File(dstFolder, srcFile.getName()).exists()); } + + @Test + public void testKind() { + final FileSystem fs = FileSystem.getLocalFileSystem(); + assertEquals(FileSystemKind.FILE_SYSTEM, fs.getKind()); + } } diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java index 5970c9d9375e9..7bc5a0f025067 100644 --- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java +++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java @@ -21,10 +21,12 @@ import org.apache.flink.core.fs.BlockLocation; import org.apache.flink.core.fs.FileStatus; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; import java.io.IOException; import java.net.URI; +import java.util.Locale; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -36,6 +38,11 @@ public class HadoopFileSystem extends FileSystem { /** The wrapped Hadoop File System. */ private final org.apache.hadoop.fs.FileSystem fs; + /* This field caches the file system kind. It is lazily set because the file system + * URL is lazily initialized. */ + private FileSystemKind fsKind; + + /** * Wraps the given Hadoop File System object as a Flink File System object. * The given Hadoop file system object is expected to be initialized already. @@ -168,4 +175,44 @@ public long getDefaultBlockSize() { public boolean isDistributedFS() { return true; } + + @Override + public FileSystemKind getKind() { + if (fsKind == null) { + fsKind = getKindForScheme(this.fs.getUri().getScheme()); + } + return fsKind; + } + + /** + * Gets the kind of the file system from its scheme. + * + *

Implementation note: Initially, especially within the Flink 1.3.x line + * (in order to not break backwards compatibility), we must only label file systems + * as 'inconsistent' or as 'not proper filesystems' if we are sure about it. + * Otherwise, we cause regression for example in the performance and cleanup handling + * of checkpoints. + * For that reason, we initially mark some filesystems as 'eventually consistent' or + * as 'object stores', and leave the others as 'consistent file systems'. + */ + static FileSystemKind getKindForScheme(String scheme) { + scheme = scheme.toLowerCase(Locale.US); + + if (scheme.startsWith("s3") || scheme.startsWith("emr")) { + // the Amazon S3 storage + return FileSystemKind.OBJECT_STORE; + } + else if (scheme.startsWith("http") || scheme.startsWith("ftp")) { + // file servers instead of file systems + // they might actually be consistent, but we have no hard guarantees + // currently to rely on that + return FileSystemKind.OBJECT_STORE; + } + else { + // the remainder should include hdfs, kosmos, ceph, ... + // this also includes federated HDFS (viewfs). + return FileSystemKind.FILE_SYSTEM; + } + } + } diff --git a/flink-filesystems/flink-mapr-fs/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java b/flink-filesystems/flink-mapr-fs/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java index 058772caec4a6..5aec4a4ade3e8 100644 --- a/flink-filesystems/flink-mapr-fs/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java +++ b/flink-filesystems/flink-mapr-fs/src/main/java/org/apache/flink/runtime/fs/maprfs/MapRFileSystem.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.fs.maprfs; +import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; import org.slf4j.Logger; @@ -172,4 +173,9 @@ private static String[] getCLDBLocations(String authority) throws IOException { throw new IOException(String.format( "Unable to find CLDB locations for cluster %s", authority)); } + + @Override + public FileSystemKind getKind() { + return FileSystemKind.FILE_SYSTEM; + } } diff --git a/flink-fs-tests/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java b/flink-fs-tests/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java new file mode 100644 index 0000000000000..69ecdb851b3dc --- /dev/null +++ b/flink-fs-tests/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.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.flink.runtime.fs.hdfs; + +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemKind; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for extracting the {@link FileSystemKind} from file systems that Flink + * accesses through Hadoop's File System interface. + * + *

This class needs to be in this package, because it accesses package private methods + * from the HDFS file system wrapper class. + */ +public class HdfsKindTest extends TestLogger { + + @Test + public void testHdfsKind() throws IOException { + final FileSystem fs = new Path("hdfs://localhost:55445/my/file").getFileSystem(); + assertEquals(FileSystemKind.FILE_SYSTEM, fs.getKind()); + } + + @Test + public void testS3Kind() throws IOException { + try { + Class.forName("org.apache.hadoop.fs.s3.S3FileSystem"); + } catch (ClassNotFoundException ignored) { + // not in the classpath, cannot run this test + log.info("Skipping test 'testS3Kind()' because the S3 file system is not in the class path"); + return; + } + + final FileSystem s3 = new Path("s3://myId:mySecret@bucket/some/bucket/some/object").getFileSystem(); + assertEquals(FileSystemKind.OBJECT_STORE, s3.getKind()); + } + + @Test + public void testS3nKind() throws IOException { + try { + Class.forName("org.apache.hadoop.fs.s3native.NativeS3FileSystem"); + } catch (ClassNotFoundException ignored) { + // not in the classpath, cannot run this test + log.info("Skipping test 'testS3nKind()' because the Native S3 file system is not in the class path"); + return; + } + + final FileSystem s3n = new Path("s3n://myId:mySecret@bucket/some/bucket/some/object").getFileSystem(); + assertEquals(FileSystemKind.OBJECT_STORE, s3n.getKind()); + } + + @Test + public void testS3aKind() throws IOException { + try { + Class.forName("org.apache.hadoop.fs.s3a.S3AFileSystem"); + } catch (ClassNotFoundException ignored) { + // not in the classpath, cannot run this test + log.info("Skipping test 'testS3aKind()' because the S3AFileSystem is not in the class path"); + return; + } + + final FileSystem s3a = new Path("s3a://myId:mySecret@bucket/some/bucket/some/object").getFileSystem(); + assertEquals(FileSystemKind.OBJECT_STORE, s3a.getKind()); + } + + @Test + public void testS3fileSystemSchemes() { + assertEquals(FileSystemKind.OBJECT_STORE, HadoopFileSystem.getKindForScheme("s3")); + assertEquals(FileSystemKind.OBJECT_STORE, HadoopFileSystem.getKindForScheme("s3n")); + assertEquals(FileSystemKind.OBJECT_STORE, HadoopFileSystem.getKindForScheme("s3a")); + assertEquals(FileSystemKind.OBJECT_STORE, HadoopFileSystem.getKindForScheme("EMRFS")); + } + + @Test + public void testViewFs() { + assertEquals(FileSystemKind.FILE_SYSTEM, HadoopFileSystem.getKindForScheme("viewfs")); + } +} From 666b1b2e62463ae4985d237535d56c9e0ab9dba9 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 25 Jul 2017 17:26:38 +0200 Subject: [PATCH 073/367] [FLINK-7266] [core] Prevent attempt for parent directory deletion for object stores This closes #4397 --- .../runtime/state/filesystem/FileStateHandle.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStateHandle.java index bdf3f42aaaddc..7655f0bc6f634 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStateHandle.java @@ -20,6 +20,7 @@ import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.FileUtils; @@ -77,14 +78,15 @@ public FSDataInputStream openInputStream() throws IOException { */ @Override public void discardState() throws Exception { - FileSystem fs = getFileSystem(); fs.delete(filePath, false); - try { - FileUtils.deletePathIfEmpty(fs, filePath.getParent()); - } catch (Exception ignored) {} + if (fs.getKind() == FileSystemKind.FILE_SYSTEM) { + try { + FileUtils.deletePathIfEmpty(fs, filePath.getParent()); + } catch (Exception ignored) {} + } } /** From 36b807567d93c431c1498241a42c20221cb6a664 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Wed, 9 Nov 2016 15:04:50 -0500 Subject: [PATCH 074/367] [FLINK-4228][yarn/s3] fix for yarn staging with s3a defaultFs + includes a new unit tests for recursive uploads to hfds:// targets + add a unit test for recursive file uploads to s3:// via s3a [FLINK-4228][yarn/s3] turn the dependencies around Instead of having flink-s3-fs-hadoop depend on flink-yarn_, let flink-yarn depend on the s3 filesystem and implement the test there. This is safer with regards to the scala-independent flink-s3-fs-hadoop project. [FLINK-4228][yarn] change the S3 upload tests to use Hadoop's S3 implementations This is how YARN would use it and what should really be tested. [FLINK-4228][yarn] enable S3 tests for newer Hadoop versions - requires the 'include_hadoop_aws' build profile (or property) to be set - requires a newer aws-sdk version (than Hadoop pulls in) to work with our httpcomponents version - we also add a check that at least one S3 implementation is tested to not silently ignore all tests because of such a missing dependency This closes #4939. --- .travis.yml | 2 +- flink-yarn/pom.xml | 63 +++++ .../yarn/AbstractYarnClusterDescriptor.java | 192 ++++++++++----- .../java/org/apache/flink/yarn/Utils.java | 86 ++++--- .../apache/flink/yarn/YarnFileStageTest.java | 218 +++++++++++++++++ .../flink/yarn/YarnFileStageTestS3ITCase.java | 220 ++++++++++++++++++ 6 files changed, 698 insertions(+), 83 deletions(-) create mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java create mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java diff --git a/.travis.yml b/.travis.yml index daf2186c09d27..5e2ef74aa8d28 100644 --- a/.travis.yml +++ b/.travis.yml @@ -59,7 +59,7 @@ matrix: - jdk: "oraclejdk8" env: - TEST="misc" - - PROFILE="-Dhadoop.version=2.8.0" + - PROFILE="-Dhadoop.version=2.8.0 -Dinclude_hadoop_aws" - CACHE_NAME=JDK8_H280_M - jdk: "openjdk8" env: diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml index db6ee88dc903b..f621acfff464f 100644 --- a/flink-yarn/pom.xml +++ b/flink-yarn/pom.xml @@ -31,6 +31,12 @@ under the License. flink-yarn jar + + + 1.11.171 + + @@ -153,6 +159,63 @@ under the License. + + + + include_hadoop_aws + + + include_hadoop_aws + + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + test + + + org.apache.avro + avro + + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + + + + com.amazonaws + aws-java-sdk-s3 + ${aws.sdk.version} + test + + + com.amazonaws + aws-java-sdk-sts + ${aws.sdk.version} + test + + + + diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 8ecc371d798f4..5ac5c4ea4fde9 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.ConfigConstants; @@ -624,6 +625,7 @@ public ApplicationReport startAppMaster( // Copy the application master jar to the filesystem // Create a local resource to point to the destination jar path final FileSystem fs = FileSystem.get(conf); + final Path homeDir = fs.getHomeDirectory(); // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && @@ -705,11 +707,25 @@ public ApplicationReport startAppMaster( StringBuilder envShipFileList = new StringBuilder(); // upload and register ship files - List systemClassPaths = uploadAndRegisterFiles(systemShipFiles, fs, appId.toString(), paths, localResources, envShipFileList); + List systemClassPaths = uploadAndRegisterFiles( + systemShipFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); List userClassPaths; if (userJarInclusion != YarnConfigOptions.UserJarInclusion.DISABLED) { - userClassPaths = uploadAndRegisterFiles(userJarFiles, fs, appId.toString(), paths, localResources, envShipFileList); + userClassPaths = uploadAndRegisterFiles( + userJarFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); } else { userClassPaths = Collections.emptyList(); } @@ -739,32 +755,29 @@ public ApplicationReport startAppMaster( } // Setup jar for ApplicationMaster - LocalResource appMasterJar = Records.newRecord(LocalResource.class); - Path remotePathJar = Utils.setupLocalResource( + Path remotePathJar = setupSingleLocalResource( + "flink.jar", fs, - appId.toString(), + appId, flinkJarPath, - appMasterJar, - fs.getHomeDirectory()); - - localResources.put("flink.jar", appMasterJar); + localResources, + homeDir, + ""); // Upload the flink configuration - LocalResource flinkConf = Records.newRecord(LocalResource.class); - // write out configuration file File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); tmpConfigurationFile.deleteOnExit(); BootstrapTools.writeConfiguration(flinkConfiguration, tmpConfigurationFile); - Path remotePathConf = Utils.setupLocalResource( + Path remotePathConf = setupSingleLocalResource( + "flink-conf.yaml", fs, - appId.toString(), + appId, new Path(tmpConfigurationFile.getAbsolutePath()), - flinkConf, - fs.getHomeDirectory()); - - localResources.put("flink-conf.yaml", flinkConf); + localResources, + homeDir, + ""); paths.add(remotePathJar); classPathBuilder.append("flink.jar").append(File.pathSeparator); @@ -781,11 +794,16 @@ public ApplicationReport startAppMaster( ObjectOutputStream obOutput = new ObjectOutputStream(output);){ obOutput.writeObject(jobGraph); } - LocalResource jobgraph = Records.newRecord(LocalResource.class); - Path remoteJobGraph = - Utils.setupLocalResource(fs, appId.toString(), new Path(fp.toURI()), jobgraph, fs.getHomeDirectory()); - localResources.put("job.graph", jobgraph); - paths.add(remoteJobGraph); + + Path pathFromYarnURL = setupSingleLocalResource( + "job.graph", + fs, + appId, + new Path(fp.toURI()), + localResources, + homeDir, + ""); + paths.add(pathFromYarnURL); classPathBuilder.append("job.graph").append(File.pathSeparator); } catch (Exception e) { LOG.warn("Add job graph to local resource fail"); @@ -793,7 +811,7 @@ public ApplicationReport startAppMaster( } } - Path yarnFilesDir = new Path(fs.getHomeDirectory(), ".flink/" + appId + '/'); + Path yarnFilesDir = new Path(homeDir, ".flink/" + appId + '/'); FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); fs.setPermission(yarnFilesDir, permission); // set permission for path. @@ -810,32 +828,44 @@ public ApplicationReport startAppMaster( if (krb5Config != null && krb5Config.length() != 0) { File krb5 = new File(krb5Config); LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); - LocalResource krb5ConfResource = Records.newRecord(LocalResource.class); Path krb5ConfPath = new Path(krb5.getAbsolutePath()); - remoteKrb5Path = Utils.setupLocalResource(fs, appId.toString(), krb5ConfPath, krb5ConfResource, fs.getHomeDirectory()); - localResources.put(Utils.KRB5_FILE_NAME, krb5ConfResource); + remoteKrb5Path = setupSingleLocalResource( + Utils.KRB5_FILE_NAME, + fs, + appId, + krb5ConfPath, + localResources, + homeDir, + ""); File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); - LocalResource yarnConfResource = Records.newRecord(LocalResource.class); Path yarnSitePath = new Path(f.getAbsolutePath()); - remoteYarnSiteXmlPath = Utils.setupLocalResource(fs, appId.toString(), yarnSitePath, yarnConfResource, fs.getHomeDirectory()); - localResources.put(Utils.YARN_SITE_FILE_NAME, yarnConfResource); - + remoteYarnSiteXmlPath = setupSingleLocalResource( + Utils.YARN_SITE_FILE_NAME, + fs, + appId, + yarnSitePath, + localResources, + homeDir, + ""); hasKrb5 = true; } } // setup security tokens - LocalResource keytabResource = null; Path remotePathKeytab = null; String keytab = flinkConfiguration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); if (keytab != null) { LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); - keytabResource = Records.newRecord(LocalResource.class); - Path keytabPath = new Path(keytab); - remotePathKeytab = Utils.setupLocalResource(fs, appId.toString(), keytabPath, keytabResource, fs.getHomeDirectory()); - localResources.put(Utils.KEYTAB_FILE_NAME, keytabResource); + remotePathKeytab = setupSingleLocalResource( + Utils.KEYTAB_FILE_NAME, + fs, + appId, + new Path(keytab), + localResources, + homeDir, + ""); } final ContainerLaunchContext amContainer = setupApplicationMasterContainer( @@ -866,7 +896,7 @@ public ApplicationReport startAppMaster( appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(clusterSpecification.getTaskManagerMemoryMB())); appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, fs.getHomeDirectory().toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(clusterSpecification.getSlotsPerTaskManager())); appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached)); @@ -876,7 +906,7 @@ public ApplicationReport startAppMaster( // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); - if (keytabResource != null) { + if (remotePathKeytab != null) { appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); String principal = flinkConfiguration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); @@ -981,25 +1011,54 @@ public ApplicationReport startAppMaster( return report; } - private static List uploadAndRegisterFiles( - Collection shipFiles, + /** + * Uploads and registers a single resource and adds it to localResources. + * + * @param key + * the key to add the resource under + * @param fs + * the remote file system to upload to + * @param appId + * application ID + * @param localSrcPath + * local path to the file + * @param localResources + * map of resources + * + * @return the remote path to the uploaded resource + */ + private static Path setupSingleLocalResource( + String key, FileSystem fs, - String appId, - List remotePaths, + ApplicationId appId, + Path localSrcPath, Map localResources, - StringBuilder envShipFileList) throws IOException { - final List classPaths = new ArrayList<>(2 + shipFiles.size()); - for (File shipFile : shipFiles) { - LocalResource shipResources = Records.newRecord(LocalResource.class); + Path targetHomeDir, + String relativeTargetPath) throws IOException, URISyntaxException { - Path shipLocalPath = new Path("file://" + shipFile.getAbsolutePath()); - Path remotePath = - Utils.setupLocalResource(fs, appId, shipLocalPath, shipResources, fs.getHomeDirectory()); + Tuple2 resource = Utils.setupLocalResource( + fs, + appId.toString(), + localSrcPath, + targetHomeDir, + relativeTargetPath); - remotePaths.add(remotePath); + localResources.put(key, resource.f1); - localResources.put(shipFile.getName(), shipResources); + return resource.f0; + } + + static List uploadAndRegisterFiles( + Collection shipFiles, + FileSystem fs, + Path targetHomeDir, + ApplicationId appId, + List remotePaths, + Map localResources, + StringBuilder envShipFileList) throws IOException, URISyntaxException { + final List classPaths = new ArrayList<>(2 + shipFiles.size()); + for (File shipFile : shipFiles) { if (shipFile.isDirectory()) { // add directories to the classpath java.nio.file.Path shipPath = shipFile.toPath(); @@ -1011,17 +1070,40 @@ public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes at throws IOException { java.nio.file.Path relativePath = parentPath.relativize(file); - classPaths.add(relativePath.toString()); - - return FileVisitResult.CONTINUE; + String key = relativePath.toString(); + try { + Path remotePath = setupSingleLocalResource( + key, + fs, + appId, + new Path(file.toUri()), + localResources, + targetHomeDir, + relativePath.getParent().toString()); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=").append(remotePath).append(","); + + // add files to the classpath + classPaths.add(key); + + return FileVisitResult.CONTINUE; + } catch (URISyntaxException e) { + throw new IOException(e); + } } }); } else { + Path shipLocalPath = new Path("file://" + shipFile.getAbsolutePath()); + String key = shipFile.getName(); + Path remotePath = setupSingleLocalResource( + key, fs, appId, shipLocalPath, localResources, targetHomeDir, ""); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=").append(remotePath).append(","); + // add files to the classpath - classPaths.add(shipFile.getName()); + classPaths.add(key); } - envShipFileList.append(remotePath).append(","); } return classPaths; } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 32cbb64cb052d..652afec370e43 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; @@ -117,33 +118,60 @@ public static void setupYarnClassPath(Configuration conf, Map ap } /** + * Copy a local file to a remote file system. + * + * @param fs + * remote filesystem + * @param appId + * application ID + * @param localSrcPath + * path to the local file + * @param homedir + * remote home directory base (will be extended) + * @param relativeTargetPath + * relative target path of the file (will be prefixed be the full home directory we set up) + * * @return Path to remote file (usually hdfs) - * @throws IOException */ - public static Path setupLocalResource( - FileSystem fs, - String appId, Path localRsrcPath, - LocalResource appMasterJar, - Path homedir) throws IOException { + static Tuple2 setupLocalResource( + FileSystem fs, + String appId, + Path localSrcPath, + Path homedir, + String relativeTargetPath) throws IOException { + + if (new File(localSrcPath.toUri().getPath()).isDirectory()) { + throw new IllegalArgumentException("File to copy must not be a directory: " + + localSrcPath); + } // copy resource to HDFS - String suffix = ".flink/" + appId + "/" + localRsrcPath.getName(); + String suffix = + ".flink/" + + appId + + (relativeTargetPath.isEmpty() ? "" : "/" + relativeTargetPath) + + "/" + localSrcPath.getName(); Path dst = new Path(homedir, suffix); - LOG.info("Copying from " + localRsrcPath + " to " + dst); - fs.copyFromLocalFile(localRsrcPath, dst); - registerLocalResource(fs, dst, appMasterJar); - return dst; + LOG.info("Copying from " + localSrcPath + " to " + dst); + + fs.copyFromLocalFile(false, true, localSrcPath, dst); + + // now create the resource instance + LocalResource resource = registerLocalResource(fs, dst); + return Tuple2.of(dst, resource); } - public static void registerLocalResource(FileSystem fs, Path remoteRsrcPath, LocalResource localResource) throws IOException { + private static LocalResource registerLocalResource(FileSystem fs, Path remoteRsrcPath) throws IOException { + LocalResource localResource = Records.newRecord(LocalResource.class); FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); localResource.setResource(ConverterUtils.getYarnUrlFromURI(remoteRsrcPath.toUri())); localResource.setSize(jarStat.getLen()); localResource.setTimestamp(jarStat.getModificationTime()); localResource.setType(LocalResourceType.FILE); localResource.setVisibility(LocalResourceVisibility.APPLICATION); + return localResource; } public static void setTokensFor(ContainerLaunchContext amContainer, List paths, Configuration conf) throws IOException { @@ -340,10 +368,9 @@ static ContainerLaunchContext createTaskExecutorContext( LocalResource keytabResource = null; if (remoteKeytabPath != null) { log.info("Adding keytab {} to the AM container local resource bucket", remoteKeytabPath); - keytabResource = Records.newRecord(LocalResource.class); Path keytabPath = new Path(remoteKeytabPath); FileSystem fs = keytabPath.getFileSystem(yarnConfig); - registerLocalResource(fs, keytabPath, keytabResource); + keytabResource = registerLocalResource(fs, keytabPath); } //To support Yarn Secure Integration Test Scenario @@ -352,30 +379,28 @@ static ContainerLaunchContext createTaskExecutorContext( boolean hasKrb5 = false; if (remoteYarnConfPath != null && remoteKrb5Path != null) { log.info("TM:Adding remoteYarnConfPath {} to the container local resource bucket", remoteYarnConfPath); - yarnConfResource = Records.newRecord(LocalResource.class); Path yarnConfPath = new Path(remoteYarnConfPath); FileSystem fs = yarnConfPath.getFileSystem(yarnConfig); - registerLocalResource(fs, yarnConfPath, yarnConfResource); + yarnConfResource = registerLocalResource(fs, yarnConfPath); log.info("TM:Adding remoteKrb5Path {} to the container local resource bucket", remoteKrb5Path); - krb5ConfResource = Records.newRecord(LocalResource.class); Path krb5ConfPath = new Path(remoteKrb5Path); fs = krb5ConfPath.getFileSystem(yarnConfig); - registerLocalResource(fs, krb5ConfPath, krb5ConfResource); + krb5ConfResource = registerLocalResource(fs, krb5ConfPath); hasKrb5 = true; } // register Flink Jar with remote HDFS - LocalResource flinkJar = Records.newRecord(LocalResource.class); + final LocalResource flinkJar; { Path remoteJarPath = new Path(remoteFlinkJarPath); FileSystem fs = remoteJarPath.getFileSystem(yarnConfig); - registerLocalResource(fs, remoteJarPath, flinkJar); + flinkJar = registerLocalResource(fs, remoteJarPath); } // register conf with local fs - LocalResource flinkConf = Records.newRecord(LocalResource.class); + final LocalResource flinkConf; { // write the TaskManager configuration to a local file final File taskManagerConfigFile = @@ -385,8 +410,13 @@ static ContainerLaunchContext createTaskExecutorContext( Path homeDirPath = new Path(clientHomeDir); FileSystem fs = homeDirPath.getFileSystem(yarnConfig); - setupLocalResource(fs, appId, - new Path(taskManagerConfigFile.toURI()), flinkConf, new Path(clientHomeDir)); + + flinkConf = setupLocalResource( + fs, + appId, + new Path(taskManagerConfigFile.toURI()), + homeDirPath, + "").f1; log.info("Prepared local resource for modified yaml: {}", flinkConf); } @@ -408,10 +438,11 @@ static ContainerLaunchContext createTaskExecutorContext( // prepare additional files to be shipped for (String pathStr : shipListString.split(",")) { if (!pathStr.isEmpty()) { - LocalResource resource = Records.newRecord(LocalResource.class); - Path path = new Path(pathStr); - registerLocalResource(path.getFileSystem(yarnConfig), path, resource); - taskManagerLocalResources.put(path.getName(), resource); + String[] keyAndPath = pathStr.split("="); + require(keyAndPath.length == 2, "Invalid entry in ship file list: %s", pathStr); + Path path = new Path(keyAndPath[1]); + LocalResource resource = registerLocalResource(path.getFileSystem(yarnConfig), path); + taskManagerLocalResources.put(keyAndPath[0], resource); } } @@ -488,4 +519,5 @@ static void require(boolean condition, String message, Object... values) { throw new RuntimeException(String.format(message, values)); } } + } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java new file mode 100644 index 0000000000000..4d3825311aef3 --- /dev/null +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java @@ -0,0 +1,218 @@ +/* + * 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.flink.yarn; + +import org.apache.flink.util.OperatingSystem; +import org.apache.flink.util.TestLogger; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * Tests for verifying file staging during submission to YARN works. + */ +public class YarnFileStageTest extends TestLogger { + + @ClassRule + public static final TemporaryFolder CLASS_TEMP_DIR = new TemporaryFolder(); + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + private static MiniDFSCluster hdfsCluster; + + private static Path hdfsRootPath; + + private org.apache.hadoop.conf.Configuration hadoopConfig; + + // ------------------------------------------------------------------------ + // Test setup and shutdown + // ------------------------------------------------------------------------ + + @BeforeClass + public static void createHDFS() throws Exception { + Assume.assumeTrue(!OperatingSystem.isWindows()); + + final File tempDir = CLASS_TEMP_DIR.newFolder(); + + org.apache.hadoop.conf.Configuration hdConf = new org.apache.hadoop.conf.Configuration(); + hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath()); + + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf); + hdfsCluster = builder.build(); + hdfsRootPath = new Path(hdfsCluster.getURI()); + } + + @AfterClass + public static void destroyHDFS() { + if (hdfsCluster != null) { + hdfsCluster.shutdown(); + } + hdfsCluster = null; + hdfsRootPath = null; + } + + @Before + public void initConfig() { + hadoopConfig = new org.apache.hadoop.conf.Configuration(); + hadoopConfig.set(org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY, hdfsRootPath.toString()); + } + + /** + * Verifies that nested directories are properly copied with a hdfs:// file + * system (from a file:///absolute/path source path). + */ + @Test + public void testCopyFromLocalRecursiveWithScheme() throws Exception { + final FileSystem targetFileSystem = hdfsRootPath.getFileSystem(hadoopConfig); + final Path targetDir = targetFileSystem.getWorkingDirectory(); + + testCopyFromLocalRecursive(targetFileSystem, targetDir, tempFolder, true); + } + + /** + * Verifies that nested directories are properly copied with a hdfs:// file + * system (from a /absolute/path source path). + */ + @Test + public void testCopyFromLocalRecursiveWithoutScheme() throws Exception { + final FileSystem targetFileSystem = hdfsRootPath.getFileSystem(hadoopConfig); + final Path targetDir = targetFileSystem.getWorkingDirectory(); + + testCopyFromLocalRecursive(targetFileSystem, targetDir, tempFolder, false); + } + + /** + * Verifies that nested directories are properly copied with the given filesystem and paths. + * + * @param targetFileSystem + * file system of the target path + * @param targetDir + * target path (URI like hdfs://...) + * @param tempFolder + * JUnit temporary folder rule to create the source directory with + * @param addSchemeToLocalPath + * whether add the file:// scheme to the local path to copy from + */ + public static void testCopyFromLocalRecursive( + FileSystem targetFileSystem, + Path targetDir, + TemporaryFolder tempFolder, + boolean addSchemeToLocalPath) throws Exception { + + // directory must not yet exist + assertFalse(targetFileSystem.exists(targetDir)); + + final File srcDir = tempFolder.newFolder(); + final Path srcPath; + if (addSchemeToLocalPath) { + srcPath = new Path("file://" + srcDir.getAbsolutePath()); + } else { + srcPath = new Path(srcDir.getAbsolutePath()); + } + + HashMap srcFiles = new HashMap<>(4); + + // create and fill source files + srcFiles.put("1", "Hello 1"); + srcFiles.put("2", "Hello 2"); + srcFiles.put("nested/3", "Hello nested/3"); + srcFiles.put("nested/4/5", "Hello nested/4/5"); + for (Map.Entry src : srcFiles.entrySet()) { + File file = new File(srcDir, src.getKey()); + //noinspection ResultOfMethodCallIgnored + file.getParentFile().mkdirs(); + try (DataOutputStream out = new DataOutputStream(new FileOutputStream(file))) { + out.writeUTF(src.getValue()); + } + } + + // copy the created directory recursively: + try { + List remotePaths = new ArrayList<>(); + HashMap localResources = new HashMap<>(); + AbstractYarnClusterDescriptor.uploadAndRegisterFiles( + Collections.singletonList(new File(srcPath.toUri().getPath())), + targetFileSystem, + targetDir, + ApplicationId.newInstance(0, 0), + remotePaths, + localResources, + new StringBuilder()); + assertEquals(srcFiles.size(), localResources.size()); + + Path workDir = ConverterUtils + .getPathFromYarnURL(localResources.get(srcPath.getName() + "/1").getResource()) + .getParent(); + + RemoteIterator targetFilesIterator = + targetFileSystem.listFiles(workDir, true); + HashMap targetFiles = + new HashMap<>(4); + + final int workDirPrefixLength = + workDir.toString().length() + 1; // one more for the concluding "/" + while (targetFilesIterator.hasNext()) { + LocatedFileStatus targetFile = targetFilesIterator.next(); + + try (FSDataInputStream in = targetFileSystem.open(targetFile.getPath())) { + String absolutePathString = targetFile.getPath().toString(); + String relativePath = absolutePathString.substring(workDirPrefixLength); + targetFiles.put(relativePath, in.readUTF()); + + assertEquals("extraneous data in file " + relativePath, -1, in.read()); + } + } + + assertThat(targetFiles, equalTo(srcFiles)); + } finally { + // clean up + targetFileSystem.delete(targetDir, true); + } + } +} diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java new file mode 100644 index 0000000000000..74fb5963179d6 --- /dev/null +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java @@ -0,0 +1,220 @@ +/* + * 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.flink.yarn; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assume.assumeFalse; +import static org.junit.Assume.assumeNoException; + +/** + * Tests for verifying file staging during submission to YARN works with the S3A file system. + * + *

Note that the setup is similar to org.apache.flink.fs.s3hadoop.HadoopS3FileSystemITCase. + */ +public class YarnFileStageTestS3ITCase extends TestLogger { + + private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); + + private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); + + private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); + private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); + + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + /** + * Number of tests executed. + */ + private static int numRecursiveUploadTests = 0; + + /** + * Will be updated by {@link #checkCredentialsAndSetup()} if the test is not skipped. + */ + private static boolean skipTest = true; + + @BeforeClass + public static void checkCredentialsAndSetup() throws IOException { + // check whether credentials exist + Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); + Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); + Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + + skipTest = false; + + setupCustomHadoopConfig(); + } + + @AfterClass + public static void resetFileSystemConfiguration() throws IOException { + FileSystem.initialize(new Configuration()); + } + + @AfterClass + public static void checkAtLeastOneTestRun() { + if (!skipTest) { + assertThat( + "No S3 filesystem upload test executed. Please activate the " + + "'include_hadoop_aws' build profile or set '-Dinclude_hadoop_aws' during build " + + "(Hadoop >= 2.6 moved S3 filesystems out of hadoop-common).", + numRecursiveUploadTests, greaterThan(0)); + } + } + + /** + * Create a Hadoop config file containing S3 access credentials. + * + *

Note that we cannot use them as part of the URL since this may fail if the credentials + * contain a "/" (see HADOOP-3733). + */ + private static void setupCustomHadoopConfig() throws IOException { + File hadoopConfig = TEMP_FOLDER.newFile(); + Map parameters = new HashMap<>(); + + // set all different S3 fs implementation variants' configuration keys + parameters.put("fs.s3a.access.key", ACCESS_KEY); + parameters.put("fs.s3a.secret.key", SECRET_KEY); + + parameters.put("fs.s3.awsAccessKeyId", ACCESS_KEY); + parameters.put("fs.s3.awsSecretAccessKey", SECRET_KEY); + + parameters.put("fs.s3n.awsAccessKeyId", ACCESS_KEY); + parameters.put("fs.s3n.awsSecretAccessKey", SECRET_KEY); + + try (PrintStream out = new PrintStream(new FileOutputStream(hadoopConfig))) { + out.println(""); + out.println(""); + out.println(""); + for (Map.Entry entry : parameters.entrySet()) { + out.println("\t"); + out.println("\t\t" + entry.getKey() + ""); + out.println("\t\t" + entry.getValue() + ""); + out.println("\t"); + } + out.println(""); + } + + final Configuration conf = new Configuration(); + conf.setString(ConfigConstants.HDFS_SITE_CONFIG, hadoopConfig.getAbsolutePath()); + + FileSystem.initialize(conf); + } + + /** + * Verifies that nested directories are properly copied with to the given S3 path (using the + * appropriate file system) during resource uploads for YARN. + * + * @param scheme + * file system scheme + * @param pathSuffix + * test path suffix which will be the test's target path + */ + private void testRecursiveUploadForYarn(String scheme, String pathSuffix) throws Exception { + ++numRecursiveUploadTests; + + final Path basePath = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR); + final HadoopFileSystem fs = (HadoopFileSystem) basePath.getFileSystem(); + + assumeFalse(fs.exists(basePath)); + + try { + final Path directory = new Path(basePath, pathSuffix); + + YarnFileStageTest.testCopyFromLocalRecursive(fs.getHadoopFileSystem(), + new org.apache.hadoop.fs.Path(directory.toUri()), tempFolder, true); + + // now directory must be gone + assertFalse(fs.exists(directory)); + } finally { + // clean up + fs.delete(basePath, true); + } + } + + /** + * Verifies that nested directories are properly copied with a s3a:// file + * systems during resource uploads for YARN. + */ + @Test + public void testRecursiveUploadForYarnS3() throws Exception { + try { + Class.forName("org.apache.hadoop.fs.s3.S3FileSystem"); + } catch (ClassNotFoundException e) { + // not in the classpath, cannot run this test + String msg = "Skipping test because S3FileSystem is not in the class path"; + log.info(msg); + assumeNoException(msg, e); + } + testRecursiveUploadForYarn("s3", "testYarn-s3"); + } + + @Test + public void testRecursiveUploadForYarnS3n() throws Exception { + try { + Class.forName("org.apache.hadoop.fs.s3native.NativeS3FileSystem"); + } catch (ClassNotFoundException e) { + // not in the classpath, cannot run this test + String msg = "Skipping test because NativeS3FileSystem is not in the class path"; + log.info(msg); + assumeNoException(msg, e); + } + testRecursiveUploadForYarn("s3n", "testYarn-s3n"); + } + + @Test + public void testRecursiveUploadForYarnS3a() throws Exception { + try { + Class.forName("org.apache.hadoop.fs.s3a.S3AFileSystem"); + } catch (ClassNotFoundException e) { + // not in the classpath, cannot run this test + String msg = "Skipping test because S3AFileSystem is not in the class path"; + log.info(msg); + assumeNoException(msg, e); + } + testRecursiveUploadForYarn("s3a", "testYarn-s3a"); + } +} From abdc7d1ecdc2e997ef6c9d1764348da00148ec28 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 2 Nov 2017 19:38:48 +0100 Subject: [PATCH 075/367] [FLINK-7988][s3] fix HadoopS3FileSystemITCase leaving test directories behind in S3 This closes #4950. --- .../fs/s3hadoop/HadoopS3FileSystemITCase.java | 50 ++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java index 88f13ed6769e9..8c646f02ecdf4 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; @@ -57,11 +58,58 @@ public class HadoopS3FileSystemITCase extends TestLogger { private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); + /** + * Will be updated by {@link #checkCredentialsAndSetup()} if the test is not skipped. + */ + private static boolean skipTest = true; + @BeforeClass - public static void checkIfCredentialsArePresent() { + public static void checkCredentialsAndSetup() throws IOException { + // check whether credentials exist Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + + // initialize configuration with valid credentials + final Configuration conf = new Configuration(); + conf.setString("s3.access.key", ACCESS_KEY); + conf.setString("s3.secret.key", SECRET_KEY); + FileSystem.initialize(conf); + + // check for uniqueness of the test directory + final Path directory = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + final FileSystem fs = directory.getFileSystem(); + + // directory must not yet exist + assertFalse(fs.exists(directory)); + + // reset configuration + FileSystem.initialize(new Configuration()); + + skipTest = false; + } + + @AfterClass + public static void cleanUp() throws IOException { + if (!skipTest) { + // initialize configuration with valid credentials + final Configuration conf = new Configuration(); + conf.setString("s3.access.key", ACCESS_KEY); + conf.setString("s3.secret.key", SECRET_KEY); + FileSystem.initialize(conf); + + final Path directory = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + final FileSystem fs = directory.getFileSystem(); + + // clean up + fs.delete(directory, true); + + // now directory must be gone + assertFalse(fs.exists(directory)); + + // reset configuration + FileSystem.initialize(new Configuration()); + } } @Test From 4d559879a15e8a79a88e20ffe3a3ced901004e08 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 17 Nov 2017 10:53:31 +0800 Subject: [PATCH 076/367] [FLINK-8096] [table] Fix time attribute materialization when writing to TableSink This closes #5025. --- .../table/api/StreamTableEnvironment.scala | 39 ++++++++++++------- .../runtime/stream/TimeAttributesITCase.scala | 26 ++++++++++++- 2 files changed, 49 insertions(+), 16 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index e80acca878193..920da2ec72261 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -234,11 +234,12 @@ abstract class StreamTableEnvironment( "UpsertStreamTableSink requires that Table has a full primary keys if it is updated.") } val outputType = sink.getOutputType + val resultType = getResultType(table.getRelNode, optimizedPlan) // translate the Table into a DataStream and provide the type that the TableSink expects. val result: DataStream[T] = translate( optimizedPlan, - table.getRelNode.getRowType, + resultType, streamQueryConfig, withChangeFlag = true)(outputType) // Give the DataStream to the TableSink to emit it. @@ -254,11 +255,12 @@ abstract class StreamTableEnvironment( "AppendStreamTableSink requires that Table has only insert changes.") } val outputType = sink.getOutputType + val resultType = getResultType(table.getRelNode, optimizedPlan) // translate the Table into a DataStream and provide the type that the TableSink expects. val result: DataStream[T] = translate( optimizedPlan, - table.getRelNode.getRowType, + resultType, streamQueryConfig, withChangeFlag = false)(outputType) // Give the DataStream to the TableSink to emit it. @@ -727,19 +729,7 @@ abstract class StreamTableEnvironment( val relNode = table.getRelNode val dataStreamPlan = optimize(relNode, updatesAsRetraction) - // zip original field names with optimized field types - val fieldTypes = relNode.getRowType.getFieldList.asScala - .zip(dataStreamPlan.getRowType.getFieldList.asScala) - // get name of original plan and type of optimized plan - .map(x => (x._1.getName, x._2.getType)) - // add field indexes - .zipWithIndex - // build new field types - .map(x => new RelDataTypeFieldImpl(x._1._1, x._2, x._1._2)) - - // build a record type from list of field types - val rowType = new RelRecordType( - fieldTypes.toList.asInstanceOf[List[RelDataTypeField]].asJava) + val rowType = getResultType(relNode, dataStreamPlan) translate(dataStreamPlan, rowType, queryConfig, withChangeFlag) } @@ -851,6 +841,25 @@ abstract class StreamTableEnvironment( } } + /** + * Returns the record type of the optimized plan with field names of the logical plan. + */ + private def getResultType(originRelNode: RelNode, optimizedPlan: RelNode): RelRecordType = { + // zip original field names with optimized field types + val fieldTypes = originRelNode.getRowType.getFieldList.asScala + .zip(optimizedPlan.getRowType.getFieldList.asScala) + // get name of original plan and type of optimized plan + .map(x => (x._1.getName, x._2.getType)) + // add field indexes + .zipWithIndex + // build new field types + .map(x => new RelDataTypeFieldImpl(x._1._1, x._2, x._1._2)) + + // build a record type from list of field types + new RelRecordType( + fieldTypes.toList.asInstanceOf[List[RelDataTypeField]].asJava) + } + /** * Returns the AST of the specified Table API and SQL queries and the execution plan to compute * the result of the given [[Table]]. diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index 508660126703b..a30135492e8f2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -36,7 +36,7 @@ import org.apache.flink.table.api.{TableEnvironment, TableSchema, Types} import org.apache.flink.table.expressions.{ExpressionParser, TimeIntervalUnit} import org.apache.flink.table.runtime.stream.TimeAttributesITCase.{AtomicTimestampWithEqualWatermark, TestPojo, TimestampWithEqualWatermark, TimestampWithEqualWatermarkPojo} import org.apache.flink.table.runtime.utils.StreamITCase -import org.apache.flink.table.utils.TestTableSourceWithTime +import org.apache.flink.table.utils.{MemoryTableSinkUtil, TestTableSourceWithTime} import org.apache.flink.types.Row import org.junit.Assert._ import org.junit.Test @@ -178,6 +178,30 @@ class TimeAttributesITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + @Test + def testTableSink(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + MemoryTableSinkUtil.clear + + val stream = env + .fromCollection(data) + .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark()) + stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string) + .filter('rowtime.cast(Types.LONG) > 4) + .select('rowtime, 'rowtime.floor(TimeIntervalUnit.DAY), 'rowtime.ceil(TimeIntervalUnit.DAY)) + .writeToSink(new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink) + + env.execute() + + val expected = Seq( + "1970-01-01 00:00:00.007,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", + "1970-01-01 00:00:00.008,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", + "1970-01-01 00:00:00.016,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0") + assertEquals(expected.sorted, MemoryTableSinkUtil.results.sorted) + } + @Test def testTableFunction(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment From 3fe70d761239d03b7433976ff19e1bbb83855b37 Mon Sep 17 00:00:00 2001 From: Xpray Date: Fri, 17 Nov 2017 11:01:27 +0800 Subject: [PATCH 077/367] [FLINK-8095] [table] Introduce ProjectSetOpTransposeRule This closes #5026. --- .../table/plan/rules/FlinkRuleSets.scala | 2 + .../api/batch/table/SetOperatorsTest.scala | 58 +++++++++++++++++++ .../api/stream/table/SetOperatorsTest.scala | 29 ++++++++++ .../plan/TimeIndicatorConversionTest.scala | 16 +++-- 4 files changed, 99 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index a20d14fe5baa0..10d68814bf24e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -54,6 +54,8 @@ object FlinkRuleSets { FilterAggregateTransposeRule.INSTANCE, // push filter through set operation FilterSetOpTransposeRule.INSTANCE, + // push project through set operation + ProjectSetOpTransposeRule.INSTANCE, // aggregation and projection rules AggregateProjectMergeRule.INSTANCE, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala index 35f4429662383..929ce9c656d9e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala @@ -215,4 +215,62 @@ class SetOperatorsTest extends TableTestBase { util.verifyTable(result, expected) } + + @Test + def testProjectUnionTranspose(): Unit = { + val util = batchTestUtil() + val left = util.addTable[(Int, Long, String)]("left", 'a, 'b, 'c) + val right = util.addTable[(Int, Long, String)]("right", 'a, 'b, 'c) + + val result = left.select('a, 'b, 'c) + .unionAll(right.select('a, 'b, 'c)) + .select('b, 'c) + + val expected = binaryNode( + "DataSetUnion", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "b", "c") + ), + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "b", "c") + ), + term("union", "b", "c") + ) + + util.verifyTable(result, expected) + + } + + @Test + def testProjectMinusTranspose(): Unit = { + val util = batchTestUtil() + val left = util.addTable[(Int, Long, String)]("left", 'a, 'b, 'c) + val right = util.addTable[(Int, Long, String)]("right", 'a, 'b, 'c) + + val result = left.select('a, 'b, 'c) + .minusAll(right.select('a, 'b, 'c)) + .select('b, 'c) + + val expected = binaryNode( + "DataSetMinus", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "b", "c") + ), + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "b", "c") + ), + term("minus", "b", "c") + ) + + util.verifyTable(result, expected) + + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala index b1b700bb522d6..c0fc05b53ee34 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala @@ -65,4 +65,33 @@ class SetOperatorsTest extends TableTestBase { util.verifyTable(result, expected) } + + @Test + def testProjectUnionTranspose(): Unit = { + val util = streamTestUtil() + val left = util.addTable[(Int, Long, String)]("left", 'a, 'b, 'c) + val right = util.addTable[(Int, Long, String)]("right", 'a, 'b, 'c) + + val result = left.select('a, 'b, 'c) + .unionAll(right.select('a, 'b, 'c)) + .select('b, 'c) + + val expected = binaryNode( + "DataStreamUnion", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "b", "c") + ), + unaryNode( + "DataStreamCalc", + streamTableNode(1), + term("select", "b", "c") + ), + term("union all", "b", "c") + ) + + util.verifyTable(result, expected) + } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala index 009ae40000f13..faca7f995bd51 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala @@ -209,15 +209,19 @@ class TimeIndicatorConversionTest extends TableTestBase { val result = t.unionAll(t).select('rowtime) - val expected = unaryNode( - "DataStreamCalc", - binaryNode( - "DataStreamUnion", + val expected = binaryNode( + "DataStreamUnion", + unaryNode( + "DataStreamCalc", streamTableNode(0), + term("select", "rowtime") + ), + unaryNode( + "DataStreamCalc", streamTableNode(0), - term("union all", "rowtime", "long", "int") + term("select", "rowtime") ), - term("select", "rowtime") + term("union all", "rowtime") ) util.verifyTable(result, expected) From 1a6121a69df86aecd5cc252b3f6142394ef68047 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 17 Nov 2017 14:31:07 +0100 Subject: [PATCH 078/367] [hotfix][kafka] Improve logging in FlinkKafkaProducer011 --- .../streaming/connectors/kafka/FlinkKafkaProducer011.java | 7 ++++++- .../connectors/kafka/internal/FlinkKafkaProducer.java | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 08599d82e3914..611a3d5d23c28 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -1022,7 +1022,12 @@ static class KafkaTransactionState { @Override public String toString() { - return String.format("%s [transactionalId=%s]", this.getClass().getSimpleName(), transactionalId); + return String.format( + "%s [transactionalId=%s, producerId=%s, epoch=%s]", + this.getClass().getSimpleName(), + transactionalId, + producerId, + epoch); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java index 9d50379b12a61..2f58d5685f832 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java @@ -188,7 +188,7 @@ public void flush() { */ public void resumeTransaction(long producerId, short epoch) { Preconditions.checkState(producerId >= 0 && epoch >= 0, "Incorrect values for producerId {} and epoch {}", producerId, epoch); - LOG.info("Attempting to resume transaction with producerId {} and epoch {}", producerId, epoch); + LOG.info("Attempting to resume transaction {} with producerId {} and epoch {}", transactionalId, producerId, epoch); Object transactionManager = getValue(kafkaProducer, "transactionManager"); synchronized (transactionManager) { From 2f3f8c773d50cca25a0c304c8348ac4db362b136 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 17 Nov 2017 14:40:30 +0100 Subject: [PATCH 079/367] [FLINK-8086][kafka] Ignore ProducerFencedException during recovery ProducerFencedException can happen if we restore twice from the same checkpoint or if we restore from an old savepoint. In both cases transactional.ids that we want to recoverAndCommit have been already committed and reused. Reusing mean that they will be known by Kafka's brokers under newer producerId/epochId, which will result in ProducerFencedException if we try to commit again some old (and already committed) transaction. Ignoring this exception might hide some bugs/issues, because instead of failing we might have a semi silent (with a warning) data loss. --- .../kafka/FlinkKafkaProducer011.java | 20 ++++---- .../kafka/FlinkKafkaProducer011ITCase.java | 47 +++++++++++++++++++ 2 files changed, 56 insertions(+), 11 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 611a3d5d23c28..6b0136d5f3b23 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -59,6 +59,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.errors.InvalidTxnStateException; +import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -678,14 +679,12 @@ protected void commit(KafkaTransactionState transaction) { protected void recoverAndCommit(KafkaTransactionState transaction) { switch (semantic) { case EXACTLY_ONCE: - FlinkKafkaProducer producer = - initTransactionalProducer(transaction.transactionalId, false); - producer.resumeTransaction(transaction.producerId, transaction.epoch); - try { + try (FlinkKafkaProducer producer = + initTransactionalProducer(transaction.transactionalId, false)) { + producer.resumeTransaction(transaction.producerId, transaction.epoch); producer.commitTransaction(); - producer.close(); } - catch (InvalidTxnStateException ex) { + catch (InvalidTxnStateException | ProducerFencedException ex) { // That means we have committed this transaction before. LOG.warn("Encountered error {} while recovering transaction {}. " + "Presumably this transaction has been already committed before", @@ -720,11 +719,10 @@ protected void abort(KafkaTransactionState transaction) { protected void recoverAndAbort(KafkaTransactionState transaction) { switch (semantic) { case EXACTLY_ONCE: - FlinkKafkaProducer producer = - initTransactionalProducer(transaction.transactionalId, false); - producer.resumeTransaction(transaction.producerId, transaction.epoch); - producer.abortTransaction(); - producer.close(); + try (FlinkKafkaProducer producer = + initTransactionalProducer(transaction.transactionalId, false)) { + producer.initTransactions(); + } break; case AT_LEAST_ONCE: case NONE: diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index 922344d18b947..07acd4f6c5dec 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -219,6 +219,53 @@ public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { deleteTestTopic(topic); } + @Test(timeout = 120_000L) + public void testFailAndRecoverSameCheckpointTwice() throws Exception { + String topic = "flink-kafka-producer-fail-and-recover-same-checkpoint-twice"; + + OperatorStateHandles snapshot1; + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic)) { + testHarness.setup(); + testHarness.open(); + testHarness.processElement(42, 0); + testHarness.snapshot(0, 1); + testHarness.processElement(43, 2); + snapshot1 = testHarness.snapshot(1, 3); + + testHarness.processElement(44, 4); + } + + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic)) { + testHarness.setup(); + // restore from snapshot1, transactions with records 44 and 45 should be aborted + testHarness.initializeState(snapshot1); + testHarness.open(); + + // write and commit more records, after potentially lingering transactions + testHarness.processElement(44, 7); + testHarness.snapshot(2, 8); + testHarness.processElement(45, 9); + } + + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic)) { + testHarness.setup(); + // restore from snapshot1, transactions with records 44 and 45 should be aborted + testHarness.initializeState(snapshot1); + testHarness.open(); + + // write and commit more records, after potentially lingering transactions + testHarness.processElement(44, 7); + testHarness.snapshot(3, 8); + testHarness.processElement(45, 9); + } + + //now we should have: + // - records 42 and 43 in committed transactions + // - aborted transactions with records 44 and 45 + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + deleteTestTopic(topic); + } + /** * This tests checks whether FlinkKafkaProducer011 correctly aborts lingering transactions after a failure, * which happened before first checkpoint and was followed up by reducing the parallelism. From d0ea9d95109a7e8ca7eb81d42f72c7f132192321 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 17 Nov 2017 17:19:51 +0100 Subject: [PATCH 080/367] [FLINK-8099] Reduce default restart delay to 1 second --- docs/ops/config.md | 4 +--- .../flink/configuration/ConfigConstants.java | 3 ++- .../restart/FixedDelayRestartStrategy.java | 21 ++++--------------- 3 files changed, 7 insertions(+), 21 deletions(-) diff --git a/docs/ops/config.md b/docs/ops/config.md index 55dec4cd3fc01..c85ce84a3c98e 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -231,9 +231,7 @@ The options are: - `restart-strategy.fixed-delay.attempts`: Number of restart attempts, used if the default restart strategy is set to "fixed-delay". Default value is 1, unless "fixed-delay" was activated by enabling checkpoints, in which case the default is `Integer.MAX_VALUE`. -- `restart-strategy.fixed-delay.delay`: Delay between restart attempts, used if the default restart strategy is set to "fixed-delay". -Default value is the `akka.ask.timeout`, unless "fixed-delay" was activated by enabling checkpoints, in which case -the default is 10s. +- `restart-strategy.fixed-delay.delay`: Delay between restart attempts, used if the default restart strategy is set to "fixed-delay". (default: `1 s`) - `restart-strategy.failure-rate.max-failures-per-interval`: Maximum number of restarts in given time interval before failing a job in "failure-rate" strategy. Default value is 1. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index f9e473578c536..fcf73b8809d44 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -67,7 +67,8 @@ public final class ConfigConstants { * FiniteDuration notation: "1 min", "20 s" */ @PublicEvolving - public static final String RESTART_STRATEGY_FIXED_DELAY_DELAY = "restart-strategy.fixed-delay.delay"; + public static final ConfigOption RESTART_STRATEGY_FIXED_DELAY_DELAY = + key("restart-strategy.fixed-delay.delay").defaultValue("1 s"); /** * Maximum number of restarts in given time interval {@link #RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL} before failing a job diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java index ca9626a7983b9..1916bea83586c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.executiongraph.restart; -import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -81,28 +80,16 @@ public void run() { public static FixedDelayRestartStrategyFactory createFactory(Configuration configuration) throws Exception { int maxAttempts = configuration.getInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); - String timeoutString = configuration.getString( - AkkaOptions.WATCH_HEARTBEAT_INTERVAL); - - String delayString = configuration.getString( - ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, - timeoutString - ); + String delayString = configuration.getString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY); long delay; try { delay = Duration.apply(delayString).toMillis(); } catch (NumberFormatException nfe) { - if (delayString.equals(timeoutString)) { - throw new Exception("Invalid config value for " + - AkkaOptions.WATCH_HEARTBEAT_INTERVAL.key() + ": " + timeoutString + - ". Value must be a valid duration (such as '10 s' or '1 min')"); - } else { - throw new Exception("Invalid config value for " + - ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY + ": " + delayString + - ". Value must be a valid duration (such as '100 milli' or '10 s')"); - } + throw new Exception("Invalid config value for " + + ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY + ": " + delayString + + ". Value must be a valid duration (such as '100 milli' or '10 s')"); } return new FixedDelayRestartStrategyFactory(maxAttempts, delay); From 8eadda357c3739af84070cec5a8c227b55b04390 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 20 Nov 2017 12:57:43 +0100 Subject: [PATCH 081/367] [FLINK-8108][py] Fix bounds check --- .../org/apache/flink/python/api/PythonPlanBinder.java | 9 ++++----- .../apache/flink/python/api/PythonPlanBinderTest.java | 10 ++++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java index 810c8cdfb5b54..b7adde1c4eb09 100644 --- a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java +++ b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java @@ -91,11 +91,6 @@ public class PythonPlanBinder { * @throws Exception */ public static void main(String[] args) throws Exception { - if (args.length < 2) { - System.out.println("Usage: ./bin/pyflink<2/3>.[sh/bat] [ [ [ ]]"); - return; - } - Configuration globalConfig = GlobalConfiguration.loadConfiguration(); PythonPlanBinder binder = new PythonPlanBinder(globalConfig); binder.runPlan(args); @@ -126,6 +121,10 @@ public PythonPlanBinder(Configuration globalConfig) { } void runPlan(String[] args) throws Exception { + if (args.length < 1) { + throw new IllegalArgumentException("Missing script file argument. Usage: ./bin/pyflink.[sh/bat] [ [ [ ]]"); + } + int split = 0; for (int x = 0; x < args.length; x++) { if (args[x].equals("-")) { diff --git a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java index 2a19a5f1e4eb8..9e63091ff1dda 100644 --- a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java +++ b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java @@ -118,6 +118,7 @@ private static String getPython3Path() throws IOException { @Override protected void testProgram() throws Exception { + testBoundCheck(); String utils = findUtilsFile(); String python2 = getPython2Path(); if (python2 != null) { @@ -136,4 +137,13 @@ protected void testProgram() throws Exception { } } } + + private void testBoundCheck() throws Exception { + log.info("Running testBoundCheck."); + try { + new PythonPlanBinder(new Configuration()).runPlan(new String[0]); + } catch (IllegalArgumentException expected) { + // we expect this exception to be thrown since no argument was passed + } + } } From 497c36f0d25c0277832a682226f6f08ba7c83635 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 20 Nov 2017 12:58:27 +0100 Subject: [PATCH 082/367] [FLINK-8109][py] Check for existence of plan/additional files --- .../flink/python/api/PythonPlanBinder.java | 21 +++++++- .../python/api/PythonPlanBinderTest.java | 53 +++++++++++++++---- 2 files changed, 61 insertions(+), 13 deletions(-) diff --git a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java index b7adde1c4eb09..e0c8215fa276f 100644 --- a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java +++ b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java @@ -50,6 +50,7 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.Arrays; import java.util.UUID; @@ -93,7 +94,12 @@ public class PythonPlanBinder { public static void main(String[] args) throws Exception { Configuration globalConfig = GlobalConfiguration.loadConfiguration(); PythonPlanBinder binder = new PythonPlanBinder(globalConfig); - binder.runPlan(args); + try { + binder.runPlan(args); + } catch (Exception e) { + System.out.println("Failed to run plan: " + e.getMessage()); + LOG.error("Failed to run plan.", e); + } } public PythonPlanBinder(Configuration globalConfig) { @@ -146,11 +152,22 @@ void runPlan(String[] args) throws Exception { operatorConfig.setString(PLAN_ARGUMENTS_KEY, planArguments); + Path planPath = new Path(planFile); + if (!FileSystem.getUnguardedFileSystem(planPath.toUri()).exists(planPath)) { + throw new FileNotFoundException("Plan file " + planFile + " does not exist."); + } + for (String file : filesToCopy) { + Path filePath = new Path(file); + if (!FileSystem.getUnguardedFileSystem(filePath.toUri()).exists(filePath)) { + throw new FileNotFoundException("Additional file " + file + " does not exist."); + } + } + // copy flink library, plan file and additional files to temporary location Path tmpPlanFilesPath = new Path(tmpPlanFilesDir); deleteIfExists(tmpPlanFilesPath); FileCache.copy(new Path(pythonLibraryPath), tmpPlanFilesPath, false); - copyFile(new Path(planFile), tmpPlanFilesPath, FLINK_PYTHON_PLAN_NAME); + copyFile(planPath, tmpPlanFilesPath, FLINK_PYTHON_PLAN_NAME); for (String file : filesToCopy) { Path source = new Path(file); copyFile(source, tmpPlanFilesPath, source.getName()); diff --git a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java index 9e63091ff1dda..55cf1dc48610a 100644 --- a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java +++ b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java @@ -21,6 +21,7 @@ import org.apache.flink.test.util.JavaProgramTestBase; import java.io.BufferedReader; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; import java.util.ArrayList; @@ -119,22 +120,52 @@ private static String getPython3Path() throws IOException { @Override protected void testProgram() throws Exception { testBoundCheck(); - String utils = findUtilsFile(); + testNotExistingPlanFile(); + testNotExistingAdditionalFile(); String python2 = getPython2Path(); if (python2 != null) { - for (String file : findTestFiles()) { - Configuration configuration = new Configuration(); - configuration.setString(PythonOptions.PYTHON_BINARY_PATH, python2); - new PythonPlanBinder(configuration).runPlan(new String[]{file, utils}); - } + log.info("Running python2 tests"); + runTestPrograms(python2); } String python3 = getPython3Path(); if (python3 != null) { - for (String file : findTestFiles()) { - Configuration configuration = new Configuration(); - configuration.setString(PythonOptions.PYTHON_BINARY_PATH, python3); - new PythonPlanBinder(configuration).runPlan(new String[]{file, utils}); - } + log.info("Running python3 tests"); + runTestPrograms(python3); + } + } + + private void runTestPrograms(String pythonBinary) throws Exception { + String utils = findUtilsFile(); + for (String file : findTestFiles()) { + log.info("Running file {}.", file); + Configuration configuration = new Configuration(); + configuration.setString(PythonOptions.PYTHON_BINARY_PATH, pythonBinary); + new PythonPlanBinder(configuration).runPlan(new String[]{file, utils}); + } + } + + private void testNotExistingPlanFile() throws Exception { + log.info("Running testNotExistingPlanFile."); + String utils = findUtilsFile(); + String nonExistingPlan = utils + "abc"; + Configuration configuration = new Configuration(); + try { + new PythonPlanBinder(configuration).runPlan(new String[]{nonExistingPlan}); + } catch (FileNotFoundException expected) { + // we expect this exception to be thrown since the plan file does not exist + } + } + + private void testNotExistingAdditionalFile() throws Exception { + log.info("Running testNotExistingAdditionalFile."); + String utils = findUtilsFile(); + String planFile = findTestFiles().iterator().next(); + String nonExistingLibrary = utils + "abc"; + Configuration configuration = new Configuration(); + try { + new PythonPlanBinder(configuration).runPlan(new String[]{planFile, utils, nonExistingLibrary}); + } catch (FileNotFoundException expected) { + // we expect this exception to be thrown since the plan file does not exist } } From ee4420f4b879ab385a4458c38af762ac0738148c Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 20 Nov 2017 15:07:32 +0100 Subject: [PATCH 083/367] [FLINK-8114][py] Fix forwarding of arguments --- .../api/streaming/data/PythonStreamer.java | 3 +- .../python/api/PythonPlanBinderTest.java | 36 +++++++++++++++---- .../flink/python/api/args/multiple_args.py | 32 +++++++++++++++++ .../apache/flink/python/api/args/no_arg.py | 32 +++++++++++++++++ 4 files changed, 96 insertions(+), 7 deletions(-) create mode 100644 flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/multiple_args.py create mode 100644 flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/no_arg.py diff --git a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/data/PythonStreamer.java b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/data/PythonStreamer.java index 3fec94720d516..864ea30f7c69a 100644 --- a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/data/PythonStreamer.java +++ b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/data/PythonStreamer.java @@ -117,7 +117,8 @@ private void startPython() throws IOException { String pythonBinaryPath = config.getString(PythonOptions.PYTHON_BINARY_PATH); - process = Runtime.getRuntime().exec(new String[] {pythonBinaryPath, "-O", "-B", planPath, config.getString(PLAN_ARGUMENTS_KEY, "")}); + String arguments = config.getString(PLAN_ARGUMENTS_KEY, ""); + process = Runtime.getRuntime().exec(pythonBinaryPath + " -O -B " + planPath + arguments); outPrinter = new Thread(new StreamPrinter(process.getInputStream())); outPrinter.start(); errorPrinter = new Thread(new StreamPrinter(process.getErrorStream(), msg)); diff --git a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java index 55cf1dc48610a..92a985c36cc32 100644 --- a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java +++ b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java @@ -37,18 +37,19 @@ protected boolean skipCollectionExecution() { return true; } - private static String findUtilsFile() throws Exception { + private static Path getBaseTestPythonDir() { FileSystem fs = FileSystem.getLocalFileSystem(); - return fs.getWorkingDirectory().toString() - + "/src/test/python/org/apache/flink/python/api/utils/utils.py"; + return new Path(fs.getWorkingDirectory(), "src/test/python/org/apache/flink/python/api"); + } + + private static String findUtilsFile() throws Exception { + return new Path(getBaseTestPythonDir(), "utils/utils.py").toString(); } private static List findTestFiles() throws Exception { List files = new ArrayList<>(); FileSystem fs = FileSystem.getLocalFileSystem(); - FileStatus[] status = fs.listStatus( - new Path(fs.getWorkingDirectory().toString() - + "/src/test/python/org/apache/flink/python/api")); + FileStatus[] status = fs.listStatus(getBaseTestPythonDir()); for (FileStatus f : status) { String file = f.getPath().toString(); if (file.endsWith(".py")) { @@ -126,11 +127,13 @@ protected void testProgram() throws Exception { if (python2 != null) { log.info("Running python2 tests"); runTestPrograms(python2); + runArgvTestPrograms(python2); } String python3 = getPython3Path(); if (python3 != null) { log.info("Running python3 tests"); runTestPrograms(python3); + runArgvTestPrograms(python3); } } @@ -177,4 +180,25 @@ private void testBoundCheck() throws Exception { // we expect this exception to be thrown since no argument was passed } } + + private void runArgvTestPrograms(String pythonBinary) throws Exception { + log.info("Running runArgvTestPrograms."); + String utils = findUtilsFile(); + + { + String noArgTestPath = new Path(getBaseTestPythonDir(), "args/no_arg.py").toString(); + + Configuration configuration = new Configuration(); + configuration.setString(PythonOptions.PYTHON_BINARY_PATH, pythonBinary); + new PythonPlanBinder(configuration).runPlan(new String[]{noArgTestPath, utils}); + } + + { + String multiArgTestPath = new Path(getBaseTestPythonDir(), "args/multiple_args.py").toString(); + + Configuration configuration = new Configuration(); + configuration.setString(PythonOptions.PYTHON_BINARY_PATH, pythonBinary); + new PythonPlanBinder(configuration).runPlan(new String[]{multiArgTestPath, utils, "-", "hello", "world"}); + } + } } diff --git a/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/multiple_args.py b/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/multiple_args.py new file mode 100644 index 0000000000000..57b44c35cb7ea --- /dev/null +++ b/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/multiple_args.py @@ -0,0 +1,32 @@ +# ############################################################################### +# 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 flink.plan.Environment import get_environment +import sys +from utils import Verify + +if __name__ == "__main__": + env = get_environment() + + d1 = env.from_elements(len(sys.argv)) + + d1.map_partition(Verify([3], "MultipleArguments")).output() + + #Execution + env.set_parallelism(1) + + env.execute(local=True) diff --git a/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/no_arg.py b/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/no_arg.py new file mode 100644 index 0000000000000..6afe7f2da8cb7 --- /dev/null +++ b/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/args/no_arg.py @@ -0,0 +1,32 @@ +# ############################################################################### +# 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 flink.plan.Environment import get_environment +import sys +from utils import Verify + +if __name__ == "__main__": + env = get_environment() + + d1 = env.from_elements(len(sys.argv)) + + d1.map_partition(Verify([1], "NoArgument")).output() + + #Execution + env.set_parallelism(1) + + env.execute(local=True) From 257032807e62879581508a353cf8fbb6f6076543 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 20 Nov 2017 13:58:21 +0100 Subject: [PATCH 084/367] [FLINK-8110][dist] Relocate jackson services in flink-dist --- flink-dist/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 6696294f405b9..95b244ca20e93 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -462,6 +462,11 @@ under the License. org.apache.maven.plugins maven-shade-plugin + + + + 3.0.0 + package @@ -496,6 +501,11 @@ under the License. org.codehaus.jackson org.apache.flink.formats.avro.shaded.org.codehaus.jackson + + + com.fasterxml.jackson + org.apache.flink.shaded.jackson2.com.fasterxml.jackson + From 2b4f1b1168e074be7ec6f57800877cccd1aef28e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 30 Oct 2017 14:15:20 +0100 Subject: [PATCH 085/367] [FLINK-7943] Make ParameterTool thread safe This commit changes the serialization of the ParameterTool such that only the data map is contained. The defaultData and the unrequestedParameters maps are not serialized because they are only used on the client side. Additionally, the defaultData and unrequestedParameters map are being made thread safe by using ConcurrentHashMaps. This closes #4921. --- .../flink/api/common/ExecutionConfigTest.java | 3 +- .../flink/api/java/utils/ParameterTool.java | 64 ++++++++++++-- .../api/java/utils/RequiredParameters.java | 22 +++-- .../api/java/utils/ParameterToolTest.java | 85 +++++++++++++++++++ .../java/utils/RequiredParametersTest.java | 18 ++-- 5 files changed, 167 insertions(+), 25 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/api/common/ExecutionConfigTest.java b/flink-core/src/test/java/org/apache/flink/api/common/ExecutionConfigTest.java index 7e98604c017e1..f9beb402bb554 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/ExecutionConfigTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/ExecutionConfigTest.java @@ -24,6 +24,7 @@ import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -37,7 +38,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class ExecutionConfigTest { +public class ExecutionConfigTest extends TestLogger { @Test public void testDoubleTypeRegistration() { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java index 894b66d0bbb17..e42a4b7c7c0e3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java @@ -32,6 +32,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.ObjectInputStream; import java.io.OutputStream; import java.io.Serializable; import java.util.Arrays; @@ -39,8 +40,10 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; /** * This class provides simple utility methods for reading and parsing program arguments from different sources. @@ -212,13 +215,38 @@ public static ParameterTool fromSystemProperties() { // ------------------ ParameterUtil ------------------------ protected final Map data; - protected final Map defaultData; - protected final Set unrequestedParameters; + + // data which is only used on the client and does not need to be transmitted + protected transient Map defaultData; + protected transient Set unrequestedParameters; private ParameterTool(Map data) { - this.data = new HashMap<>(data); - this.defaultData = new HashMap<>(); - this.unrequestedParameters = new HashSet<>(data.keySet()); + this.data = Collections.unmodifiableMap(new HashMap<>(data)); + + this.defaultData = new ConcurrentHashMap<>(data.size()); + + this.unrequestedParameters = Collections.newSetFromMap(new ConcurrentHashMap<>(data.size())); + + unrequestedParameters.addAll(data.keySet()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ParameterTool that = (ParameterTool) o; + return Objects.equals(data, that.data) && + Objects.equals(defaultData, that.defaultData) && + Objects.equals(unrequestedParameters, that.unrequestedParameters); + } + + @Override + public int hashCode() { + return Objects.hash(data, defaultData, unrequestedParameters); } /** @@ -560,9 +588,21 @@ protected Object clone() throws CloneNotSupportedException { * @return The Merged {@link ParameterTool} */ public ParameterTool mergeWith(ParameterTool other) { - ParameterTool ret = new ParameterTool(this.data); - ret.data.putAll(other.data); - ret.unrequestedParameters.addAll(other.unrequestedParameters); + Map resultData = new HashMap<>(data.size() + other.data.size()); + resultData.putAll(data); + resultData.putAll(other.data); + + ParameterTool ret = new ParameterTool(resultData); + + final HashSet requestedParametersLeft = new HashSet<>(data.keySet()); + requestedParametersLeft.removeAll(unrequestedParameters); + + final HashSet requestedParametersRight = new HashSet<>(other.data.keySet()); + requestedParametersRight.removeAll(other.unrequestedParameters); + + ret.unrequestedParameters.removeAll(requestedParametersLeft); + ret.unrequestedParameters.removeAll(requestedParametersRight); + return ret; } @@ -573,4 +613,12 @@ public Map toMap() { return data; } + // ------------------------- Serialization --------------------------------------------- + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + defaultData = Collections.emptyMap(); + unrequestedParameters = Collections.emptySet(); + } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/RequiredParameters.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/RequiredParameters.java index 1d2e73a109f3f..676a4726deb83 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/RequiredParameters.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/RequiredParameters.java @@ -83,24 +83,28 @@ public void add(Option option) throws RequiredParametersException { *

If any check fails, a RequiredParametersException is thrown * * @param parameterTool - parameters supplied by the user. + * @return the updated ParameterTool containing all the required parameters * @throws RequiredParametersException if any of the specified checks fail */ - public void applyTo(ParameterTool parameterTool) throws RequiredParametersException { + public ParameterTool applyTo(ParameterTool parameterTool) throws RequiredParametersException { List missingArguments = new LinkedList<>(); + + HashMap newParameters = new HashMap<>(parameterTool.toMap()); + for (Option o : data.values()) { - if (parameterTool.data.containsKey(o.getName())) { - if (Objects.equals(parameterTool.data.get(o.getName()), ParameterTool.NO_VALUE_KEY)) { + if (newParameters.containsKey(o.getName())) { + if (Objects.equals(newParameters.get(o.getName()), ParameterTool.NO_VALUE_KEY)) { // the parameter has been passed, but no value, check if there is a default value - checkAndApplyDefaultValue(o, parameterTool.data); + checkAndApplyDefaultValue(o, newParameters); } else { // a value has been passed in the parameterTool, now check if it adheres to all constraints - checkAmbiguousValues(o, parameterTool.data); - checkIsCastableToDefinedType(o, parameterTool.data); - checkChoices(o, parameterTool.data); + checkAmbiguousValues(o, newParameters); + checkIsCastableToDefinedType(o, newParameters); + checkChoices(o, newParameters); } } else { // check if there is a default name or a value passed for a possibly defined alternative name. - if (hasNoDefaultValueAndNoValuePassedOnAlternativeName(o, parameterTool.data)) { + if (hasNoDefaultValueAndNoValuePassedOnAlternativeName(o, newParameters)) { missingArguments.add(o.getName()); } } @@ -108,6 +112,8 @@ public void applyTo(ParameterTool parameterTool) throws RequiredParametersExcept if (!missingArguments.isEmpty()) { throw new RequiredParametersException(this.missingArgumentsText(missingArguments), missingArguments); } + + return ParameterTool.fromMap(newParameters); } // check if the given parameter has a default value and add it to the passed map if that is the case diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java index 6ad2022cf5d95..ccd472baabad6 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java @@ -23,16 +23,29 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; /** * Tests for {@link ParameterTool}. @@ -574,6 +587,78 @@ public void testUnrequestedUnknown() { Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); } + /** + * Tests that we can concurrently serialize and access the ParameterTool. See FLINK-7943 + */ + @Test + public void testConcurrentExecutionConfigSerialization() throws ExecutionException, InterruptedException { + + final int numInputs = 10; + Collection input = new ArrayList<>(numInputs); + + for (int i = 0; i < numInputs; i++) { + input.add("--" + UUID.randomUUID()); + input.add(UUID.randomUUID().toString()); + } + + final String[] args = input.toArray(new String[0]); + + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + final int numThreads = 5; + final int numSerializations = 100; + + final Collection> futures = new ArrayList<>(numSerializations); + + final ExecutorService executorService = Executors.newFixedThreadPool(numThreads); + + try { + for (int i = 0; i < numSerializations; i++) { + futures.add( + CompletableFuture.runAsync( + () -> { + try { + serializeDeserialize(parameterTool); + } catch (Exception e) { + throw new CompletionException(e); + } + }, + executorService)); + } + + for (CompletableFuture future : futures) { + future.get(); + } + } finally { + executorService.shutdownNow(); + executorService.awaitTermination(1000L, TimeUnit.MILLISECONDS); + } + } + + /** + * Accesses parameter tool parameters and then serializes the given parameter tool and deserializes again. + * @param parameterTool to serialize/deserialize + */ + private void serializeDeserialize(ParameterTool parameterTool) throws IOException, ClassNotFoundException { + // weirdly enough, this call has side effects making the ParameterTool serialization fail if not + // using a concurrent data structure. + parameterTool.get(UUID.randomUUID().toString()); + + try ( + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos)) { + oos.writeObject(parameterTool); + oos.close(); + baos.close(); + + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + ObjectInputStream ois = new ObjectInputStream(bais); + + // this should work :-) + ParameterTool deserializedParameterTool = ((ParameterTool) ois.readObject()); + } + } + private static Set createHashSet(T... elements) { Set set = new HashSet<>(); for (T element : elements) { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/RequiredParametersTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/RequiredParametersTest.java index 11d1267ca1add..e8273ef5643ab 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/RequiredParametersTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/RequiredParametersTest.java @@ -18,6 +18,8 @@ package org.apache.flink.api.java.utils; +import org.apache.flink.util.TestLogger; + import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -33,7 +35,7 @@ /** * Tests for RequiredParameter class and its interactions with ParameterTool. */ -public class RequiredParametersTest { +public class RequiredParametersTest extends TestLogger { @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -122,7 +124,7 @@ public void testApplyToMovesValuePassedOnShortNameToLongNameIfLongNameIsUndefine try { required.add(new Option("berlin").alt("b")); - required.applyTo(parameter); + parameter = required.applyTo(parameter); Assert.assertEquals(parameter.data.get("berlin"), "value"); Assert.assertEquals(parameter.data.get("b"), "value"); } catch (RequiredParametersException e) { @@ -137,7 +139,7 @@ public void testDefaultValueDoesNotOverrideValuePassedOnShortKeyIfLongKeyIsNotPa try { required.add(new Option("berlin").alt("b").defaultValue("something")); - required.applyTo(parameter); + parameter = required.applyTo(parameter); Assert.assertEquals(parameter.data.get("berlin"), "value"); Assert.assertEquals(parameter.data.get("b"), "value"); } catch (RequiredParametersException e) { @@ -164,7 +166,7 @@ public void testApplyToWithSimpleOption() { RequiredParameters required = new RequiredParameters(); try { required.add(new Option("berlin")); - required.applyTo(parameter); + parameter = required.applyTo(parameter); Assert.assertEquals(parameter.data.get("berlin"), "value"); } catch (RequiredParametersException e) { fail("Exception thrown " + e.getMessage()); @@ -177,7 +179,7 @@ public void testApplyToWithOptionAndDefaultValue() { RequiredParameters required = new RequiredParameters(); try { required.add(new Option("berlin").defaultValue("value")); - required.applyTo(parameter); + parameter = required.applyTo(parameter); Assert.assertEquals(parameter.data.get("berlin"), "value"); } catch (RequiredParametersException e) { fail("Exception thrown " + e.getMessage()); @@ -190,7 +192,7 @@ public void testApplyToWithOptionWithLongAndShortNameAndDefaultValue() { RequiredParameters required = new RequiredParameters(); try { required.add(new Option("berlin").alt("b").defaultValue("value")); - required.applyTo(parameter); + parameter = required.applyTo(parameter); Assert.assertEquals(parameter.data.get("berlin"), "value"); Assert.assertEquals(parameter.data.get("b"), "value"); } catch (RequiredParametersException e) { @@ -205,7 +207,7 @@ public void testApplyToWithOptionMultipleOptionsAndOneDefaultValue() { try { rq.add("input"); rq.add(new Option("parallelism").alt("p").defaultValue("1").type(OptionType.INTEGER)); - rq.applyTo(parameter); + parameter = rq.applyTo(parameter); Assert.assertEquals(parameter.data.get("parallelism"), "1"); Assert.assertEquals(parameter.data.get("p"), "1"); Assert.assertEquals(parameter.data.get("input"), "abc"); @@ -223,7 +225,7 @@ public void testApplyToWithMultipleTypes() { required.add(new Option("count").defaultValue("15")); required.add(new Option("someFlag").alt("sf").defaultValue("true")); - required.applyTo(parameter); + parameter = required.applyTo(parameter); Assert.assertEquals(parameter.data.get("berlin"), "value"); Assert.assertEquals(parameter.data.get("count"), "15"); From f0012645986be200f248e9c469fe69e81bbe8d4b Mon Sep 17 00:00:00 2001 From: yew1eb Date: Wed, 11 Oct 2017 02:52:35 +0800 Subject: [PATCH 086/367] [hotfix][license] Add missing licenses This close #4794. --- flink-dist/src/main/flink-bin/conf/zoo.cfg | 18 ++++++++++++++++++ .../assets/images/browserconfig.xml | 19 +++++++++++++++++++ .../web/images/browserconfig.xml | 19 +++++++++++++++++++ pom.xml | 2 -- 4 files changed, 56 insertions(+), 2 deletions(-) diff --git a/flink-dist/src/main/flink-bin/conf/zoo.cfg b/flink-dist/src/main/flink-bin/conf/zoo.cfg index a14ec66745b9c..f59899746c367 100644 --- a/flink-dist/src/main/flink-bin/conf/zoo.cfg +++ b/flink-dist/src/main/flink-bin/conf/zoo.cfg @@ -1,3 +1,21 @@ +################################################################################ +# 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. +################################################################################ + # The number of milliseconds of each tick tickTime=2000 diff --git a/flink-runtime-web/web-dashboard/assets/images/browserconfig.xml b/flink-runtime-web/web-dashboard/assets/images/browserconfig.xml index 81ec113f55c0a..68bf6388807ea 100644 --- a/flink-runtime-web/web-dashboard/assets/images/browserconfig.xml +++ b/flink-runtime-web/web-dashboard/assets/images/browserconfig.xml @@ -1,4 +1,23 @@ + + diff --git a/flink-runtime-web/web-dashboard/web/images/browserconfig.xml b/flink-runtime-web/web-dashboard/web/images/browserconfig.xml index 81ec113f55c0a..68bf6388807ea 100644 --- a/flink-runtime-web/web-dashboard/web/images/browserconfig.xml +++ b/flink-runtime-web/web-dashboard/web/images/browserconfig.xml @@ -1,4 +1,23 @@ + + diff --git a/pom.xml b/pom.xml index 890e7f690e68e..d36ac3feeaef1 100644 --- a/pom.xml +++ b/pom.xml @@ -996,7 +996,6 @@ under the License. flink-runtime-web/web-dashboard/assets/fonts/fontawesome* - flink-runtime-web/web-dashboard/assets/images/browserconfig.xml flink-runtime-web/web-dashboard/assets/images/manifest.json flink-runtime-web/web-dashboard/assets/images/safari-pinned-tab.svg @@ -1031,7 +1030,6 @@ under the License. **/flink-bin/conf/slaves **/flink-bin/conf/masters - **/flink-bin/conf/zoo.cfg **/README.md .github/** From 28b3115d23ab6844583578f7f2c7c37316c199be Mon Sep 17 00:00:00 2001 From: Joerg Schad Date: Fri, 17 Nov 2017 22:23:02 -0800 Subject: [PATCH 087/367] [FLINK-8102][docs] Fixed formatting issues in Mesos documentation. --- docs/ops/config.md | 4 ++-- docs/ops/deployment/mesos.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ops/config.md b/docs/ops/config.md index c85ce84a3c98e..bcf7671bd9ac3 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -523,7 +523,7 @@ May be set to -1 to disable this feature. - `mesos.resourcemanager.tasks.container.image.name`: Image name to use for the container (**NO DEFAULT**) -- `mesos.resourcemanager.tasks.container.volumes`: A comma seperated list of [host_path:]container_path[:RO|RW]. This allows for mounting additional volumes into your container. (**NO DEFAULT**) +- `mesos.resourcemanager.tasks.container.volumes`: A comma separated list of `[host_path:]`container_path`[:RO|RW]`. This allows for mounting additional volumes into your container. (**NO DEFAULT**) - `high-availability.zookeeper.path.mesos-workers`: The ZooKeeper root path for persisting the Mesos worker information. @@ -559,7 +559,7 @@ Previously this key was named `recovery.mode` and the default value was `standal - `high-availability.job.delay`: (Default `akka.ask.timeout`) Defines the delay before persisted jobs are recovered in case of a master recovery situation. Previously this key was named `recovery.job.delay`. -- `high-availability.zookeeper.client.acl`: (Default `open`) Defines the ACL (open|creator) to be configured on ZK node. The configuration value can be set to "creator" if the ZooKeeper server configuration has the "authProvider" property mapped to use SASLAuthenticationProvider and the cluster is configured to run in secure mode (Kerberos). The ACL options are based on https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_BuiltinACLSchemes +- `high-availability.zookeeper.client.acl`: (Default `open`) Defines the ACL (open\|creator) to be configured on ZK node. The configuration value can be set to "creator" if the ZooKeeper server configuration has the "authProvider" property mapped to use SASLAuthenticationProvider and the cluster is configured to run in secure mode (Kerberos). The ACL options are based on https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_BuiltinACLSchemes #### ZooKeeper Security diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md index 36b6df7ffdb42..56d0cded7d169 100644 --- a/docs/ops/deployment/mesos.md +++ b/docs/ops/deployment/mesos.md @@ -262,7 +262,7 @@ May be set to -1 to disable this feature. `mesos.resourcemanager.tasks.container.image.name`: Image name to use for the container (**NO DEFAULT**) -`mesos.resourcemanager.tasks.container.volumes`: A comma seperated list of [host_path:]container_path[:RO|RW]. This allows for mounting additional volumes into your container. (**NO DEFAULT**) +`mesos.resourcemanager.tasks.container.volumes`: A comma separated list of `[host_path:]`container_path`[:RO|RW]`. This allows for mounting additional volumes into your container. (**NO DEFAULT**) `mesos.resourcemanager.tasks.hostname`: Optional value to define the TaskManager's hostname. The pattern `_TASK_` is replaced by the actual id of the Mesos task. This can be used to configure the TaskManager to use Mesos DNS (e.g. `_TASK_.flink-service.mesos`) for name lookups. (**NO DEFAULT**) From 3d4146c10bf3d2f7df6d9992670017247d1952dd Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 20 Nov 2017 16:09:10 +0100 Subject: [PATCH 088/367] [FLINK-8115] Fix Kafka download link in end-to-end test --- test-infra/end-to-end-test/test_streaming_kafka010.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test-infra/end-to-end-test/test_streaming_kafka010.sh b/test-infra/end-to-end-test/test_streaming_kafka010.sh index 1324e5aead3d0..dda2db566b853 100755 --- a/test-infra/end-to-end-test/test_streaming_kafka010.sh +++ b/test-infra/end-to-end-test/test_streaming_kafka010.sh @@ -25,7 +25,7 @@ start_cluster mkdir -p $TEST_DATA_DIR if [ -z "$3" ]; then # need to download Kafka because no Kafka was specified on the invocation - KAFKA_URL="http://mirror.netcologne.de/apache.org/kafka/0.10.2.0/kafka_2.11-0.10.2.0.tgz" + KAFKA_URL="https://archive.apache.org/dist/kafka/0.10.2.0/kafka_2.11-0.10.2.0.tgz" echo "Downloading Kafka from $KAFKA_URL" curl "$KAFKA_URL" > $TEST_DATA_DIR/kafka.tgz else @@ -81,4 +81,4 @@ if [[ "$DATA_FROM_KAFKA" != "$EXPECTED" ]]; then echo -e "EXPECTED: --$EXPECTED--" echo -e "ACTUAL: --$DATA_FROM_KAFKA--" PASS="" -fi \ No newline at end of file +fi From 314087e8d31b7965afa5445ad4a8d9c467d0b940 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 20 Nov 2017 16:32:06 +0100 Subject: [PATCH 089/367] [hotfix] Fix create_release_branch.sh to use correct branch name --- tools/releasing/create_release_branch.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/releasing/create_release_branch.sh b/tools/releasing/create_release_branch.sh index c2bca88ff959c..84c259206da43 100755 --- a/tools/releasing/create_release_branch.sh +++ b/tools/releasing/create_release_branch.sh @@ -43,7 +43,7 @@ cd .. target_branch=release-$NEW_VERSION if [ "$RELEASE_CANDIDATE" != "none" ]; then - target_branch=$target_branch-$RELEASE_CANDIDATE + target_branch=$target_branch-rc$RELEASE_CANDIDATE fi git checkout -b $target_branch From ddbc0150d97d5d26de6f9ab9d64d9fb4495c80aa Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Fri, 3 Nov 2017 21:08:24 -0700 Subject: [PATCH 090/367] [FLINK-7977][build] Bump version of compatibility check for Flink 1.4 This closes #4945. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d36ac3feeaef1..cef9e643cc119 100644 --- a/pom.xml +++ b/pom.xml @@ -1442,7 +1442,7 @@ under the License. org.apache.flink ${project.artifactId} - 1.1.4 + 1.3.2 ${project.packaging} From 5f523e6ab31afeab5b1d9bbf62c6d4ef726ffe1b Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 20 Nov 2017 16:51:43 +0100 Subject: [PATCH 091/367] [hotfix][docs] Fix some typos in the documentation. This closes #5039. --- docs/dev/connectors/kafka.md | 4 ++-- docs/dev/stream/operators/windows.md | 4 ++-- docs/ops/production_ready.md | 2 +- .../api/environment/StreamExecutionEnvironment.java | 6 +++--- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 5d3e66d66e850..ad4cc2fe5894c 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -537,7 +537,7 @@ chosen by passing appropriate `semantic` parameter to the `FlinkKafkaProducer011 * `Semantic.NONE`: Flink will not guarantee anything. Produced records can be lost or they can be duplicated. * `Semantic.AT_LEAST_ONCE` (default setting): similar to `setFlushOnCheckpoint(true)` in - `FlinkKafkaProducer010`. his guarantees that no records will be lost (although they can be duplicated). + `FlinkKafkaProducer010`. This guarantees that no records will be lost (although they can be duplicated). * `Semantic.EXACTLY_ONCE`: uses Kafka transactions to provide exactly-once semantic.

@@ -579,7 +579,7 @@ un-finished transaction. In other words after following sequence of events: 3. User committed `transaction2` Even if records from `transaction2` are already committed, they will not be visible to -the consumers until `transaction1` is committed or aborted. This hastwo implications: +the consumers until `transaction1` is committed or aborted. This has two implications: * First of all, during normal working of Flink applications, user can expect a delay in visibility of the records produced into Kafka topics, equal to average time between completed checkpoints. diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 3c0cd8509d3fa..e161854bdcd9d 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -29,7 +29,7 @@ programmer can benefit to the maximum from its offered functionality. The general structure of a windowed Flink program is presented below. The first snippet refers to *keyed* streams, while the second to *non-keyed* ones. As one can see, the only difference is the `keyBy(...)` call for the keyed streams -and the `window(...)` which becomes `windowAll(...)` for non-keyed streams. These is also going to serve as a roadmap +and the `window(...)` which becomes `windowAll(...)` for non-keyed streams. This is also going to serve as a roadmap for the rest of the page. **Keyed Windows** @@ -1383,7 +1383,7 @@ and then calculating the top-k elements within the same window in the second ope Windows can be defined over long periods of time (such as days, weeks, or months) and therefore accumulate very large state. There are a couple of rules to keep in mind when estimating the storage requirements of your windowing computation: -1. Flink creates one copy of each element per window to which it belongs. Given this, tumbling windows keep one copy of each element (an element belongs to exactly window unless it is dropped late). In contrast, sliding windows create several of each element, as explained in the [Window Assigners](#window-assigners) section. Hence, a sliding window of size 1 day and slide 1 second might not be a good idea. +1. Flink creates one copy of each element per window to which it belongs. Given this, tumbling windows keep one copy of each element (an element belongs to exactly one window unless it is dropped late). In contrast, sliding windows create several of each element, as explained in the [Window Assigners](#window-assigners) section. Hence, a sliding window of size 1 day and slide 1 second might not be a good idea. 2. `ReduceFunction`, `AggregateFunction`, and `FoldFunction` can significantly reduce the storage requirements, as they eagerly aggregate elements and store only one value per window. In contrast, just using a `ProcessWindowFunction` requires accumulating all elements. diff --git a/docs/ops/production_ready.md b/docs/ops/production_ready.md index 303e7a71bb6cc..0d11b8a1866b1 100644 --- a/docs/ops/production_ready.md +++ b/docs/ops/production_ready.md @@ -32,7 +32,7 @@ important and need **careful considerations** if you plan to bring your Flink jo Flink provides out-of-the-box defaults to make usage and adoption of Flink easier. For many users and scenarios, those defaults are good starting points for development and completely sufficient for "one-shot" jobs. -However, once you are planning to bring a Flink appplication to production the requirements typically increase. For example, +However, once you are planning to bring a Flink application to production the requirements typically increase. For example, you want your job to be (re-)scalable and to have a good upgrade story for your job and new Flink versions. In the following, we present a collection of configuration options that you should check before your job goes into production. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 46c821edfa2e8..cc45ddc580ffb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -747,7 +747,7 @@ public final DataStreamSource fromElements(Class type, OUT... da * elements, it may be necessary to manually supply the type information via * {@link #fromCollection(java.util.Collection, org.apache.flink.api.common.typeinfo.TypeInformation)}. * - *

Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with a + *

Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with * parallelism one. * * @param data @@ -784,7 +784,7 @@ public DataStreamSource fromCollection(Collection data) { * Creates a data stream from the given non-empty collection. * *

Note that this operation will result in a non-parallel data stream source, - * i.e., a data stream source with a parallelism one. + * i.e., a data stream source with parallelism one. * * @param data * The collection of elements to create the data stream from @@ -843,7 +843,7 @@ public DataStreamSource fromCollection(Iterator data, Class * {@link #fromCollection(java.util.Iterator, Class)} does not supply all type information. * *

Note that this operation will result in a non-parallel data stream source, i.e., - * a data stream source with a parallelism one. + * a data stream source with parallelism one. * * @param data * The iterator of elements to create the data stream from From f26edb88d19623d93e563be417e50df969177c6a Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 20 Nov 2017 15:12:17 +0100 Subject: [PATCH 092/367] [FLINK-8117] [runtime] Eliminate modulo operation from round-robin partitioners This closes #5041 --- .../io/network/api/writer/RoundRobinChannelSelector.java | 5 ++++- .../runtime/partitioner/CustomPartitionerWrapper.java | 2 +- .../streaming/runtime/partitioner/ForwardPartitioner.java | 2 +- .../streaming/runtime/partitioner/GlobalPartitioner.java | 2 +- .../runtime/partitioner/RebalancePartitioner.java | 7 +++++-- .../streaming/runtime/partitioner/RescalePartitioner.java | 7 +++++-- .../streaming/runtime/partitioner/ShufflePartitioner.java | 2 +- 7 files changed, 18 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RoundRobinChannelSelector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RoundRobinChannelSelector.java index 46af5a74914c0..c7d25e5ca4e48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RoundRobinChannelSelector.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RoundRobinChannelSelector.java @@ -46,7 +46,10 @@ public RoundRobinChannelSelector() { @Override public int[] selectChannels(final T record, final int numberOfOutputChannels) { - this.nextChannelToSendTo[0] = (this.nextChannelToSendTo[0] + 1) % numberOfOutputChannels; + int newChannel = ++this.nextChannelToSendTo[0]; + if (newChannel >= numberOfOutputChannels) { + this.nextChannelToSendTo[0] = 0; + } return this.nextChannelToSendTo; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java index a51cedeb542a5..f19c87d7dfcca 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java @@ -35,7 +35,7 @@ public class CustomPartitionerWrapper extends StreamPartitioner { private static final long serialVersionUID = 1L; - private int[] returnArray = new int[1]; + private final int[] returnArray = new int[1]; Partitioner partitioner; KeySelector keySelector; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java index 0ae737c344d18..c952282a81043 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java @@ -30,7 +30,7 @@ public class ForwardPartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; - private int[] returnArray = new int[] {0}; + private final int[] returnArray = new int[] {0}; @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java index 67eaa7300569f..69c8d0073b967 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java @@ -30,7 +30,7 @@ public class GlobalPartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; - private int[] returnArray = new int[] { 0 }; + private final int[] returnArray = new int[] { 0 }; @Override public int[] selectChannels(SerializationDelegate> record, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java index a81f9739d5b0f..bb88d17cf9c99 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java @@ -31,12 +31,15 @@ public class RebalancePartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; - private int[] returnArray = new int[] {-1}; + private final int[] returnArray = new int[] {-1}; @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { - this.returnArray[0] = (this.returnArray[0] + 1) % numberOfOutputChannels; + int newChannel = ++this.returnArray[0]; + if (newChannel >= numberOfOutputChannels) { + this.returnArray[0] = 0; + } return this.returnArray; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitioner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitioner.java index 90615230048d3..b9af629b89ab8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitioner.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitioner.java @@ -48,11 +48,14 @@ public class RescalePartitioner extends StreamPartitioner { private static final long serialVersionUID = 1L; - private int[] returnArray = new int[] {-1}; + private final int[] returnArray = new int[] {-1}; @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { - this.returnArray[0] = (this.returnArray[0] + 1) % numberOfOutputChannels; + int newChannel = ++this.returnArray[0]; + if (newChannel >= numberOfOutputChannels) { + this.returnArray[0] = 0; + } return this.returnArray; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java index 60c3fbc4e1534..ddcbec7213061 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java @@ -36,7 +36,7 @@ public class ShufflePartitioner extends StreamPartitioner { private Random random = new Random(); - private int[] returnArray = new int[1]; + private final int[] returnArray = new int[1]; @Override public int[] selectChannels(SerializationDelegate> record, From 9d28619467c33dec1fe44c6521001298ea1500e2 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 17 Nov 2017 14:13:30 +0100 Subject: [PATCH 093/367] [FLINK-7841] [docs] Update AWS docs with respect to S3 file system changes This closes #5029 --- docs/ops/deployment/aws.md | 126 ++++++++++++++++++++++++++----------- 1 file changed, 91 insertions(+), 35 deletions(-) diff --git a/docs/ops/deployment/aws.md b/docs/ops/deployment/aws.md index bd11bec05a9ef..b5ae1e907163d 100644 --- a/docs/ops/deployment/aws.md +++ b/docs/ops/deployment/aws.md @@ -59,14 +59,14 @@ After creating your cluster, you can [connect to the master node](http://docs.aw 2. Extract the Flink distribution and you are ready to deploy [Flink jobs via YARN](yarn_setup.html) after **setting the Hadoop config directory**: ```bash -HADOOP_CONF_DIR=/etc/hadoop/conf bin/flink run -m yarn-cluster -yn 1 examples/streaming/WordCount.jar +HADOOP_CONF_DIR=/etc/hadoop/conf ./bin/flink run -m yarn-cluster -yn 1 examples/streaming/WordCount.jar ``` {% top %} ## S3: Simple Storage Service -[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html). +[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html) or even as a YARN object storage. You can use S3 objects like regular files by specifying paths in the following format: @@ -89,20 +89,62 @@ env.setStateBackend(new FsStateBackend("s3:///")); Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI. -### Set S3 FileSystem +For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3 +filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as +YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem +implementation. Both ways are described below. + +### Shaded Hadoop/Presto S3 file systems (recommended) + +{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %} + +To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the +`opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g. +``` +cp ./opt/flink-s3-fs-presto-1.4-SNAPSHOT.jar ./lib/ +``` + +#### Configure Access Credentials + +After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets. + +##### Identity and Access Management (IAM) (Recommended) + +The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html). + +If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink. + +##### Access Keys (Discouraged) + +Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2). + +You need to configure both `s3.access-key` and `s3.secret-key` in Flink's `flink-conf.yaml`: + +``` +s3.access-key: your-access-key +s3.secret-key: your-secret-key +``` + +{% top %} + +### Hadoop-provided S3 file systems - manual setup {% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %} -S3 is treated by Flink as a regular FileSystem. Interaction with S3 happens via a Hadoop [S3 FileSystem client](https://wiki.apache.org/hadoop/AmazonS3). +This setup is a bit more complex and we recommend using our shaded Hadoop/Presto file systems +instead (see above) unless required otherwise, e.g. for using S3 as YARN's resource storage dir +via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`. + +#### Set S3 FileSystem -There are two popular S3 file system implementations available: +Interaction with S3 happens via one of [Hadoop's S3 FileSystem clients](https://wiki.apache.org/hadoop/AmazonS3): -1. `S3AFileSystem` (**recommended**): file system for reading and writing regular files using Amazon's SDK internally. No maximum file size and works with IAM roles. -2. `NativeS3FileSystem`: file system for reading and writing regular files. Maximum object size is 5GB and does not work with IAM roles. +1. `S3AFileSystem` (**recommended** for Hadoop 2.7 and later): file system for reading and writing regular files using Amazon's SDK internally. No maximum file size and works with IAM roles. +2. `NativeS3FileSystem` (for Hadoop 2.6 and earlier): file system for reading and writing regular files. Maximum object size is 5GB and does not work with IAM roles. -#### `S3AFileSystem` (Recommended) +##### `S3AFileSystem` (Recommended) -This is the recommended S3 FileSystem implementation to use. It uses Amazon's SDK internally and works with IAM roles (see [Configure Access Credential](#configure-access-credentials)). +This is the recommended S3 FileSystem implementation to use. It uses Amazon's SDK internally and works with IAM roles (see [Configure Access Credentials](#configure-access-credentials-1)). You need to point Flink to a valid Hadoop configuration, which contains the following properties in `core-site.xml`: @@ -126,9 +168,9 @@ You need to point Flink to a valid Hadoop configuration, which contains the foll This registers `S3AFileSystem` as the default FileSystem for URIs with the `s3a://` scheme. -#### `NativeS3FileSystem` +##### `NativeS3FileSystem` -This file system is limited to files up to 5GB in size and it does not work IAM roles (see [Configure Access Credential](#configure-access-credentials)), meaning that you have to manually configure your AWS credentials in the Hadoop config file. +This file system is limited to files up to 5GB in size and it does not work with IAM roles (see [Configure Access Credentials](#configure-access-credentials-1)), meaning that you have to manually configure your AWS credentials in the Hadoop config file. You need to point Flink to a valid Hadoop configuration, which contains the following property in `core-site.xml`: @@ -141,10 +183,14 @@ You need to point Flink to a valid Hadoop configuration, which contains the foll This registers `NativeS3FileSystem` as the default FileSystem for URIs with the `s3://` scheme. -#### Hadoop Configuration +{% top %} -You can specify the [Hadoop configuration](../config.html#hdfs) in various ways, for example by configuring the path to the Hadoop configuration directory in `flink-conf.yaml`: +#### Hadoop Configuration +You can specify the [Hadoop configuration](../config.html#hdfs) in various ways pointing Flink to +the path of the Hadoop configuration directory, for example +- by setting the environment variable `HADOOP_CONF_DIR`, or +- by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`: ``` fs.hdfs.hadoopconf: /path/to/etc/hadoop ``` @@ -153,15 +199,15 @@ This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Fl {% top %} -### Configure Access Credentials +#### Configure Access Credentials {% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %} After setting up the S3 FileSystem, you need to make sure that Flink is allowed to access your S3 buckets. -#### Identity and Access Management (IAM) (Recommended) +##### Identity and Access Management (IAM) (Recommended) -When using `S3AFileSystem` the recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html). +When using `S3AFileSystem`, the recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html). If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink. @@ -169,7 +215,7 @@ Note that this only works with `S3AFileSystem` and not `NativeS3FileSystem`. {% top %} -#### Access Keys with S3AFileSystem (Discouraged) +##### Access Keys with `S3AFileSystem` (Discouraged) Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2). @@ -189,7 +235,7 @@ For `S3AFileSystem` you need to configure both `fs.s3a.access.key` and `fs.s3a.s {% top %} -#### Access Keys with NativeS3FileSystem (Discouraged) +##### Access Keys with `NativeS3FileSystem` (Discouraged) Access to S3 can be granted via your **access and secret key pair**. But this is discouraged and you should use `S3AFileSystem` [with the required IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2). @@ -209,39 +255,49 @@ For `NativeS3FileSystem` you need to configure both `fs.s3.awsAccessKeyId` and ` {% top %} -### Provide S3 FileSystem Dependency +#### Provide S3 FileSystem Dependency {% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %} -Hadoop's S3 FileSystem clients are packaged in the `hadoop-aws`. This JAR and all its dependencies need to be added to Flink's classpath, i.e. the class path of both Job and TaskManagers. Depending on which FileSystem implementation and which Flink and Hadoop version you use, you need to provide different dependencies (see below). +Hadoop's S3 FileSystem clients are packaged in the `hadoop-aws` artifact (Hadoop version 2.6 and later). This JAR and all its dependencies need to be added to Flink's classpath, i.e. the class path of both Job and TaskManagers. Depending on which FileSystem implementation and which Flink and Hadoop version you use, you need to provide different dependencies (see below). -There are multiple ways of adding JARs to Flink's class path, the easiest being simply to drop the JARs in Flink's `/lib` folder. You need to copy the `hadoop-aws` JAR with all its dependencies. You can also export the directory containing these JARs as part of the `HADOOP_CLASSPATH` environment variable on all machines. +There are multiple ways of adding JARs to Flink's class path, the easiest being simply to drop the JARs in Flink's `lib` folder. You need to copy the `hadoop-aws` JAR with all its dependencies. You can also export the directory containing these JARs as part of the `HADOOP_CLASSPATH` environment variable on all machines. -#### Flink for Hadoop 2.7 +##### Flink for Hadoop 2.7 Depending on which file system you use, please add the following dependencies. You can find these as part of the Hadoop binaries in `hadoop-2.7/share/hadoop/tools/lib`: - `S3AFileSystem`: - - `hadoop-aws-2.7.2.jar` - - `aws-java-sdk-1.7.4.jar` - - `httpcore-4.2.5.jar` - - `httpclient-4.2.5.jar` + - `hadoop-aws-2.7.3.jar` + - `aws-java-sdk-s3-1.11.183.jar` and its dependencies: + - `aws-java-sdk-core-1.11.183.jar` + - `aws-java-sdk-kms-1.11.183.jar` + - `jackson-annotations-2.6.7.jar` + - `jackson-core-2.6.7.jar` + - `jackson-databind-2.6.7.jar` + - `joda-time-2.8.1.jar` + - `httpcore-4.4.4.jar` + - `httpclient-4.5.3.jar` - `NativeS3FileSystem`: - - `hadoop-aws-2.7.2.jar` + - `hadoop-aws-2.7.3.jar` - `guava-11.0.2.jar` Note that `hadoop-common` is available as part of Flink, but Guava is shaded by Flink. -#### Flink for Hadoop 2.6 +##### Flink for Hadoop 2.6 Depending on which file system you use, please add the following dependencies. You can find these as part of the Hadoop binaries in `hadoop-2.6/share/hadoop/tools/lib`: - `S3AFileSystem`: - `hadoop-aws-2.6.4.jar` - - `aws-java-sdk-1.7.4.jar` - - `httpcore-4.2.5.jar` - - `httpclient-4.2.5.jar` + - `aws-java-sdk-1.7.4.jar` and its dependencies: + - `jackson-annotations-2.1.1.jar` + - `jackson-core-2.1.1.jar` + - `jackson-databind-2.1.1.jar` + - `joda-time-2.2.jar` + - `httpcore-4.2.5.jar` + - `httpclient-4.2.5.jar` - `NativeS3FileSystem`: - `hadoop-aws-2.6.4.jar` @@ -249,7 +305,7 @@ Depending on which file system you use, please add the following dependencies. Y Note that `hadoop-common` is available as part of Flink, but Guava is shaded by Flink. -#### Flink for Hadoop 2.4 and earlier +##### Flink for Hadoop 2.4 and earlier These Hadoop versions only have support for `NativeS3FileSystem`. This comes pre-packaged with Flink for Hadoop 2 as part of `hadoop-common`. You don't need to add anything to the classpath. @@ -261,7 +317,7 @@ The following sections lists common issues when working with Flink on AWS. ### Missing S3 FileSystem Configuration -If your job submission fails with an Exception message noting that `No file system found with scheme s3` this means that no FileSystem has been configured for S3. Please check out the [FileSystem Configuration section](#set-s3-filesystem) for details on how to configure this properly. +If your job submission fails with an Exception message noting that `No file system found with scheme s3` this means that no FileSystem has been configured for S3. Please check out the configuration sections for our [shaded Hadoop/Presto](#shaded-hadooppresto-s3-file-systems-recommended) or [generic Hadoop](#set-s3-filesystem) file systems for details on how to configure this properly. ``` org.apache.flink.client.program.ProgramInvocationException: The program execution failed: @@ -281,7 +337,7 @@ Caused by: java.io.IOException: No file system found with scheme s3, ### AWS Access Key ID and Secret Access Key Not Specified -If you see your job failing with an Exception noting that the `AWS Access Key ID and Secret Access Key must be specified as the username or password`, your access credentials have not been set up properly. Please refer to the [access credential section](#configure-access-credentials) for details on how to configure this. +If you see your job failing with an Exception noting that the `AWS Access Key ID and Secret Access Key must be specified as the username or password`, your access credentials have not been set up properly. Please refer to the access credential section for our [shaded Hadoop/Presto](#configure-access-credentials) or [generic Hadoop](#configure-access-credentials-1) file systems for details on how to configure this. ``` org.apache.flink.client.program.ProgramInvocationException: The program execution failed: @@ -354,7 +410,7 @@ Caused by: org.jets3t.service.impl.rest.HttpException [...] ### NullPointerException at org.apache.hadoop.fs.LocalDirAllocator -This Exception is usually caused by skipping the local buffer directory configuration `fs.s3.buffer.dir` for the `S3AFileSystem`. Please refer to the [S3AFileSystem configuration](#s3afilesystem-recommended) section to see how to configure the `S3AFileSystem` properly. +This Exception is usually caused by skipping the local buffer directory configuration `fs.s3a.buffer.dir` for the `S3AFileSystem`. Please refer to the [S3AFileSystem configuration](#s3afilesystem-recommended) section to see how to configure the `S3AFileSystem` properly. ``` [...] From 56c78cbcfd6fddd7af0fee0ee8c179b7a75fb1a7 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 14 Nov 2017 11:55:33 +0100 Subject: [PATCH 094/367] [FLINK-8070][yarn][tests] Print errors found in log files This closes #5012. --- .../org/apache/flink/yarn/YarnTestBase.java | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index a701c909c205c..4fc41c45b2ffe 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -69,6 +69,7 @@ import java.io.IOException; import java.io.PrintStream; import java.io.PrintWriter; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -307,6 +308,7 @@ public static void ensureNoProhibitedStringInLogFiles(final String[] prohibited, Assert.assertTrue("Expecting directory " + cwd.getAbsolutePath() + " to exist", cwd.exists()); Assert.assertTrue("Expecting directory " + cwd.getAbsolutePath() + " to be a directory", cwd.isDirectory()); + List prohibitedExcerpts = new ArrayList<>(); File foundFile = findFile(cwd.getAbsolutePath(), new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -331,6 +333,24 @@ public boolean accept(File dir, String name) { // logging in FATAL to see the actual message in TRAVIS tests. Marker fatal = MarkerFactory.getMarker("FATAL"); LOG.error(fatal, "Prohibited String '{}' in line '{}'", aProhibited, lineFromFile); + + StringBuilder logExcerpt = new StringBuilder(); + + logExcerpt.append(System.lineSeparator()); + logExcerpt.append(lineFromFile); + logExcerpt.append(System.lineSeparator()); + // extract potential stack trace from log + while (scanner.hasNextLine()) { + String line = scanner.nextLine(); + if (!line.isEmpty() && (Character.isWhitespace(line.charAt(0)) || line.startsWith("Caused by"))) { + logExcerpt.append(line); + logExcerpt.append(System.lineSeparator()); + } else { + break; + } + } + prohibitedExcerpts.add(logExcerpt.toString()); + return true; } } @@ -355,7 +375,9 @@ public boolean accept(File dir, String name) { while (scanner.hasNextLine()) { LOG.warn("LINE: " + scanner.nextLine()); } - Assert.fail("Found a file " + foundFile + " with a prohibited string: " + Arrays.toString(prohibited)); + Assert.fail( + "Found a file " + foundFile + " with a prohibited string (one of " + Arrays.toString(prohibited) + "). " + + "Excerpts:" + System.lineSeparator() + prohibitedExcerpts); } } From 7ed2cef83723fadb63f5cccf35ade1c9c78725e2 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 22 Nov 2017 12:00:09 +0100 Subject: [PATCH 095/367] [FLINK-8131] Update to Kafka 0.11.0.2 --- flink-connectors/flink-connector-kafka-0.11/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml index c39c146374e3a..f67be32b0e255 100644 --- a/flink-connectors/flink-connector-kafka-0.11/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml @@ -37,7 +37,7 @@ under the License. - 0.11.0.1 + 0.11.0.2 From a964ef6e218263c664d7b30607b8461c964e6fc8 Mon Sep 17 00:00:00 2001 From: Cristian Date: Wed, 22 Nov 2017 04:17:39 -0800 Subject: [PATCH 096/367] [hotfix][docs] Fix typo in Trigger doc This closes #5051. --- .../apache/flink/streaming/api/windowing/triggers/Trigger.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java index f29fc6f66b1e4..f41cce87f9d49 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java @@ -36,7 +36,7 @@ * *

A pane is the bucket of elements that have the same key (assigned by the * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can - * be in multiple panes of it was assigned to multiple windows by the + * be in multiple panes if it was assigned to multiple windows by the * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all * have their own instance of the {@code Trigger}. * From 6cb8b5b1286e52f72667f6c4f5794742580c9145 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 22 Nov 2017 16:22:28 +0100 Subject: [PATCH 097/367] [hotfix] Make aws docs version agnostic --- docs/ops/deployment/aws.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/ops/deployment/aws.md b/docs/ops/deployment/aws.md index b5ae1e907163d..d9d26478ee30c 100644 --- a/docs/ops/deployment/aws.md +++ b/docs/ops/deployment/aws.md @@ -100,8 +100,9 @@ implementation. Both ways are described below. To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g. + ``` -cp ./opt/flink-s3-fs-presto-1.4-SNAPSHOT.jar ./lib/ +cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./lib/ ``` #### Configure Access Credentials @@ -195,7 +196,7 @@ the path of the Hadoop configuration directory, for example fs.hdfs.hadoopconf: /path/to/etc/hadoop ``` -This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the specified directory. +This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the specified directory. {% top %} From 28157962196cecb94a59720e92cbf3682418e821 Mon Sep 17 00:00:00 2001 From: twalthr Date: Wed, 22 Nov 2017 17:43:08 +0100 Subject: [PATCH 098/367] [FLINK-8136] [table] Fix code generation with JodaTime shading This closes #5054. --- .../scala/org/apache/flink/table/codegen/CodeGenerator.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index a794b08993317..7660e2ba9b99b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -44,6 +44,7 @@ import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} import org.apache.flink.table.typeutils.TypeCheckUtils._ +import org.joda.time.format.DateTimeFormatter import scala.collection.JavaConversions._ import scala.collection.mutable @@ -1457,7 +1458,7 @@ abstract class CodeGenerator( val field = s""" - |final org.joda.time.format.DateTimeFormatter $fieldTerm; + |final ${classOf[DateTimeFormatter].getCanonicalName} $fieldTerm; |""".stripMargin reusableMemberStatements.add(field) From 7868ea4a6986ac46917f63a81c00957bee6fb1e6 Mon Sep 17 00:00:00 2001 From: uybhatti Date: Fri, 3 Mar 2017 23:55:22 +0100 Subject: [PATCH 099/367] [FLINK-2170] [connectors] Add OrcRowInputFormat and OrcTableSource. This closes #4670. --- flink-connectors/flink-orc/pom.xml | 152 + .../org/apache/flink/orc/OrcTableSource.java | 128 + .../java/org/apache/flink/orc/OrcUtils.java | 2229 ++++ .../apache/flink/orc/RowOrcInputFormat.java | 241 + .../flink/orc/OrcTableSourceITCase.java | 142 + .../apache/flink/orc/OrcTableSourceTest.java | 113 + .../flink/orc/RowOrcInputFormatTest.java | 472 + .../test/resources/TestOrcFile.emptyFile.orc | Bin 0 -> 523 bytes .../TestOrcFile.listliststructlong.orc | Bin 0 -> 845 bytes .../test/resources/TestOrcFile.listlong.orc | Bin 0 -> 627 bytes .../test/resources/TestOrcFile.liststring.orc | Bin 0 -> 1298 bytes .../src/test/resources/TestOrcFile.test1.orc | Bin 0 -> 1711 bytes .../resources/TestOrcFile.testDate1900.dat | 10000 ++++++++++++++++ .../resources/TestOrcFile.testDate1900.orc | Bin 0 -> 30941 bytes .../flink-orc/src/test/resources/decimal.dat | 6000 ++++++++++ .../flink-orc/src/test/resources/decimal.orc | Bin 0 -> 16337 bytes .../src/test/resources/demo-11-none.orc | Bin 0 -> 5147970 bytes .../src/test/resources/log4j-test.properties | 27 + flink-connectors/pom.xml | 1 + 19 files changed, 19505 insertions(+) create mode 100644 flink-connectors/flink-orc/pom.xml create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.listliststructlong.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/decimal.dat create mode 100644 flink-connectors/flink-orc/src/test/resources/decimal.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/demo-11-none.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/log4j-test.properties diff --git a/flink-connectors/flink-orc/pom.xml b/flink-connectors/flink-orc/pom.xml new file mode 100644 index 0000000000000..1ac7eaa7a9f2f --- /dev/null +++ b/flink-connectors/flink-orc/pom.xml @@ -0,0 +1,152 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.4-SNAPSHOT + .. + + + flink-orc_${scala.binary.version} + flink-orc + + jar + + + + + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + compile + + + + org.apache.flink + flink-streaming-scala_${scala.binary.version} + ${project.version} + compile + + + + org.apache.orc + orc-core + 1.4.0 + + + + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-core + ${project.version} + test + test-jar + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.maven.plugins + maven-assembly-plugin + [2.4,) + + single + + + + + + + + + org.apache.maven.plugins + maven-clean-plugin + [1,) + + clean + + + + + + + + + org.apache.avro + avro-maven-plugin + [1.7.7,) + + schema + + + + + + + + + + + + + + + diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java new file mode 100644 index 0000000000000..0454ba45af16a --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java @@ -0,0 +1,128 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.sources.BatchTableSource; +import org.apache.flink.table.sources.ProjectableTableSource; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.TypeDescription; + +/** + * Creates a TableSource to read ORC file. + * + *

The ORC file path and schema is passed during {@link OrcTableSource} construction. configuration is optional. + * + *

The OrcTableSource is used as shown in the example below. + * + *

+ * {@code
+ * String path = testInputURL.getPath();
+ * String schema = "struct"
+ * OrcTableSource orcSrc = new OrcTableSource(path, schema);
+ * tEnv.registerTableSource("orcTable", orcSrc);
+ * Table res = tableEnv.sql("SELECT * FROM orcTable");
+ * }
+ * 
+ */ +public class OrcTableSource implements BatchTableSource, ProjectableTableSource { + + private String path; + private TypeDescription orcSchema; + private RowTypeInfo typeInfo; + private Configuration orcConfig; + private int[] fieldMapping; + + /** + * The ORC file path and schema. + * + * @param path the path of orc file + * @param orcSchema schema of orc file + */ + public OrcTableSource(String path, String orcSchema) { + this(path, orcSchema, new Configuration()); + } + + /** + * The file path and schema of orc file, and configuration to read orc file . + * + * @param path the path of orc file + * @param orcSchema schema of orc file + * @param orcConfig configuration to read orc file + */ + public OrcTableSource(String path, String orcSchema, Configuration orcConfig) { + this(path, TypeDescription.fromString(orcSchema), orcConfig); + } + + public OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig) { + this.path = path; + this.orcSchema = orcSchema; + this.orcConfig = orcConfig; + + this.typeInfo = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema); + + } + + @Override + public DataSet getDataSet(ExecutionEnvironment execEnv) { + + RowOrcInputFormat orcIF = new RowOrcInputFormat(path, orcSchema, orcConfig); + if (fieldMapping != null) { + orcIF.setFieldMapping(fieldMapping); + } + return execEnv.createInput(orcIF); + } + + @Override + public TypeInformation getReturnType() { + return typeInfo; + } + + @Override + public TableSource projectFields(int[] fields) { + + OrcTableSource copy = new OrcTableSource(path, orcSchema, orcConfig); + + // set field mapping + copy.fieldMapping = fields; + + // adapt TypeInfo + TypeInformation[] fieldTypes = new TypeInformation[fields.length]; + String[] fieldNames = new String[fields.length]; + for (int i = 0; i < fields.length; i++) { + fieldTypes[i] = this.typeInfo.getTypeAt(fields[i]); + fieldNames[i] = this.typeInfo.getFieldNames()[fields[i]]; + } + copy.typeInfo = new RowTypeInfo(fieldTypes, fieldNames); + + return copy; + } + + @Override + public String explainSource() { + return "ORC Source file at path " + this.path + " with schema " + this.orcSchema; + } + +} diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java new file mode 100644 index 0000000000000..c7557c7477be1 --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java @@ -0,0 +1,2229 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; + +import org.apache.orc.TypeDescription; + +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +/** + * A class that provides utility methods for orc file reading. + */ +public class OrcUtils { + + /** + * Convert ORC schema types to Flink types. + * + * @param schema schema of orc file + * + */ + public static TypeInformation schemaToTypeInfo(TypeDescription schema) { + switch (schema.getCategory()) { + case BOOLEAN: + return BasicTypeInfo.BOOLEAN_TYPE_INFO; + case BYTE: + return BasicTypeInfo.BYTE_TYPE_INFO; + case SHORT: + return BasicTypeInfo.SHORT_TYPE_INFO; + case INT: + return BasicTypeInfo.INT_TYPE_INFO; + case LONG: + return BasicTypeInfo.LONG_TYPE_INFO; + case FLOAT: + return BasicTypeInfo.FLOAT_TYPE_INFO; + case DOUBLE: + return BasicTypeInfo.DOUBLE_TYPE_INFO; + case STRING: + case CHAR: + case VARCHAR: + return BasicTypeInfo.STRING_TYPE_INFO; + case DATE: + return SqlTimeTypeInfo.DATE; + case TIMESTAMP: + return SqlTimeTypeInfo.TIMESTAMP; + case BINARY: + return PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO; + case STRUCT: + List fieldSchemas = schema.getChildren(); + TypeInformation[] fieldTypes = new TypeInformation[fieldSchemas.size()]; + for (int i = 0; i < fieldSchemas.size(); i++) { + fieldTypes[i] = schemaToTypeInfo(fieldSchemas.get(i)); + } + String[] fieldNames = schema.getFieldNames().toArray(new String[]{}); + return new RowTypeInfo(fieldTypes, fieldNames); + case LIST: + TypeDescription elementSchema = schema.getChildren().get(0); + TypeInformation elementType = schemaToTypeInfo(elementSchema); + return ObjectArrayTypeInfo.getInfoFor(elementType); + case MAP: + TypeDescription keySchema = schema.getChildren().get(0); + TypeDescription valSchema = schema.getChildren().get(1); + TypeInformation keyType = schemaToTypeInfo(keySchema); + TypeInformation valType = schemaToTypeInfo(valSchema); + return new MapTypeInfo(keyType, valType); + case DECIMAL: + return BasicTypeInfo.BIG_DEC_TYPE_INFO; + case UNION: + throw new UnsupportedOperationException("UNION type not supported yet."); + default: + throw new IllegalArgumentException("Unknown type " + schema); + } + } + + /** + * Fill rows from orc batch. + * + * @param rows the batch of rows need to be filled + * @param schema schema of orc file + * @param batch current orc batch data used to fill the rows + * @param fieldMapping field mapping + * + */ + public static void fillRows(Object[] rows, TypeDescription schema, VectorizedRowBatch batch, int[] fieldMapping) { + + int totalRowsInBatch = (int) batch.count(); + + List fieldTypes = schema.getChildren(); + for (int outIdx = 0; outIdx < fieldMapping.length; outIdx++) { + int inIdx = fieldMapping[outIdx]; + readField(rows, outIdx, fieldTypes.get(inIdx), batch.cols[inIdx], null, Math.min((int) totalRowsInBatch, rows.length)); + } + } + + private static void readField(Object[] rows, int fieldIdx, TypeDescription schema, ColumnVector vector, long[] lengthVector, int childCount) { + + switch (schema.getCategory()) { + case BOOLEAN: + if (vector.noNulls) { + readNonNullBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case BYTE: + if (vector.noNulls) { + readNonNullByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case SHORT: + if (vector.noNulls) { + readNonNullShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case INT: + if (vector.noNulls) { + readNonNullIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case LONG: + if (vector.noNulls) { + readNonNullLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case FLOAT: + if (vector.noNulls) { + readNonNullFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } else { + readFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } + break; + case DOUBLE: + if (vector.noNulls) { + readNonNullDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } else { + readDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } + break; + case CHAR: + case VARCHAR: + case STRING: + if (vector.noNulls) { + readNonNullStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } else { + readStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } + break; + case DATE: + if (vector.noNulls) { + readNonNullDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case TIMESTAMP: + if (vector.noNulls) { + readNonNullTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + } else { + readTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + } + break; + case BINARY: + if (vector.noNulls) { + readNonNullBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } else { + readBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } + break; + case DECIMAL: + if (vector.noNulls) { + readNonNullDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + } + else { + readDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + } + break; + case STRUCT: + if (vector.noNulls) { + readNonNullStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + } else { + readStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + } + break; + case LIST: + if (vector.noNulls) { + readNonNullListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + } + else { + readListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + } + break; + case MAP: + if (vector.noNulls) { + readNonNullMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + } + else { + readMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + } + break; + case UNION: + throw new UnsupportedOperationException("UNION type not supported yet"); + default: + throw new IllegalArgumentException("Unknown type " + schema); + } + } + + private static void readNonNullBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if boolean is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + boolean repeatingValue = vector.vector[0] != 0; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = vector.vector[i] != 0; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); + } + } + } + } else { // in a list + boolean[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + boolean repeatingValue = vector.vector[0] != 0; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++] != 0; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++] != 0; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if byte is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + byte repeatingValue = (byte) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (byte) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); + } + } + } + } else { // in a list + byte[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + byte repeatingValue = (byte) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (byte) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (byte) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if short is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + short repeatingValue = (short) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (short) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); + } + } + } + } else { // in a list + short[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + short repeatingValue = (short) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (short) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (short) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if int is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + int repeatingValue = (int) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (int) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); + } + } + } + } else { // in a list + int[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + int repeatingValue = (int) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (int) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (int) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if long is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + long repeatingValue = vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (Long) vector.vector[i]); + } + } + } + } else { // in a list + long[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + long repeatingValue = vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if float is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + float repeatingValue = (float) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (float) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); + } + } + } + } else { // in a list + float[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + float repeatingValue = (float) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (float) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (float) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if double is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + double repeatingValue = vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + } + } + } + } else { // in a list + double[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + double repeatingValue = vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } + else { // in a list + String[] temp; + int offset = 0; + if (bytes.isRepeating) { // fill list with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); + if (fieldIdx == -1) { // set list as an object + for (int i = 0; i < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field + for (int i = 0; i < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readNonNullDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if date is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readDate(vector.vector[0]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[0])); + } + } + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readDate(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); + } + } + } + } else { + Date[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[0]); + } + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[0]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + + // check if timestamp is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readTimeStamp(vector.time[0], vector.nanos[0]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[0], vector.nanos[0])); + } + } + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readTimeStamp(vector.time[i], vector.nanos[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[i], vector.nanos[i])); + } + } + } + } else { + Timestamp[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); + } + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[0], bytes.start[0], bytes.length[0])); + } + } + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } else { + byte[][] temp; + int offset = 0; + if (bytes.isRepeating) { // fill complete list with first value + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readNonNullDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { + + // check if decimal is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + fillColumnWithRepeatingValue(rows, fieldIdx, readBigDecimal(vector.vector[0]), childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readBigDecimal(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); + } + } + } + } else { + BigDecimal[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + BigDecimal repeatingValue = readBigDecimal(vector.vector[0]); + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBigDecimal(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBigDecimal(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readNonNullStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { + + List childrenTypes = schema.getChildren(); + + int numChildren = childrenTypes.size(); + Row[] nestedFields = new Row[childCount]; + for (int i = 0; i < childCount; i++) { + nestedFields[i] = new Row(numChildren); + } + for (int i = 0; i < numChildren; i++) { + readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); + } + + // check if struct is directly in a list or not, e.g, array> + if (lengthVector == null) { + if (fieldIdx == -1) { // set struct as an object + System.arraycopy(nestedFields, 0, rows, 0, childCount); + } + else { // set struct as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); + } + } + } + else { // struct in a list + int offset = 0; + Row[] temp; + if (fieldIdx == -1) { // set list of struct as an object + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + System.arraycopy(nestedFields, offset, temp, 0, temp.length); + offset = offset + temp.length; + rows[i] = temp; + } + } + else { // set list of struct as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + System.arraycopy(nestedFields, offset, temp, 0, temp.length); + offset = offset + temp.length; + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void readNonNullListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + + TypeDescription fieldType = schema.getChildren().get(0); + if (lengthVector == null) { + long[] lengthVectorNested = list.lengths; + readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } + else { // list in a list + + Object[] nestedList = new Object[childCount]; + + // length vector for nested list + long[] lengthVectorNested = list.lengths; + + // read nested list + readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); + + // get type of nestedList + Class classType = nestedList[0].getClass(); + + // fill outer list with nested list + int offset = 0; + int length; + if (fieldIdx == -1) { // set list of list as an object + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + rows[i] = temp; + + } + } else { // set list of list as an field on Row + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + + } + + private static void readNonNullMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { + + List fieldType = schema.getChildren(); + TypeDescription keyType = fieldType.get(0); + TypeDescription valueType = fieldType.get(1); + + ColumnVector keys = map.keys; + ColumnVector values = map.values; + Object[] keyRows = new Object[map.childCount]; + Object[] valueRows = new Object[map.childCount]; + + // read map kes and values + readField(keyRows, -1, keyType, keys, null, keyRows.length); + readField(valueRows, -1, valueType, values, null, valueRows.length); + + // check if map is directly in a list or not, e.g, array> + if (lengthVector == null) { + long[] lengthVectorMap = map.lengths; + int offset = 0; + if (fieldIdx == -1) { + for (int i = 0; i < childCount; i++) { + rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); + offset += lengthVectorMap[i]; + } + } else { + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); + offset += lengthVectorMap[i]; + } + } + } else { // list of map + + long[] lengthVectorMap = map.lengths; + int mapOffset = 0; // offset of map element + int offset = 0; // offset of map + HashMap[] temp; + if (fieldIdx == -1) { // set map list as an object + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + rows[i] = temp; + } + } else { // set map list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void fillColumnWithRepeatingValue(Object[] rows, int fieldIdx, Object repeatingValue, int childCount) { + + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = repeatingValue; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, repeatingValue); + } + } + } + + private static void fillListWithRepeatingNull(Object[] rows, int fieldIdx, Class classType, long[] lengthVector, int childCount) { + + int length; + if (fieldIdx == -1) { + for (int i = 0; i < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + rows[i] = temp; + } + } else { + for (int i = 0; i < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + + private static void readBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = vector.vector[i] != 0; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, boolean[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + boolean[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++] != 0; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++] != 0; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (byte) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, byte[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + byte[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (byte) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (byte) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (short) vector.vector[i]; + } + } else { // set as field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, short[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + short[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (short) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (short) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (int) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, int[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + int[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (int) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (int) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, long[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + long[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (float) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, float[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + float[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (float) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; i < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (float) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = vector.vector[i]; + } + } else { // set as field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, double[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + double[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = bytes.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } else { // in a list + if (bytes.isRepeating) { // fill list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, String[].class, lengthVector, childCount); + } else { + int offset = 0; + String[] temp; + boolean[] isNullVector = bytes.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; // skip null value + } + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; // skip null value + } + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if date is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = readDate(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); + } + } + } + } else { + if (vector.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, Date[].class, lengthVector, childCount); + } else { + int offset = 0; + Date[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readDate(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readDate(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + + // check if timestamp is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + Timestamp ts = new Timestamp(vector.time[i]); + ts.setNanos(vector.nanos[i]); + rows[i] = ts; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + Timestamp ts = new Timestamp(vector.time[i]); + ts.setNanos(vector.nanos[i]); + ((Row) rows[i]).setField(fieldIdx, ts); + } + } + } + } + else { + if (vector.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, Timestamp[].class, lengthVector, childCount); + } else { + int offset = 0; + Timestamp[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = new Timestamp(vector.time[offset]); + temp[j].setNanos(vector.nanos[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = new Timestamp(vector.time[offset]); + temp[j].setNanos(vector.nanos[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVectorIndex = bytes.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVectorIndex[i]) { + rows[i] = null; + continue; + } + rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVectorIndex[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } else { + if (bytes.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, byte[][].class, lengthVector, childCount); + } else { + int offset = 0; + byte[][] temp; + boolean[] isNullVector = bytes.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { + + // check if decimal is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = readBigDecimal(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); + } + } + } + } else { + if (vector.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, BigDecimal[].class, lengthVector, childCount); + } else { + int offset = 0; + BigDecimal[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBigDecimal(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBigDecimal(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { + + List childrenTypes = schema.getChildren(); + + int numChildren = childrenTypes.size(); + Row[] nestedFields = new Row[childCount]; + for (int i = 0; i < childCount; i++) { + nestedFields[i] = new Row(numChildren); + } + for (int i = 0; i < numChildren; i++) { + readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); + } + + boolean[] isNullVector = struct.isNull; + + // check if struct is directly in a list or not, e.g, array> + if (lengthVector == null) { + if (fieldIdx == -1) { // set struct as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = nestedFields[i]; + } + } else { // set struct as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); + } + } + } else { // struct in a list + int offset = 0; + Row[] temp; + if (fieldIdx == -1) { // set list of struct as an object + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = nestedFields[offset++]; + } + rows[i] = temp; + } + } + else { // set list of struct as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = nestedFields[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void readListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + + TypeDescription fieldType = schema.getChildren().get(0); + if (lengthVector == null) { + long[] lengthVectorNested = list.lengths; + readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } + else { // list in a list + + Object[] nestedList = new Object[childCount]; + + // length vector for nested list + long[] lengthVectorNested = list.lengths; + + // read nested list + readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); + + // get type of nestedList + Class classType = nestedList[0].getClass(); + + // fill outer list with nested list + int offset = 0; + int length; + if (fieldIdx == -1) { // set list of list as an object + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + rows[i] = temp; + + } + } else { // set list of list as an field on Row + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { + + List fieldType = schema.getChildren(); + TypeDescription keyType = fieldType.get(0); + TypeDescription valueType = fieldType.get(1); + + ColumnVector keys = map.keys; + ColumnVector values = map.values; + Object[] keyRows = new Object[map.childCount]; + Object[] valueRows = new Object[map.childCount]; + + // read map kes and values + readField(keyRows, -1, keyType, keys, null, keyRows.length); + readField(valueRows, -1, valueType, values, null, valueRows.length); + + boolean[] isNullVector = map.isNull; + + // check if map is directly in a list or not, e.g, array> + if (lengthVector == null) { + long[] lengthVectorMap = map.lengths; + int offset = 0; + if (fieldIdx == -1) { // set map as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); + offset += lengthVectorMap[i]; + } + } else { // set map as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); + offset += lengthVectorMap[i]; + } + } + } else { // list of map + long[] lengthVectorMap = map.lengths; + int mapOffset = 0; // offset of map element + int offset = 0; // offset of map + HashMap[] temp; + if (fieldIdx == -1) { // set map list as an object + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + rows[i] = temp; + } + } else { // set map list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static BigDecimal readBigDecimal(HiveDecimalWritable hiveDecimalWritable) { + HiveDecimal hiveDecimal = hiveDecimalWritable.getHiveDecimal(); + return hiveDecimal.bigDecimalValue(); + } + + private static byte[] readBinary(byte[] src, int srcPos, int length) { + byte[] result = new byte[length]; + System.arraycopy(src, srcPos, result, 0, length); + return result; + } + + private static Timestamp readTimeStamp(long time, int nanos) { + Timestamp ts = new Timestamp(time); + ts.setNanos(nanos); + return ts; + } + + private static Date readDate(long days) { + // day to milliseconds + return new Date(days * 24 * 60 * 60 * 1000); + } + + private static HashMap readHashMap(Object[] keyRows, Object[] valueRows, int offset, long length) { + + HashMap resultMap = new HashMap<>(); + for (int j = 0; j < length; j++) { + resultMap.put(keyRows[offset], valueRows[offset]); + offset++; + } + return resultMap; + } + +} diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java new file mode 100644 index 0000000000000..0c9c549aed2cf --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java @@ -0,0 +1,241 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.io.FileInputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; + +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.apache.flink.orc.OrcUtils.fillRows; + +/** + * InputFormat to read ORC data. + * For Optimization, reading is done in batch instead of a single row. + */ +public class RowOrcInputFormat + extends FileInputFormat + implements ResultTypeQueryable { + + private static final Logger LOG = LoggerFactory.getLogger(RowOrcInputFormat.class); + private static final int BATCH_SIZE = 1024; + + private org.apache.hadoop.conf.Configuration config; + private TypeDescription schema; + private int[] fieldMapping; + + private transient RowTypeInfo rowType; + private transient RecordReader orcRowsReader; + private transient VectorizedRowBatch rowBatch; + private transient Row[] rows; + + private transient int rowInBatch; + + public RowOrcInputFormat(String path, String schemaString, Configuration orcConfig) { + this(path, TypeDescription.fromString(schemaString), orcConfig); + } + + public RowOrcInputFormat(String path, TypeDescription orcSchema, Configuration orcConfig) { + super(new Path(path)); + this.unsplittable = false; + this.schema = orcSchema; + this.rowType = (RowTypeInfo) OrcUtils.schemaToTypeInfo(schema); + this.config = orcConfig; + + this.fieldMapping = new int[this.schema.getChildren().size()]; + for (int i = 0; i < fieldMapping.length; i++) { + this.fieldMapping[i] = i; + } + + } + + public void setFieldMapping(int[] fieldMapping) { + this.fieldMapping = fieldMapping; + // adapt result type + + TypeInformation[] fieldTypes = new TypeInformation[fieldMapping.length]; + String[] fieldNames = new String[fieldMapping.length]; + for (int i = 0; i < fieldMapping.length; i++) { + fieldTypes[i] = this.rowType.getTypeAt(fieldMapping[i]); + fieldNames[i] = this.rowType.getFieldNames()[fieldMapping[i]]; + } + this.rowType = new RowTypeInfo(fieldTypes, fieldNames); + } + + private boolean[] computeProjectionMask() { + boolean[] projectionMask = new boolean[schema.getMaximumId() + 1]; + for (int inIdx : fieldMapping) { + TypeDescription fieldSchema = schema.getChildren().get(inIdx); + for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) { + projectionMask[i] = true; + } + } + return projectionMask; + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + this.rows = new Row[BATCH_SIZE]; + for (int i = 0; i < BATCH_SIZE; i++) { + rows[i] = new Row(fieldMapping.length); + } + } + + @Override + public void open(FileInputSplit fileSplit) throws IOException { + + this.currentSplit = fileSplit; + Preconditions.checkArgument(this.splitStart == 0, "ORC files must be read from the start."); + + if (LOG.isDebugEnabled()) { + LOG.debug("Opening ORC file " + fileSplit.getPath()); + } + + org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(fileSplit.getPath().getPath()); + + Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(config)); + + Reader.Options options = orcReader.options() + .range(fileSplit.getStart(), fileSplit.getLength()) + .useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(config)) + .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(config)) + .tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(config)); + + options.include(computeProjectionMask()); + + // check that schema of file is as expected + if (!this.schema.equals(orcReader.getSchema())) { + + throw new RuntimeException("Invalid schema for file at " + this.filePath + + " Expected:" + this.schema + " Actual: " + orcReader.getSchema()); + } + + this.orcRowsReader = orcReader.rows(options); + + // assign ids + this.schema.getId(); + + this.rowBatch = schema.createRowBatch(BATCH_SIZE); + rowInBatch = 0; + } + + @Override + public void close() throws IOException { + + if (orcRowsReader != null) { + this.orcRowsReader.close(); + } + this.orcRowsReader = null; + + } + + @Override + public void closeInputFormat() throws IOException { + this.rows = null; + this.rows = null; + this.schema = null; + this.rowBatch = null; + } + + @Override + public boolean reachedEnd() throws IOException { + return !ensureBatch(); + } + + private boolean ensureBatch() throws IOException { + + if (rowInBatch >= rowBatch.size) { + rowInBatch = 0; + boolean moreRows = orcRowsReader.nextBatch(rowBatch); + + if (moreRows) { + // read rows + fillRows(rows, schema, rowBatch, fieldMapping); + } + return moreRows; + } + + return true; + } + + @Override + public Row nextRecord(Row reuse) throws IOException { + return rows[this.rowInBatch++]; + } + + @Override + public TypeInformation getProducedType() { + return rowType; + } + + // -------------------------------------------------------------------------------------------- + // Custom serialization methods + // -------------------------------------------------------------------------------------------- + + private void writeObject(ObjectOutputStream out) throws IOException { + this.config.write(out); + out.writeUTF(schema.toString()); + + out.writeInt(fieldMapping.length); + for (int f : fieldMapping) { + out.writeInt(f); + } + + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + + org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); + configuration.readFields(in); + + if (this.config == null) { + this.config = configuration; + } + this.schema = TypeDescription.fromString(in.readUTF()); + + this.fieldMapping = new int[in.readInt()]; + for (int i = 0; i < fieldMapping.length; i++) { + this.fieldMapping[i] = in.readInt(); + } + + } + +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java new file mode 100644 index 0000000000000..3de6ab31010a5 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java @@ -0,0 +1,142 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.apache.flink.types.Row; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; + +import java.net.URL; +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for {@link OrcTableSource}. + */ +public class OrcTableSourceITCase extends MultipleProgramsTestBase { + + private static final String TEST1_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + + + private static final String[] TEST1_DATA = new String[] { + "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", + "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + + "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; + + public OrcTableSourceITCase() { + super(TestExecutionMode.COLLECTION); + } + + @Test + public void testOrcTableSource() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + + assert (test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + tEnv.registerTableSource("orcTable", orc); + + String query = "Select * from orcTable"; + Table t = tEnv.sql(query); + + DataSet dataSet = tEnv.toDataSet(t, Row.class); + List records = dataSet.collect(); + + Assert.assertEquals(records.size(), 2); + + List actualRecords = new ArrayList<>(); + for (Row record : records) { + Assert.assertEquals(record.getArity(), 12); + actualRecords.add(record.toString()); + } + + Assert.assertThat(actualRecords, CoreMatchers.hasItems(TEST1_DATA)); + } + + @Test + public void testOrcTableProjection() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + + assert(test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + tEnv.registerTableSource("orcTable", orc); + + String query = "Select middle,list,map from orcTable"; + Table t = tEnv.sql(query); + + String[] colNames = new String[] {"middle", "list", "map"}; + + RowTypeInfo rowTypeInfo = new RowTypeInfo( + new TypeInformation[] { + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}, + new String[] {"int1", "string1"}); + + RowTypeInfo structTypeInfo = new RowTypeInfo( + new TypeInformation[] {ObjectArrayTypeInfo.getInfoFor(rowTypeInfo)}, + new String[] {"list"}); + + TypeInformation[] colTypes = new TypeInformation[] { + structTypeInfo, + ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), + new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) + }; + + TableSchema actualTableSchema = new TableSchema(colNames, colTypes); + + Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); + Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); + Assert.assertEquals(actualTableSchema.toString(), t.getSchema().toString()); + + DataSet dataSet = tEnv.toDataSet(t, Row.class); + List records = dataSet.collect(); + + Assert.assertEquals(records.size(), 2); + for (Row record: records) { + Assert.assertEquals(record.getArity(), 3); + } + + } + +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java new file mode 100644 index 0000000000000..c285054c6e155 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java @@ -0,0 +1,113 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.java.BatchTableEnvironment; + +import org.junit.Assert; +import org.junit.Test; + +import java.net.URL; + +/** + * Unit Tests for {@link OrcTableSource}. + */ +public class OrcTableSourceTest { + + private static final String TEST1_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + + @Test + public void testOrcSchema() throws Exception { + + assert(test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + String expectedSchema = "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer, long1: Long, " + + "float1: Float, double1: Double, bytes1: byte[], string1: String, " + + "middle: Row(list: ObjectArrayTypeInfo), " + + "list: ObjectArrayTypeInfo, " + + "map: Map)"; + + Assert.assertEquals(expectedSchema, orc.getReturnType().toString()); + + } + + @Test + public void testOrcTableSchema() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + + assert(test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + tEnv.registerTableSource("orcTable", orc); + String query = "Select * from orcTable"; + Table t = tEnv.sql(query); + + String[] colNames = new String[] { + "boolean1", "byte1", "short1", "int1", "long1", "float1", + "double1", "bytes1", "string1", "list", "list0", "map" + }; + + RowTypeInfo rowTypeInfo = new RowTypeInfo( + new TypeInformation[] { + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}, + new String[] {"int1", "string1"}); + + TypeInformation[] colTypes = new TypeInformation[] { + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.BYTE_TYPE_INFO, + BasicTypeInfo.SHORT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.FLOAT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), + ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), + new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) + }; + TableSchema expectedTableSchema = new TableSchema(colNames, colTypes); + + Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); + Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); + Assert.assertEquals(expectedTableSchema.toString(), t.getSchema().toString()); + + } + +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java new file mode 100644 index 0000000000000..60008a0838612 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java @@ -0,0 +1,472 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for the {@link RowOrcInputFormat}. + */ + +public class RowOrcInputFormatTest { + + private RowOrcInputFormat rowOrcInputFormat; + + @After + public void tearDown() throws IOException { + if (rowOrcInputFormat != null) { + rowOrcInputFormat.close(); + rowOrcInputFormat.closeInputFormat(); + } + rowOrcInputFormat = null; + } + + private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + + private static final String TEST1_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + private static final String[] TEST1_DATA = new String[] { + "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", + "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + + "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; + + private static final String[] TEST1_PROJECTED_DATA = new String[] { + "{},[3,good, 4,bad],[1,bye, 2,sigh],hi,[0, 1, 2, 3, 4],-15.0,1.0,9223372036854775807,65536,1024,1,false", + "{chani=5,chani, mauddib=1,mauddib},[100000000,cat, -100000,in, 1234,hat],[1,bye, 2,sigh],bye," + + "[],-5.0,2.0,9223372036854775807,65536,2048,100,true" }; + + private static final String TEST1_INVALID_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + @Test(expected = FileNotFoundException.class) + public void testInvalidPath() throws IOException{ + + rowOrcInputFormat = new RowOrcInputFormat("TestOrcFile.test2.orc", TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.open(inputSplits[0]); + + } + + @Test(expected = RuntimeException.class) + public void testInvalidSchema() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_INVALID_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.open(inputSplits[0]); + + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testInvalidProjection() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + int[] projectionMask = {14}; + rowOrcInputFormat.setFieldMapping(projectionMask); + } + + @Test + public void testMajorDataTypes() throws IOException{ + + // test for boolean,byte,short,int,long,float,double,bytes,string,struct,list,map + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.toString(), TEST1_DATA[count++]); + } + } + } + + @Test + public void testProjection() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + int[] projectionMask = {11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}; + rowOrcInputFormat.setFieldMapping(projectionMask); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.toString(), TEST1_PROJECTED_DATA[count++]); + } + } + + } + + @Test + public void testTimeStampAndDate() throws IOException{ + + URL expectedDataURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.dat"); + assert(expectedDataURL != null); + List expectedTimeStampAndDate = Files.readAllLines(Paths.get(expectedDataURL.getPath())); + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct"; + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + List actualTimeStampAndDate = new ArrayList<>(); + + Row row = null; + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + count++; + if (count <= 10000) { + actualTimeStampAndDate.add(row.getField(0) + "," + row.getField(1)); + } + + } + } + Assert.assertEquals(count, 70000); + Assert.assertEquals(expectedTimeStampAndDate.size(), actualTimeStampAndDate.size()); + Assert.assertEquals(expectedTimeStampAndDate.toString(), actualTimeStampAndDate.toString()); + + } + + @Test + public void testDecimal() throws IOException{ + + URL expectedDataURL = getClass().getClassLoader().getResource("decimal.dat"); + List expectedDecimal = Files.readAllLines(Paths.get(expectedDataURL.getPath())); + + URL testInputURL = getClass().getClassLoader().getResource("decimal.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct<_col0:decimal(10,5)>"; + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + List actualDecimal = new ArrayList<>(); + + Row row = null; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + actualDecimal.add(row.getField(0)); + } + } + + Assert.assertEquals(expectedDecimal.size(), actualDecimal.size()); + Assert.assertEquals(expectedDecimal.toString(), actualDecimal.toString()); + + } + + @Test + public void testEmptyFile() throws IOException{ + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.emptyFile.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + + rowOrcInputFormat = new RowOrcInputFormat(path, TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = new Row(1); + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + count++; + } + } + + Assert.assertEquals(count, 0); + } + + @Test + public void testLargeFile() throws IOException{ + + URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + + "_col5:string,_col6:int,_col7:int,_col8:int>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = new Row(1); + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + count++; + } + } + + Assert.assertEquals(count, 1920800); + } + + @Test + public void testProducedType() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + rowOrcInputFormat.open(inputSplits[0]); + + TypeInformation type = rowOrcInputFormat.getProducedType(); + Assert.assertEquals(type.toString(), "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer," + + " long1: Long, float1: Float, double1: Double, bytes1: byte[], string1: String," + + " middle: Row(list: ObjectArrayTypeInfo)," + + " list: ObjectArrayTypeInfo," + + " map: Map)"); + + } + + @Test + public void testProducedTypeWithProjection() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + int[] projectionMask = {9, 10, 11}; + rowOrcInputFormat.setFieldMapping(projectionMask); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + rowOrcInputFormat.open(inputSplits[0]); + + TypeInformation type = rowOrcInputFormat.getProducedType(); + Assert.assertEquals(type.toString(), "Row(middle: Row(list: ObjectArrayTypeInfo)," + + " list: ObjectArrayTypeInfo," + + " map: Map)"); + + } + + @Test + public void testLongList() throws Exception { + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listlong.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + long count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.getArity(), 1); + Object object = row.getField(0); + long[] l = (long[]) object; + + Assert.assertEquals(l.length, 2); + if (count < 50) { + Assert.assertArrayEquals(l, new long[]{count, count + 1}); + } + else { + Assert.assertArrayEquals(l, new long[]{0L, 0L}); + } + count = count + 2; + } + } + Assert.assertEquals(count, 100); + } + + @Test + public void testStringList() throws Exception { + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.liststring.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + long count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.getArity(), 1); + Object object = row.getField(0); + String[] l = (String[]) object; + + Assert.assertEquals(l.length, 2); + Assert.assertArrayEquals(l, new String[]{"hello" + count, "hello" + (count + 1) }); + count = count + 2; + } + } + Assert.assertEquals(count, 200); + } + + @Test + public void testListOfListOfStructOfLong() throws Exception { + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listliststructlong.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct>>>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + long count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.getArity(), 1); + + Object[] objects = (Object[]) row.getField(0); + Assert.assertEquals(objects.length, 1); + + Object[] objects1 = (Object[]) objects[0]; + Assert.assertEquals(objects1.length, 1); + + Row[] nestedRows = Arrays.copyOf(objects1, objects1.length, Row[].class); + Assert.assertEquals(nestedRows.length, 1); + + Assert.assertEquals(nestedRows[0].getArity(), 1); + + Assert.assertEquals(nestedRows[0].getField(0), count); + + count++; + } + } + Assert.assertEquals(count, 100); + } + + @Test + public void testSplit() throws IOException{ + + URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + + "_col5:string,_col6:int,_col7:int,_col8:int>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(10); + + Assert.assertEquals(inputSplits.length, 10); + + Row row = null; + int countTotalRecords = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + int countSplitRecords = 0; + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + countSplitRecords++; + } + Assert.assertNotEquals(countSplitRecords, 1920800); + countTotalRecords += countSplitRecords; + } + + Assert.assertEquals(countTotalRecords, 1920800); + } + +} diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc new file mode 100644 index 0000000000000000000000000000000000000000..ecdadcbff134615d7eefcb740d55fe710cee059b GIT binary patch literal 523 zcmaix$xg#C5Qe>EnJmx{S{5I3LY$gwPl(Ki0VfCx86O*nAn33yj{0RALJfJ%g= z-yY9=v&depmn0*ZJW?2kTu94yT=M+Mv7+18fd!+++rKoYVb8*(6m+$6w2JQ%VP zoBWuE$qYVmaJZY`s{`<^gga2tKpn+nltV!*rn5eyFL-h~g3j5(B6iCb91v J8R&e0I%!PbNv8+gamelvDCX%*5(l1yh8mvCbhTm+DWG ztwa_dU#Z@DIhV8B;OhB~-R0@ul$5?`y7+ZIH($|mqj$k$U8AmcUDj$pnUI%_5|g`{ z_pu*k`hH3v+BCv+irL?_?^!%_DtEq7PmP?caWOG^s&}!BC1(=&Qg19mLDn{_(&)5@POtR&tadvR9xSX{NJBb%7= zt4}R)n|QOUwU+;q*euNVx90xK%Z_|`W-cOGt!Dr3c^5Rx+Ua`FUijqE)I)rq=R7`g zH81qX@ei{tH+_qD-ms;qt}$#=>vH zo3i@)wRdg~T6DH-QB3@q2N{blifa8lXk2r0E7z_EGHI^AbR}Yc7b^ZTJC>R<@7R;f z(~G7?F<){0!_LXTa3GnH0hwZ8n!v!|+ilI);K1QB-9Mt~?6G5=Yumr=Qc^sY7I)8} zW#`&P-7~h8M_#vt`t3NibMEb>-!D}kl{Ahm^cI`bzxjmgsl#`!vTIDP|7pr^226;T z9wa^V;7mB+^r>(OM`3{hhfMO6h{w_g7#P+8WfR119M?K}_MlF|Y*(j{gVD;ao&C-3 z9TNk)Y_}V!Z{BevpmS=ocgI9dCqw6y$OD^xR=#ZW@{!SD(+ty*(PZR_@UZI>QRL%e pYfSDD%C2FSkcr#C%)k*Pz$nq6pv1($qakO&*7Sv$*+0lx3;+xlS?&M; literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc new file mode 100644 index 0000000000000000000000000000000000000000..648ea18084ed061f03afc0e6453778c1d617d280 GIT binary patch literal 627 zcmV-(0*w7nQbQ2{015^Y1PC%v04M+eV7f4AZ$rdMGkni%gxIAM^?8)@NzR1G03_ zQ~I=$l8)`w(zKm!g}HebqmEB z$(}8Vks5ovjmb^kX-N*YbktC-?e0-&YOo*RHb1o4jT#G6Uq?2aIh-Ryw8c%=ROlA(7bVdZ!>jqI83wO`#jQ!BZ1BcvtYw$6ASzM8ou-0iYb#i<<5?Y8 zcR86{`PbuAyu-G0%+|g?R_Q}}d}YnIwm1mjJj`x;n`S?P$jk3QlgcznjWaUDr?(61 z3Bbqtmr$)JCquHRII7bMwFr!QAr#AIzr0d}oOz2A#o{B}df>n)gR+|a0pW23?LA#& zcAvMyU=0E<{s9R90KhQ<01*HHz%c*h0OOM8V&O0fV1Sam z94SKV91H?U5|e}j7yy(20OMd5_{=QD&A}|NfLTIF!A`?SiI0Ovh>=l>J-0F^v$(`i ziIsy(h>=N&iG#(!$clx-D1ZS<@^YjIv2!p8BuPvX4q$M2!43cjO%MSXfFL3Q01PM` NFb0D30|QS|Llz^%2`T^p literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc new file mode 100644 index 0000000000000000000000000000000000000000..75a5f2a10a3fe703394fd3700e56d06149f2785b GIT binary patch literal 1298 zcmV+t1?~DzQbQ2{015^Y1PEkM04M+e|ol`QeugC00RIb1OOdU!HN?>5PjWgV`Yg@6M_T} zQj^_<=xJ8=;;j=RGjzO2ys!ex3fVHFMPf}N0~_|J$b(oPe=BEqIe z@P+Q{3BlH(q^KS_Q66s*qbnS;<%}RL3%{1=BQ6o&kDZt%K`b7imB4z^u%)5M`$3|1 z!$5h0e8-8HVU#(PcXGQlE7Kax-Uj?& z!#wOIPOfEmByAH>txpV$-a8&5P(_YG26PHYo!MV84}7~>7RQl&sAf< zk-72I{bY7VtVnN>nrTZf<~#-2&t3rAgjOZ}g{>kyvM5fFwQb86006*a0stBV z06o(|uADFw1<@^07-PJkNPiG1`=3RjicH>&qi(4C^XvETKgZ9%%X`oFj`v>go$tNh zyWab}cfXe&_Gx?^pGN=%B+x(vl~d*HH>jK{r^=~vs+=mP%BeC`hRW!#s0@{%GE|1j zP#G$h%B6CtT)h#MOXX6zR4$cEWvWb-sWMe&FIHu$OqHoJRc@7AZk4<5qjIa< zD!0l~St?6qsVtSHvii&_OJ%7%Dv!#e@~Av2kIJL+REx@^vQ@UqR@o|BWvgtJt+G|N z3Wp4b4u=qj5{DFr7Ka#z8iyQ5-v5mAzL39?zm&h0znH(8zntrtf~KN5`+YS{QB&2F zHFZs4Q`wX@wM}s|`oHB(eN*66I3-SvQ{+@RWlo)Q_5LcIQm57_cB-9nr`{=eDxQ*O z_C1Q8s;BI!dkUY*r}U|Pil6G|?)%n14S*Iv6QB*y2xtW~1KI%%fmJU}fwn+npf%7O zXb&_9S_DmkHo;SWt%7DjyP#pvGH4pK4H^fngXY2Be*qN$00#!ZSqA{XV+a6%001?K zK?;CC36VJHcrhQ)_j>Tb#)6j#Gf5Ahjca;@_Q=nTv)p)8Qgd?h4TRXiw56rQ67c{A0IUE2 z<6su}%^}6Z!7MP5RbmOVg0n`75+4VT5F?`$dv0Y;W^sw35-SIn5F?Wk69>CNiWLh- zN&o|tl;$|Ws3gM0mXVs1lW!oz4yG+FC6FgUzm2LK3J5CIr~AR+<)3@9fs27>eh I15Z*z7I1J@L;wH) literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc new file mode 100644 index 0000000000000000000000000000000000000000..4fb0beff868971efb653739fe6ae47a37e4a1c66 GIT binary patch literal 1711 zcma)53s6%>6y2AX#Fs$e5wL~=SI1j7^ppUN+?iL7eMX!KP?|U(2PGykr zMnDEm3tYcYD8Tt@fkL4XffW78WSpFmVtur#7q0|w$w2R*$@rIbh!<9^<^mL0Lk+rG z7y%GVJw2ukQ|}5C{s@8!l@@f?W3G^{))X}OTNY*dP!`yK*kA&C*IDdcYs=phkq172 zl;I+!ixKH$=mn9X%L9%Oe|Hf##E4cjDk@BfpcCUH;z))m5B!%Ri3irhN|VJ(Ba8$c zuJ~RTmmMt)WsJ?{hw@(UIzZdC(#W!?aRcQS*!7#jZVTou4o-nHTCV#KW zY{|-Mw5FuLKI&`p5VZ497v}k2IwSz*xk~_eg@|YXgdiBe7i^Rw1OVVq2jF7(tUfX? zU@yP`$Tq>iWdIN^0LUFwkm0NbKnG$70VP5mcZ3H|krHSfJh>oc+L%Xy3~Fr+4_uU| zO=If=0FxHnT})gBzyTr@BEyC04BZ6)rNPT61?B*VqoYN_Fj08eeu*?PY(Kogo#?uj zr7g5DhRsBVNhN#3;zc5fP^%uOjrnh-!L9pc6=7+7%vM9Q$0swq zYndaP)cv+kU8bw-28{y;gN3drVcdBz9t7dhuMmb;CnuK$&m)N*ABg3G{t2zI3h)7JHQ`w$?;Yul6&u ziwZDH8s)Zzzn)e#+n)4aslGibo1Dd}Z!(O7*z|E%ly3M1cAf zYt1z)_kqG1Uqzcg?G&q_?Z#etp>J!{dl`+{7n_4>OPBj?lslX5vp~G+Qn9goge**% z^tY~Wx04^}8SVY?Oi#J%6t%R_-}~zcW<%i-&9}MC=;kgDljiWlR`QBJa-XMfic6|J zaZJ9)>Wb5cGce^ly-Ls2ct8(Y?Re=hf9M( zLo**0tb4TQEc>Otekshx*T)2S*OK9$LOJF>!JUR*2#VMT+3 literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat new file mode 100644 index 0000000000000..59b933ddf0b5c --- /dev/null +++ b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat @@ -0,0 +1,10000 @@ +1900-05-05 12:34:56.1,1900-12-25 +1900-05-05 12:34:56.1001,1900-12-25 +1900-05-05 12:34:56.1002,1900-12-25 +1900-05-05 12:34:56.1003,1900-12-25 +1900-05-05 12:34:56.1004,1900-12-25 +1900-05-05 12:34:56.1005,1900-12-25 +1900-05-05 12:34:56.1006,1900-12-25 +1900-05-05 12:34:56.1007,1900-12-25 +1900-05-05 12:34:56.1008,1900-12-25 +1900-05-05 12:34:56.1009,1900-12-25 +1900-05-05 12:34:56.101,1900-12-25 +1900-05-05 12:34:56.1011,1900-12-25 +1900-05-05 12:34:56.1012,1900-12-25 +1900-05-05 12:34:56.1013,1900-12-25 +1900-05-05 12:34:56.1014,1900-12-25 +1900-05-05 12:34:56.1015,1900-12-25 +1900-05-05 12:34:56.1016,1900-12-25 +1900-05-05 12:34:56.1017,1900-12-25 +1900-05-05 12:34:56.1018,1900-12-25 +1900-05-05 12:34:56.1019,1900-12-25 +1900-05-05 12:34:56.102,1900-12-25 +1900-05-05 12:34:56.1021,1900-12-25 +1900-05-05 12:34:56.1022,1900-12-25 +1900-05-05 12:34:56.1023,1900-12-25 +1900-05-05 12:34:56.1024,1900-12-25 +1900-05-05 12:34:56.1025,1900-12-25 +1900-05-05 12:34:56.1026,1900-12-25 +1900-05-05 12:34:56.1027,1900-12-25 +1900-05-05 12:34:56.1028,1900-12-25 +1900-05-05 12:34:56.1029,1900-12-25 +1900-05-05 12:34:56.103,1900-12-25 +1900-05-05 12:34:56.1031,1900-12-25 +1900-05-05 12:34:56.1032,1900-12-25 +1900-05-05 12:34:56.1033,1900-12-25 +1900-05-05 12:34:56.1034,1900-12-25 +1900-05-05 12:34:56.1035,1900-12-25 +1900-05-05 12:34:56.1036,1900-12-25 +1900-05-05 12:34:56.1037,1900-12-25 +1900-05-05 12:34:56.1038,1900-12-25 +1900-05-05 12:34:56.1039,1900-12-25 +1900-05-05 12:34:56.104,1900-12-25 +1900-05-05 12:34:56.1041,1900-12-25 +1900-05-05 12:34:56.1042,1900-12-25 +1900-05-05 12:34:56.1043,1900-12-25 +1900-05-05 12:34:56.1044,1900-12-25 +1900-05-05 12:34:56.1045,1900-12-25 +1900-05-05 12:34:56.1046,1900-12-25 +1900-05-05 12:34:56.1047,1900-12-25 +1900-05-05 12:34:56.1048,1900-12-25 +1900-05-05 12:34:56.1049,1900-12-25 +1900-05-05 12:34:56.105,1900-12-25 +1900-05-05 12:34:56.1051,1900-12-25 +1900-05-05 12:34:56.1052,1900-12-25 +1900-05-05 12:34:56.1053,1900-12-25 +1900-05-05 12:34:56.1054,1900-12-25 +1900-05-05 12:34:56.1055,1900-12-25 +1900-05-05 12:34:56.1056,1900-12-25 +1900-05-05 12:34:56.1057,1900-12-25 +1900-05-05 12:34:56.1058,1900-12-25 +1900-05-05 12:34:56.1059,1900-12-25 +1900-05-05 12:34:56.106,1900-12-25 +1900-05-05 12:34:56.1061,1900-12-25 +1900-05-05 12:34:56.1062,1900-12-25 +1900-05-05 12:34:56.1063,1900-12-25 +1900-05-05 12:34:56.1064,1900-12-25 +1900-05-05 12:34:56.1065,1900-12-25 +1900-05-05 12:34:56.1066,1900-12-25 +1900-05-05 12:34:56.1067,1900-12-25 +1900-05-05 12:34:56.1068,1900-12-25 +1900-05-05 12:34:56.1069,1900-12-25 +1900-05-05 12:34:56.107,1900-12-25 +1900-05-05 12:34:56.1071,1900-12-25 +1900-05-05 12:34:56.1072,1900-12-25 +1900-05-05 12:34:56.1073,1900-12-25 +1900-05-05 12:34:56.1074,1900-12-25 +1900-05-05 12:34:56.1075,1900-12-25 +1900-05-05 12:34:56.1076,1900-12-25 +1900-05-05 12:34:56.1077,1900-12-25 +1900-05-05 12:34:56.1078,1900-12-25 +1900-05-05 12:34:56.1079,1900-12-25 +1900-05-05 12:34:56.108,1900-12-25 +1900-05-05 12:34:56.1081,1900-12-25 +1900-05-05 12:34:56.1082,1900-12-25 +1900-05-05 12:34:56.1083,1900-12-25 +1900-05-05 12:34:56.1084,1900-12-25 +1900-05-05 12:34:56.1085,1900-12-25 +1900-05-05 12:34:56.1086,1900-12-25 +1900-05-05 12:34:56.1087,1900-12-25 +1900-05-05 12:34:56.1088,1900-12-25 +1900-05-05 12:34:56.1089,1900-12-25 +1900-05-05 12:34:56.109,1900-12-25 +1900-05-05 12:34:56.1091,1900-12-25 +1900-05-05 12:34:56.1092,1900-12-25 +1900-05-05 12:34:56.1093,1900-12-25 +1900-05-05 12:34:56.1094,1900-12-25 +1900-05-05 12:34:56.1095,1900-12-25 +1900-05-05 12:34:56.1096,1900-12-25 +1900-05-05 12:34:56.1097,1900-12-25 +1900-05-05 12:34:56.1098,1900-12-25 +1900-05-05 12:34:56.1099,1900-12-25 +1900-05-05 12:34:56.11,1900-12-25 +1900-05-05 12:34:56.1101,1900-12-25 +1900-05-05 12:34:56.1102,1900-12-25 +1900-05-05 12:34:56.1103,1900-12-25 +1900-05-05 12:34:56.1104,1900-12-25 +1900-05-05 12:34:56.1105,1900-12-25 +1900-05-05 12:34:56.1106,1900-12-25 +1900-05-05 12:34:56.1107,1900-12-25 +1900-05-05 12:34:56.1108,1900-12-25 +1900-05-05 12:34:56.1109,1900-12-25 +1900-05-05 12:34:56.111,1900-12-25 +1900-05-05 12:34:56.1111,1900-12-25 +1900-05-05 12:34:56.1112,1900-12-25 +1900-05-05 12:34:56.1113,1900-12-25 +1900-05-05 12:34:56.1114,1900-12-25 +1900-05-05 12:34:56.1115,1900-12-25 +1900-05-05 12:34:56.1116,1900-12-25 +1900-05-05 12:34:56.1117,1900-12-25 +1900-05-05 12:34:56.1118,1900-12-25 +1900-05-05 12:34:56.1119,1900-12-25 +1900-05-05 12:34:56.112,1900-12-25 +1900-05-05 12:34:56.1121,1900-12-25 +1900-05-05 12:34:56.1122,1900-12-25 +1900-05-05 12:34:56.1123,1900-12-25 +1900-05-05 12:34:56.1124,1900-12-25 +1900-05-05 12:34:56.1125,1900-12-25 +1900-05-05 12:34:56.1126,1900-12-25 +1900-05-05 12:34:56.1127,1900-12-25 +1900-05-05 12:34:56.1128,1900-12-25 +1900-05-05 12:34:56.1129,1900-12-25 +1900-05-05 12:34:56.113,1900-12-25 +1900-05-05 12:34:56.1131,1900-12-25 +1900-05-05 12:34:56.1132,1900-12-25 +1900-05-05 12:34:56.1133,1900-12-25 +1900-05-05 12:34:56.1134,1900-12-25 +1900-05-05 12:34:56.1135,1900-12-25 +1900-05-05 12:34:56.1136,1900-12-25 +1900-05-05 12:34:56.1137,1900-12-25 +1900-05-05 12:34:56.1138,1900-12-25 +1900-05-05 12:34:56.1139,1900-12-25 +1900-05-05 12:34:56.114,1900-12-25 +1900-05-05 12:34:56.1141,1900-12-25 +1900-05-05 12:34:56.1142,1900-12-25 +1900-05-05 12:34:56.1143,1900-12-25 +1900-05-05 12:34:56.1144,1900-12-25 +1900-05-05 12:34:56.1145,1900-12-25 +1900-05-05 12:34:56.1146,1900-12-25 +1900-05-05 12:34:56.1147,1900-12-25 +1900-05-05 12:34:56.1148,1900-12-25 +1900-05-05 12:34:56.1149,1900-12-25 +1900-05-05 12:34:56.115,1900-12-25 +1900-05-05 12:34:56.1151,1900-12-25 +1900-05-05 12:34:56.1152,1900-12-25 +1900-05-05 12:34:56.1153,1900-12-25 +1900-05-05 12:34:56.1154,1900-12-25 +1900-05-05 12:34:56.1155,1900-12-25 +1900-05-05 12:34:56.1156,1900-12-25 +1900-05-05 12:34:56.1157,1900-12-25 +1900-05-05 12:34:56.1158,1900-12-25 +1900-05-05 12:34:56.1159,1900-12-25 +1900-05-05 12:34:56.116,1900-12-25 +1900-05-05 12:34:56.1161,1900-12-25 +1900-05-05 12:34:56.1162,1900-12-25 +1900-05-05 12:34:56.1163,1900-12-25 +1900-05-05 12:34:56.1164,1900-12-25 +1900-05-05 12:34:56.1165,1900-12-25 +1900-05-05 12:34:56.1166,1900-12-25 +1900-05-05 12:34:56.1167,1900-12-25 +1900-05-05 12:34:56.1168,1900-12-25 +1900-05-05 12:34:56.1169,1900-12-25 +1900-05-05 12:34:56.117,1900-12-25 +1900-05-05 12:34:56.1171,1900-12-25 +1900-05-05 12:34:56.1172,1900-12-25 +1900-05-05 12:34:56.1173,1900-12-25 +1900-05-05 12:34:56.1174,1900-12-25 +1900-05-05 12:34:56.1175,1900-12-25 +1900-05-05 12:34:56.1176,1900-12-25 +1900-05-05 12:34:56.1177,1900-12-25 +1900-05-05 12:34:56.1178,1900-12-25 +1900-05-05 12:34:56.1179,1900-12-25 +1900-05-05 12:34:56.118,1900-12-25 +1900-05-05 12:34:56.1181,1900-12-25 +1900-05-05 12:34:56.1182,1900-12-25 +1900-05-05 12:34:56.1183,1900-12-25 +1900-05-05 12:34:56.1184,1900-12-25 +1900-05-05 12:34:56.1185,1900-12-25 +1900-05-05 12:34:56.1186,1900-12-25 +1900-05-05 12:34:56.1187,1900-12-25 +1900-05-05 12:34:56.1188,1900-12-25 +1900-05-05 12:34:56.1189,1900-12-25 +1900-05-05 12:34:56.119,1900-12-25 +1900-05-05 12:34:56.1191,1900-12-25 +1900-05-05 12:34:56.1192,1900-12-25 +1900-05-05 12:34:56.1193,1900-12-25 +1900-05-05 12:34:56.1194,1900-12-25 +1900-05-05 12:34:56.1195,1900-12-25 +1900-05-05 12:34:56.1196,1900-12-25 +1900-05-05 12:34:56.1197,1900-12-25 +1900-05-05 12:34:56.1198,1900-12-25 +1900-05-05 12:34:56.1199,1900-12-25 +1900-05-05 12:34:56.12,1900-12-25 +1900-05-05 12:34:56.1201,1900-12-25 +1900-05-05 12:34:56.1202,1900-12-25 +1900-05-05 12:34:56.1203,1900-12-25 +1900-05-05 12:34:56.1204,1900-12-25 +1900-05-05 12:34:56.1205,1900-12-25 +1900-05-05 12:34:56.1206,1900-12-25 +1900-05-05 12:34:56.1207,1900-12-25 +1900-05-05 12:34:56.1208,1900-12-25 +1900-05-05 12:34:56.1209,1900-12-25 +1900-05-05 12:34:56.121,1900-12-25 +1900-05-05 12:34:56.1211,1900-12-25 +1900-05-05 12:34:56.1212,1900-12-25 +1900-05-05 12:34:56.1213,1900-12-25 +1900-05-05 12:34:56.1214,1900-12-25 +1900-05-05 12:34:56.1215,1900-12-25 +1900-05-05 12:34:56.1216,1900-12-25 +1900-05-05 12:34:56.1217,1900-12-25 +1900-05-05 12:34:56.1218,1900-12-25 +1900-05-05 12:34:56.1219,1900-12-25 +1900-05-05 12:34:56.122,1900-12-25 +1900-05-05 12:34:56.1221,1900-12-25 +1900-05-05 12:34:56.1222,1900-12-25 +1900-05-05 12:34:56.1223,1900-12-25 +1900-05-05 12:34:56.1224,1900-12-25 +1900-05-05 12:34:56.1225,1900-12-25 +1900-05-05 12:34:56.1226,1900-12-25 +1900-05-05 12:34:56.1227,1900-12-25 +1900-05-05 12:34:56.1228,1900-12-25 +1900-05-05 12:34:56.1229,1900-12-25 +1900-05-05 12:34:56.123,1900-12-25 +1900-05-05 12:34:56.1231,1900-12-25 +1900-05-05 12:34:56.1232,1900-12-25 +1900-05-05 12:34:56.1233,1900-12-25 +1900-05-05 12:34:56.1234,1900-12-25 +1900-05-05 12:34:56.1235,1900-12-25 +1900-05-05 12:34:56.1236,1900-12-25 +1900-05-05 12:34:56.1237,1900-12-25 +1900-05-05 12:34:56.1238,1900-12-25 +1900-05-05 12:34:56.1239,1900-12-25 +1900-05-05 12:34:56.124,1900-12-25 +1900-05-05 12:34:56.1241,1900-12-25 +1900-05-05 12:34:56.1242,1900-12-25 +1900-05-05 12:34:56.1243,1900-12-25 +1900-05-05 12:34:56.1244,1900-12-25 +1900-05-05 12:34:56.1245,1900-12-25 +1900-05-05 12:34:56.1246,1900-12-25 +1900-05-05 12:34:56.1247,1900-12-25 +1900-05-05 12:34:56.1248,1900-12-25 +1900-05-05 12:34:56.1249,1900-12-25 +1900-05-05 12:34:56.125,1900-12-25 +1900-05-05 12:34:56.1251,1900-12-25 +1900-05-05 12:34:56.1252,1900-12-25 +1900-05-05 12:34:56.1253,1900-12-25 +1900-05-05 12:34:56.1254,1900-12-25 +1900-05-05 12:34:56.1255,1900-12-25 +1900-05-05 12:34:56.1256,1900-12-25 +1900-05-05 12:34:56.1257,1900-12-25 +1900-05-05 12:34:56.1258,1900-12-25 +1900-05-05 12:34:56.1259,1900-12-25 +1900-05-05 12:34:56.126,1900-12-25 +1900-05-05 12:34:56.1261,1900-12-25 +1900-05-05 12:34:56.1262,1900-12-25 +1900-05-05 12:34:56.1263,1900-12-25 +1900-05-05 12:34:56.1264,1900-12-25 +1900-05-05 12:34:56.1265,1900-12-25 +1900-05-05 12:34:56.1266,1900-12-25 +1900-05-05 12:34:56.1267,1900-12-25 +1900-05-05 12:34:56.1268,1900-12-25 +1900-05-05 12:34:56.1269,1900-12-25 +1900-05-05 12:34:56.127,1900-12-25 +1900-05-05 12:34:56.1271,1900-12-25 +1900-05-05 12:34:56.1272,1900-12-25 +1900-05-05 12:34:56.1273,1900-12-25 +1900-05-05 12:34:56.1274,1900-12-25 +1900-05-05 12:34:56.1275,1900-12-25 +1900-05-05 12:34:56.1276,1900-12-25 +1900-05-05 12:34:56.1277,1900-12-25 +1900-05-05 12:34:56.1278,1900-12-25 +1900-05-05 12:34:56.1279,1900-12-25 +1900-05-05 12:34:56.128,1900-12-25 +1900-05-05 12:34:56.1281,1900-12-25 +1900-05-05 12:34:56.1282,1900-12-25 +1900-05-05 12:34:56.1283,1900-12-25 +1900-05-05 12:34:56.1284,1900-12-25 +1900-05-05 12:34:56.1285,1900-12-25 +1900-05-05 12:34:56.1286,1900-12-25 +1900-05-05 12:34:56.1287,1900-12-25 +1900-05-05 12:34:56.1288,1900-12-25 +1900-05-05 12:34:56.1289,1900-12-25 +1900-05-05 12:34:56.129,1900-12-25 +1900-05-05 12:34:56.1291,1900-12-25 +1900-05-05 12:34:56.1292,1900-12-25 +1900-05-05 12:34:56.1293,1900-12-25 +1900-05-05 12:34:56.1294,1900-12-25 +1900-05-05 12:34:56.1295,1900-12-25 +1900-05-05 12:34:56.1296,1900-12-25 +1900-05-05 12:34:56.1297,1900-12-25 +1900-05-05 12:34:56.1298,1900-12-25 +1900-05-05 12:34:56.1299,1900-12-25 +1900-05-05 12:34:56.13,1900-12-25 +1900-05-05 12:34:56.1301,1900-12-25 +1900-05-05 12:34:56.1302,1900-12-25 +1900-05-05 12:34:56.1303,1900-12-25 +1900-05-05 12:34:56.1304,1900-12-25 +1900-05-05 12:34:56.1305,1900-12-25 +1900-05-05 12:34:56.1306,1900-12-25 +1900-05-05 12:34:56.1307,1900-12-25 +1900-05-05 12:34:56.1308,1900-12-25 +1900-05-05 12:34:56.1309,1900-12-25 +1900-05-05 12:34:56.131,1900-12-25 +1900-05-05 12:34:56.1311,1900-12-25 +1900-05-05 12:34:56.1312,1900-12-25 +1900-05-05 12:34:56.1313,1900-12-25 +1900-05-05 12:34:56.1314,1900-12-25 +1900-05-05 12:34:56.1315,1900-12-25 +1900-05-05 12:34:56.1316,1900-12-25 +1900-05-05 12:34:56.1317,1900-12-25 +1900-05-05 12:34:56.1318,1900-12-25 +1900-05-05 12:34:56.1319,1900-12-25 +1900-05-05 12:34:56.132,1900-12-25 +1900-05-05 12:34:56.1321,1900-12-25 +1900-05-05 12:34:56.1322,1900-12-25 +1900-05-05 12:34:56.1323,1900-12-25 +1900-05-05 12:34:56.1324,1900-12-25 +1900-05-05 12:34:56.1325,1900-12-25 +1900-05-05 12:34:56.1326,1900-12-25 +1900-05-05 12:34:56.1327,1900-12-25 +1900-05-05 12:34:56.1328,1900-12-25 +1900-05-05 12:34:56.1329,1900-12-25 +1900-05-05 12:34:56.133,1900-12-25 +1900-05-05 12:34:56.1331,1900-12-25 +1900-05-05 12:34:56.1332,1900-12-25 +1900-05-05 12:34:56.1333,1900-12-25 +1900-05-05 12:34:56.1334,1900-12-25 +1900-05-05 12:34:56.1335,1900-12-25 +1900-05-05 12:34:56.1336,1900-12-25 +1900-05-05 12:34:56.1337,1900-12-25 +1900-05-05 12:34:56.1338,1900-12-25 +1900-05-05 12:34:56.1339,1900-12-25 +1900-05-05 12:34:56.134,1900-12-25 +1900-05-05 12:34:56.1341,1900-12-25 +1900-05-05 12:34:56.1342,1900-12-25 +1900-05-05 12:34:56.1343,1900-12-25 +1900-05-05 12:34:56.1344,1900-12-25 +1900-05-05 12:34:56.1345,1900-12-25 +1900-05-05 12:34:56.1346,1900-12-25 +1900-05-05 12:34:56.1347,1900-12-25 +1900-05-05 12:34:56.1348,1900-12-25 +1900-05-05 12:34:56.1349,1900-12-25 +1900-05-05 12:34:56.135,1900-12-25 +1900-05-05 12:34:56.1351,1900-12-25 +1900-05-05 12:34:56.1352,1900-12-25 +1900-05-05 12:34:56.1353,1900-12-25 +1900-05-05 12:34:56.1354,1900-12-25 +1900-05-05 12:34:56.1355,1900-12-25 +1900-05-05 12:34:56.1356,1900-12-25 +1900-05-05 12:34:56.1357,1900-12-25 +1900-05-05 12:34:56.1358,1900-12-25 +1900-05-05 12:34:56.1359,1900-12-25 +1900-05-05 12:34:56.136,1900-12-25 +1900-05-05 12:34:56.1361,1900-12-25 +1900-05-05 12:34:56.1362,1900-12-25 +1900-05-05 12:34:56.1363,1900-12-25 +1900-05-05 12:34:56.1364,1900-12-25 +1900-05-05 12:34:56.1365,1900-12-25 +1900-05-05 12:34:56.1366,1900-12-25 +1900-05-05 12:34:56.1367,1900-12-25 +1900-05-05 12:34:56.1368,1900-12-25 +1900-05-05 12:34:56.1369,1900-12-25 +1900-05-05 12:34:56.137,1900-12-25 +1900-05-05 12:34:56.1371,1900-12-25 +1900-05-05 12:34:56.1372,1900-12-25 +1900-05-05 12:34:56.1373,1900-12-25 +1900-05-05 12:34:56.1374,1900-12-25 +1900-05-05 12:34:56.1375,1900-12-25 +1900-05-05 12:34:56.1376,1900-12-25 +1900-05-05 12:34:56.1377,1900-12-25 +1900-05-05 12:34:56.1378,1900-12-25 +1900-05-05 12:34:56.1379,1900-12-25 +1900-05-05 12:34:56.138,1900-12-25 +1900-05-05 12:34:56.1381,1900-12-25 +1900-05-05 12:34:56.1382,1900-12-25 +1900-05-05 12:34:56.1383,1900-12-25 +1900-05-05 12:34:56.1384,1900-12-25 +1900-05-05 12:34:56.1385,1900-12-25 +1900-05-05 12:34:56.1386,1900-12-25 +1900-05-05 12:34:56.1387,1900-12-25 +1900-05-05 12:34:56.1388,1900-12-25 +1900-05-05 12:34:56.1389,1900-12-25 +1900-05-05 12:34:56.139,1900-12-25 +1900-05-05 12:34:56.1391,1900-12-25 +1900-05-05 12:34:56.1392,1900-12-25 +1900-05-05 12:34:56.1393,1900-12-25 +1900-05-05 12:34:56.1394,1900-12-25 +1900-05-05 12:34:56.1395,1900-12-25 +1900-05-05 12:34:56.1396,1900-12-25 +1900-05-05 12:34:56.1397,1900-12-25 +1900-05-05 12:34:56.1398,1900-12-25 +1900-05-05 12:34:56.1399,1900-12-25 +1900-05-05 12:34:56.14,1900-12-25 +1900-05-05 12:34:56.1401,1900-12-25 +1900-05-05 12:34:56.1402,1900-12-25 +1900-05-05 12:34:56.1403,1900-12-25 +1900-05-05 12:34:56.1404,1900-12-25 +1900-05-05 12:34:56.1405,1900-12-25 +1900-05-05 12:34:56.1406,1900-12-25 +1900-05-05 12:34:56.1407,1900-12-25 +1900-05-05 12:34:56.1408,1900-12-25 +1900-05-05 12:34:56.1409,1900-12-25 +1900-05-05 12:34:56.141,1900-12-25 +1900-05-05 12:34:56.1411,1900-12-25 +1900-05-05 12:34:56.1412,1900-12-25 +1900-05-05 12:34:56.1413,1900-12-25 +1900-05-05 12:34:56.1414,1900-12-25 +1900-05-05 12:34:56.1415,1900-12-25 +1900-05-05 12:34:56.1416,1900-12-25 +1900-05-05 12:34:56.1417,1900-12-25 +1900-05-05 12:34:56.1418,1900-12-25 +1900-05-05 12:34:56.1419,1900-12-25 +1900-05-05 12:34:56.142,1900-12-25 +1900-05-05 12:34:56.1421,1900-12-25 +1900-05-05 12:34:56.1422,1900-12-25 +1900-05-05 12:34:56.1423,1900-12-25 +1900-05-05 12:34:56.1424,1900-12-25 +1900-05-05 12:34:56.1425,1900-12-25 +1900-05-05 12:34:56.1426,1900-12-25 +1900-05-05 12:34:56.1427,1900-12-25 +1900-05-05 12:34:56.1428,1900-12-25 +1900-05-05 12:34:56.1429,1900-12-25 +1900-05-05 12:34:56.143,1900-12-25 +1900-05-05 12:34:56.1431,1900-12-25 +1900-05-05 12:34:56.1432,1900-12-25 +1900-05-05 12:34:56.1433,1900-12-25 +1900-05-05 12:34:56.1434,1900-12-25 +1900-05-05 12:34:56.1435,1900-12-25 +1900-05-05 12:34:56.1436,1900-12-25 +1900-05-05 12:34:56.1437,1900-12-25 +1900-05-05 12:34:56.1438,1900-12-25 +1900-05-05 12:34:56.1439,1900-12-25 +1900-05-05 12:34:56.144,1900-12-25 +1900-05-05 12:34:56.1441,1900-12-25 +1900-05-05 12:34:56.1442,1900-12-25 +1900-05-05 12:34:56.1443,1900-12-25 +1900-05-05 12:34:56.1444,1900-12-25 +1900-05-05 12:34:56.1445,1900-12-25 +1900-05-05 12:34:56.1446,1900-12-25 +1900-05-05 12:34:56.1447,1900-12-25 +1900-05-05 12:34:56.1448,1900-12-25 +1900-05-05 12:34:56.1449,1900-12-25 +1900-05-05 12:34:56.145,1900-12-25 +1900-05-05 12:34:56.1451,1900-12-25 +1900-05-05 12:34:56.1452,1900-12-25 +1900-05-05 12:34:56.1453,1900-12-25 +1900-05-05 12:34:56.1454,1900-12-25 +1900-05-05 12:34:56.1455,1900-12-25 +1900-05-05 12:34:56.1456,1900-12-25 +1900-05-05 12:34:56.1457,1900-12-25 +1900-05-05 12:34:56.1458,1900-12-25 +1900-05-05 12:34:56.1459,1900-12-25 +1900-05-05 12:34:56.146,1900-12-25 +1900-05-05 12:34:56.1461,1900-12-25 +1900-05-05 12:34:56.1462,1900-12-25 +1900-05-05 12:34:56.1463,1900-12-25 +1900-05-05 12:34:56.1464,1900-12-25 +1900-05-05 12:34:56.1465,1900-12-25 +1900-05-05 12:34:56.1466,1900-12-25 +1900-05-05 12:34:56.1467,1900-12-25 +1900-05-05 12:34:56.1468,1900-12-25 +1900-05-05 12:34:56.1469,1900-12-25 +1900-05-05 12:34:56.147,1900-12-25 +1900-05-05 12:34:56.1471,1900-12-25 +1900-05-05 12:34:56.1472,1900-12-25 +1900-05-05 12:34:56.1473,1900-12-25 +1900-05-05 12:34:56.1474,1900-12-25 +1900-05-05 12:34:56.1475,1900-12-25 +1900-05-05 12:34:56.1476,1900-12-25 +1900-05-05 12:34:56.1477,1900-12-25 +1900-05-05 12:34:56.1478,1900-12-25 +1900-05-05 12:34:56.1479,1900-12-25 +1900-05-05 12:34:56.148,1900-12-25 +1900-05-05 12:34:56.1481,1900-12-25 +1900-05-05 12:34:56.1482,1900-12-25 +1900-05-05 12:34:56.1483,1900-12-25 +1900-05-05 12:34:56.1484,1900-12-25 +1900-05-05 12:34:56.1485,1900-12-25 +1900-05-05 12:34:56.1486,1900-12-25 +1900-05-05 12:34:56.1487,1900-12-25 +1900-05-05 12:34:56.1488,1900-12-25 +1900-05-05 12:34:56.1489,1900-12-25 +1900-05-05 12:34:56.149,1900-12-25 +1900-05-05 12:34:56.1491,1900-12-25 +1900-05-05 12:34:56.1492,1900-12-25 +1900-05-05 12:34:56.1493,1900-12-25 +1900-05-05 12:34:56.1494,1900-12-25 +1900-05-05 12:34:56.1495,1900-12-25 +1900-05-05 12:34:56.1496,1900-12-25 +1900-05-05 12:34:56.1497,1900-12-25 +1900-05-05 12:34:56.1498,1900-12-25 +1900-05-05 12:34:56.1499,1900-12-25 +1900-05-05 12:34:56.15,1900-12-25 +1900-05-05 12:34:56.1501,1900-12-25 +1900-05-05 12:34:56.1502,1900-12-25 +1900-05-05 12:34:56.1503,1900-12-25 +1900-05-05 12:34:56.1504,1900-12-25 +1900-05-05 12:34:56.1505,1900-12-25 +1900-05-05 12:34:56.1506,1900-12-25 +1900-05-05 12:34:56.1507,1900-12-25 +1900-05-05 12:34:56.1508,1900-12-25 +1900-05-05 12:34:56.1509,1900-12-25 +1900-05-05 12:34:56.151,1900-12-25 +1900-05-05 12:34:56.1511,1900-12-25 +1900-05-05 12:34:56.1512,1900-12-25 +1900-05-05 12:34:56.1513,1900-12-25 +1900-05-05 12:34:56.1514,1900-12-25 +1900-05-05 12:34:56.1515,1900-12-25 +1900-05-05 12:34:56.1516,1900-12-25 +1900-05-05 12:34:56.1517,1900-12-25 +1900-05-05 12:34:56.1518,1900-12-25 +1900-05-05 12:34:56.1519,1900-12-25 +1900-05-05 12:34:56.152,1900-12-25 +1900-05-05 12:34:56.1521,1900-12-25 +1900-05-05 12:34:56.1522,1900-12-25 +1900-05-05 12:34:56.1523,1900-12-25 +1900-05-05 12:34:56.1524,1900-12-25 +1900-05-05 12:34:56.1525,1900-12-25 +1900-05-05 12:34:56.1526,1900-12-25 +1900-05-05 12:34:56.1527,1900-12-25 +1900-05-05 12:34:56.1528,1900-12-25 +1900-05-05 12:34:56.1529,1900-12-25 +1900-05-05 12:34:56.153,1900-12-25 +1900-05-05 12:34:56.1531,1900-12-25 +1900-05-05 12:34:56.1532,1900-12-25 +1900-05-05 12:34:56.1533,1900-12-25 +1900-05-05 12:34:56.1534,1900-12-25 +1900-05-05 12:34:56.1535,1900-12-25 +1900-05-05 12:34:56.1536,1900-12-25 +1900-05-05 12:34:56.1537,1900-12-25 +1900-05-05 12:34:56.1538,1900-12-25 +1900-05-05 12:34:56.1539,1900-12-25 +1900-05-05 12:34:56.154,1900-12-25 +1900-05-05 12:34:56.1541,1900-12-25 +1900-05-05 12:34:56.1542,1900-12-25 +1900-05-05 12:34:56.1543,1900-12-25 +1900-05-05 12:34:56.1544,1900-12-25 +1900-05-05 12:34:56.1545,1900-12-25 +1900-05-05 12:34:56.1546,1900-12-25 +1900-05-05 12:34:56.1547,1900-12-25 +1900-05-05 12:34:56.1548,1900-12-25 +1900-05-05 12:34:56.1549,1900-12-25 +1900-05-05 12:34:56.155,1900-12-25 +1900-05-05 12:34:56.1551,1900-12-25 +1900-05-05 12:34:56.1552,1900-12-25 +1900-05-05 12:34:56.1553,1900-12-25 +1900-05-05 12:34:56.1554,1900-12-25 +1900-05-05 12:34:56.1555,1900-12-25 +1900-05-05 12:34:56.1556,1900-12-25 +1900-05-05 12:34:56.1557,1900-12-25 +1900-05-05 12:34:56.1558,1900-12-25 +1900-05-05 12:34:56.1559,1900-12-25 +1900-05-05 12:34:56.156,1900-12-25 +1900-05-05 12:34:56.1561,1900-12-25 +1900-05-05 12:34:56.1562,1900-12-25 +1900-05-05 12:34:56.1563,1900-12-25 +1900-05-05 12:34:56.1564,1900-12-25 +1900-05-05 12:34:56.1565,1900-12-25 +1900-05-05 12:34:56.1566,1900-12-25 +1900-05-05 12:34:56.1567,1900-12-25 +1900-05-05 12:34:56.1568,1900-12-25 +1900-05-05 12:34:56.1569,1900-12-25 +1900-05-05 12:34:56.157,1900-12-25 +1900-05-05 12:34:56.1571,1900-12-25 +1900-05-05 12:34:56.1572,1900-12-25 +1900-05-05 12:34:56.1573,1900-12-25 +1900-05-05 12:34:56.1574,1900-12-25 +1900-05-05 12:34:56.1575,1900-12-25 +1900-05-05 12:34:56.1576,1900-12-25 +1900-05-05 12:34:56.1577,1900-12-25 +1900-05-05 12:34:56.1578,1900-12-25 +1900-05-05 12:34:56.1579,1900-12-25 +1900-05-05 12:34:56.158,1900-12-25 +1900-05-05 12:34:56.1581,1900-12-25 +1900-05-05 12:34:56.1582,1900-12-25 +1900-05-05 12:34:56.1583,1900-12-25 +1900-05-05 12:34:56.1584,1900-12-25 +1900-05-05 12:34:56.1585,1900-12-25 +1900-05-05 12:34:56.1586,1900-12-25 +1900-05-05 12:34:56.1587,1900-12-25 +1900-05-05 12:34:56.1588,1900-12-25 +1900-05-05 12:34:56.1589,1900-12-25 +1900-05-05 12:34:56.159,1900-12-25 +1900-05-05 12:34:56.1591,1900-12-25 +1900-05-05 12:34:56.1592,1900-12-25 +1900-05-05 12:34:56.1593,1900-12-25 +1900-05-05 12:34:56.1594,1900-12-25 +1900-05-05 12:34:56.1595,1900-12-25 +1900-05-05 12:34:56.1596,1900-12-25 +1900-05-05 12:34:56.1597,1900-12-25 +1900-05-05 12:34:56.1598,1900-12-25 +1900-05-05 12:34:56.1599,1900-12-25 +1900-05-05 12:34:56.16,1900-12-25 +1900-05-05 12:34:56.1601,1900-12-25 +1900-05-05 12:34:56.1602,1900-12-25 +1900-05-05 12:34:56.1603,1900-12-25 +1900-05-05 12:34:56.1604,1900-12-25 +1900-05-05 12:34:56.1605,1900-12-25 +1900-05-05 12:34:56.1606,1900-12-25 +1900-05-05 12:34:56.1607,1900-12-25 +1900-05-05 12:34:56.1608,1900-12-25 +1900-05-05 12:34:56.1609,1900-12-25 +1900-05-05 12:34:56.161,1900-12-25 +1900-05-05 12:34:56.1611,1900-12-25 +1900-05-05 12:34:56.1612,1900-12-25 +1900-05-05 12:34:56.1613,1900-12-25 +1900-05-05 12:34:56.1614,1900-12-25 +1900-05-05 12:34:56.1615,1900-12-25 +1900-05-05 12:34:56.1616,1900-12-25 +1900-05-05 12:34:56.1617,1900-12-25 +1900-05-05 12:34:56.1618,1900-12-25 +1900-05-05 12:34:56.1619,1900-12-25 +1900-05-05 12:34:56.162,1900-12-25 +1900-05-05 12:34:56.1621,1900-12-25 +1900-05-05 12:34:56.1622,1900-12-25 +1900-05-05 12:34:56.1623,1900-12-25 +1900-05-05 12:34:56.1624,1900-12-25 +1900-05-05 12:34:56.1625,1900-12-25 +1900-05-05 12:34:56.1626,1900-12-25 +1900-05-05 12:34:56.1627,1900-12-25 +1900-05-05 12:34:56.1628,1900-12-25 +1900-05-05 12:34:56.1629,1900-12-25 +1900-05-05 12:34:56.163,1900-12-25 +1900-05-05 12:34:56.1631,1900-12-25 +1900-05-05 12:34:56.1632,1900-12-25 +1900-05-05 12:34:56.1633,1900-12-25 +1900-05-05 12:34:56.1634,1900-12-25 +1900-05-05 12:34:56.1635,1900-12-25 +1900-05-05 12:34:56.1636,1900-12-25 +1900-05-05 12:34:56.1637,1900-12-25 +1900-05-05 12:34:56.1638,1900-12-25 +1900-05-05 12:34:56.1639,1900-12-25 +1900-05-05 12:34:56.164,1900-12-25 +1900-05-05 12:34:56.1641,1900-12-25 +1900-05-05 12:34:56.1642,1900-12-25 +1900-05-05 12:34:56.1643,1900-12-25 +1900-05-05 12:34:56.1644,1900-12-25 +1900-05-05 12:34:56.1645,1900-12-25 +1900-05-05 12:34:56.1646,1900-12-25 +1900-05-05 12:34:56.1647,1900-12-25 +1900-05-05 12:34:56.1648,1900-12-25 +1900-05-05 12:34:56.1649,1900-12-25 +1900-05-05 12:34:56.165,1900-12-25 +1900-05-05 12:34:56.1651,1900-12-25 +1900-05-05 12:34:56.1652,1900-12-25 +1900-05-05 12:34:56.1653,1900-12-25 +1900-05-05 12:34:56.1654,1900-12-25 +1900-05-05 12:34:56.1655,1900-12-25 +1900-05-05 12:34:56.1656,1900-12-25 +1900-05-05 12:34:56.1657,1900-12-25 +1900-05-05 12:34:56.1658,1900-12-25 +1900-05-05 12:34:56.1659,1900-12-25 +1900-05-05 12:34:56.166,1900-12-25 +1900-05-05 12:34:56.1661,1900-12-25 +1900-05-05 12:34:56.1662,1900-12-25 +1900-05-05 12:34:56.1663,1900-12-25 +1900-05-05 12:34:56.1664,1900-12-25 +1900-05-05 12:34:56.1665,1900-12-25 +1900-05-05 12:34:56.1666,1900-12-25 +1900-05-05 12:34:56.1667,1900-12-25 +1900-05-05 12:34:56.1668,1900-12-25 +1900-05-05 12:34:56.1669,1900-12-25 +1900-05-05 12:34:56.167,1900-12-25 +1900-05-05 12:34:56.1671,1900-12-25 +1900-05-05 12:34:56.1672,1900-12-25 +1900-05-05 12:34:56.1673,1900-12-25 +1900-05-05 12:34:56.1674,1900-12-25 +1900-05-05 12:34:56.1675,1900-12-25 +1900-05-05 12:34:56.1676,1900-12-25 +1900-05-05 12:34:56.1677,1900-12-25 +1900-05-05 12:34:56.1678,1900-12-25 +1900-05-05 12:34:56.1679,1900-12-25 +1900-05-05 12:34:56.168,1900-12-25 +1900-05-05 12:34:56.1681,1900-12-25 +1900-05-05 12:34:56.1682,1900-12-25 +1900-05-05 12:34:56.1683,1900-12-25 +1900-05-05 12:34:56.1684,1900-12-25 +1900-05-05 12:34:56.1685,1900-12-25 +1900-05-05 12:34:56.1686,1900-12-25 +1900-05-05 12:34:56.1687,1900-12-25 +1900-05-05 12:34:56.1688,1900-12-25 +1900-05-05 12:34:56.1689,1900-12-25 +1900-05-05 12:34:56.169,1900-12-25 +1900-05-05 12:34:56.1691,1900-12-25 +1900-05-05 12:34:56.1692,1900-12-25 +1900-05-05 12:34:56.1693,1900-12-25 +1900-05-05 12:34:56.1694,1900-12-25 +1900-05-05 12:34:56.1695,1900-12-25 +1900-05-05 12:34:56.1696,1900-12-25 +1900-05-05 12:34:56.1697,1900-12-25 +1900-05-05 12:34:56.1698,1900-12-25 +1900-05-05 12:34:56.1699,1900-12-25 +1900-05-05 12:34:56.17,1900-12-25 +1900-05-05 12:34:56.1701,1900-12-25 +1900-05-05 12:34:56.1702,1900-12-25 +1900-05-05 12:34:56.1703,1900-12-25 +1900-05-05 12:34:56.1704,1900-12-25 +1900-05-05 12:34:56.1705,1900-12-25 +1900-05-05 12:34:56.1706,1900-12-25 +1900-05-05 12:34:56.1707,1900-12-25 +1900-05-05 12:34:56.1708,1900-12-25 +1900-05-05 12:34:56.1709,1900-12-25 +1900-05-05 12:34:56.171,1900-12-25 +1900-05-05 12:34:56.1711,1900-12-25 +1900-05-05 12:34:56.1712,1900-12-25 +1900-05-05 12:34:56.1713,1900-12-25 +1900-05-05 12:34:56.1714,1900-12-25 +1900-05-05 12:34:56.1715,1900-12-25 +1900-05-05 12:34:56.1716,1900-12-25 +1900-05-05 12:34:56.1717,1900-12-25 +1900-05-05 12:34:56.1718,1900-12-25 +1900-05-05 12:34:56.1719,1900-12-25 +1900-05-05 12:34:56.172,1900-12-25 +1900-05-05 12:34:56.1721,1900-12-25 +1900-05-05 12:34:56.1722,1900-12-25 +1900-05-05 12:34:56.1723,1900-12-25 +1900-05-05 12:34:56.1724,1900-12-25 +1900-05-05 12:34:56.1725,1900-12-25 +1900-05-05 12:34:56.1726,1900-12-25 +1900-05-05 12:34:56.1727,1900-12-25 +1900-05-05 12:34:56.1728,1900-12-25 +1900-05-05 12:34:56.1729,1900-12-25 +1900-05-05 12:34:56.173,1900-12-25 +1900-05-05 12:34:56.1731,1900-12-25 +1900-05-05 12:34:56.1732,1900-12-25 +1900-05-05 12:34:56.1733,1900-12-25 +1900-05-05 12:34:56.1734,1900-12-25 +1900-05-05 12:34:56.1735,1900-12-25 +1900-05-05 12:34:56.1736,1900-12-25 +1900-05-05 12:34:56.1737,1900-12-25 +1900-05-05 12:34:56.1738,1900-12-25 +1900-05-05 12:34:56.1739,1900-12-25 +1900-05-05 12:34:56.174,1900-12-25 +1900-05-05 12:34:56.1741,1900-12-25 +1900-05-05 12:34:56.1742,1900-12-25 +1900-05-05 12:34:56.1743,1900-12-25 +1900-05-05 12:34:56.1744,1900-12-25 +1900-05-05 12:34:56.1745,1900-12-25 +1900-05-05 12:34:56.1746,1900-12-25 +1900-05-05 12:34:56.1747,1900-12-25 +1900-05-05 12:34:56.1748,1900-12-25 +1900-05-05 12:34:56.1749,1900-12-25 +1900-05-05 12:34:56.175,1900-12-25 +1900-05-05 12:34:56.1751,1900-12-25 +1900-05-05 12:34:56.1752,1900-12-25 +1900-05-05 12:34:56.1753,1900-12-25 +1900-05-05 12:34:56.1754,1900-12-25 +1900-05-05 12:34:56.1755,1900-12-25 +1900-05-05 12:34:56.1756,1900-12-25 +1900-05-05 12:34:56.1757,1900-12-25 +1900-05-05 12:34:56.1758,1900-12-25 +1900-05-05 12:34:56.1759,1900-12-25 +1900-05-05 12:34:56.176,1900-12-25 +1900-05-05 12:34:56.1761,1900-12-25 +1900-05-05 12:34:56.1762,1900-12-25 +1900-05-05 12:34:56.1763,1900-12-25 +1900-05-05 12:34:56.1764,1900-12-25 +1900-05-05 12:34:56.1765,1900-12-25 +1900-05-05 12:34:56.1766,1900-12-25 +1900-05-05 12:34:56.1767,1900-12-25 +1900-05-05 12:34:56.1768,1900-12-25 +1900-05-05 12:34:56.1769,1900-12-25 +1900-05-05 12:34:56.177,1900-12-25 +1900-05-05 12:34:56.1771,1900-12-25 +1900-05-05 12:34:56.1772,1900-12-25 +1900-05-05 12:34:56.1773,1900-12-25 +1900-05-05 12:34:56.1774,1900-12-25 +1900-05-05 12:34:56.1775,1900-12-25 +1900-05-05 12:34:56.1776,1900-12-25 +1900-05-05 12:34:56.1777,1900-12-25 +1900-05-05 12:34:56.1778,1900-12-25 +1900-05-05 12:34:56.1779,1900-12-25 +1900-05-05 12:34:56.178,1900-12-25 +1900-05-05 12:34:56.1781,1900-12-25 +1900-05-05 12:34:56.1782,1900-12-25 +1900-05-05 12:34:56.1783,1900-12-25 +1900-05-05 12:34:56.1784,1900-12-25 +1900-05-05 12:34:56.1785,1900-12-25 +1900-05-05 12:34:56.1786,1900-12-25 +1900-05-05 12:34:56.1787,1900-12-25 +1900-05-05 12:34:56.1788,1900-12-25 +1900-05-05 12:34:56.1789,1900-12-25 +1900-05-05 12:34:56.179,1900-12-25 +1900-05-05 12:34:56.1791,1900-12-25 +1900-05-05 12:34:56.1792,1900-12-25 +1900-05-05 12:34:56.1793,1900-12-25 +1900-05-05 12:34:56.1794,1900-12-25 +1900-05-05 12:34:56.1795,1900-12-25 +1900-05-05 12:34:56.1796,1900-12-25 +1900-05-05 12:34:56.1797,1900-12-25 +1900-05-05 12:34:56.1798,1900-12-25 +1900-05-05 12:34:56.1799,1900-12-25 +1900-05-05 12:34:56.18,1900-12-25 +1900-05-05 12:34:56.1801,1900-12-25 +1900-05-05 12:34:56.1802,1900-12-25 +1900-05-05 12:34:56.1803,1900-12-25 +1900-05-05 12:34:56.1804,1900-12-25 +1900-05-05 12:34:56.1805,1900-12-25 +1900-05-05 12:34:56.1806,1900-12-25 +1900-05-05 12:34:56.1807,1900-12-25 +1900-05-05 12:34:56.1808,1900-12-25 +1900-05-05 12:34:56.1809,1900-12-25 +1900-05-05 12:34:56.181,1900-12-25 +1900-05-05 12:34:56.1811,1900-12-25 +1900-05-05 12:34:56.1812,1900-12-25 +1900-05-05 12:34:56.1813,1900-12-25 +1900-05-05 12:34:56.1814,1900-12-25 +1900-05-05 12:34:56.1815,1900-12-25 +1900-05-05 12:34:56.1816,1900-12-25 +1900-05-05 12:34:56.1817,1900-12-25 +1900-05-05 12:34:56.1818,1900-12-25 +1900-05-05 12:34:56.1819,1900-12-25 +1900-05-05 12:34:56.182,1900-12-25 +1900-05-05 12:34:56.1821,1900-12-25 +1900-05-05 12:34:56.1822,1900-12-25 +1900-05-05 12:34:56.1823,1900-12-25 +1900-05-05 12:34:56.1824,1900-12-25 +1900-05-05 12:34:56.1825,1900-12-25 +1900-05-05 12:34:56.1826,1900-12-25 +1900-05-05 12:34:56.1827,1900-12-25 +1900-05-05 12:34:56.1828,1900-12-25 +1900-05-05 12:34:56.1829,1900-12-25 +1900-05-05 12:34:56.183,1900-12-25 +1900-05-05 12:34:56.1831,1900-12-25 +1900-05-05 12:34:56.1832,1900-12-25 +1900-05-05 12:34:56.1833,1900-12-25 +1900-05-05 12:34:56.1834,1900-12-25 +1900-05-05 12:34:56.1835,1900-12-25 +1900-05-05 12:34:56.1836,1900-12-25 +1900-05-05 12:34:56.1837,1900-12-25 +1900-05-05 12:34:56.1838,1900-12-25 +1900-05-05 12:34:56.1839,1900-12-25 +1900-05-05 12:34:56.184,1900-12-25 +1900-05-05 12:34:56.1841,1900-12-25 +1900-05-05 12:34:56.1842,1900-12-25 +1900-05-05 12:34:56.1843,1900-12-25 +1900-05-05 12:34:56.1844,1900-12-25 +1900-05-05 12:34:56.1845,1900-12-25 +1900-05-05 12:34:56.1846,1900-12-25 +1900-05-05 12:34:56.1847,1900-12-25 +1900-05-05 12:34:56.1848,1900-12-25 +1900-05-05 12:34:56.1849,1900-12-25 +1900-05-05 12:34:56.185,1900-12-25 +1900-05-05 12:34:56.1851,1900-12-25 +1900-05-05 12:34:56.1852,1900-12-25 +1900-05-05 12:34:56.1853,1900-12-25 +1900-05-05 12:34:56.1854,1900-12-25 +1900-05-05 12:34:56.1855,1900-12-25 +1900-05-05 12:34:56.1856,1900-12-25 +1900-05-05 12:34:56.1857,1900-12-25 +1900-05-05 12:34:56.1858,1900-12-25 +1900-05-05 12:34:56.1859,1900-12-25 +1900-05-05 12:34:56.186,1900-12-25 +1900-05-05 12:34:56.1861,1900-12-25 +1900-05-05 12:34:56.1862,1900-12-25 +1900-05-05 12:34:56.1863,1900-12-25 +1900-05-05 12:34:56.1864,1900-12-25 +1900-05-05 12:34:56.1865,1900-12-25 +1900-05-05 12:34:56.1866,1900-12-25 +1900-05-05 12:34:56.1867,1900-12-25 +1900-05-05 12:34:56.1868,1900-12-25 +1900-05-05 12:34:56.1869,1900-12-25 +1900-05-05 12:34:56.187,1900-12-25 +1900-05-05 12:34:56.1871,1900-12-25 +1900-05-05 12:34:56.1872,1900-12-25 +1900-05-05 12:34:56.1873,1900-12-25 +1900-05-05 12:34:56.1874,1900-12-25 +1900-05-05 12:34:56.1875,1900-12-25 +1900-05-05 12:34:56.1876,1900-12-25 +1900-05-05 12:34:56.1877,1900-12-25 +1900-05-05 12:34:56.1878,1900-12-25 +1900-05-05 12:34:56.1879,1900-12-25 +1900-05-05 12:34:56.188,1900-12-25 +1900-05-05 12:34:56.1881,1900-12-25 +1900-05-05 12:34:56.1882,1900-12-25 +1900-05-05 12:34:56.1883,1900-12-25 +1900-05-05 12:34:56.1884,1900-12-25 +1900-05-05 12:34:56.1885,1900-12-25 +1900-05-05 12:34:56.1886,1900-12-25 +1900-05-05 12:34:56.1887,1900-12-25 +1900-05-05 12:34:56.1888,1900-12-25 +1900-05-05 12:34:56.1889,1900-12-25 +1900-05-05 12:34:56.189,1900-12-25 +1900-05-05 12:34:56.1891,1900-12-25 +1900-05-05 12:34:56.1892,1900-12-25 +1900-05-05 12:34:56.1893,1900-12-25 +1900-05-05 12:34:56.1894,1900-12-25 +1900-05-05 12:34:56.1895,1900-12-25 +1900-05-05 12:34:56.1896,1900-12-25 +1900-05-05 12:34:56.1897,1900-12-25 +1900-05-05 12:34:56.1898,1900-12-25 +1900-05-05 12:34:56.1899,1900-12-25 +1900-05-05 12:34:56.19,1900-12-25 +1900-05-05 12:34:56.1901,1900-12-25 +1900-05-05 12:34:56.1902,1900-12-25 +1900-05-05 12:34:56.1903,1900-12-25 +1900-05-05 12:34:56.1904,1900-12-25 +1900-05-05 12:34:56.1905,1900-12-25 +1900-05-05 12:34:56.1906,1900-12-25 +1900-05-05 12:34:56.1907,1900-12-25 +1900-05-05 12:34:56.1908,1900-12-25 +1900-05-05 12:34:56.1909,1900-12-25 +1900-05-05 12:34:56.191,1900-12-25 +1900-05-05 12:34:56.1911,1900-12-25 +1900-05-05 12:34:56.1912,1900-12-25 +1900-05-05 12:34:56.1913,1900-12-25 +1900-05-05 12:34:56.1914,1900-12-25 +1900-05-05 12:34:56.1915,1900-12-25 +1900-05-05 12:34:56.1916,1900-12-25 +1900-05-05 12:34:56.1917,1900-12-25 +1900-05-05 12:34:56.1918,1900-12-25 +1900-05-05 12:34:56.1919,1900-12-25 +1900-05-05 12:34:56.192,1900-12-25 +1900-05-05 12:34:56.1921,1900-12-25 +1900-05-05 12:34:56.1922,1900-12-25 +1900-05-05 12:34:56.1923,1900-12-25 +1900-05-05 12:34:56.1924,1900-12-25 +1900-05-05 12:34:56.1925,1900-12-25 +1900-05-05 12:34:56.1926,1900-12-25 +1900-05-05 12:34:56.1927,1900-12-25 +1900-05-05 12:34:56.1928,1900-12-25 +1900-05-05 12:34:56.1929,1900-12-25 +1900-05-05 12:34:56.193,1900-12-25 +1900-05-05 12:34:56.1931,1900-12-25 +1900-05-05 12:34:56.1932,1900-12-25 +1900-05-05 12:34:56.1933,1900-12-25 +1900-05-05 12:34:56.1934,1900-12-25 +1900-05-05 12:34:56.1935,1900-12-25 +1900-05-05 12:34:56.1936,1900-12-25 +1900-05-05 12:34:56.1937,1900-12-25 +1900-05-05 12:34:56.1938,1900-12-25 +1900-05-05 12:34:56.1939,1900-12-25 +1900-05-05 12:34:56.194,1900-12-25 +1900-05-05 12:34:56.1941,1900-12-25 +1900-05-05 12:34:56.1942,1900-12-25 +1900-05-05 12:34:56.1943,1900-12-25 +1900-05-05 12:34:56.1944,1900-12-25 +1900-05-05 12:34:56.1945,1900-12-25 +1900-05-05 12:34:56.1946,1900-12-25 +1900-05-05 12:34:56.1947,1900-12-25 +1900-05-05 12:34:56.1948,1900-12-25 +1900-05-05 12:34:56.1949,1900-12-25 +1900-05-05 12:34:56.195,1900-12-25 +1900-05-05 12:34:56.1951,1900-12-25 +1900-05-05 12:34:56.1952,1900-12-25 +1900-05-05 12:34:56.1953,1900-12-25 +1900-05-05 12:34:56.1954,1900-12-25 +1900-05-05 12:34:56.1955,1900-12-25 +1900-05-05 12:34:56.1956,1900-12-25 +1900-05-05 12:34:56.1957,1900-12-25 +1900-05-05 12:34:56.1958,1900-12-25 +1900-05-05 12:34:56.1959,1900-12-25 +1900-05-05 12:34:56.196,1900-12-25 +1900-05-05 12:34:56.1961,1900-12-25 +1900-05-05 12:34:56.1962,1900-12-25 +1900-05-05 12:34:56.1963,1900-12-25 +1900-05-05 12:34:56.1964,1900-12-25 +1900-05-05 12:34:56.1965,1900-12-25 +1900-05-05 12:34:56.1966,1900-12-25 +1900-05-05 12:34:56.1967,1900-12-25 +1900-05-05 12:34:56.1968,1900-12-25 +1900-05-05 12:34:56.1969,1900-12-25 +1900-05-05 12:34:56.197,1900-12-25 +1900-05-05 12:34:56.1971,1900-12-25 +1900-05-05 12:34:56.1972,1900-12-25 +1900-05-05 12:34:56.1973,1900-12-25 +1900-05-05 12:34:56.1974,1900-12-25 +1900-05-05 12:34:56.1975,1900-12-25 +1900-05-05 12:34:56.1976,1900-12-25 +1900-05-05 12:34:56.1977,1900-12-25 +1900-05-05 12:34:56.1978,1900-12-25 +1900-05-05 12:34:56.1979,1900-12-25 +1900-05-05 12:34:56.198,1900-12-25 +1900-05-05 12:34:56.1981,1900-12-25 +1900-05-05 12:34:56.1982,1900-12-25 +1900-05-05 12:34:56.1983,1900-12-25 +1900-05-05 12:34:56.1984,1900-12-25 +1900-05-05 12:34:56.1985,1900-12-25 +1900-05-05 12:34:56.1986,1900-12-25 +1900-05-05 12:34:56.1987,1900-12-25 +1900-05-05 12:34:56.1988,1900-12-25 +1900-05-05 12:34:56.1989,1900-12-25 +1900-05-05 12:34:56.199,1900-12-25 +1900-05-05 12:34:56.1991,1900-12-25 +1900-05-05 12:34:56.1992,1900-12-25 +1900-05-05 12:34:56.1993,1900-12-25 +1900-05-05 12:34:56.1994,1900-12-25 +1900-05-05 12:34:56.1995,1900-12-25 +1900-05-05 12:34:56.1996,1900-12-25 +1900-05-05 12:34:56.1997,1900-12-25 +1900-05-05 12:34:56.1998,1900-12-25 +1900-05-05 12:34:56.1999,1900-12-25 +1901-05-05 12:34:56.1,1901-12-25 +1901-05-05 12:34:56.1001,1901-12-25 +1901-05-05 12:34:56.1002,1901-12-25 +1901-05-05 12:34:56.1003,1901-12-25 +1901-05-05 12:34:56.1004,1901-12-25 +1901-05-05 12:34:56.1005,1901-12-25 +1901-05-05 12:34:56.1006,1901-12-25 +1901-05-05 12:34:56.1007,1901-12-25 +1901-05-05 12:34:56.1008,1901-12-25 +1901-05-05 12:34:56.1009,1901-12-25 +1901-05-05 12:34:56.101,1901-12-25 +1901-05-05 12:34:56.1011,1901-12-25 +1901-05-05 12:34:56.1012,1901-12-25 +1901-05-05 12:34:56.1013,1901-12-25 +1901-05-05 12:34:56.1014,1901-12-25 +1901-05-05 12:34:56.1015,1901-12-25 +1901-05-05 12:34:56.1016,1901-12-25 +1901-05-05 12:34:56.1017,1901-12-25 +1901-05-05 12:34:56.1018,1901-12-25 +1901-05-05 12:34:56.1019,1901-12-25 +1901-05-05 12:34:56.102,1901-12-25 +1901-05-05 12:34:56.1021,1901-12-25 +1901-05-05 12:34:56.1022,1901-12-25 +1901-05-05 12:34:56.1023,1901-12-25 +1901-05-05 12:34:56.1024,1901-12-25 +1901-05-05 12:34:56.1025,1901-12-25 +1901-05-05 12:34:56.1026,1901-12-25 +1901-05-05 12:34:56.1027,1901-12-25 +1901-05-05 12:34:56.1028,1901-12-25 +1901-05-05 12:34:56.1029,1901-12-25 +1901-05-05 12:34:56.103,1901-12-25 +1901-05-05 12:34:56.1031,1901-12-25 +1901-05-05 12:34:56.1032,1901-12-25 +1901-05-05 12:34:56.1033,1901-12-25 +1901-05-05 12:34:56.1034,1901-12-25 +1901-05-05 12:34:56.1035,1901-12-25 +1901-05-05 12:34:56.1036,1901-12-25 +1901-05-05 12:34:56.1037,1901-12-25 +1901-05-05 12:34:56.1038,1901-12-25 +1901-05-05 12:34:56.1039,1901-12-25 +1901-05-05 12:34:56.104,1901-12-25 +1901-05-05 12:34:56.1041,1901-12-25 +1901-05-05 12:34:56.1042,1901-12-25 +1901-05-05 12:34:56.1043,1901-12-25 +1901-05-05 12:34:56.1044,1901-12-25 +1901-05-05 12:34:56.1045,1901-12-25 +1901-05-05 12:34:56.1046,1901-12-25 +1901-05-05 12:34:56.1047,1901-12-25 +1901-05-05 12:34:56.1048,1901-12-25 +1901-05-05 12:34:56.1049,1901-12-25 +1901-05-05 12:34:56.105,1901-12-25 +1901-05-05 12:34:56.1051,1901-12-25 +1901-05-05 12:34:56.1052,1901-12-25 +1901-05-05 12:34:56.1053,1901-12-25 +1901-05-05 12:34:56.1054,1901-12-25 +1901-05-05 12:34:56.1055,1901-12-25 +1901-05-05 12:34:56.1056,1901-12-25 +1901-05-05 12:34:56.1057,1901-12-25 +1901-05-05 12:34:56.1058,1901-12-25 +1901-05-05 12:34:56.1059,1901-12-25 +1901-05-05 12:34:56.106,1901-12-25 +1901-05-05 12:34:56.1061,1901-12-25 +1901-05-05 12:34:56.1062,1901-12-25 +1901-05-05 12:34:56.1063,1901-12-25 +1901-05-05 12:34:56.1064,1901-12-25 +1901-05-05 12:34:56.1065,1901-12-25 +1901-05-05 12:34:56.1066,1901-12-25 +1901-05-05 12:34:56.1067,1901-12-25 +1901-05-05 12:34:56.1068,1901-12-25 +1901-05-05 12:34:56.1069,1901-12-25 +1901-05-05 12:34:56.107,1901-12-25 +1901-05-05 12:34:56.1071,1901-12-25 +1901-05-05 12:34:56.1072,1901-12-25 +1901-05-05 12:34:56.1073,1901-12-25 +1901-05-05 12:34:56.1074,1901-12-25 +1901-05-05 12:34:56.1075,1901-12-25 +1901-05-05 12:34:56.1076,1901-12-25 +1901-05-05 12:34:56.1077,1901-12-25 +1901-05-05 12:34:56.1078,1901-12-25 +1901-05-05 12:34:56.1079,1901-12-25 +1901-05-05 12:34:56.108,1901-12-25 +1901-05-05 12:34:56.1081,1901-12-25 +1901-05-05 12:34:56.1082,1901-12-25 +1901-05-05 12:34:56.1083,1901-12-25 +1901-05-05 12:34:56.1084,1901-12-25 +1901-05-05 12:34:56.1085,1901-12-25 +1901-05-05 12:34:56.1086,1901-12-25 +1901-05-05 12:34:56.1087,1901-12-25 +1901-05-05 12:34:56.1088,1901-12-25 +1901-05-05 12:34:56.1089,1901-12-25 +1901-05-05 12:34:56.109,1901-12-25 +1901-05-05 12:34:56.1091,1901-12-25 +1901-05-05 12:34:56.1092,1901-12-25 +1901-05-05 12:34:56.1093,1901-12-25 +1901-05-05 12:34:56.1094,1901-12-25 +1901-05-05 12:34:56.1095,1901-12-25 +1901-05-05 12:34:56.1096,1901-12-25 +1901-05-05 12:34:56.1097,1901-12-25 +1901-05-05 12:34:56.1098,1901-12-25 +1901-05-05 12:34:56.1099,1901-12-25 +1901-05-05 12:34:56.11,1901-12-25 +1901-05-05 12:34:56.1101,1901-12-25 +1901-05-05 12:34:56.1102,1901-12-25 +1901-05-05 12:34:56.1103,1901-12-25 +1901-05-05 12:34:56.1104,1901-12-25 +1901-05-05 12:34:56.1105,1901-12-25 +1901-05-05 12:34:56.1106,1901-12-25 +1901-05-05 12:34:56.1107,1901-12-25 +1901-05-05 12:34:56.1108,1901-12-25 +1901-05-05 12:34:56.1109,1901-12-25 +1901-05-05 12:34:56.111,1901-12-25 +1901-05-05 12:34:56.1111,1901-12-25 +1901-05-05 12:34:56.1112,1901-12-25 +1901-05-05 12:34:56.1113,1901-12-25 +1901-05-05 12:34:56.1114,1901-12-25 +1901-05-05 12:34:56.1115,1901-12-25 +1901-05-05 12:34:56.1116,1901-12-25 +1901-05-05 12:34:56.1117,1901-12-25 +1901-05-05 12:34:56.1118,1901-12-25 +1901-05-05 12:34:56.1119,1901-12-25 +1901-05-05 12:34:56.112,1901-12-25 +1901-05-05 12:34:56.1121,1901-12-25 +1901-05-05 12:34:56.1122,1901-12-25 +1901-05-05 12:34:56.1123,1901-12-25 +1901-05-05 12:34:56.1124,1901-12-25 +1901-05-05 12:34:56.1125,1901-12-25 +1901-05-05 12:34:56.1126,1901-12-25 +1901-05-05 12:34:56.1127,1901-12-25 +1901-05-05 12:34:56.1128,1901-12-25 +1901-05-05 12:34:56.1129,1901-12-25 +1901-05-05 12:34:56.113,1901-12-25 +1901-05-05 12:34:56.1131,1901-12-25 +1901-05-05 12:34:56.1132,1901-12-25 +1901-05-05 12:34:56.1133,1901-12-25 +1901-05-05 12:34:56.1134,1901-12-25 +1901-05-05 12:34:56.1135,1901-12-25 +1901-05-05 12:34:56.1136,1901-12-25 +1901-05-05 12:34:56.1137,1901-12-25 +1901-05-05 12:34:56.1138,1901-12-25 +1901-05-05 12:34:56.1139,1901-12-25 +1901-05-05 12:34:56.114,1901-12-25 +1901-05-05 12:34:56.1141,1901-12-25 +1901-05-05 12:34:56.1142,1901-12-25 +1901-05-05 12:34:56.1143,1901-12-25 +1901-05-05 12:34:56.1144,1901-12-25 +1901-05-05 12:34:56.1145,1901-12-25 +1901-05-05 12:34:56.1146,1901-12-25 +1901-05-05 12:34:56.1147,1901-12-25 +1901-05-05 12:34:56.1148,1901-12-25 +1901-05-05 12:34:56.1149,1901-12-25 +1901-05-05 12:34:56.115,1901-12-25 +1901-05-05 12:34:56.1151,1901-12-25 +1901-05-05 12:34:56.1152,1901-12-25 +1901-05-05 12:34:56.1153,1901-12-25 +1901-05-05 12:34:56.1154,1901-12-25 +1901-05-05 12:34:56.1155,1901-12-25 +1901-05-05 12:34:56.1156,1901-12-25 +1901-05-05 12:34:56.1157,1901-12-25 +1901-05-05 12:34:56.1158,1901-12-25 +1901-05-05 12:34:56.1159,1901-12-25 +1901-05-05 12:34:56.116,1901-12-25 +1901-05-05 12:34:56.1161,1901-12-25 +1901-05-05 12:34:56.1162,1901-12-25 +1901-05-05 12:34:56.1163,1901-12-25 +1901-05-05 12:34:56.1164,1901-12-25 +1901-05-05 12:34:56.1165,1901-12-25 +1901-05-05 12:34:56.1166,1901-12-25 +1901-05-05 12:34:56.1167,1901-12-25 +1901-05-05 12:34:56.1168,1901-12-25 +1901-05-05 12:34:56.1169,1901-12-25 +1901-05-05 12:34:56.117,1901-12-25 +1901-05-05 12:34:56.1171,1901-12-25 +1901-05-05 12:34:56.1172,1901-12-25 +1901-05-05 12:34:56.1173,1901-12-25 +1901-05-05 12:34:56.1174,1901-12-25 +1901-05-05 12:34:56.1175,1901-12-25 +1901-05-05 12:34:56.1176,1901-12-25 +1901-05-05 12:34:56.1177,1901-12-25 +1901-05-05 12:34:56.1178,1901-12-25 +1901-05-05 12:34:56.1179,1901-12-25 +1901-05-05 12:34:56.118,1901-12-25 +1901-05-05 12:34:56.1181,1901-12-25 +1901-05-05 12:34:56.1182,1901-12-25 +1901-05-05 12:34:56.1183,1901-12-25 +1901-05-05 12:34:56.1184,1901-12-25 +1901-05-05 12:34:56.1185,1901-12-25 +1901-05-05 12:34:56.1186,1901-12-25 +1901-05-05 12:34:56.1187,1901-12-25 +1901-05-05 12:34:56.1188,1901-12-25 +1901-05-05 12:34:56.1189,1901-12-25 +1901-05-05 12:34:56.119,1901-12-25 +1901-05-05 12:34:56.1191,1901-12-25 +1901-05-05 12:34:56.1192,1901-12-25 +1901-05-05 12:34:56.1193,1901-12-25 +1901-05-05 12:34:56.1194,1901-12-25 +1901-05-05 12:34:56.1195,1901-12-25 +1901-05-05 12:34:56.1196,1901-12-25 +1901-05-05 12:34:56.1197,1901-12-25 +1901-05-05 12:34:56.1198,1901-12-25 +1901-05-05 12:34:56.1199,1901-12-25 +1901-05-05 12:34:56.12,1901-12-25 +1901-05-05 12:34:56.1201,1901-12-25 +1901-05-05 12:34:56.1202,1901-12-25 +1901-05-05 12:34:56.1203,1901-12-25 +1901-05-05 12:34:56.1204,1901-12-25 +1901-05-05 12:34:56.1205,1901-12-25 +1901-05-05 12:34:56.1206,1901-12-25 +1901-05-05 12:34:56.1207,1901-12-25 +1901-05-05 12:34:56.1208,1901-12-25 +1901-05-05 12:34:56.1209,1901-12-25 +1901-05-05 12:34:56.121,1901-12-25 +1901-05-05 12:34:56.1211,1901-12-25 +1901-05-05 12:34:56.1212,1901-12-25 +1901-05-05 12:34:56.1213,1901-12-25 +1901-05-05 12:34:56.1214,1901-12-25 +1901-05-05 12:34:56.1215,1901-12-25 +1901-05-05 12:34:56.1216,1901-12-25 +1901-05-05 12:34:56.1217,1901-12-25 +1901-05-05 12:34:56.1218,1901-12-25 +1901-05-05 12:34:56.1219,1901-12-25 +1901-05-05 12:34:56.122,1901-12-25 +1901-05-05 12:34:56.1221,1901-12-25 +1901-05-05 12:34:56.1222,1901-12-25 +1901-05-05 12:34:56.1223,1901-12-25 +1901-05-05 12:34:56.1224,1901-12-25 +1901-05-05 12:34:56.1225,1901-12-25 +1901-05-05 12:34:56.1226,1901-12-25 +1901-05-05 12:34:56.1227,1901-12-25 +1901-05-05 12:34:56.1228,1901-12-25 +1901-05-05 12:34:56.1229,1901-12-25 +1901-05-05 12:34:56.123,1901-12-25 +1901-05-05 12:34:56.1231,1901-12-25 +1901-05-05 12:34:56.1232,1901-12-25 +1901-05-05 12:34:56.1233,1901-12-25 +1901-05-05 12:34:56.1234,1901-12-25 +1901-05-05 12:34:56.1235,1901-12-25 +1901-05-05 12:34:56.1236,1901-12-25 +1901-05-05 12:34:56.1237,1901-12-25 +1901-05-05 12:34:56.1238,1901-12-25 +1901-05-05 12:34:56.1239,1901-12-25 +1901-05-05 12:34:56.124,1901-12-25 +1901-05-05 12:34:56.1241,1901-12-25 +1901-05-05 12:34:56.1242,1901-12-25 +1901-05-05 12:34:56.1243,1901-12-25 +1901-05-05 12:34:56.1244,1901-12-25 +1901-05-05 12:34:56.1245,1901-12-25 +1901-05-05 12:34:56.1246,1901-12-25 +1901-05-05 12:34:56.1247,1901-12-25 +1901-05-05 12:34:56.1248,1901-12-25 +1901-05-05 12:34:56.1249,1901-12-25 +1901-05-05 12:34:56.125,1901-12-25 +1901-05-05 12:34:56.1251,1901-12-25 +1901-05-05 12:34:56.1252,1901-12-25 +1901-05-05 12:34:56.1253,1901-12-25 +1901-05-05 12:34:56.1254,1901-12-25 +1901-05-05 12:34:56.1255,1901-12-25 +1901-05-05 12:34:56.1256,1901-12-25 +1901-05-05 12:34:56.1257,1901-12-25 +1901-05-05 12:34:56.1258,1901-12-25 +1901-05-05 12:34:56.1259,1901-12-25 +1901-05-05 12:34:56.126,1901-12-25 +1901-05-05 12:34:56.1261,1901-12-25 +1901-05-05 12:34:56.1262,1901-12-25 +1901-05-05 12:34:56.1263,1901-12-25 +1901-05-05 12:34:56.1264,1901-12-25 +1901-05-05 12:34:56.1265,1901-12-25 +1901-05-05 12:34:56.1266,1901-12-25 +1901-05-05 12:34:56.1267,1901-12-25 +1901-05-05 12:34:56.1268,1901-12-25 +1901-05-05 12:34:56.1269,1901-12-25 +1901-05-05 12:34:56.127,1901-12-25 +1901-05-05 12:34:56.1271,1901-12-25 +1901-05-05 12:34:56.1272,1901-12-25 +1901-05-05 12:34:56.1273,1901-12-25 +1901-05-05 12:34:56.1274,1901-12-25 +1901-05-05 12:34:56.1275,1901-12-25 +1901-05-05 12:34:56.1276,1901-12-25 +1901-05-05 12:34:56.1277,1901-12-25 +1901-05-05 12:34:56.1278,1901-12-25 +1901-05-05 12:34:56.1279,1901-12-25 +1901-05-05 12:34:56.128,1901-12-25 +1901-05-05 12:34:56.1281,1901-12-25 +1901-05-05 12:34:56.1282,1901-12-25 +1901-05-05 12:34:56.1283,1901-12-25 +1901-05-05 12:34:56.1284,1901-12-25 +1901-05-05 12:34:56.1285,1901-12-25 +1901-05-05 12:34:56.1286,1901-12-25 +1901-05-05 12:34:56.1287,1901-12-25 +1901-05-05 12:34:56.1288,1901-12-25 +1901-05-05 12:34:56.1289,1901-12-25 +1901-05-05 12:34:56.129,1901-12-25 +1901-05-05 12:34:56.1291,1901-12-25 +1901-05-05 12:34:56.1292,1901-12-25 +1901-05-05 12:34:56.1293,1901-12-25 +1901-05-05 12:34:56.1294,1901-12-25 +1901-05-05 12:34:56.1295,1901-12-25 +1901-05-05 12:34:56.1296,1901-12-25 +1901-05-05 12:34:56.1297,1901-12-25 +1901-05-05 12:34:56.1298,1901-12-25 +1901-05-05 12:34:56.1299,1901-12-25 +1901-05-05 12:34:56.13,1901-12-25 +1901-05-05 12:34:56.1301,1901-12-25 +1901-05-05 12:34:56.1302,1901-12-25 +1901-05-05 12:34:56.1303,1901-12-25 +1901-05-05 12:34:56.1304,1901-12-25 +1901-05-05 12:34:56.1305,1901-12-25 +1901-05-05 12:34:56.1306,1901-12-25 +1901-05-05 12:34:56.1307,1901-12-25 +1901-05-05 12:34:56.1308,1901-12-25 +1901-05-05 12:34:56.1309,1901-12-25 +1901-05-05 12:34:56.131,1901-12-25 +1901-05-05 12:34:56.1311,1901-12-25 +1901-05-05 12:34:56.1312,1901-12-25 +1901-05-05 12:34:56.1313,1901-12-25 +1901-05-05 12:34:56.1314,1901-12-25 +1901-05-05 12:34:56.1315,1901-12-25 +1901-05-05 12:34:56.1316,1901-12-25 +1901-05-05 12:34:56.1317,1901-12-25 +1901-05-05 12:34:56.1318,1901-12-25 +1901-05-05 12:34:56.1319,1901-12-25 +1901-05-05 12:34:56.132,1901-12-25 +1901-05-05 12:34:56.1321,1901-12-25 +1901-05-05 12:34:56.1322,1901-12-25 +1901-05-05 12:34:56.1323,1901-12-25 +1901-05-05 12:34:56.1324,1901-12-25 +1901-05-05 12:34:56.1325,1901-12-25 +1901-05-05 12:34:56.1326,1901-12-25 +1901-05-05 12:34:56.1327,1901-12-25 +1901-05-05 12:34:56.1328,1901-12-25 +1901-05-05 12:34:56.1329,1901-12-25 +1901-05-05 12:34:56.133,1901-12-25 +1901-05-05 12:34:56.1331,1901-12-25 +1901-05-05 12:34:56.1332,1901-12-25 +1901-05-05 12:34:56.1333,1901-12-25 +1901-05-05 12:34:56.1334,1901-12-25 +1901-05-05 12:34:56.1335,1901-12-25 +1901-05-05 12:34:56.1336,1901-12-25 +1901-05-05 12:34:56.1337,1901-12-25 +1901-05-05 12:34:56.1338,1901-12-25 +1901-05-05 12:34:56.1339,1901-12-25 +1901-05-05 12:34:56.134,1901-12-25 +1901-05-05 12:34:56.1341,1901-12-25 +1901-05-05 12:34:56.1342,1901-12-25 +1901-05-05 12:34:56.1343,1901-12-25 +1901-05-05 12:34:56.1344,1901-12-25 +1901-05-05 12:34:56.1345,1901-12-25 +1901-05-05 12:34:56.1346,1901-12-25 +1901-05-05 12:34:56.1347,1901-12-25 +1901-05-05 12:34:56.1348,1901-12-25 +1901-05-05 12:34:56.1349,1901-12-25 +1901-05-05 12:34:56.135,1901-12-25 +1901-05-05 12:34:56.1351,1901-12-25 +1901-05-05 12:34:56.1352,1901-12-25 +1901-05-05 12:34:56.1353,1901-12-25 +1901-05-05 12:34:56.1354,1901-12-25 +1901-05-05 12:34:56.1355,1901-12-25 +1901-05-05 12:34:56.1356,1901-12-25 +1901-05-05 12:34:56.1357,1901-12-25 +1901-05-05 12:34:56.1358,1901-12-25 +1901-05-05 12:34:56.1359,1901-12-25 +1901-05-05 12:34:56.136,1901-12-25 +1901-05-05 12:34:56.1361,1901-12-25 +1901-05-05 12:34:56.1362,1901-12-25 +1901-05-05 12:34:56.1363,1901-12-25 +1901-05-05 12:34:56.1364,1901-12-25 +1901-05-05 12:34:56.1365,1901-12-25 +1901-05-05 12:34:56.1366,1901-12-25 +1901-05-05 12:34:56.1367,1901-12-25 +1901-05-05 12:34:56.1368,1901-12-25 +1901-05-05 12:34:56.1369,1901-12-25 +1901-05-05 12:34:56.137,1901-12-25 +1901-05-05 12:34:56.1371,1901-12-25 +1901-05-05 12:34:56.1372,1901-12-25 +1901-05-05 12:34:56.1373,1901-12-25 +1901-05-05 12:34:56.1374,1901-12-25 +1901-05-05 12:34:56.1375,1901-12-25 +1901-05-05 12:34:56.1376,1901-12-25 +1901-05-05 12:34:56.1377,1901-12-25 +1901-05-05 12:34:56.1378,1901-12-25 +1901-05-05 12:34:56.1379,1901-12-25 +1901-05-05 12:34:56.138,1901-12-25 +1901-05-05 12:34:56.1381,1901-12-25 +1901-05-05 12:34:56.1382,1901-12-25 +1901-05-05 12:34:56.1383,1901-12-25 +1901-05-05 12:34:56.1384,1901-12-25 +1901-05-05 12:34:56.1385,1901-12-25 +1901-05-05 12:34:56.1386,1901-12-25 +1901-05-05 12:34:56.1387,1901-12-25 +1901-05-05 12:34:56.1388,1901-12-25 +1901-05-05 12:34:56.1389,1901-12-25 +1901-05-05 12:34:56.139,1901-12-25 +1901-05-05 12:34:56.1391,1901-12-25 +1901-05-05 12:34:56.1392,1901-12-25 +1901-05-05 12:34:56.1393,1901-12-25 +1901-05-05 12:34:56.1394,1901-12-25 +1901-05-05 12:34:56.1395,1901-12-25 +1901-05-05 12:34:56.1396,1901-12-25 +1901-05-05 12:34:56.1397,1901-12-25 +1901-05-05 12:34:56.1398,1901-12-25 +1901-05-05 12:34:56.1399,1901-12-25 +1901-05-05 12:34:56.14,1901-12-25 +1901-05-05 12:34:56.1401,1901-12-25 +1901-05-05 12:34:56.1402,1901-12-25 +1901-05-05 12:34:56.1403,1901-12-25 +1901-05-05 12:34:56.1404,1901-12-25 +1901-05-05 12:34:56.1405,1901-12-25 +1901-05-05 12:34:56.1406,1901-12-25 +1901-05-05 12:34:56.1407,1901-12-25 +1901-05-05 12:34:56.1408,1901-12-25 +1901-05-05 12:34:56.1409,1901-12-25 +1901-05-05 12:34:56.141,1901-12-25 +1901-05-05 12:34:56.1411,1901-12-25 +1901-05-05 12:34:56.1412,1901-12-25 +1901-05-05 12:34:56.1413,1901-12-25 +1901-05-05 12:34:56.1414,1901-12-25 +1901-05-05 12:34:56.1415,1901-12-25 +1901-05-05 12:34:56.1416,1901-12-25 +1901-05-05 12:34:56.1417,1901-12-25 +1901-05-05 12:34:56.1418,1901-12-25 +1901-05-05 12:34:56.1419,1901-12-25 +1901-05-05 12:34:56.142,1901-12-25 +1901-05-05 12:34:56.1421,1901-12-25 +1901-05-05 12:34:56.1422,1901-12-25 +1901-05-05 12:34:56.1423,1901-12-25 +1901-05-05 12:34:56.1424,1901-12-25 +1901-05-05 12:34:56.1425,1901-12-25 +1901-05-05 12:34:56.1426,1901-12-25 +1901-05-05 12:34:56.1427,1901-12-25 +1901-05-05 12:34:56.1428,1901-12-25 +1901-05-05 12:34:56.1429,1901-12-25 +1901-05-05 12:34:56.143,1901-12-25 +1901-05-05 12:34:56.1431,1901-12-25 +1901-05-05 12:34:56.1432,1901-12-25 +1901-05-05 12:34:56.1433,1901-12-25 +1901-05-05 12:34:56.1434,1901-12-25 +1901-05-05 12:34:56.1435,1901-12-25 +1901-05-05 12:34:56.1436,1901-12-25 +1901-05-05 12:34:56.1437,1901-12-25 +1901-05-05 12:34:56.1438,1901-12-25 +1901-05-05 12:34:56.1439,1901-12-25 +1901-05-05 12:34:56.144,1901-12-25 +1901-05-05 12:34:56.1441,1901-12-25 +1901-05-05 12:34:56.1442,1901-12-25 +1901-05-05 12:34:56.1443,1901-12-25 +1901-05-05 12:34:56.1444,1901-12-25 +1901-05-05 12:34:56.1445,1901-12-25 +1901-05-05 12:34:56.1446,1901-12-25 +1901-05-05 12:34:56.1447,1901-12-25 +1901-05-05 12:34:56.1448,1901-12-25 +1901-05-05 12:34:56.1449,1901-12-25 +1901-05-05 12:34:56.145,1901-12-25 +1901-05-05 12:34:56.1451,1901-12-25 +1901-05-05 12:34:56.1452,1901-12-25 +1901-05-05 12:34:56.1453,1901-12-25 +1901-05-05 12:34:56.1454,1901-12-25 +1901-05-05 12:34:56.1455,1901-12-25 +1901-05-05 12:34:56.1456,1901-12-25 +1901-05-05 12:34:56.1457,1901-12-25 +1901-05-05 12:34:56.1458,1901-12-25 +1901-05-05 12:34:56.1459,1901-12-25 +1901-05-05 12:34:56.146,1901-12-25 +1901-05-05 12:34:56.1461,1901-12-25 +1901-05-05 12:34:56.1462,1901-12-25 +1901-05-05 12:34:56.1463,1901-12-25 +1901-05-05 12:34:56.1464,1901-12-25 +1901-05-05 12:34:56.1465,1901-12-25 +1901-05-05 12:34:56.1466,1901-12-25 +1901-05-05 12:34:56.1467,1901-12-25 +1901-05-05 12:34:56.1468,1901-12-25 +1901-05-05 12:34:56.1469,1901-12-25 +1901-05-05 12:34:56.147,1901-12-25 +1901-05-05 12:34:56.1471,1901-12-25 +1901-05-05 12:34:56.1472,1901-12-25 +1901-05-05 12:34:56.1473,1901-12-25 +1901-05-05 12:34:56.1474,1901-12-25 +1901-05-05 12:34:56.1475,1901-12-25 +1901-05-05 12:34:56.1476,1901-12-25 +1901-05-05 12:34:56.1477,1901-12-25 +1901-05-05 12:34:56.1478,1901-12-25 +1901-05-05 12:34:56.1479,1901-12-25 +1901-05-05 12:34:56.148,1901-12-25 +1901-05-05 12:34:56.1481,1901-12-25 +1901-05-05 12:34:56.1482,1901-12-25 +1901-05-05 12:34:56.1483,1901-12-25 +1901-05-05 12:34:56.1484,1901-12-25 +1901-05-05 12:34:56.1485,1901-12-25 +1901-05-05 12:34:56.1486,1901-12-25 +1901-05-05 12:34:56.1487,1901-12-25 +1901-05-05 12:34:56.1488,1901-12-25 +1901-05-05 12:34:56.1489,1901-12-25 +1901-05-05 12:34:56.149,1901-12-25 +1901-05-05 12:34:56.1491,1901-12-25 +1901-05-05 12:34:56.1492,1901-12-25 +1901-05-05 12:34:56.1493,1901-12-25 +1901-05-05 12:34:56.1494,1901-12-25 +1901-05-05 12:34:56.1495,1901-12-25 +1901-05-05 12:34:56.1496,1901-12-25 +1901-05-05 12:34:56.1497,1901-12-25 +1901-05-05 12:34:56.1498,1901-12-25 +1901-05-05 12:34:56.1499,1901-12-25 +1901-05-05 12:34:56.15,1901-12-25 +1901-05-05 12:34:56.1501,1901-12-25 +1901-05-05 12:34:56.1502,1901-12-25 +1901-05-05 12:34:56.1503,1901-12-25 +1901-05-05 12:34:56.1504,1901-12-25 +1901-05-05 12:34:56.1505,1901-12-25 +1901-05-05 12:34:56.1506,1901-12-25 +1901-05-05 12:34:56.1507,1901-12-25 +1901-05-05 12:34:56.1508,1901-12-25 +1901-05-05 12:34:56.1509,1901-12-25 +1901-05-05 12:34:56.151,1901-12-25 +1901-05-05 12:34:56.1511,1901-12-25 +1901-05-05 12:34:56.1512,1901-12-25 +1901-05-05 12:34:56.1513,1901-12-25 +1901-05-05 12:34:56.1514,1901-12-25 +1901-05-05 12:34:56.1515,1901-12-25 +1901-05-05 12:34:56.1516,1901-12-25 +1901-05-05 12:34:56.1517,1901-12-25 +1901-05-05 12:34:56.1518,1901-12-25 +1901-05-05 12:34:56.1519,1901-12-25 +1901-05-05 12:34:56.152,1901-12-25 +1901-05-05 12:34:56.1521,1901-12-25 +1901-05-05 12:34:56.1522,1901-12-25 +1901-05-05 12:34:56.1523,1901-12-25 +1901-05-05 12:34:56.1524,1901-12-25 +1901-05-05 12:34:56.1525,1901-12-25 +1901-05-05 12:34:56.1526,1901-12-25 +1901-05-05 12:34:56.1527,1901-12-25 +1901-05-05 12:34:56.1528,1901-12-25 +1901-05-05 12:34:56.1529,1901-12-25 +1901-05-05 12:34:56.153,1901-12-25 +1901-05-05 12:34:56.1531,1901-12-25 +1901-05-05 12:34:56.1532,1901-12-25 +1901-05-05 12:34:56.1533,1901-12-25 +1901-05-05 12:34:56.1534,1901-12-25 +1901-05-05 12:34:56.1535,1901-12-25 +1901-05-05 12:34:56.1536,1901-12-25 +1901-05-05 12:34:56.1537,1901-12-25 +1901-05-05 12:34:56.1538,1901-12-25 +1901-05-05 12:34:56.1539,1901-12-25 +1901-05-05 12:34:56.154,1901-12-25 +1901-05-05 12:34:56.1541,1901-12-25 +1901-05-05 12:34:56.1542,1901-12-25 +1901-05-05 12:34:56.1543,1901-12-25 +1901-05-05 12:34:56.1544,1901-12-25 +1901-05-05 12:34:56.1545,1901-12-25 +1901-05-05 12:34:56.1546,1901-12-25 +1901-05-05 12:34:56.1547,1901-12-25 +1901-05-05 12:34:56.1548,1901-12-25 +1901-05-05 12:34:56.1549,1901-12-25 +1901-05-05 12:34:56.155,1901-12-25 +1901-05-05 12:34:56.1551,1901-12-25 +1901-05-05 12:34:56.1552,1901-12-25 +1901-05-05 12:34:56.1553,1901-12-25 +1901-05-05 12:34:56.1554,1901-12-25 +1901-05-05 12:34:56.1555,1901-12-25 +1901-05-05 12:34:56.1556,1901-12-25 +1901-05-05 12:34:56.1557,1901-12-25 +1901-05-05 12:34:56.1558,1901-12-25 +1901-05-05 12:34:56.1559,1901-12-25 +1901-05-05 12:34:56.156,1901-12-25 +1901-05-05 12:34:56.1561,1901-12-25 +1901-05-05 12:34:56.1562,1901-12-25 +1901-05-05 12:34:56.1563,1901-12-25 +1901-05-05 12:34:56.1564,1901-12-25 +1901-05-05 12:34:56.1565,1901-12-25 +1901-05-05 12:34:56.1566,1901-12-25 +1901-05-05 12:34:56.1567,1901-12-25 +1901-05-05 12:34:56.1568,1901-12-25 +1901-05-05 12:34:56.1569,1901-12-25 +1901-05-05 12:34:56.157,1901-12-25 +1901-05-05 12:34:56.1571,1901-12-25 +1901-05-05 12:34:56.1572,1901-12-25 +1901-05-05 12:34:56.1573,1901-12-25 +1901-05-05 12:34:56.1574,1901-12-25 +1901-05-05 12:34:56.1575,1901-12-25 +1901-05-05 12:34:56.1576,1901-12-25 +1901-05-05 12:34:56.1577,1901-12-25 +1901-05-05 12:34:56.1578,1901-12-25 +1901-05-05 12:34:56.1579,1901-12-25 +1901-05-05 12:34:56.158,1901-12-25 +1901-05-05 12:34:56.1581,1901-12-25 +1901-05-05 12:34:56.1582,1901-12-25 +1901-05-05 12:34:56.1583,1901-12-25 +1901-05-05 12:34:56.1584,1901-12-25 +1901-05-05 12:34:56.1585,1901-12-25 +1901-05-05 12:34:56.1586,1901-12-25 +1901-05-05 12:34:56.1587,1901-12-25 +1901-05-05 12:34:56.1588,1901-12-25 +1901-05-05 12:34:56.1589,1901-12-25 +1901-05-05 12:34:56.159,1901-12-25 +1901-05-05 12:34:56.1591,1901-12-25 +1901-05-05 12:34:56.1592,1901-12-25 +1901-05-05 12:34:56.1593,1901-12-25 +1901-05-05 12:34:56.1594,1901-12-25 +1901-05-05 12:34:56.1595,1901-12-25 +1901-05-05 12:34:56.1596,1901-12-25 +1901-05-05 12:34:56.1597,1901-12-25 +1901-05-05 12:34:56.1598,1901-12-25 +1901-05-05 12:34:56.1599,1901-12-25 +1901-05-05 12:34:56.16,1901-12-25 +1901-05-05 12:34:56.1601,1901-12-25 +1901-05-05 12:34:56.1602,1901-12-25 +1901-05-05 12:34:56.1603,1901-12-25 +1901-05-05 12:34:56.1604,1901-12-25 +1901-05-05 12:34:56.1605,1901-12-25 +1901-05-05 12:34:56.1606,1901-12-25 +1901-05-05 12:34:56.1607,1901-12-25 +1901-05-05 12:34:56.1608,1901-12-25 +1901-05-05 12:34:56.1609,1901-12-25 +1901-05-05 12:34:56.161,1901-12-25 +1901-05-05 12:34:56.1611,1901-12-25 +1901-05-05 12:34:56.1612,1901-12-25 +1901-05-05 12:34:56.1613,1901-12-25 +1901-05-05 12:34:56.1614,1901-12-25 +1901-05-05 12:34:56.1615,1901-12-25 +1901-05-05 12:34:56.1616,1901-12-25 +1901-05-05 12:34:56.1617,1901-12-25 +1901-05-05 12:34:56.1618,1901-12-25 +1901-05-05 12:34:56.1619,1901-12-25 +1901-05-05 12:34:56.162,1901-12-25 +1901-05-05 12:34:56.1621,1901-12-25 +1901-05-05 12:34:56.1622,1901-12-25 +1901-05-05 12:34:56.1623,1901-12-25 +1901-05-05 12:34:56.1624,1901-12-25 +1901-05-05 12:34:56.1625,1901-12-25 +1901-05-05 12:34:56.1626,1901-12-25 +1901-05-05 12:34:56.1627,1901-12-25 +1901-05-05 12:34:56.1628,1901-12-25 +1901-05-05 12:34:56.1629,1901-12-25 +1901-05-05 12:34:56.163,1901-12-25 +1901-05-05 12:34:56.1631,1901-12-25 +1901-05-05 12:34:56.1632,1901-12-25 +1901-05-05 12:34:56.1633,1901-12-25 +1901-05-05 12:34:56.1634,1901-12-25 +1901-05-05 12:34:56.1635,1901-12-25 +1901-05-05 12:34:56.1636,1901-12-25 +1901-05-05 12:34:56.1637,1901-12-25 +1901-05-05 12:34:56.1638,1901-12-25 +1901-05-05 12:34:56.1639,1901-12-25 +1901-05-05 12:34:56.164,1901-12-25 +1901-05-05 12:34:56.1641,1901-12-25 +1901-05-05 12:34:56.1642,1901-12-25 +1901-05-05 12:34:56.1643,1901-12-25 +1901-05-05 12:34:56.1644,1901-12-25 +1901-05-05 12:34:56.1645,1901-12-25 +1901-05-05 12:34:56.1646,1901-12-25 +1901-05-05 12:34:56.1647,1901-12-25 +1901-05-05 12:34:56.1648,1901-12-25 +1901-05-05 12:34:56.1649,1901-12-25 +1901-05-05 12:34:56.165,1901-12-25 +1901-05-05 12:34:56.1651,1901-12-25 +1901-05-05 12:34:56.1652,1901-12-25 +1901-05-05 12:34:56.1653,1901-12-25 +1901-05-05 12:34:56.1654,1901-12-25 +1901-05-05 12:34:56.1655,1901-12-25 +1901-05-05 12:34:56.1656,1901-12-25 +1901-05-05 12:34:56.1657,1901-12-25 +1901-05-05 12:34:56.1658,1901-12-25 +1901-05-05 12:34:56.1659,1901-12-25 +1901-05-05 12:34:56.166,1901-12-25 +1901-05-05 12:34:56.1661,1901-12-25 +1901-05-05 12:34:56.1662,1901-12-25 +1901-05-05 12:34:56.1663,1901-12-25 +1901-05-05 12:34:56.1664,1901-12-25 +1901-05-05 12:34:56.1665,1901-12-25 +1901-05-05 12:34:56.1666,1901-12-25 +1901-05-05 12:34:56.1667,1901-12-25 +1901-05-05 12:34:56.1668,1901-12-25 +1901-05-05 12:34:56.1669,1901-12-25 +1901-05-05 12:34:56.167,1901-12-25 +1901-05-05 12:34:56.1671,1901-12-25 +1901-05-05 12:34:56.1672,1901-12-25 +1901-05-05 12:34:56.1673,1901-12-25 +1901-05-05 12:34:56.1674,1901-12-25 +1901-05-05 12:34:56.1675,1901-12-25 +1901-05-05 12:34:56.1676,1901-12-25 +1901-05-05 12:34:56.1677,1901-12-25 +1901-05-05 12:34:56.1678,1901-12-25 +1901-05-05 12:34:56.1679,1901-12-25 +1901-05-05 12:34:56.168,1901-12-25 +1901-05-05 12:34:56.1681,1901-12-25 +1901-05-05 12:34:56.1682,1901-12-25 +1901-05-05 12:34:56.1683,1901-12-25 +1901-05-05 12:34:56.1684,1901-12-25 +1901-05-05 12:34:56.1685,1901-12-25 +1901-05-05 12:34:56.1686,1901-12-25 +1901-05-05 12:34:56.1687,1901-12-25 +1901-05-05 12:34:56.1688,1901-12-25 +1901-05-05 12:34:56.1689,1901-12-25 +1901-05-05 12:34:56.169,1901-12-25 +1901-05-05 12:34:56.1691,1901-12-25 +1901-05-05 12:34:56.1692,1901-12-25 +1901-05-05 12:34:56.1693,1901-12-25 +1901-05-05 12:34:56.1694,1901-12-25 +1901-05-05 12:34:56.1695,1901-12-25 +1901-05-05 12:34:56.1696,1901-12-25 +1901-05-05 12:34:56.1697,1901-12-25 +1901-05-05 12:34:56.1698,1901-12-25 +1901-05-05 12:34:56.1699,1901-12-25 +1901-05-05 12:34:56.17,1901-12-25 +1901-05-05 12:34:56.1701,1901-12-25 +1901-05-05 12:34:56.1702,1901-12-25 +1901-05-05 12:34:56.1703,1901-12-25 +1901-05-05 12:34:56.1704,1901-12-25 +1901-05-05 12:34:56.1705,1901-12-25 +1901-05-05 12:34:56.1706,1901-12-25 +1901-05-05 12:34:56.1707,1901-12-25 +1901-05-05 12:34:56.1708,1901-12-25 +1901-05-05 12:34:56.1709,1901-12-25 +1901-05-05 12:34:56.171,1901-12-25 +1901-05-05 12:34:56.1711,1901-12-25 +1901-05-05 12:34:56.1712,1901-12-25 +1901-05-05 12:34:56.1713,1901-12-25 +1901-05-05 12:34:56.1714,1901-12-25 +1901-05-05 12:34:56.1715,1901-12-25 +1901-05-05 12:34:56.1716,1901-12-25 +1901-05-05 12:34:56.1717,1901-12-25 +1901-05-05 12:34:56.1718,1901-12-25 +1901-05-05 12:34:56.1719,1901-12-25 +1901-05-05 12:34:56.172,1901-12-25 +1901-05-05 12:34:56.1721,1901-12-25 +1901-05-05 12:34:56.1722,1901-12-25 +1901-05-05 12:34:56.1723,1901-12-25 +1901-05-05 12:34:56.1724,1901-12-25 +1901-05-05 12:34:56.1725,1901-12-25 +1901-05-05 12:34:56.1726,1901-12-25 +1901-05-05 12:34:56.1727,1901-12-25 +1901-05-05 12:34:56.1728,1901-12-25 +1901-05-05 12:34:56.1729,1901-12-25 +1901-05-05 12:34:56.173,1901-12-25 +1901-05-05 12:34:56.1731,1901-12-25 +1901-05-05 12:34:56.1732,1901-12-25 +1901-05-05 12:34:56.1733,1901-12-25 +1901-05-05 12:34:56.1734,1901-12-25 +1901-05-05 12:34:56.1735,1901-12-25 +1901-05-05 12:34:56.1736,1901-12-25 +1901-05-05 12:34:56.1737,1901-12-25 +1901-05-05 12:34:56.1738,1901-12-25 +1901-05-05 12:34:56.1739,1901-12-25 +1901-05-05 12:34:56.174,1901-12-25 +1901-05-05 12:34:56.1741,1901-12-25 +1901-05-05 12:34:56.1742,1901-12-25 +1901-05-05 12:34:56.1743,1901-12-25 +1901-05-05 12:34:56.1744,1901-12-25 +1901-05-05 12:34:56.1745,1901-12-25 +1901-05-05 12:34:56.1746,1901-12-25 +1901-05-05 12:34:56.1747,1901-12-25 +1901-05-05 12:34:56.1748,1901-12-25 +1901-05-05 12:34:56.1749,1901-12-25 +1901-05-05 12:34:56.175,1901-12-25 +1901-05-05 12:34:56.1751,1901-12-25 +1901-05-05 12:34:56.1752,1901-12-25 +1901-05-05 12:34:56.1753,1901-12-25 +1901-05-05 12:34:56.1754,1901-12-25 +1901-05-05 12:34:56.1755,1901-12-25 +1901-05-05 12:34:56.1756,1901-12-25 +1901-05-05 12:34:56.1757,1901-12-25 +1901-05-05 12:34:56.1758,1901-12-25 +1901-05-05 12:34:56.1759,1901-12-25 +1901-05-05 12:34:56.176,1901-12-25 +1901-05-05 12:34:56.1761,1901-12-25 +1901-05-05 12:34:56.1762,1901-12-25 +1901-05-05 12:34:56.1763,1901-12-25 +1901-05-05 12:34:56.1764,1901-12-25 +1901-05-05 12:34:56.1765,1901-12-25 +1901-05-05 12:34:56.1766,1901-12-25 +1901-05-05 12:34:56.1767,1901-12-25 +1901-05-05 12:34:56.1768,1901-12-25 +1901-05-05 12:34:56.1769,1901-12-25 +1901-05-05 12:34:56.177,1901-12-25 +1901-05-05 12:34:56.1771,1901-12-25 +1901-05-05 12:34:56.1772,1901-12-25 +1901-05-05 12:34:56.1773,1901-12-25 +1901-05-05 12:34:56.1774,1901-12-25 +1901-05-05 12:34:56.1775,1901-12-25 +1901-05-05 12:34:56.1776,1901-12-25 +1901-05-05 12:34:56.1777,1901-12-25 +1901-05-05 12:34:56.1778,1901-12-25 +1901-05-05 12:34:56.1779,1901-12-25 +1901-05-05 12:34:56.178,1901-12-25 +1901-05-05 12:34:56.1781,1901-12-25 +1901-05-05 12:34:56.1782,1901-12-25 +1901-05-05 12:34:56.1783,1901-12-25 +1901-05-05 12:34:56.1784,1901-12-25 +1901-05-05 12:34:56.1785,1901-12-25 +1901-05-05 12:34:56.1786,1901-12-25 +1901-05-05 12:34:56.1787,1901-12-25 +1901-05-05 12:34:56.1788,1901-12-25 +1901-05-05 12:34:56.1789,1901-12-25 +1901-05-05 12:34:56.179,1901-12-25 +1901-05-05 12:34:56.1791,1901-12-25 +1901-05-05 12:34:56.1792,1901-12-25 +1901-05-05 12:34:56.1793,1901-12-25 +1901-05-05 12:34:56.1794,1901-12-25 +1901-05-05 12:34:56.1795,1901-12-25 +1901-05-05 12:34:56.1796,1901-12-25 +1901-05-05 12:34:56.1797,1901-12-25 +1901-05-05 12:34:56.1798,1901-12-25 +1901-05-05 12:34:56.1799,1901-12-25 +1901-05-05 12:34:56.18,1901-12-25 +1901-05-05 12:34:56.1801,1901-12-25 +1901-05-05 12:34:56.1802,1901-12-25 +1901-05-05 12:34:56.1803,1901-12-25 +1901-05-05 12:34:56.1804,1901-12-25 +1901-05-05 12:34:56.1805,1901-12-25 +1901-05-05 12:34:56.1806,1901-12-25 +1901-05-05 12:34:56.1807,1901-12-25 +1901-05-05 12:34:56.1808,1901-12-25 +1901-05-05 12:34:56.1809,1901-12-25 +1901-05-05 12:34:56.181,1901-12-25 +1901-05-05 12:34:56.1811,1901-12-25 +1901-05-05 12:34:56.1812,1901-12-25 +1901-05-05 12:34:56.1813,1901-12-25 +1901-05-05 12:34:56.1814,1901-12-25 +1901-05-05 12:34:56.1815,1901-12-25 +1901-05-05 12:34:56.1816,1901-12-25 +1901-05-05 12:34:56.1817,1901-12-25 +1901-05-05 12:34:56.1818,1901-12-25 +1901-05-05 12:34:56.1819,1901-12-25 +1901-05-05 12:34:56.182,1901-12-25 +1901-05-05 12:34:56.1821,1901-12-25 +1901-05-05 12:34:56.1822,1901-12-25 +1901-05-05 12:34:56.1823,1901-12-25 +1901-05-05 12:34:56.1824,1901-12-25 +1901-05-05 12:34:56.1825,1901-12-25 +1901-05-05 12:34:56.1826,1901-12-25 +1901-05-05 12:34:56.1827,1901-12-25 +1901-05-05 12:34:56.1828,1901-12-25 +1901-05-05 12:34:56.1829,1901-12-25 +1901-05-05 12:34:56.183,1901-12-25 +1901-05-05 12:34:56.1831,1901-12-25 +1901-05-05 12:34:56.1832,1901-12-25 +1901-05-05 12:34:56.1833,1901-12-25 +1901-05-05 12:34:56.1834,1901-12-25 +1901-05-05 12:34:56.1835,1901-12-25 +1901-05-05 12:34:56.1836,1901-12-25 +1901-05-05 12:34:56.1837,1901-12-25 +1901-05-05 12:34:56.1838,1901-12-25 +1901-05-05 12:34:56.1839,1901-12-25 +1901-05-05 12:34:56.184,1901-12-25 +1901-05-05 12:34:56.1841,1901-12-25 +1901-05-05 12:34:56.1842,1901-12-25 +1901-05-05 12:34:56.1843,1901-12-25 +1901-05-05 12:34:56.1844,1901-12-25 +1901-05-05 12:34:56.1845,1901-12-25 +1901-05-05 12:34:56.1846,1901-12-25 +1901-05-05 12:34:56.1847,1901-12-25 +1901-05-05 12:34:56.1848,1901-12-25 +1901-05-05 12:34:56.1849,1901-12-25 +1901-05-05 12:34:56.185,1901-12-25 +1901-05-05 12:34:56.1851,1901-12-25 +1901-05-05 12:34:56.1852,1901-12-25 +1901-05-05 12:34:56.1853,1901-12-25 +1901-05-05 12:34:56.1854,1901-12-25 +1901-05-05 12:34:56.1855,1901-12-25 +1901-05-05 12:34:56.1856,1901-12-25 +1901-05-05 12:34:56.1857,1901-12-25 +1901-05-05 12:34:56.1858,1901-12-25 +1901-05-05 12:34:56.1859,1901-12-25 +1901-05-05 12:34:56.186,1901-12-25 +1901-05-05 12:34:56.1861,1901-12-25 +1901-05-05 12:34:56.1862,1901-12-25 +1901-05-05 12:34:56.1863,1901-12-25 +1901-05-05 12:34:56.1864,1901-12-25 +1901-05-05 12:34:56.1865,1901-12-25 +1901-05-05 12:34:56.1866,1901-12-25 +1901-05-05 12:34:56.1867,1901-12-25 +1901-05-05 12:34:56.1868,1901-12-25 +1901-05-05 12:34:56.1869,1901-12-25 +1901-05-05 12:34:56.187,1901-12-25 +1901-05-05 12:34:56.1871,1901-12-25 +1901-05-05 12:34:56.1872,1901-12-25 +1901-05-05 12:34:56.1873,1901-12-25 +1901-05-05 12:34:56.1874,1901-12-25 +1901-05-05 12:34:56.1875,1901-12-25 +1901-05-05 12:34:56.1876,1901-12-25 +1901-05-05 12:34:56.1877,1901-12-25 +1901-05-05 12:34:56.1878,1901-12-25 +1901-05-05 12:34:56.1879,1901-12-25 +1901-05-05 12:34:56.188,1901-12-25 +1901-05-05 12:34:56.1881,1901-12-25 +1901-05-05 12:34:56.1882,1901-12-25 +1901-05-05 12:34:56.1883,1901-12-25 +1901-05-05 12:34:56.1884,1901-12-25 +1901-05-05 12:34:56.1885,1901-12-25 +1901-05-05 12:34:56.1886,1901-12-25 +1901-05-05 12:34:56.1887,1901-12-25 +1901-05-05 12:34:56.1888,1901-12-25 +1901-05-05 12:34:56.1889,1901-12-25 +1901-05-05 12:34:56.189,1901-12-25 +1901-05-05 12:34:56.1891,1901-12-25 +1901-05-05 12:34:56.1892,1901-12-25 +1901-05-05 12:34:56.1893,1901-12-25 +1901-05-05 12:34:56.1894,1901-12-25 +1901-05-05 12:34:56.1895,1901-12-25 +1901-05-05 12:34:56.1896,1901-12-25 +1901-05-05 12:34:56.1897,1901-12-25 +1901-05-05 12:34:56.1898,1901-12-25 +1901-05-05 12:34:56.1899,1901-12-25 +1901-05-05 12:34:56.19,1901-12-25 +1901-05-05 12:34:56.1901,1901-12-25 +1901-05-05 12:34:56.1902,1901-12-25 +1901-05-05 12:34:56.1903,1901-12-25 +1901-05-05 12:34:56.1904,1901-12-25 +1901-05-05 12:34:56.1905,1901-12-25 +1901-05-05 12:34:56.1906,1901-12-25 +1901-05-05 12:34:56.1907,1901-12-25 +1901-05-05 12:34:56.1908,1901-12-25 +1901-05-05 12:34:56.1909,1901-12-25 +1901-05-05 12:34:56.191,1901-12-25 +1901-05-05 12:34:56.1911,1901-12-25 +1901-05-05 12:34:56.1912,1901-12-25 +1901-05-05 12:34:56.1913,1901-12-25 +1901-05-05 12:34:56.1914,1901-12-25 +1901-05-05 12:34:56.1915,1901-12-25 +1901-05-05 12:34:56.1916,1901-12-25 +1901-05-05 12:34:56.1917,1901-12-25 +1901-05-05 12:34:56.1918,1901-12-25 +1901-05-05 12:34:56.1919,1901-12-25 +1901-05-05 12:34:56.192,1901-12-25 +1901-05-05 12:34:56.1921,1901-12-25 +1901-05-05 12:34:56.1922,1901-12-25 +1901-05-05 12:34:56.1923,1901-12-25 +1901-05-05 12:34:56.1924,1901-12-25 +1901-05-05 12:34:56.1925,1901-12-25 +1901-05-05 12:34:56.1926,1901-12-25 +1901-05-05 12:34:56.1927,1901-12-25 +1901-05-05 12:34:56.1928,1901-12-25 +1901-05-05 12:34:56.1929,1901-12-25 +1901-05-05 12:34:56.193,1901-12-25 +1901-05-05 12:34:56.1931,1901-12-25 +1901-05-05 12:34:56.1932,1901-12-25 +1901-05-05 12:34:56.1933,1901-12-25 +1901-05-05 12:34:56.1934,1901-12-25 +1901-05-05 12:34:56.1935,1901-12-25 +1901-05-05 12:34:56.1936,1901-12-25 +1901-05-05 12:34:56.1937,1901-12-25 +1901-05-05 12:34:56.1938,1901-12-25 +1901-05-05 12:34:56.1939,1901-12-25 +1901-05-05 12:34:56.194,1901-12-25 +1901-05-05 12:34:56.1941,1901-12-25 +1901-05-05 12:34:56.1942,1901-12-25 +1901-05-05 12:34:56.1943,1901-12-25 +1901-05-05 12:34:56.1944,1901-12-25 +1901-05-05 12:34:56.1945,1901-12-25 +1901-05-05 12:34:56.1946,1901-12-25 +1901-05-05 12:34:56.1947,1901-12-25 +1901-05-05 12:34:56.1948,1901-12-25 +1901-05-05 12:34:56.1949,1901-12-25 +1901-05-05 12:34:56.195,1901-12-25 +1901-05-05 12:34:56.1951,1901-12-25 +1901-05-05 12:34:56.1952,1901-12-25 +1901-05-05 12:34:56.1953,1901-12-25 +1901-05-05 12:34:56.1954,1901-12-25 +1901-05-05 12:34:56.1955,1901-12-25 +1901-05-05 12:34:56.1956,1901-12-25 +1901-05-05 12:34:56.1957,1901-12-25 +1901-05-05 12:34:56.1958,1901-12-25 +1901-05-05 12:34:56.1959,1901-12-25 +1901-05-05 12:34:56.196,1901-12-25 +1901-05-05 12:34:56.1961,1901-12-25 +1901-05-05 12:34:56.1962,1901-12-25 +1901-05-05 12:34:56.1963,1901-12-25 +1901-05-05 12:34:56.1964,1901-12-25 +1901-05-05 12:34:56.1965,1901-12-25 +1901-05-05 12:34:56.1966,1901-12-25 +1901-05-05 12:34:56.1967,1901-12-25 +1901-05-05 12:34:56.1968,1901-12-25 +1901-05-05 12:34:56.1969,1901-12-25 +1901-05-05 12:34:56.197,1901-12-25 +1901-05-05 12:34:56.1971,1901-12-25 +1901-05-05 12:34:56.1972,1901-12-25 +1901-05-05 12:34:56.1973,1901-12-25 +1901-05-05 12:34:56.1974,1901-12-25 +1901-05-05 12:34:56.1975,1901-12-25 +1901-05-05 12:34:56.1976,1901-12-25 +1901-05-05 12:34:56.1977,1901-12-25 +1901-05-05 12:34:56.1978,1901-12-25 +1901-05-05 12:34:56.1979,1901-12-25 +1901-05-05 12:34:56.198,1901-12-25 +1901-05-05 12:34:56.1981,1901-12-25 +1901-05-05 12:34:56.1982,1901-12-25 +1901-05-05 12:34:56.1983,1901-12-25 +1901-05-05 12:34:56.1984,1901-12-25 +1901-05-05 12:34:56.1985,1901-12-25 +1901-05-05 12:34:56.1986,1901-12-25 +1901-05-05 12:34:56.1987,1901-12-25 +1901-05-05 12:34:56.1988,1901-12-25 +1901-05-05 12:34:56.1989,1901-12-25 +1901-05-05 12:34:56.199,1901-12-25 +1901-05-05 12:34:56.1991,1901-12-25 +1901-05-05 12:34:56.1992,1901-12-25 +1901-05-05 12:34:56.1993,1901-12-25 +1901-05-05 12:34:56.1994,1901-12-25 +1901-05-05 12:34:56.1995,1901-12-25 +1901-05-05 12:34:56.1996,1901-12-25 +1901-05-05 12:34:56.1997,1901-12-25 +1901-05-05 12:34:56.1998,1901-12-25 +1901-05-05 12:34:56.1999,1901-12-25 +1902-05-05 12:34:56.1,1902-12-25 +1902-05-05 12:34:56.1001,1902-12-25 +1902-05-05 12:34:56.1002,1902-12-25 +1902-05-05 12:34:56.1003,1902-12-25 +1902-05-05 12:34:56.1004,1902-12-25 +1902-05-05 12:34:56.1005,1902-12-25 +1902-05-05 12:34:56.1006,1902-12-25 +1902-05-05 12:34:56.1007,1902-12-25 +1902-05-05 12:34:56.1008,1902-12-25 +1902-05-05 12:34:56.1009,1902-12-25 +1902-05-05 12:34:56.101,1902-12-25 +1902-05-05 12:34:56.1011,1902-12-25 +1902-05-05 12:34:56.1012,1902-12-25 +1902-05-05 12:34:56.1013,1902-12-25 +1902-05-05 12:34:56.1014,1902-12-25 +1902-05-05 12:34:56.1015,1902-12-25 +1902-05-05 12:34:56.1016,1902-12-25 +1902-05-05 12:34:56.1017,1902-12-25 +1902-05-05 12:34:56.1018,1902-12-25 +1902-05-05 12:34:56.1019,1902-12-25 +1902-05-05 12:34:56.102,1902-12-25 +1902-05-05 12:34:56.1021,1902-12-25 +1902-05-05 12:34:56.1022,1902-12-25 +1902-05-05 12:34:56.1023,1902-12-25 +1902-05-05 12:34:56.1024,1902-12-25 +1902-05-05 12:34:56.1025,1902-12-25 +1902-05-05 12:34:56.1026,1902-12-25 +1902-05-05 12:34:56.1027,1902-12-25 +1902-05-05 12:34:56.1028,1902-12-25 +1902-05-05 12:34:56.1029,1902-12-25 +1902-05-05 12:34:56.103,1902-12-25 +1902-05-05 12:34:56.1031,1902-12-25 +1902-05-05 12:34:56.1032,1902-12-25 +1902-05-05 12:34:56.1033,1902-12-25 +1902-05-05 12:34:56.1034,1902-12-25 +1902-05-05 12:34:56.1035,1902-12-25 +1902-05-05 12:34:56.1036,1902-12-25 +1902-05-05 12:34:56.1037,1902-12-25 +1902-05-05 12:34:56.1038,1902-12-25 +1902-05-05 12:34:56.1039,1902-12-25 +1902-05-05 12:34:56.104,1902-12-25 +1902-05-05 12:34:56.1041,1902-12-25 +1902-05-05 12:34:56.1042,1902-12-25 +1902-05-05 12:34:56.1043,1902-12-25 +1902-05-05 12:34:56.1044,1902-12-25 +1902-05-05 12:34:56.1045,1902-12-25 +1902-05-05 12:34:56.1046,1902-12-25 +1902-05-05 12:34:56.1047,1902-12-25 +1902-05-05 12:34:56.1048,1902-12-25 +1902-05-05 12:34:56.1049,1902-12-25 +1902-05-05 12:34:56.105,1902-12-25 +1902-05-05 12:34:56.1051,1902-12-25 +1902-05-05 12:34:56.1052,1902-12-25 +1902-05-05 12:34:56.1053,1902-12-25 +1902-05-05 12:34:56.1054,1902-12-25 +1902-05-05 12:34:56.1055,1902-12-25 +1902-05-05 12:34:56.1056,1902-12-25 +1902-05-05 12:34:56.1057,1902-12-25 +1902-05-05 12:34:56.1058,1902-12-25 +1902-05-05 12:34:56.1059,1902-12-25 +1902-05-05 12:34:56.106,1902-12-25 +1902-05-05 12:34:56.1061,1902-12-25 +1902-05-05 12:34:56.1062,1902-12-25 +1902-05-05 12:34:56.1063,1902-12-25 +1902-05-05 12:34:56.1064,1902-12-25 +1902-05-05 12:34:56.1065,1902-12-25 +1902-05-05 12:34:56.1066,1902-12-25 +1902-05-05 12:34:56.1067,1902-12-25 +1902-05-05 12:34:56.1068,1902-12-25 +1902-05-05 12:34:56.1069,1902-12-25 +1902-05-05 12:34:56.107,1902-12-25 +1902-05-05 12:34:56.1071,1902-12-25 +1902-05-05 12:34:56.1072,1902-12-25 +1902-05-05 12:34:56.1073,1902-12-25 +1902-05-05 12:34:56.1074,1902-12-25 +1902-05-05 12:34:56.1075,1902-12-25 +1902-05-05 12:34:56.1076,1902-12-25 +1902-05-05 12:34:56.1077,1902-12-25 +1902-05-05 12:34:56.1078,1902-12-25 +1902-05-05 12:34:56.1079,1902-12-25 +1902-05-05 12:34:56.108,1902-12-25 +1902-05-05 12:34:56.1081,1902-12-25 +1902-05-05 12:34:56.1082,1902-12-25 +1902-05-05 12:34:56.1083,1902-12-25 +1902-05-05 12:34:56.1084,1902-12-25 +1902-05-05 12:34:56.1085,1902-12-25 +1902-05-05 12:34:56.1086,1902-12-25 +1902-05-05 12:34:56.1087,1902-12-25 +1902-05-05 12:34:56.1088,1902-12-25 +1902-05-05 12:34:56.1089,1902-12-25 +1902-05-05 12:34:56.109,1902-12-25 +1902-05-05 12:34:56.1091,1902-12-25 +1902-05-05 12:34:56.1092,1902-12-25 +1902-05-05 12:34:56.1093,1902-12-25 +1902-05-05 12:34:56.1094,1902-12-25 +1902-05-05 12:34:56.1095,1902-12-25 +1902-05-05 12:34:56.1096,1902-12-25 +1902-05-05 12:34:56.1097,1902-12-25 +1902-05-05 12:34:56.1098,1902-12-25 +1902-05-05 12:34:56.1099,1902-12-25 +1902-05-05 12:34:56.11,1902-12-25 +1902-05-05 12:34:56.1101,1902-12-25 +1902-05-05 12:34:56.1102,1902-12-25 +1902-05-05 12:34:56.1103,1902-12-25 +1902-05-05 12:34:56.1104,1902-12-25 +1902-05-05 12:34:56.1105,1902-12-25 +1902-05-05 12:34:56.1106,1902-12-25 +1902-05-05 12:34:56.1107,1902-12-25 +1902-05-05 12:34:56.1108,1902-12-25 +1902-05-05 12:34:56.1109,1902-12-25 +1902-05-05 12:34:56.111,1902-12-25 +1902-05-05 12:34:56.1111,1902-12-25 +1902-05-05 12:34:56.1112,1902-12-25 +1902-05-05 12:34:56.1113,1902-12-25 +1902-05-05 12:34:56.1114,1902-12-25 +1902-05-05 12:34:56.1115,1902-12-25 +1902-05-05 12:34:56.1116,1902-12-25 +1902-05-05 12:34:56.1117,1902-12-25 +1902-05-05 12:34:56.1118,1902-12-25 +1902-05-05 12:34:56.1119,1902-12-25 +1902-05-05 12:34:56.112,1902-12-25 +1902-05-05 12:34:56.1121,1902-12-25 +1902-05-05 12:34:56.1122,1902-12-25 +1902-05-05 12:34:56.1123,1902-12-25 +1902-05-05 12:34:56.1124,1902-12-25 +1902-05-05 12:34:56.1125,1902-12-25 +1902-05-05 12:34:56.1126,1902-12-25 +1902-05-05 12:34:56.1127,1902-12-25 +1902-05-05 12:34:56.1128,1902-12-25 +1902-05-05 12:34:56.1129,1902-12-25 +1902-05-05 12:34:56.113,1902-12-25 +1902-05-05 12:34:56.1131,1902-12-25 +1902-05-05 12:34:56.1132,1902-12-25 +1902-05-05 12:34:56.1133,1902-12-25 +1902-05-05 12:34:56.1134,1902-12-25 +1902-05-05 12:34:56.1135,1902-12-25 +1902-05-05 12:34:56.1136,1902-12-25 +1902-05-05 12:34:56.1137,1902-12-25 +1902-05-05 12:34:56.1138,1902-12-25 +1902-05-05 12:34:56.1139,1902-12-25 +1902-05-05 12:34:56.114,1902-12-25 +1902-05-05 12:34:56.1141,1902-12-25 +1902-05-05 12:34:56.1142,1902-12-25 +1902-05-05 12:34:56.1143,1902-12-25 +1902-05-05 12:34:56.1144,1902-12-25 +1902-05-05 12:34:56.1145,1902-12-25 +1902-05-05 12:34:56.1146,1902-12-25 +1902-05-05 12:34:56.1147,1902-12-25 +1902-05-05 12:34:56.1148,1902-12-25 +1902-05-05 12:34:56.1149,1902-12-25 +1902-05-05 12:34:56.115,1902-12-25 +1902-05-05 12:34:56.1151,1902-12-25 +1902-05-05 12:34:56.1152,1902-12-25 +1902-05-05 12:34:56.1153,1902-12-25 +1902-05-05 12:34:56.1154,1902-12-25 +1902-05-05 12:34:56.1155,1902-12-25 +1902-05-05 12:34:56.1156,1902-12-25 +1902-05-05 12:34:56.1157,1902-12-25 +1902-05-05 12:34:56.1158,1902-12-25 +1902-05-05 12:34:56.1159,1902-12-25 +1902-05-05 12:34:56.116,1902-12-25 +1902-05-05 12:34:56.1161,1902-12-25 +1902-05-05 12:34:56.1162,1902-12-25 +1902-05-05 12:34:56.1163,1902-12-25 +1902-05-05 12:34:56.1164,1902-12-25 +1902-05-05 12:34:56.1165,1902-12-25 +1902-05-05 12:34:56.1166,1902-12-25 +1902-05-05 12:34:56.1167,1902-12-25 +1902-05-05 12:34:56.1168,1902-12-25 +1902-05-05 12:34:56.1169,1902-12-25 +1902-05-05 12:34:56.117,1902-12-25 +1902-05-05 12:34:56.1171,1902-12-25 +1902-05-05 12:34:56.1172,1902-12-25 +1902-05-05 12:34:56.1173,1902-12-25 +1902-05-05 12:34:56.1174,1902-12-25 +1902-05-05 12:34:56.1175,1902-12-25 +1902-05-05 12:34:56.1176,1902-12-25 +1902-05-05 12:34:56.1177,1902-12-25 +1902-05-05 12:34:56.1178,1902-12-25 +1902-05-05 12:34:56.1179,1902-12-25 +1902-05-05 12:34:56.118,1902-12-25 +1902-05-05 12:34:56.1181,1902-12-25 +1902-05-05 12:34:56.1182,1902-12-25 +1902-05-05 12:34:56.1183,1902-12-25 +1902-05-05 12:34:56.1184,1902-12-25 +1902-05-05 12:34:56.1185,1902-12-25 +1902-05-05 12:34:56.1186,1902-12-25 +1902-05-05 12:34:56.1187,1902-12-25 +1902-05-05 12:34:56.1188,1902-12-25 +1902-05-05 12:34:56.1189,1902-12-25 +1902-05-05 12:34:56.119,1902-12-25 +1902-05-05 12:34:56.1191,1902-12-25 +1902-05-05 12:34:56.1192,1902-12-25 +1902-05-05 12:34:56.1193,1902-12-25 +1902-05-05 12:34:56.1194,1902-12-25 +1902-05-05 12:34:56.1195,1902-12-25 +1902-05-05 12:34:56.1196,1902-12-25 +1902-05-05 12:34:56.1197,1902-12-25 +1902-05-05 12:34:56.1198,1902-12-25 +1902-05-05 12:34:56.1199,1902-12-25 +1902-05-05 12:34:56.12,1902-12-25 +1902-05-05 12:34:56.1201,1902-12-25 +1902-05-05 12:34:56.1202,1902-12-25 +1902-05-05 12:34:56.1203,1902-12-25 +1902-05-05 12:34:56.1204,1902-12-25 +1902-05-05 12:34:56.1205,1902-12-25 +1902-05-05 12:34:56.1206,1902-12-25 +1902-05-05 12:34:56.1207,1902-12-25 +1902-05-05 12:34:56.1208,1902-12-25 +1902-05-05 12:34:56.1209,1902-12-25 +1902-05-05 12:34:56.121,1902-12-25 +1902-05-05 12:34:56.1211,1902-12-25 +1902-05-05 12:34:56.1212,1902-12-25 +1902-05-05 12:34:56.1213,1902-12-25 +1902-05-05 12:34:56.1214,1902-12-25 +1902-05-05 12:34:56.1215,1902-12-25 +1902-05-05 12:34:56.1216,1902-12-25 +1902-05-05 12:34:56.1217,1902-12-25 +1902-05-05 12:34:56.1218,1902-12-25 +1902-05-05 12:34:56.1219,1902-12-25 +1902-05-05 12:34:56.122,1902-12-25 +1902-05-05 12:34:56.1221,1902-12-25 +1902-05-05 12:34:56.1222,1902-12-25 +1902-05-05 12:34:56.1223,1902-12-25 +1902-05-05 12:34:56.1224,1902-12-25 +1902-05-05 12:34:56.1225,1902-12-25 +1902-05-05 12:34:56.1226,1902-12-25 +1902-05-05 12:34:56.1227,1902-12-25 +1902-05-05 12:34:56.1228,1902-12-25 +1902-05-05 12:34:56.1229,1902-12-25 +1902-05-05 12:34:56.123,1902-12-25 +1902-05-05 12:34:56.1231,1902-12-25 +1902-05-05 12:34:56.1232,1902-12-25 +1902-05-05 12:34:56.1233,1902-12-25 +1902-05-05 12:34:56.1234,1902-12-25 +1902-05-05 12:34:56.1235,1902-12-25 +1902-05-05 12:34:56.1236,1902-12-25 +1902-05-05 12:34:56.1237,1902-12-25 +1902-05-05 12:34:56.1238,1902-12-25 +1902-05-05 12:34:56.1239,1902-12-25 +1902-05-05 12:34:56.124,1902-12-25 +1902-05-05 12:34:56.1241,1902-12-25 +1902-05-05 12:34:56.1242,1902-12-25 +1902-05-05 12:34:56.1243,1902-12-25 +1902-05-05 12:34:56.1244,1902-12-25 +1902-05-05 12:34:56.1245,1902-12-25 +1902-05-05 12:34:56.1246,1902-12-25 +1902-05-05 12:34:56.1247,1902-12-25 +1902-05-05 12:34:56.1248,1902-12-25 +1902-05-05 12:34:56.1249,1902-12-25 +1902-05-05 12:34:56.125,1902-12-25 +1902-05-05 12:34:56.1251,1902-12-25 +1902-05-05 12:34:56.1252,1902-12-25 +1902-05-05 12:34:56.1253,1902-12-25 +1902-05-05 12:34:56.1254,1902-12-25 +1902-05-05 12:34:56.1255,1902-12-25 +1902-05-05 12:34:56.1256,1902-12-25 +1902-05-05 12:34:56.1257,1902-12-25 +1902-05-05 12:34:56.1258,1902-12-25 +1902-05-05 12:34:56.1259,1902-12-25 +1902-05-05 12:34:56.126,1902-12-25 +1902-05-05 12:34:56.1261,1902-12-25 +1902-05-05 12:34:56.1262,1902-12-25 +1902-05-05 12:34:56.1263,1902-12-25 +1902-05-05 12:34:56.1264,1902-12-25 +1902-05-05 12:34:56.1265,1902-12-25 +1902-05-05 12:34:56.1266,1902-12-25 +1902-05-05 12:34:56.1267,1902-12-25 +1902-05-05 12:34:56.1268,1902-12-25 +1902-05-05 12:34:56.1269,1902-12-25 +1902-05-05 12:34:56.127,1902-12-25 +1902-05-05 12:34:56.1271,1902-12-25 +1902-05-05 12:34:56.1272,1902-12-25 +1902-05-05 12:34:56.1273,1902-12-25 +1902-05-05 12:34:56.1274,1902-12-25 +1902-05-05 12:34:56.1275,1902-12-25 +1902-05-05 12:34:56.1276,1902-12-25 +1902-05-05 12:34:56.1277,1902-12-25 +1902-05-05 12:34:56.1278,1902-12-25 +1902-05-05 12:34:56.1279,1902-12-25 +1902-05-05 12:34:56.128,1902-12-25 +1902-05-05 12:34:56.1281,1902-12-25 +1902-05-05 12:34:56.1282,1902-12-25 +1902-05-05 12:34:56.1283,1902-12-25 +1902-05-05 12:34:56.1284,1902-12-25 +1902-05-05 12:34:56.1285,1902-12-25 +1902-05-05 12:34:56.1286,1902-12-25 +1902-05-05 12:34:56.1287,1902-12-25 +1902-05-05 12:34:56.1288,1902-12-25 +1902-05-05 12:34:56.1289,1902-12-25 +1902-05-05 12:34:56.129,1902-12-25 +1902-05-05 12:34:56.1291,1902-12-25 +1902-05-05 12:34:56.1292,1902-12-25 +1902-05-05 12:34:56.1293,1902-12-25 +1902-05-05 12:34:56.1294,1902-12-25 +1902-05-05 12:34:56.1295,1902-12-25 +1902-05-05 12:34:56.1296,1902-12-25 +1902-05-05 12:34:56.1297,1902-12-25 +1902-05-05 12:34:56.1298,1902-12-25 +1902-05-05 12:34:56.1299,1902-12-25 +1902-05-05 12:34:56.13,1902-12-25 +1902-05-05 12:34:56.1301,1902-12-25 +1902-05-05 12:34:56.1302,1902-12-25 +1902-05-05 12:34:56.1303,1902-12-25 +1902-05-05 12:34:56.1304,1902-12-25 +1902-05-05 12:34:56.1305,1902-12-25 +1902-05-05 12:34:56.1306,1902-12-25 +1902-05-05 12:34:56.1307,1902-12-25 +1902-05-05 12:34:56.1308,1902-12-25 +1902-05-05 12:34:56.1309,1902-12-25 +1902-05-05 12:34:56.131,1902-12-25 +1902-05-05 12:34:56.1311,1902-12-25 +1902-05-05 12:34:56.1312,1902-12-25 +1902-05-05 12:34:56.1313,1902-12-25 +1902-05-05 12:34:56.1314,1902-12-25 +1902-05-05 12:34:56.1315,1902-12-25 +1902-05-05 12:34:56.1316,1902-12-25 +1902-05-05 12:34:56.1317,1902-12-25 +1902-05-05 12:34:56.1318,1902-12-25 +1902-05-05 12:34:56.1319,1902-12-25 +1902-05-05 12:34:56.132,1902-12-25 +1902-05-05 12:34:56.1321,1902-12-25 +1902-05-05 12:34:56.1322,1902-12-25 +1902-05-05 12:34:56.1323,1902-12-25 +1902-05-05 12:34:56.1324,1902-12-25 +1902-05-05 12:34:56.1325,1902-12-25 +1902-05-05 12:34:56.1326,1902-12-25 +1902-05-05 12:34:56.1327,1902-12-25 +1902-05-05 12:34:56.1328,1902-12-25 +1902-05-05 12:34:56.1329,1902-12-25 +1902-05-05 12:34:56.133,1902-12-25 +1902-05-05 12:34:56.1331,1902-12-25 +1902-05-05 12:34:56.1332,1902-12-25 +1902-05-05 12:34:56.1333,1902-12-25 +1902-05-05 12:34:56.1334,1902-12-25 +1902-05-05 12:34:56.1335,1902-12-25 +1902-05-05 12:34:56.1336,1902-12-25 +1902-05-05 12:34:56.1337,1902-12-25 +1902-05-05 12:34:56.1338,1902-12-25 +1902-05-05 12:34:56.1339,1902-12-25 +1902-05-05 12:34:56.134,1902-12-25 +1902-05-05 12:34:56.1341,1902-12-25 +1902-05-05 12:34:56.1342,1902-12-25 +1902-05-05 12:34:56.1343,1902-12-25 +1902-05-05 12:34:56.1344,1902-12-25 +1902-05-05 12:34:56.1345,1902-12-25 +1902-05-05 12:34:56.1346,1902-12-25 +1902-05-05 12:34:56.1347,1902-12-25 +1902-05-05 12:34:56.1348,1902-12-25 +1902-05-05 12:34:56.1349,1902-12-25 +1902-05-05 12:34:56.135,1902-12-25 +1902-05-05 12:34:56.1351,1902-12-25 +1902-05-05 12:34:56.1352,1902-12-25 +1902-05-05 12:34:56.1353,1902-12-25 +1902-05-05 12:34:56.1354,1902-12-25 +1902-05-05 12:34:56.1355,1902-12-25 +1902-05-05 12:34:56.1356,1902-12-25 +1902-05-05 12:34:56.1357,1902-12-25 +1902-05-05 12:34:56.1358,1902-12-25 +1902-05-05 12:34:56.1359,1902-12-25 +1902-05-05 12:34:56.136,1902-12-25 +1902-05-05 12:34:56.1361,1902-12-25 +1902-05-05 12:34:56.1362,1902-12-25 +1902-05-05 12:34:56.1363,1902-12-25 +1902-05-05 12:34:56.1364,1902-12-25 +1902-05-05 12:34:56.1365,1902-12-25 +1902-05-05 12:34:56.1366,1902-12-25 +1902-05-05 12:34:56.1367,1902-12-25 +1902-05-05 12:34:56.1368,1902-12-25 +1902-05-05 12:34:56.1369,1902-12-25 +1902-05-05 12:34:56.137,1902-12-25 +1902-05-05 12:34:56.1371,1902-12-25 +1902-05-05 12:34:56.1372,1902-12-25 +1902-05-05 12:34:56.1373,1902-12-25 +1902-05-05 12:34:56.1374,1902-12-25 +1902-05-05 12:34:56.1375,1902-12-25 +1902-05-05 12:34:56.1376,1902-12-25 +1902-05-05 12:34:56.1377,1902-12-25 +1902-05-05 12:34:56.1378,1902-12-25 +1902-05-05 12:34:56.1379,1902-12-25 +1902-05-05 12:34:56.138,1902-12-25 +1902-05-05 12:34:56.1381,1902-12-25 +1902-05-05 12:34:56.1382,1902-12-25 +1902-05-05 12:34:56.1383,1902-12-25 +1902-05-05 12:34:56.1384,1902-12-25 +1902-05-05 12:34:56.1385,1902-12-25 +1902-05-05 12:34:56.1386,1902-12-25 +1902-05-05 12:34:56.1387,1902-12-25 +1902-05-05 12:34:56.1388,1902-12-25 +1902-05-05 12:34:56.1389,1902-12-25 +1902-05-05 12:34:56.139,1902-12-25 +1902-05-05 12:34:56.1391,1902-12-25 +1902-05-05 12:34:56.1392,1902-12-25 +1902-05-05 12:34:56.1393,1902-12-25 +1902-05-05 12:34:56.1394,1902-12-25 +1902-05-05 12:34:56.1395,1902-12-25 +1902-05-05 12:34:56.1396,1902-12-25 +1902-05-05 12:34:56.1397,1902-12-25 +1902-05-05 12:34:56.1398,1902-12-25 +1902-05-05 12:34:56.1399,1902-12-25 +1902-05-05 12:34:56.14,1902-12-25 +1902-05-05 12:34:56.1401,1902-12-25 +1902-05-05 12:34:56.1402,1902-12-25 +1902-05-05 12:34:56.1403,1902-12-25 +1902-05-05 12:34:56.1404,1902-12-25 +1902-05-05 12:34:56.1405,1902-12-25 +1902-05-05 12:34:56.1406,1902-12-25 +1902-05-05 12:34:56.1407,1902-12-25 +1902-05-05 12:34:56.1408,1902-12-25 +1902-05-05 12:34:56.1409,1902-12-25 +1902-05-05 12:34:56.141,1902-12-25 +1902-05-05 12:34:56.1411,1902-12-25 +1902-05-05 12:34:56.1412,1902-12-25 +1902-05-05 12:34:56.1413,1902-12-25 +1902-05-05 12:34:56.1414,1902-12-25 +1902-05-05 12:34:56.1415,1902-12-25 +1902-05-05 12:34:56.1416,1902-12-25 +1902-05-05 12:34:56.1417,1902-12-25 +1902-05-05 12:34:56.1418,1902-12-25 +1902-05-05 12:34:56.1419,1902-12-25 +1902-05-05 12:34:56.142,1902-12-25 +1902-05-05 12:34:56.1421,1902-12-25 +1902-05-05 12:34:56.1422,1902-12-25 +1902-05-05 12:34:56.1423,1902-12-25 +1902-05-05 12:34:56.1424,1902-12-25 +1902-05-05 12:34:56.1425,1902-12-25 +1902-05-05 12:34:56.1426,1902-12-25 +1902-05-05 12:34:56.1427,1902-12-25 +1902-05-05 12:34:56.1428,1902-12-25 +1902-05-05 12:34:56.1429,1902-12-25 +1902-05-05 12:34:56.143,1902-12-25 +1902-05-05 12:34:56.1431,1902-12-25 +1902-05-05 12:34:56.1432,1902-12-25 +1902-05-05 12:34:56.1433,1902-12-25 +1902-05-05 12:34:56.1434,1902-12-25 +1902-05-05 12:34:56.1435,1902-12-25 +1902-05-05 12:34:56.1436,1902-12-25 +1902-05-05 12:34:56.1437,1902-12-25 +1902-05-05 12:34:56.1438,1902-12-25 +1902-05-05 12:34:56.1439,1902-12-25 +1902-05-05 12:34:56.144,1902-12-25 +1902-05-05 12:34:56.1441,1902-12-25 +1902-05-05 12:34:56.1442,1902-12-25 +1902-05-05 12:34:56.1443,1902-12-25 +1902-05-05 12:34:56.1444,1902-12-25 +1902-05-05 12:34:56.1445,1902-12-25 +1902-05-05 12:34:56.1446,1902-12-25 +1902-05-05 12:34:56.1447,1902-12-25 +1902-05-05 12:34:56.1448,1902-12-25 +1902-05-05 12:34:56.1449,1902-12-25 +1902-05-05 12:34:56.145,1902-12-25 +1902-05-05 12:34:56.1451,1902-12-25 +1902-05-05 12:34:56.1452,1902-12-25 +1902-05-05 12:34:56.1453,1902-12-25 +1902-05-05 12:34:56.1454,1902-12-25 +1902-05-05 12:34:56.1455,1902-12-25 +1902-05-05 12:34:56.1456,1902-12-25 +1902-05-05 12:34:56.1457,1902-12-25 +1902-05-05 12:34:56.1458,1902-12-25 +1902-05-05 12:34:56.1459,1902-12-25 +1902-05-05 12:34:56.146,1902-12-25 +1902-05-05 12:34:56.1461,1902-12-25 +1902-05-05 12:34:56.1462,1902-12-25 +1902-05-05 12:34:56.1463,1902-12-25 +1902-05-05 12:34:56.1464,1902-12-25 +1902-05-05 12:34:56.1465,1902-12-25 +1902-05-05 12:34:56.1466,1902-12-25 +1902-05-05 12:34:56.1467,1902-12-25 +1902-05-05 12:34:56.1468,1902-12-25 +1902-05-05 12:34:56.1469,1902-12-25 +1902-05-05 12:34:56.147,1902-12-25 +1902-05-05 12:34:56.1471,1902-12-25 +1902-05-05 12:34:56.1472,1902-12-25 +1902-05-05 12:34:56.1473,1902-12-25 +1902-05-05 12:34:56.1474,1902-12-25 +1902-05-05 12:34:56.1475,1902-12-25 +1902-05-05 12:34:56.1476,1902-12-25 +1902-05-05 12:34:56.1477,1902-12-25 +1902-05-05 12:34:56.1478,1902-12-25 +1902-05-05 12:34:56.1479,1902-12-25 +1902-05-05 12:34:56.148,1902-12-25 +1902-05-05 12:34:56.1481,1902-12-25 +1902-05-05 12:34:56.1482,1902-12-25 +1902-05-05 12:34:56.1483,1902-12-25 +1902-05-05 12:34:56.1484,1902-12-25 +1902-05-05 12:34:56.1485,1902-12-25 +1902-05-05 12:34:56.1486,1902-12-25 +1902-05-05 12:34:56.1487,1902-12-25 +1902-05-05 12:34:56.1488,1902-12-25 +1902-05-05 12:34:56.1489,1902-12-25 +1902-05-05 12:34:56.149,1902-12-25 +1902-05-05 12:34:56.1491,1902-12-25 +1902-05-05 12:34:56.1492,1902-12-25 +1902-05-05 12:34:56.1493,1902-12-25 +1902-05-05 12:34:56.1494,1902-12-25 +1902-05-05 12:34:56.1495,1902-12-25 +1902-05-05 12:34:56.1496,1902-12-25 +1902-05-05 12:34:56.1497,1902-12-25 +1902-05-05 12:34:56.1498,1902-12-25 +1902-05-05 12:34:56.1499,1902-12-25 +1902-05-05 12:34:56.15,1902-12-25 +1902-05-05 12:34:56.1501,1902-12-25 +1902-05-05 12:34:56.1502,1902-12-25 +1902-05-05 12:34:56.1503,1902-12-25 +1902-05-05 12:34:56.1504,1902-12-25 +1902-05-05 12:34:56.1505,1902-12-25 +1902-05-05 12:34:56.1506,1902-12-25 +1902-05-05 12:34:56.1507,1902-12-25 +1902-05-05 12:34:56.1508,1902-12-25 +1902-05-05 12:34:56.1509,1902-12-25 +1902-05-05 12:34:56.151,1902-12-25 +1902-05-05 12:34:56.1511,1902-12-25 +1902-05-05 12:34:56.1512,1902-12-25 +1902-05-05 12:34:56.1513,1902-12-25 +1902-05-05 12:34:56.1514,1902-12-25 +1902-05-05 12:34:56.1515,1902-12-25 +1902-05-05 12:34:56.1516,1902-12-25 +1902-05-05 12:34:56.1517,1902-12-25 +1902-05-05 12:34:56.1518,1902-12-25 +1902-05-05 12:34:56.1519,1902-12-25 +1902-05-05 12:34:56.152,1902-12-25 +1902-05-05 12:34:56.1521,1902-12-25 +1902-05-05 12:34:56.1522,1902-12-25 +1902-05-05 12:34:56.1523,1902-12-25 +1902-05-05 12:34:56.1524,1902-12-25 +1902-05-05 12:34:56.1525,1902-12-25 +1902-05-05 12:34:56.1526,1902-12-25 +1902-05-05 12:34:56.1527,1902-12-25 +1902-05-05 12:34:56.1528,1902-12-25 +1902-05-05 12:34:56.1529,1902-12-25 +1902-05-05 12:34:56.153,1902-12-25 +1902-05-05 12:34:56.1531,1902-12-25 +1902-05-05 12:34:56.1532,1902-12-25 +1902-05-05 12:34:56.1533,1902-12-25 +1902-05-05 12:34:56.1534,1902-12-25 +1902-05-05 12:34:56.1535,1902-12-25 +1902-05-05 12:34:56.1536,1902-12-25 +1902-05-05 12:34:56.1537,1902-12-25 +1902-05-05 12:34:56.1538,1902-12-25 +1902-05-05 12:34:56.1539,1902-12-25 +1902-05-05 12:34:56.154,1902-12-25 +1902-05-05 12:34:56.1541,1902-12-25 +1902-05-05 12:34:56.1542,1902-12-25 +1902-05-05 12:34:56.1543,1902-12-25 +1902-05-05 12:34:56.1544,1902-12-25 +1902-05-05 12:34:56.1545,1902-12-25 +1902-05-05 12:34:56.1546,1902-12-25 +1902-05-05 12:34:56.1547,1902-12-25 +1902-05-05 12:34:56.1548,1902-12-25 +1902-05-05 12:34:56.1549,1902-12-25 +1902-05-05 12:34:56.155,1902-12-25 +1902-05-05 12:34:56.1551,1902-12-25 +1902-05-05 12:34:56.1552,1902-12-25 +1902-05-05 12:34:56.1553,1902-12-25 +1902-05-05 12:34:56.1554,1902-12-25 +1902-05-05 12:34:56.1555,1902-12-25 +1902-05-05 12:34:56.1556,1902-12-25 +1902-05-05 12:34:56.1557,1902-12-25 +1902-05-05 12:34:56.1558,1902-12-25 +1902-05-05 12:34:56.1559,1902-12-25 +1902-05-05 12:34:56.156,1902-12-25 +1902-05-05 12:34:56.1561,1902-12-25 +1902-05-05 12:34:56.1562,1902-12-25 +1902-05-05 12:34:56.1563,1902-12-25 +1902-05-05 12:34:56.1564,1902-12-25 +1902-05-05 12:34:56.1565,1902-12-25 +1902-05-05 12:34:56.1566,1902-12-25 +1902-05-05 12:34:56.1567,1902-12-25 +1902-05-05 12:34:56.1568,1902-12-25 +1902-05-05 12:34:56.1569,1902-12-25 +1902-05-05 12:34:56.157,1902-12-25 +1902-05-05 12:34:56.1571,1902-12-25 +1902-05-05 12:34:56.1572,1902-12-25 +1902-05-05 12:34:56.1573,1902-12-25 +1902-05-05 12:34:56.1574,1902-12-25 +1902-05-05 12:34:56.1575,1902-12-25 +1902-05-05 12:34:56.1576,1902-12-25 +1902-05-05 12:34:56.1577,1902-12-25 +1902-05-05 12:34:56.1578,1902-12-25 +1902-05-05 12:34:56.1579,1902-12-25 +1902-05-05 12:34:56.158,1902-12-25 +1902-05-05 12:34:56.1581,1902-12-25 +1902-05-05 12:34:56.1582,1902-12-25 +1902-05-05 12:34:56.1583,1902-12-25 +1902-05-05 12:34:56.1584,1902-12-25 +1902-05-05 12:34:56.1585,1902-12-25 +1902-05-05 12:34:56.1586,1902-12-25 +1902-05-05 12:34:56.1587,1902-12-25 +1902-05-05 12:34:56.1588,1902-12-25 +1902-05-05 12:34:56.1589,1902-12-25 +1902-05-05 12:34:56.159,1902-12-25 +1902-05-05 12:34:56.1591,1902-12-25 +1902-05-05 12:34:56.1592,1902-12-25 +1902-05-05 12:34:56.1593,1902-12-25 +1902-05-05 12:34:56.1594,1902-12-25 +1902-05-05 12:34:56.1595,1902-12-25 +1902-05-05 12:34:56.1596,1902-12-25 +1902-05-05 12:34:56.1597,1902-12-25 +1902-05-05 12:34:56.1598,1902-12-25 +1902-05-05 12:34:56.1599,1902-12-25 +1902-05-05 12:34:56.16,1902-12-25 +1902-05-05 12:34:56.1601,1902-12-25 +1902-05-05 12:34:56.1602,1902-12-25 +1902-05-05 12:34:56.1603,1902-12-25 +1902-05-05 12:34:56.1604,1902-12-25 +1902-05-05 12:34:56.1605,1902-12-25 +1902-05-05 12:34:56.1606,1902-12-25 +1902-05-05 12:34:56.1607,1902-12-25 +1902-05-05 12:34:56.1608,1902-12-25 +1902-05-05 12:34:56.1609,1902-12-25 +1902-05-05 12:34:56.161,1902-12-25 +1902-05-05 12:34:56.1611,1902-12-25 +1902-05-05 12:34:56.1612,1902-12-25 +1902-05-05 12:34:56.1613,1902-12-25 +1902-05-05 12:34:56.1614,1902-12-25 +1902-05-05 12:34:56.1615,1902-12-25 +1902-05-05 12:34:56.1616,1902-12-25 +1902-05-05 12:34:56.1617,1902-12-25 +1902-05-05 12:34:56.1618,1902-12-25 +1902-05-05 12:34:56.1619,1902-12-25 +1902-05-05 12:34:56.162,1902-12-25 +1902-05-05 12:34:56.1621,1902-12-25 +1902-05-05 12:34:56.1622,1902-12-25 +1902-05-05 12:34:56.1623,1902-12-25 +1902-05-05 12:34:56.1624,1902-12-25 +1902-05-05 12:34:56.1625,1902-12-25 +1902-05-05 12:34:56.1626,1902-12-25 +1902-05-05 12:34:56.1627,1902-12-25 +1902-05-05 12:34:56.1628,1902-12-25 +1902-05-05 12:34:56.1629,1902-12-25 +1902-05-05 12:34:56.163,1902-12-25 +1902-05-05 12:34:56.1631,1902-12-25 +1902-05-05 12:34:56.1632,1902-12-25 +1902-05-05 12:34:56.1633,1902-12-25 +1902-05-05 12:34:56.1634,1902-12-25 +1902-05-05 12:34:56.1635,1902-12-25 +1902-05-05 12:34:56.1636,1902-12-25 +1902-05-05 12:34:56.1637,1902-12-25 +1902-05-05 12:34:56.1638,1902-12-25 +1902-05-05 12:34:56.1639,1902-12-25 +1902-05-05 12:34:56.164,1902-12-25 +1902-05-05 12:34:56.1641,1902-12-25 +1902-05-05 12:34:56.1642,1902-12-25 +1902-05-05 12:34:56.1643,1902-12-25 +1902-05-05 12:34:56.1644,1902-12-25 +1902-05-05 12:34:56.1645,1902-12-25 +1902-05-05 12:34:56.1646,1902-12-25 +1902-05-05 12:34:56.1647,1902-12-25 +1902-05-05 12:34:56.1648,1902-12-25 +1902-05-05 12:34:56.1649,1902-12-25 +1902-05-05 12:34:56.165,1902-12-25 +1902-05-05 12:34:56.1651,1902-12-25 +1902-05-05 12:34:56.1652,1902-12-25 +1902-05-05 12:34:56.1653,1902-12-25 +1902-05-05 12:34:56.1654,1902-12-25 +1902-05-05 12:34:56.1655,1902-12-25 +1902-05-05 12:34:56.1656,1902-12-25 +1902-05-05 12:34:56.1657,1902-12-25 +1902-05-05 12:34:56.1658,1902-12-25 +1902-05-05 12:34:56.1659,1902-12-25 +1902-05-05 12:34:56.166,1902-12-25 +1902-05-05 12:34:56.1661,1902-12-25 +1902-05-05 12:34:56.1662,1902-12-25 +1902-05-05 12:34:56.1663,1902-12-25 +1902-05-05 12:34:56.1664,1902-12-25 +1902-05-05 12:34:56.1665,1902-12-25 +1902-05-05 12:34:56.1666,1902-12-25 +1902-05-05 12:34:56.1667,1902-12-25 +1902-05-05 12:34:56.1668,1902-12-25 +1902-05-05 12:34:56.1669,1902-12-25 +1902-05-05 12:34:56.167,1902-12-25 +1902-05-05 12:34:56.1671,1902-12-25 +1902-05-05 12:34:56.1672,1902-12-25 +1902-05-05 12:34:56.1673,1902-12-25 +1902-05-05 12:34:56.1674,1902-12-25 +1902-05-05 12:34:56.1675,1902-12-25 +1902-05-05 12:34:56.1676,1902-12-25 +1902-05-05 12:34:56.1677,1902-12-25 +1902-05-05 12:34:56.1678,1902-12-25 +1902-05-05 12:34:56.1679,1902-12-25 +1902-05-05 12:34:56.168,1902-12-25 +1902-05-05 12:34:56.1681,1902-12-25 +1902-05-05 12:34:56.1682,1902-12-25 +1902-05-05 12:34:56.1683,1902-12-25 +1902-05-05 12:34:56.1684,1902-12-25 +1902-05-05 12:34:56.1685,1902-12-25 +1902-05-05 12:34:56.1686,1902-12-25 +1902-05-05 12:34:56.1687,1902-12-25 +1902-05-05 12:34:56.1688,1902-12-25 +1902-05-05 12:34:56.1689,1902-12-25 +1902-05-05 12:34:56.169,1902-12-25 +1902-05-05 12:34:56.1691,1902-12-25 +1902-05-05 12:34:56.1692,1902-12-25 +1902-05-05 12:34:56.1693,1902-12-25 +1902-05-05 12:34:56.1694,1902-12-25 +1902-05-05 12:34:56.1695,1902-12-25 +1902-05-05 12:34:56.1696,1902-12-25 +1902-05-05 12:34:56.1697,1902-12-25 +1902-05-05 12:34:56.1698,1902-12-25 +1902-05-05 12:34:56.1699,1902-12-25 +1902-05-05 12:34:56.17,1902-12-25 +1902-05-05 12:34:56.1701,1902-12-25 +1902-05-05 12:34:56.1702,1902-12-25 +1902-05-05 12:34:56.1703,1902-12-25 +1902-05-05 12:34:56.1704,1902-12-25 +1902-05-05 12:34:56.1705,1902-12-25 +1902-05-05 12:34:56.1706,1902-12-25 +1902-05-05 12:34:56.1707,1902-12-25 +1902-05-05 12:34:56.1708,1902-12-25 +1902-05-05 12:34:56.1709,1902-12-25 +1902-05-05 12:34:56.171,1902-12-25 +1902-05-05 12:34:56.1711,1902-12-25 +1902-05-05 12:34:56.1712,1902-12-25 +1902-05-05 12:34:56.1713,1902-12-25 +1902-05-05 12:34:56.1714,1902-12-25 +1902-05-05 12:34:56.1715,1902-12-25 +1902-05-05 12:34:56.1716,1902-12-25 +1902-05-05 12:34:56.1717,1902-12-25 +1902-05-05 12:34:56.1718,1902-12-25 +1902-05-05 12:34:56.1719,1902-12-25 +1902-05-05 12:34:56.172,1902-12-25 +1902-05-05 12:34:56.1721,1902-12-25 +1902-05-05 12:34:56.1722,1902-12-25 +1902-05-05 12:34:56.1723,1902-12-25 +1902-05-05 12:34:56.1724,1902-12-25 +1902-05-05 12:34:56.1725,1902-12-25 +1902-05-05 12:34:56.1726,1902-12-25 +1902-05-05 12:34:56.1727,1902-12-25 +1902-05-05 12:34:56.1728,1902-12-25 +1902-05-05 12:34:56.1729,1902-12-25 +1902-05-05 12:34:56.173,1902-12-25 +1902-05-05 12:34:56.1731,1902-12-25 +1902-05-05 12:34:56.1732,1902-12-25 +1902-05-05 12:34:56.1733,1902-12-25 +1902-05-05 12:34:56.1734,1902-12-25 +1902-05-05 12:34:56.1735,1902-12-25 +1902-05-05 12:34:56.1736,1902-12-25 +1902-05-05 12:34:56.1737,1902-12-25 +1902-05-05 12:34:56.1738,1902-12-25 +1902-05-05 12:34:56.1739,1902-12-25 +1902-05-05 12:34:56.174,1902-12-25 +1902-05-05 12:34:56.1741,1902-12-25 +1902-05-05 12:34:56.1742,1902-12-25 +1902-05-05 12:34:56.1743,1902-12-25 +1902-05-05 12:34:56.1744,1902-12-25 +1902-05-05 12:34:56.1745,1902-12-25 +1902-05-05 12:34:56.1746,1902-12-25 +1902-05-05 12:34:56.1747,1902-12-25 +1902-05-05 12:34:56.1748,1902-12-25 +1902-05-05 12:34:56.1749,1902-12-25 +1902-05-05 12:34:56.175,1902-12-25 +1902-05-05 12:34:56.1751,1902-12-25 +1902-05-05 12:34:56.1752,1902-12-25 +1902-05-05 12:34:56.1753,1902-12-25 +1902-05-05 12:34:56.1754,1902-12-25 +1902-05-05 12:34:56.1755,1902-12-25 +1902-05-05 12:34:56.1756,1902-12-25 +1902-05-05 12:34:56.1757,1902-12-25 +1902-05-05 12:34:56.1758,1902-12-25 +1902-05-05 12:34:56.1759,1902-12-25 +1902-05-05 12:34:56.176,1902-12-25 +1902-05-05 12:34:56.1761,1902-12-25 +1902-05-05 12:34:56.1762,1902-12-25 +1902-05-05 12:34:56.1763,1902-12-25 +1902-05-05 12:34:56.1764,1902-12-25 +1902-05-05 12:34:56.1765,1902-12-25 +1902-05-05 12:34:56.1766,1902-12-25 +1902-05-05 12:34:56.1767,1902-12-25 +1902-05-05 12:34:56.1768,1902-12-25 +1902-05-05 12:34:56.1769,1902-12-25 +1902-05-05 12:34:56.177,1902-12-25 +1902-05-05 12:34:56.1771,1902-12-25 +1902-05-05 12:34:56.1772,1902-12-25 +1902-05-05 12:34:56.1773,1902-12-25 +1902-05-05 12:34:56.1774,1902-12-25 +1902-05-05 12:34:56.1775,1902-12-25 +1902-05-05 12:34:56.1776,1902-12-25 +1902-05-05 12:34:56.1777,1902-12-25 +1902-05-05 12:34:56.1778,1902-12-25 +1902-05-05 12:34:56.1779,1902-12-25 +1902-05-05 12:34:56.178,1902-12-25 +1902-05-05 12:34:56.1781,1902-12-25 +1902-05-05 12:34:56.1782,1902-12-25 +1902-05-05 12:34:56.1783,1902-12-25 +1902-05-05 12:34:56.1784,1902-12-25 +1902-05-05 12:34:56.1785,1902-12-25 +1902-05-05 12:34:56.1786,1902-12-25 +1902-05-05 12:34:56.1787,1902-12-25 +1902-05-05 12:34:56.1788,1902-12-25 +1902-05-05 12:34:56.1789,1902-12-25 +1902-05-05 12:34:56.179,1902-12-25 +1902-05-05 12:34:56.1791,1902-12-25 +1902-05-05 12:34:56.1792,1902-12-25 +1902-05-05 12:34:56.1793,1902-12-25 +1902-05-05 12:34:56.1794,1902-12-25 +1902-05-05 12:34:56.1795,1902-12-25 +1902-05-05 12:34:56.1796,1902-12-25 +1902-05-05 12:34:56.1797,1902-12-25 +1902-05-05 12:34:56.1798,1902-12-25 +1902-05-05 12:34:56.1799,1902-12-25 +1902-05-05 12:34:56.18,1902-12-25 +1902-05-05 12:34:56.1801,1902-12-25 +1902-05-05 12:34:56.1802,1902-12-25 +1902-05-05 12:34:56.1803,1902-12-25 +1902-05-05 12:34:56.1804,1902-12-25 +1902-05-05 12:34:56.1805,1902-12-25 +1902-05-05 12:34:56.1806,1902-12-25 +1902-05-05 12:34:56.1807,1902-12-25 +1902-05-05 12:34:56.1808,1902-12-25 +1902-05-05 12:34:56.1809,1902-12-25 +1902-05-05 12:34:56.181,1902-12-25 +1902-05-05 12:34:56.1811,1902-12-25 +1902-05-05 12:34:56.1812,1902-12-25 +1902-05-05 12:34:56.1813,1902-12-25 +1902-05-05 12:34:56.1814,1902-12-25 +1902-05-05 12:34:56.1815,1902-12-25 +1902-05-05 12:34:56.1816,1902-12-25 +1902-05-05 12:34:56.1817,1902-12-25 +1902-05-05 12:34:56.1818,1902-12-25 +1902-05-05 12:34:56.1819,1902-12-25 +1902-05-05 12:34:56.182,1902-12-25 +1902-05-05 12:34:56.1821,1902-12-25 +1902-05-05 12:34:56.1822,1902-12-25 +1902-05-05 12:34:56.1823,1902-12-25 +1902-05-05 12:34:56.1824,1902-12-25 +1902-05-05 12:34:56.1825,1902-12-25 +1902-05-05 12:34:56.1826,1902-12-25 +1902-05-05 12:34:56.1827,1902-12-25 +1902-05-05 12:34:56.1828,1902-12-25 +1902-05-05 12:34:56.1829,1902-12-25 +1902-05-05 12:34:56.183,1902-12-25 +1902-05-05 12:34:56.1831,1902-12-25 +1902-05-05 12:34:56.1832,1902-12-25 +1902-05-05 12:34:56.1833,1902-12-25 +1902-05-05 12:34:56.1834,1902-12-25 +1902-05-05 12:34:56.1835,1902-12-25 +1902-05-05 12:34:56.1836,1902-12-25 +1902-05-05 12:34:56.1837,1902-12-25 +1902-05-05 12:34:56.1838,1902-12-25 +1902-05-05 12:34:56.1839,1902-12-25 +1902-05-05 12:34:56.184,1902-12-25 +1902-05-05 12:34:56.1841,1902-12-25 +1902-05-05 12:34:56.1842,1902-12-25 +1902-05-05 12:34:56.1843,1902-12-25 +1902-05-05 12:34:56.1844,1902-12-25 +1902-05-05 12:34:56.1845,1902-12-25 +1902-05-05 12:34:56.1846,1902-12-25 +1902-05-05 12:34:56.1847,1902-12-25 +1902-05-05 12:34:56.1848,1902-12-25 +1902-05-05 12:34:56.1849,1902-12-25 +1902-05-05 12:34:56.185,1902-12-25 +1902-05-05 12:34:56.1851,1902-12-25 +1902-05-05 12:34:56.1852,1902-12-25 +1902-05-05 12:34:56.1853,1902-12-25 +1902-05-05 12:34:56.1854,1902-12-25 +1902-05-05 12:34:56.1855,1902-12-25 +1902-05-05 12:34:56.1856,1902-12-25 +1902-05-05 12:34:56.1857,1902-12-25 +1902-05-05 12:34:56.1858,1902-12-25 +1902-05-05 12:34:56.1859,1902-12-25 +1902-05-05 12:34:56.186,1902-12-25 +1902-05-05 12:34:56.1861,1902-12-25 +1902-05-05 12:34:56.1862,1902-12-25 +1902-05-05 12:34:56.1863,1902-12-25 +1902-05-05 12:34:56.1864,1902-12-25 +1902-05-05 12:34:56.1865,1902-12-25 +1902-05-05 12:34:56.1866,1902-12-25 +1902-05-05 12:34:56.1867,1902-12-25 +1902-05-05 12:34:56.1868,1902-12-25 +1902-05-05 12:34:56.1869,1902-12-25 +1902-05-05 12:34:56.187,1902-12-25 +1902-05-05 12:34:56.1871,1902-12-25 +1902-05-05 12:34:56.1872,1902-12-25 +1902-05-05 12:34:56.1873,1902-12-25 +1902-05-05 12:34:56.1874,1902-12-25 +1902-05-05 12:34:56.1875,1902-12-25 +1902-05-05 12:34:56.1876,1902-12-25 +1902-05-05 12:34:56.1877,1902-12-25 +1902-05-05 12:34:56.1878,1902-12-25 +1902-05-05 12:34:56.1879,1902-12-25 +1902-05-05 12:34:56.188,1902-12-25 +1902-05-05 12:34:56.1881,1902-12-25 +1902-05-05 12:34:56.1882,1902-12-25 +1902-05-05 12:34:56.1883,1902-12-25 +1902-05-05 12:34:56.1884,1902-12-25 +1902-05-05 12:34:56.1885,1902-12-25 +1902-05-05 12:34:56.1886,1902-12-25 +1902-05-05 12:34:56.1887,1902-12-25 +1902-05-05 12:34:56.1888,1902-12-25 +1902-05-05 12:34:56.1889,1902-12-25 +1902-05-05 12:34:56.189,1902-12-25 +1902-05-05 12:34:56.1891,1902-12-25 +1902-05-05 12:34:56.1892,1902-12-25 +1902-05-05 12:34:56.1893,1902-12-25 +1902-05-05 12:34:56.1894,1902-12-25 +1902-05-05 12:34:56.1895,1902-12-25 +1902-05-05 12:34:56.1896,1902-12-25 +1902-05-05 12:34:56.1897,1902-12-25 +1902-05-05 12:34:56.1898,1902-12-25 +1902-05-05 12:34:56.1899,1902-12-25 +1902-05-05 12:34:56.19,1902-12-25 +1902-05-05 12:34:56.1901,1902-12-25 +1902-05-05 12:34:56.1902,1902-12-25 +1902-05-05 12:34:56.1903,1902-12-25 +1902-05-05 12:34:56.1904,1902-12-25 +1902-05-05 12:34:56.1905,1902-12-25 +1902-05-05 12:34:56.1906,1902-12-25 +1902-05-05 12:34:56.1907,1902-12-25 +1902-05-05 12:34:56.1908,1902-12-25 +1902-05-05 12:34:56.1909,1902-12-25 +1902-05-05 12:34:56.191,1902-12-25 +1902-05-05 12:34:56.1911,1902-12-25 +1902-05-05 12:34:56.1912,1902-12-25 +1902-05-05 12:34:56.1913,1902-12-25 +1902-05-05 12:34:56.1914,1902-12-25 +1902-05-05 12:34:56.1915,1902-12-25 +1902-05-05 12:34:56.1916,1902-12-25 +1902-05-05 12:34:56.1917,1902-12-25 +1902-05-05 12:34:56.1918,1902-12-25 +1902-05-05 12:34:56.1919,1902-12-25 +1902-05-05 12:34:56.192,1902-12-25 +1902-05-05 12:34:56.1921,1902-12-25 +1902-05-05 12:34:56.1922,1902-12-25 +1902-05-05 12:34:56.1923,1902-12-25 +1902-05-05 12:34:56.1924,1902-12-25 +1902-05-05 12:34:56.1925,1902-12-25 +1902-05-05 12:34:56.1926,1902-12-25 +1902-05-05 12:34:56.1927,1902-12-25 +1902-05-05 12:34:56.1928,1902-12-25 +1902-05-05 12:34:56.1929,1902-12-25 +1902-05-05 12:34:56.193,1902-12-25 +1902-05-05 12:34:56.1931,1902-12-25 +1902-05-05 12:34:56.1932,1902-12-25 +1902-05-05 12:34:56.1933,1902-12-25 +1902-05-05 12:34:56.1934,1902-12-25 +1902-05-05 12:34:56.1935,1902-12-25 +1902-05-05 12:34:56.1936,1902-12-25 +1902-05-05 12:34:56.1937,1902-12-25 +1902-05-05 12:34:56.1938,1902-12-25 +1902-05-05 12:34:56.1939,1902-12-25 +1902-05-05 12:34:56.194,1902-12-25 +1902-05-05 12:34:56.1941,1902-12-25 +1902-05-05 12:34:56.1942,1902-12-25 +1902-05-05 12:34:56.1943,1902-12-25 +1902-05-05 12:34:56.1944,1902-12-25 +1902-05-05 12:34:56.1945,1902-12-25 +1902-05-05 12:34:56.1946,1902-12-25 +1902-05-05 12:34:56.1947,1902-12-25 +1902-05-05 12:34:56.1948,1902-12-25 +1902-05-05 12:34:56.1949,1902-12-25 +1902-05-05 12:34:56.195,1902-12-25 +1902-05-05 12:34:56.1951,1902-12-25 +1902-05-05 12:34:56.1952,1902-12-25 +1902-05-05 12:34:56.1953,1902-12-25 +1902-05-05 12:34:56.1954,1902-12-25 +1902-05-05 12:34:56.1955,1902-12-25 +1902-05-05 12:34:56.1956,1902-12-25 +1902-05-05 12:34:56.1957,1902-12-25 +1902-05-05 12:34:56.1958,1902-12-25 +1902-05-05 12:34:56.1959,1902-12-25 +1902-05-05 12:34:56.196,1902-12-25 +1902-05-05 12:34:56.1961,1902-12-25 +1902-05-05 12:34:56.1962,1902-12-25 +1902-05-05 12:34:56.1963,1902-12-25 +1902-05-05 12:34:56.1964,1902-12-25 +1902-05-05 12:34:56.1965,1902-12-25 +1902-05-05 12:34:56.1966,1902-12-25 +1902-05-05 12:34:56.1967,1902-12-25 +1902-05-05 12:34:56.1968,1902-12-25 +1902-05-05 12:34:56.1969,1902-12-25 +1902-05-05 12:34:56.197,1902-12-25 +1902-05-05 12:34:56.1971,1902-12-25 +1902-05-05 12:34:56.1972,1902-12-25 +1902-05-05 12:34:56.1973,1902-12-25 +1902-05-05 12:34:56.1974,1902-12-25 +1902-05-05 12:34:56.1975,1902-12-25 +1902-05-05 12:34:56.1976,1902-12-25 +1902-05-05 12:34:56.1977,1902-12-25 +1902-05-05 12:34:56.1978,1902-12-25 +1902-05-05 12:34:56.1979,1902-12-25 +1902-05-05 12:34:56.198,1902-12-25 +1902-05-05 12:34:56.1981,1902-12-25 +1902-05-05 12:34:56.1982,1902-12-25 +1902-05-05 12:34:56.1983,1902-12-25 +1902-05-05 12:34:56.1984,1902-12-25 +1902-05-05 12:34:56.1985,1902-12-25 +1902-05-05 12:34:56.1986,1902-12-25 +1902-05-05 12:34:56.1987,1902-12-25 +1902-05-05 12:34:56.1988,1902-12-25 +1902-05-05 12:34:56.1989,1902-12-25 +1902-05-05 12:34:56.199,1902-12-25 +1902-05-05 12:34:56.1991,1902-12-25 +1902-05-05 12:34:56.1992,1902-12-25 +1902-05-05 12:34:56.1993,1902-12-25 +1902-05-05 12:34:56.1994,1902-12-25 +1902-05-05 12:34:56.1995,1902-12-25 +1902-05-05 12:34:56.1996,1902-12-25 +1902-05-05 12:34:56.1997,1902-12-25 +1902-05-05 12:34:56.1998,1902-12-25 +1902-05-05 12:34:56.1999,1902-12-25 +1903-05-05 12:34:56.1,1903-12-25 +1903-05-05 12:34:56.1001,1903-12-25 +1903-05-05 12:34:56.1002,1903-12-25 +1903-05-05 12:34:56.1003,1903-12-25 +1903-05-05 12:34:56.1004,1903-12-25 +1903-05-05 12:34:56.1005,1903-12-25 +1903-05-05 12:34:56.1006,1903-12-25 +1903-05-05 12:34:56.1007,1903-12-25 +1903-05-05 12:34:56.1008,1903-12-25 +1903-05-05 12:34:56.1009,1903-12-25 +1903-05-05 12:34:56.101,1903-12-25 +1903-05-05 12:34:56.1011,1903-12-25 +1903-05-05 12:34:56.1012,1903-12-25 +1903-05-05 12:34:56.1013,1903-12-25 +1903-05-05 12:34:56.1014,1903-12-25 +1903-05-05 12:34:56.1015,1903-12-25 +1903-05-05 12:34:56.1016,1903-12-25 +1903-05-05 12:34:56.1017,1903-12-25 +1903-05-05 12:34:56.1018,1903-12-25 +1903-05-05 12:34:56.1019,1903-12-25 +1903-05-05 12:34:56.102,1903-12-25 +1903-05-05 12:34:56.1021,1903-12-25 +1903-05-05 12:34:56.1022,1903-12-25 +1903-05-05 12:34:56.1023,1903-12-25 +1903-05-05 12:34:56.1024,1903-12-25 +1903-05-05 12:34:56.1025,1903-12-25 +1903-05-05 12:34:56.1026,1903-12-25 +1903-05-05 12:34:56.1027,1903-12-25 +1903-05-05 12:34:56.1028,1903-12-25 +1903-05-05 12:34:56.1029,1903-12-25 +1903-05-05 12:34:56.103,1903-12-25 +1903-05-05 12:34:56.1031,1903-12-25 +1903-05-05 12:34:56.1032,1903-12-25 +1903-05-05 12:34:56.1033,1903-12-25 +1903-05-05 12:34:56.1034,1903-12-25 +1903-05-05 12:34:56.1035,1903-12-25 +1903-05-05 12:34:56.1036,1903-12-25 +1903-05-05 12:34:56.1037,1903-12-25 +1903-05-05 12:34:56.1038,1903-12-25 +1903-05-05 12:34:56.1039,1903-12-25 +1903-05-05 12:34:56.104,1903-12-25 +1903-05-05 12:34:56.1041,1903-12-25 +1903-05-05 12:34:56.1042,1903-12-25 +1903-05-05 12:34:56.1043,1903-12-25 +1903-05-05 12:34:56.1044,1903-12-25 +1903-05-05 12:34:56.1045,1903-12-25 +1903-05-05 12:34:56.1046,1903-12-25 +1903-05-05 12:34:56.1047,1903-12-25 +1903-05-05 12:34:56.1048,1903-12-25 +1903-05-05 12:34:56.1049,1903-12-25 +1903-05-05 12:34:56.105,1903-12-25 +1903-05-05 12:34:56.1051,1903-12-25 +1903-05-05 12:34:56.1052,1903-12-25 +1903-05-05 12:34:56.1053,1903-12-25 +1903-05-05 12:34:56.1054,1903-12-25 +1903-05-05 12:34:56.1055,1903-12-25 +1903-05-05 12:34:56.1056,1903-12-25 +1903-05-05 12:34:56.1057,1903-12-25 +1903-05-05 12:34:56.1058,1903-12-25 +1903-05-05 12:34:56.1059,1903-12-25 +1903-05-05 12:34:56.106,1903-12-25 +1903-05-05 12:34:56.1061,1903-12-25 +1903-05-05 12:34:56.1062,1903-12-25 +1903-05-05 12:34:56.1063,1903-12-25 +1903-05-05 12:34:56.1064,1903-12-25 +1903-05-05 12:34:56.1065,1903-12-25 +1903-05-05 12:34:56.1066,1903-12-25 +1903-05-05 12:34:56.1067,1903-12-25 +1903-05-05 12:34:56.1068,1903-12-25 +1903-05-05 12:34:56.1069,1903-12-25 +1903-05-05 12:34:56.107,1903-12-25 +1903-05-05 12:34:56.1071,1903-12-25 +1903-05-05 12:34:56.1072,1903-12-25 +1903-05-05 12:34:56.1073,1903-12-25 +1903-05-05 12:34:56.1074,1903-12-25 +1903-05-05 12:34:56.1075,1903-12-25 +1903-05-05 12:34:56.1076,1903-12-25 +1903-05-05 12:34:56.1077,1903-12-25 +1903-05-05 12:34:56.1078,1903-12-25 +1903-05-05 12:34:56.1079,1903-12-25 +1903-05-05 12:34:56.108,1903-12-25 +1903-05-05 12:34:56.1081,1903-12-25 +1903-05-05 12:34:56.1082,1903-12-25 +1903-05-05 12:34:56.1083,1903-12-25 +1903-05-05 12:34:56.1084,1903-12-25 +1903-05-05 12:34:56.1085,1903-12-25 +1903-05-05 12:34:56.1086,1903-12-25 +1903-05-05 12:34:56.1087,1903-12-25 +1903-05-05 12:34:56.1088,1903-12-25 +1903-05-05 12:34:56.1089,1903-12-25 +1903-05-05 12:34:56.109,1903-12-25 +1903-05-05 12:34:56.1091,1903-12-25 +1903-05-05 12:34:56.1092,1903-12-25 +1903-05-05 12:34:56.1093,1903-12-25 +1903-05-05 12:34:56.1094,1903-12-25 +1903-05-05 12:34:56.1095,1903-12-25 +1903-05-05 12:34:56.1096,1903-12-25 +1903-05-05 12:34:56.1097,1903-12-25 +1903-05-05 12:34:56.1098,1903-12-25 +1903-05-05 12:34:56.1099,1903-12-25 +1903-05-05 12:34:56.11,1903-12-25 +1903-05-05 12:34:56.1101,1903-12-25 +1903-05-05 12:34:56.1102,1903-12-25 +1903-05-05 12:34:56.1103,1903-12-25 +1903-05-05 12:34:56.1104,1903-12-25 +1903-05-05 12:34:56.1105,1903-12-25 +1903-05-05 12:34:56.1106,1903-12-25 +1903-05-05 12:34:56.1107,1903-12-25 +1903-05-05 12:34:56.1108,1903-12-25 +1903-05-05 12:34:56.1109,1903-12-25 +1903-05-05 12:34:56.111,1903-12-25 +1903-05-05 12:34:56.1111,1903-12-25 +1903-05-05 12:34:56.1112,1903-12-25 +1903-05-05 12:34:56.1113,1903-12-25 +1903-05-05 12:34:56.1114,1903-12-25 +1903-05-05 12:34:56.1115,1903-12-25 +1903-05-05 12:34:56.1116,1903-12-25 +1903-05-05 12:34:56.1117,1903-12-25 +1903-05-05 12:34:56.1118,1903-12-25 +1903-05-05 12:34:56.1119,1903-12-25 +1903-05-05 12:34:56.112,1903-12-25 +1903-05-05 12:34:56.1121,1903-12-25 +1903-05-05 12:34:56.1122,1903-12-25 +1903-05-05 12:34:56.1123,1903-12-25 +1903-05-05 12:34:56.1124,1903-12-25 +1903-05-05 12:34:56.1125,1903-12-25 +1903-05-05 12:34:56.1126,1903-12-25 +1903-05-05 12:34:56.1127,1903-12-25 +1903-05-05 12:34:56.1128,1903-12-25 +1903-05-05 12:34:56.1129,1903-12-25 +1903-05-05 12:34:56.113,1903-12-25 +1903-05-05 12:34:56.1131,1903-12-25 +1903-05-05 12:34:56.1132,1903-12-25 +1903-05-05 12:34:56.1133,1903-12-25 +1903-05-05 12:34:56.1134,1903-12-25 +1903-05-05 12:34:56.1135,1903-12-25 +1903-05-05 12:34:56.1136,1903-12-25 +1903-05-05 12:34:56.1137,1903-12-25 +1903-05-05 12:34:56.1138,1903-12-25 +1903-05-05 12:34:56.1139,1903-12-25 +1903-05-05 12:34:56.114,1903-12-25 +1903-05-05 12:34:56.1141,1903-12-25 +1903-05-05 12:34:56.1142,1903-12-25 +1903-05-05 12:34:56.1143,1903-12-25 +1903-05-05 12:34:56.1144,1903-12-25 +1903-05-05 12:34:56.1145,1903-12-25 +1903-05-05 12:34:56.1146,1903-12-25 +1903-05-05 12:34:56.1147,1903-12-25 +1903-05-05 12:34:56.1148,1903-12-25 +1903-05-05 12:34:56.1149,1903-12-25 +1903-05-05 12:34:56.115,1903-12-25 +1903-05-05 12:34:56.1151,1903-12-25 +1903-05-05 12:34:56.1152,1903-12-25 +1903-05-05 12:34:56.1153,1903-12-25 +1903-05-05 12:34:56.1154,1903-12-25 +1903-05-05 12:34:56.1155,1903-12-25 +1903-05-05 12:34:56.1156,1903-12-25 +1903-05-05 12:34:56.1157,1903-12-25 +1903-05-05 12:34:56.1158,1903-12-25 +1903-05-05 12:34:56.1159,1903-12-25 +1903-05-05 12:34:56.116,1903-12-25 +1903-05-05 12:34:56.1161,1903-12-25 +1903-05-05 12:34:56.1162,1903-12-25 +1903-05-05 12:34:56.1163,1903-12-25 +1903-05-05 12:34:56.1164,1903-12-25 +1903-05-05 12:34:56.1165,1903-12-25 +1903-05-05 12:34:56.1166,1903-12-25 +1903-05-05 12:34:56.1167,1903-12-25 +1903-05-05 12:34:56.1168,1903-12-25 +1903-05-05 12:34:56.1169,1903-12-25 +1903-05-05 12:34:56.117,1903-12-25 +1903-05-05 12:34:56.1171,1903-12-25 +1903-05-05 12:34:56.1172,1903-12-25 +1903-05-05 12:34:56.1173,1903-12-25 +1903-05-05 12:34:56.1174,1903-12-25 +1903-05-05 12:34:56.1175,1903-12-25 +1903-05-05 12:34:56.1176,1903-12-25 +1903-05-05 12:34:56.1177,1903-12-25 +1903-05-05 12:34:56.1178,1903-12-25 +1903-05-05 12:34:56.1179,1903-12-25 +1903-05-05 12:34:56.118,1903-12-25 +1903-05-05 12:34:56.1181,1903-12-25 +1903-05-05 12:34:56.1182,1903-12-25 +1903-05-05 12:34:56.1183,1903-12-25 +1903-05-05 12:34:56.1184,1903-12-25 +1903-05-05 12:34:56.1185,1903-12-25 +1903-05-05 12:34:56.1186,1903-12-25 +1903-05-05 12:34:56.1187,1903-12-25 +1903-05-05 12:34:56.1188,1903-12-25 +1903-05-05 12:34:56.1189,1903-12-25 +1903-05-05 12:34:56.119,1903-12-25 +1903-05-05 12:34:56.1191,1903-12-25 +1903-05-05 12:34:56.1192,1903-12-25 +1903-05-05 12:34:56.1193,1903-12-25 +1903-05-05 12:34:56.1194,1903-12-25 +1903-05-05 12:34:56.1195,1903-12-25 +1903-05-05 12:34:56.1196,1903-12-25 +1903-05-05 12:34:56.1197,1903-12-25 +1903-05-05 12:34:56.1198,1903-12-25 +1903-05-05 12:34:56.1199,1903-12-25 +1903-05-05 12:34:56.12,1903-12-25 +1903-05-05 12:34:56.1201,1903-12-25 +1903-05-05 12:34:56.1202,1903-12-25 +1903-05-05 12:34:56.1203,1903-12-25 +1903-05-05 12:34:56.1204,1903-12-25 +1903-05-05 12:34:56.1205,1903-12-25 +1903-05-05 12:34:56.1206,1903-12-25 +1903-05-05 12:34:56.1207,1903-12-25 +1903-05-05 12:34:56.1208,1903-12-25 +1903-05-05 12:34:56.1209,1903-12-25 +1903-05-05 12:34:56.121,1903-12-25 +1903-05-05 12:34:56.1211,1903-12-25 +1903-05-05 12:34:56.1212,1903-12-25 +1903-05-05 12:34:56.1213,1903-12-25 +1903-05-05 12:34:56.1214,1903-12-25 +1903-05-05 12:34:56.1215,1903-12-25 +1903-05-05 12:34:56.1216,1903-12-25 +1903-05-05 12:34:56.1217,1903-12-25 +1903-05-05 12:34:56.1218,1903-12-25 +1903-05-05 12:34:56.1219,1903-12-25 +1903-05-05 12:34:56.122,1903-12-25 +1903-05-05 12:34:56.1221,1903-12-25 +1903-05-05 12:34:56.1222,1903-12-25 +1903-05-05 12:34:56.1223,1903-12-25 +1903-05-05 12:34:56.1224,1903-12-25 +1903-05-05 12:34:56.1225,1903-12-25 +1903-05-05 12:34:56.1226,1903-12-25 +1903-05-05 12:34:56.1227,1903-12-25 +1903-05-05 12:34:56.1228,1903-12-25 +1903-05-05 12:34:56.1229,1903-12-25 +1903-05-05 12:34:56.123,1903-12-25 +1903-05-05 12:34:56.1231,1903-12-25 +1903-05-05 12:34:56.1232,1903-12-25 +1903-05-05 12:34:56.1233,1903-12-25 +1903-05-05 12:34:56.1234,1903-12-25 +1903-05-05 12:34:56.1235,1903-12-25 +1903-05-05 12:34:56.1236,1903-12-25 +1903-05-05 12:34:56.1237,1903-12-25 +1903-05-05 12:34:56.1238,1903-12-25 +1903-05-05 12:34:56.1239,1903-12-25 +1903-05-05 12:34:56.124,1903-12-25 +1903-05-05 12:34:56.1241,1903-12-25 +1903-05-05 12:34:56.1242,1903-12-25 +1903-05-05 12:34:56.1243,1903-12-25 +1903-05-05 12:34:56.1244,1903-12-25 +1903-05-05 12:34:56.1245,1903-12-25 +1903-05-05 12:34:56.1246,1903-12-25 +1903-05-05 12:34:56.1247,1903-12-25 +1903-05-05 12:34:56.1248,1903-12-25 +1903-05-05 12:34:56.1249,1903-12-25 +1903-05-05 12:34:56.125,1903-12-25 +1903-05-05 12:34:56.1251,1903-12-25 +1903-05-05 12:34:56.1252,1903-12-25 +1903-05-05 12:34:56.1253,1903-12-25 +1903-05-05 12:34:56.1254,1903-12-25 +1903-05-05 12:34:56.1255,1903-12-25 +1903-05-05 12:34:56.1256,1903-12-25 +1903-05-05 12:34:56.1257,1903-12-25 +1903-05-05 12:34:56.1258,1903-12-25 +1903-05-05 12:34:56.1259,1903-12-25 +1903-05-05 12:34:56.126,1903-12-25 +1903-05-05 12:34:56.1261,1903-12-25 +1903-05-05 12:34:56.1262,1903-12-25 +1903-05-05 12:34:56.1263,1903-12-25 +1903-05-05 12:34:56.1264,1903-12-25 +1903-05-05 12:34:56.1265,1903-12-25 +1903-05-05 12:34:56.1266,1903-12-25 +1903-05-05 12:34:56.1267,1903-12-25 +1903-05-05 12:34:56.1268,1903-12-25 +1903-05-05 12:34:56.1269,1903-12-25 +1903-05-05 12:34:56.127,1903-12-25 +1903-05-05 12:34:56.1271,1903-12-25 +1903-05-05 12:34:56.1272,1903-12-25 +1903-05-05 12:34:56.1273,1903-12-25 +1903-05-05 12:34:56.1274,1903-12-25 +1903-05-05 12:34:56.1275,1903-12-25 +1903-05-05 12:34:56.1276,1903-12-25 +1903-05-05 12:34:56.1277,1903-12-25 +1903-05-05 12:34:56.1278,1903-12-25 +1903-05-05 12:34:56.1279,1903-12-25 +1903-05-05 12:34:56.128,1903-12-25 +1903-05-05 12:34:56.1281,1903-12-25 +1903-05-05 12:34:56.1282,1903-12-25 +1903-05-05 12:34:56.1283,1903-12-25 +1903-05-05 12:34:56.1284,1903-12-25 +1903-05-05 12:34:56.1285,1903-12-25 +1903-05-05 12:34:56.1286,1903-12-25 +1903-05-05 12:34:56.1287,1903-12-25 +1903-05-05 12:34:56.1288,1903-12-25 +1903-05-05 12:34:56.1289,1903-12-25 +1903-05-05 12:34:56.129,1903-12-25 +1903-05-05 12:34:56.1291,1903-12-25 +1903-05-05 12:34:56.1292,1903-12-25 +1903-05-05 12:34:56.1293,1903-12-25 +1903-05-05 12:34:56.1294,1903-12-25 +1903-05-05 12:34:56.1295,1903-12-25 +1903-05-05 12:34:56.1296,1903-12-25 +1903-05-05 12:34:56.1297,1903-12-25 +1903-05-05 12:34:56.1298,1903-12-25 +1903-05-05 12:34:56.1299,1903-12-25 +1903-05-05 12:34:56.13,1903-12-25 +1903-05-05 12:34:56.1301,1903-12-25 +1903-05-05 12:34:56.1302,1903-12-25 +1903-05-05 12:34:56.1303,1903-12-25 +1903-05-05 12:34:56.1304,1903-12-25 +1903-05-05 12:34:56.1305,1903-12-25 +1903-05-05 12:34:56.1306,1903-12-25 +1903-05-05 12:34:56.1307,1903-12-25 +1903-05-05 12:34:56.1308,1903-12-25 +1903-05-05 12:34:56.1309,1903-12-25 +1903-05-05 12:34:56.131,1903-12-25 +1903-05-05 12:34:56.1311,1903-12-25 +1903-05-05 12:34:56.1312,1903-12-25 +1903-05-05 12:34:56.1313,1903-12-25 +1903-05-05 12:34:56.1314,1903-12-25 +1903-05-05 12:34:56.1315,1903-12-25 +1903-05-05 12:34:56.1316,1903-12-25 +1903-05-05 12:34:56.1317,1903-12-25 +1903-05-05 12:34:56.1318,1903-12-25 +1903-05-05 12:34:56.1319,1903-12-25 +1903-05-05 12:34:56.132,1903-12-25 +1903-05-05 12:34:56.1321,1903-12-25 +1903-05-05 12:34:56.1322,1903-12-25 +1903-05-05 12:34:56.1323,1903-12-25 +1903-05-05 12:34:56.1324,1903-12-25 +1903-05-05 12:34:56.1325,1903-12-25 +1903-05-05 12:34:56.1326,1903-12-25 +1903-05-05 12:34:56.1327,1903-12-25 +1903-05-05 12:34:56.1328,1903-12-25 +1903-05-05 12:34:56.1329,1903-12-25 +1903-05-05 12:34:56.133,1903-12-25 +1903-05-05 12:34:56.1331,1903-12-25 +1903-05-05 12:34:56.1332,1903-12-25 +1903-05-05 12:34:56.1333,1903-12-25 +1903-05-05 12:34:56.1334,1903-12-25 +1903-05-05 12:34:56.1335,1903-12-25 +1903-05-05 12:34:56.1336,1903-12-25 +1903-05-05 12:34:56.1337,1903-12-25 +1903-05-05 12:34:56.1338,1903-12-25 +1903-05-05 12:34:56.1339,1903-12-25 +1903-05-05 12:34:56.134,1903-12-25 +1903-05-05 12:34:56.1341,1903-12-25 +1903-05-05 12:34:56.1342,1903-12-25 +1903-05-05 12:34:56.1343,1903-12-25 +1903-05-05 12:34:56.1344,1903-12-25 +1903-05-05 12:34:56.1345,1903-12-25 +1903-05-05 12:34:56.1346,1903-12-25 +1903-05-05 12:34:56.1347,1903-12-25 +1903-05-05 12:34:56.1348,1903-12-25 +1903-05-05 12:34:56.1349,1903-12-25 +1903-05-05 12:34:56.135,1903-12-25 +1903-05-05 12:34:56.1351,1903-12-25 +1903-05-05 12:34:56.1352,1903-12-25 +1903-05-05 12:34:56.1353,1903-12-25 +1903-05-05 12:34:56.1354,1903-12-25 +1903-05-05 12:34:56.1355,1903-12-25 +1903-05-05 12:34:56.1356,1903-12-25 +1903-05-05 12:34:56.1357,1903-12-25 +1903-05-05 12:34:56.1358,1903-12-25 +1903-05-05 12:34:56.1359,1903-12-25 +1903-05-05 12:34:56.136,1903-12-25 +1903-05-05 12:34:56.1361,1903-12-25 +1903-05-05 12:34:56.1362,1903-12-25 +1903-05-05 12:34:56.1363,1903-12-25 +1903-05-05 12:34:56.1364,1903-12-25 +1903-05-05 12:34:56.1365,1903-12-25 +1903-05-05 12:34:56.1366,1903-12-25 +1903-05-05 12:34:56.1367,1903-12-25 +1903-05-05 12:34:56.1368,1903-12-25 +1903-05-05 12:34:56.1369,1903-12-25 +1903-05-05 12:34:56.137,1903-12-25 +1903-05-05 12:34:56.1371,1903-12-25 +1903-05-05 12:34:56.1372,1903-12-25 +1903-05-05 12:34:56.1373,1903-12-25 +1903-05-05 12:34:56.1374,1903-12-25 +1903-05-05 12:34:56.1375,1903-12-25 +1903-05-05 12:34:56.1376,1903-12-25 +1903-05-05 12:34:56.1377,1903-12-25 +1903-05-05 12:34:56.1378,1903-12-25 +1903-05-05 12:34:56.1379,1903-12-25 +1903-05-05 12:34:56.138,1903-12-25 +1903-05-05 12:34:56.1381,1903-12-25 +1903-05-05 12:34:56.1382,1903-12-25 +1903-05-05 12:34:56.1383,1903-12-25 +1903-05-05 12:34:56.1384,1903-12-25 +1903-05-05 12:34:56.1385,1903-12-25 +1903-05-05 12:34:56.1386,1903-12-25 +1903-05-05 12:34:56.1387,1903-12-25 +1903-05-05 12:34:56.1388,1903-12-25 +1903-05-05 12:34:56.1389,1903-12-25 +1903-05-05 12:34:56.139,1903-12-25 +1903-05-05 12:34:56.1391,1903-12-25 +1903-05-05 12:34:56.1392,1903-12-25 +1903-05-05 12:34:56.1393,1903-12-25 +1903-05-05 12:34:56.1394,1903-12-25 +1903-05-05 12:34:56.1395,1903-12-25 +1903-05-05 12:34:56.1396,1903-12-25 +1903-05-05 12:34:56.1397,1903-12-25 +1903-05-05 12:34:56.1398,1903-12-25 +1903-05-05 12:34:56.1399,1903-12-25 +1903-05-05 12:34:56.14,1903-12-25 +1903-05-05 12:34:56.1401,1903-12-25 +1903-05-05 12:34:56.1402,1903-12-25 +1903-05-05 12:34:56.1403,1903-12-25 +1903-05-05 12:34:56.1404,1903-12-25 +1903-05-05 12:34:56.1405,1903-12-25 +1903-05-05 12:34:56.1406,1903-12-25 +1903-05-05 12:34:56.1407,1903-12-25 +1903-05-05 12:34:56.1408,1903-12-25 +1903-05-05 12:34:56.1409,1903-12-25 +1903-05-05 12:34:56.141,1903-12-25 +1903-05-05 12:34:56.1411,1903-12-25 +1903-05-05 12:34:56.1412,1903-12-25 +1903-05-05 12:34:56.1413,1903-12-25 +1903-05-05 12:34:56.1414,1903-12-25 +1903-05-05 12:34:56.1415,1903-12-25 +1903-05-05 12:34:56.1416,1903-12-25 +1903-05-05 12:34:56.1417,1903-12-25 +1903-05-05 12:34:56.1418,1903-12-25 +1903-05-05 12:34:56.1419,1903-12-25 +1903-05-05 12:34:56.142,1903-12-25 +1903-05-05 12:34:56.1421,1903-12-25 +1903-05-05 12:34:56.1422,1903-12-25 +1903-05-05 12:34:56.1423,1903-12-25 +1903-05-05 12:34:56.1424,1903-12-25 +1903-05-05 12:34:56.1425,1903-12-25 +1903-05-05 12:34:56.1426,1903-12-25 +1903-05-05 12:34:56.1427,1903-12-25 +1903-05-05 12:34:56.1428,1903-12-25 +1903-05-05 12:34:56.1429,1903-12-25 +1903-05-05 12:34:56.143,1903-12-25 +1903-05-05 12:34:56.1431,1903-12-25 +1903-05-05 12:34:56.1432,1903-12-25 +1903-05-05 12:34:56.1433,1903-12-25 +1903-05-05 12:34:56.1434,1903-12-25 +1903-05-05 12:34:56.1435,1903-12-25 +1903-05-05 12:34:56.1436,1903-12-25 +1903-05-05 12:34:56.1437,1903-12-25 +1903-05-05 12:34:56.1438,1903-12-25 +1903-05-05 12:34:56.1439,1903-12-25 +1903-05-05 12:34:56.144,1903-12-25 +1903-05-05 12:34:56.1441,1903-12-25 +1903-05-05 12:34:56.1442,1903-12-25 +1903-05-05 12:34:56.1443,1903-12-25 +1903-05-05 12:34:56.1444,1903-12-25 +1903-05-05 12:34:56.1445,1903-12-25 +1903-05-05 12:34:56.1446,1903-12-25 +1903-05-05 12:34:56.1447,1903-12-25 +1903-05-05 12:34:56.1448,1903-12-25 +1903-05-05 12:34:56.1449,1903-12-25 +1903-05-05 12:34:56.145,1903-12-25 +1903-05-05 12:34:56.1451,1903-12-25 +1903-05-05 12:34:56.1452,1903-12-25 +1903-05-05 12:34:56.1453,1903-12-25 +1903-05-05 12:34:56.1454,1903-12-25 +1903-05-05 12:34:56.1455,1903-12-25 +1903-05-05 12:34:56.1456,1903-12-25 +1903-05-05 12:34:56.1457,1903-12-25 +1903-05-05 12:34:56.1458,1903-12-25 +1903-05-05 12:34:56.1459,1903-12-25 +1903-05-05 12:34:56.146,1903-12-25 +1903-05-05 12:34:56.1461,1903-12-25 +1903-05-05 12:34:56.1462,1903-12-25 +1903-05-05 12:34:56.1463,1903-12-25 +1903-05-05 12:34:56.1464,1903-12-25 +1903-05-05 12:34:56.1465,1903-12-25 +1903-05-05 12:34:56.1466,1903-12-25 +1903-05-05 12:34:56.1467,1903-12-25 +1903-05-05 12:34:56.1468,1903-12-25 +1903-05-05 12:34:56.1469,1903-12-25 +1903-05-05 12:34:56.147,1903-12-25 +1903-05-05 12:34:56.1471,1903-12-25 +1903-05-05 12:34:56.1472,1903-12-25 +1903-05-05 12:34:56.1473,1903-12-25 +1903-05-05 12:34:56.1474,1903-12-25 +1903-05-05 12:34:56.1475,1903-12-25 +1903-05-05 12:34:56.1476,1903-12-25 +1903-05-05 12:34:56.1477,1903-12-25 +1903-05-05 12:34:56.1478,1903-12-25 +1903-05-05 12:34:56.1479,1903-12-25 +1903-05-05 12:34:56.148,1903-12-25 +1903-05-05 12:34:56.1481,1903-12-25 +1903-05-05 12:34:56.1482,1903-12-25 +1903-05-05 12:34:56.1483,1903-12-25 +1903-05-05 12:34:56.1484,1903-12-25 +1903-05-05 12:34:56.1485,1903-12-25 +1903-05-05 12:34:56.1486,1903-12-25 +1903-05-05 12:34:56.1487,1903-12-25 +1903-05-05 12:34:56.1488,1903-12-25 +1903-05-05 12:34:56.1489,1903-12-25 +1903-05-05 12:34:56.149,1903-12-25 +1903-05-05 12:34:56.1491,1903-12-25 +1903-05-05 12:34:56.1492,1903-12-25 +1903-05-05 12:34:56.1493,1903-12-25 +1903-05-05 12:34:56.1494,1903-12-25 +1903-05-05 12:34:56.1495,1903-12-25 +1903-05-05 12:34:56.1496,1903-12-25 +1903-05-05 12:34:56.1497,1903-12-25 +1903-05-05 12:34:56.1498,1903-12-25 +1903-05-05 12:34:56.1499,1903-12-25 +1903-05-05 12:34:56.15,1903-12-25 +1903-05-05 12:34:56.1501,1903-12-25 +1903-05-05 12:34:56.1502,1903-12-25 +1903-05-05 12:34:56.1503,1903-12-25 +1903-05-05 12:34:56.1504,1903-12-25 +1903-05-05 12:34:56.1505,1903-12-25 +1903-05-05 12:34:56.1506,1903-12-25 +1903-05-05 12:34:56.1507,1903-12-25 +1903-05-05 12:34:56.1508,1903-12-25 +1903-05-05 12:34:56.1509,1903-12-25 +1903-05-05 12:34:56.151,1903-12-25 +1903-05-05 12:34:56.1511,1903-12-25 +1903-05-05 12:34:56.1512,1903-12-25 +1903-05-05 12:34:56.1513,1903-12-25 +1903-05-05 12:34:56.1514,1903-12-25 +1903-05-05 12:34:56.1515,1903-12-25 +1903-05-05 12:34:56.1516,1903-12-25 +1903-05-05 12:34:56.1517,1903-12-25 +1903-05-05 12:34:56.1518,1903-12-25 +1903-05-05 12:34:56.1519,1903-12-25 +1903-05-05 12:34:56.152,1903-12-25 +1903-05-05 12:34:56.1521,1903-12-25 +1903-05-05 12:34:56.1522,1903-12-25 +1903-05-05 12:34:56.1523,1903-12-25 +1903-05-05 12:34:56.1524,1903-12-25 +1903-05-05 12:34:56.1525,1903-12-25 +1903-05-05 12:34:56.1526,1903-12-25 +1903-05-05 12:34:56.1527,1903-12-25 +1903-05-05 12:34:56.1528,1903-12-25 +1903-05-05 12:34:56.1529,1903-12-25 +1903-05-05 12:34:56.153,1903-12-25 +1903-05-05 12:34:56.1531,1903-12-25 +1903-05-05 12:34:56.1532,1903-12-25 +1903-05-05 12:34:56.1533,1903-12-25 +1903-05-05 12:34:56.1534,1903-12-25 +1903-05-05 12:34:56.1535,1903-12-25 +1903-05-05 12:34:56.1536,1903-12-25 +1903-05-05 12:34:56.1537,1903-12-25 +1903-05-05 12:34:56.1538,1903-12-25 +1903-05-05 12:34:56.1539,1903-12-25 +1903-05-05 12:34:56.154,1903-12-25 +1903-05-05 12:34:56.1541,1903-12-25 +1903-05-05 12:34:56.1542,1903-12-25 +1903-05-05 12:34:56.1543,1903-12-25 +1903-05-05 12:34:56.1544,1903-12-25 +1903-05-05 12:34:56.1545,1903-12-25 +1903-05-05 12:34:56.1546,1903-12-25 +1903-05-05 12:34:56.1547,1903-12-25 +1903-05-05 12:34:56.1548,1903-12-25 +1903-05-05 12:34:56.1549,1903-12-25 +1903-05-05 12:34:56.155,1903-12-25 +1903-05-05 12:34:56.1551,1903-12-25 +1903-05-05 12:34:56.1552,1903-12-25 +1903-05-05 12:34:56.1553,1903-12-25 +1903-05-05 12:34:56.1554,1903-12-25 +1903-05-05 12:34:56.1555,1903-12-25 +1903-05-05 12:34:56.1556,1903-12-25 +1903-05-05 12:34:56.1557,1903-12-25 +1903-05-05 12:34:56.1558,1903-12-25 +1903-05-05 12:34:56.1559,1903-12-25 +1903-05-05 12:34:56.156,1903-12-25 +1903-05-05 12:34:56.1561,1903-12-25 +1903-05-05 12:34:56.1562,1903-12-25 +1903-05-05 12:34:56.1563,1903-12-25 +1903-05-05 12:34:56.1564,1903-12-25 +1903-05-05 12:34:56.1565,1903-12-25 +1903-05-05 12:34:56.1566,1903-12-25 +1903-05-05 12:34:56.1567,1903-12-25 +1903-05-05 12:34:56.1568,1903-12-25 +1903-05-05 12:34:56.1569,1903-12-25 +1903-05-05 12:34:56.157,1903-12-25 +1903-05-05 12:34:56.1571,1903-12-25 +1903-05-05 12:34:56.1572,1903-12-25 +1903-05-05 12:34:56.1573,1903-12-25 +1903-05-05 12:34:56.1574,1903-12-25 +1903-05-05 12:34:56.1575,1903-12-25 +1903-05-05 12:34:56.1576,1903-12-25 +1903-05-05 12:34:56.1577,1903-12-25 +1903-05-05 12:34:56.1578,1903-12-25 +1903-05-05 12:34:56.1579,1903-12-25 +1903-05-05 12:34:56.158,1903-12-25 +1903-05-05 12:34:56.1581,1903-12-25 +1903-05-05 12:34:56.1582,1903-12-25 +1903-05-05 12:34:56.1583,1903-12-25 +1903-05-05 12:34:56.1584,1903-12-25 +1903-05-05 12:34:56.1585,1903-12-25 +1903-05-05 12:34:56.1586,1903-12-25 +1903-05-05 12:34:56.1587,1903-12-25 +1903-05-05 12:34:56.1588,1903-12-25 +1903-05-05 12:34:56.1589,1903-12-25 +1903-05-05 12:34:56.159,1903-12-25 +1903-05-05 12:34:56.1591,1903-12-25 +1903-05-05 12:34:56.1592,1903-12-25 +1903-05-05 12:34:56.1593,1903-12-25 +1903-05-05 12:34:56.1594,1903-12-25 +1903-05-05 12:34:56.1595,1903-12-25 +1903-05-05 12:34:56.1596,1903-12-25 +1903-05-05 12:34:56.1597,1903-12-25 +1903-05-05 12:34:56.1598,1903-12-25 +1903-05-05 12:34:56.1599,1903-12-25 +1903-05-05 12:34:56.16,1903-12-25 +1903-05-05 12:34:56.1601,1903-12-25 +1903-05-05 12:34:56.1602,1903-12-25 +1903-05-05 12:34:56.1603,1903-12-25 +1903-05-05 12:34:56.1604,1903-12-25 +1903-05-05 12:34:56.1605,1903-12-25 +1903-05-05 12:34:56.1606,1903-12-25 +1903-05-05 12:34:56.1607,1903-12-25 +1903-05-05 12:34:56.1608,1903-12-25 +1903-05-05 12:34:56.1609,1903-12-25 +1903-05-05 12:34:56.161,1903-12-25 +1903-05-05 12:34:56.1611,1903-12-25 +1903-05-05 12:34:56.1612,1903-12-25 +1903-05-05 12:34:56.1613,1903-12-25 +1903-05-05 12:34:56.1614,1903-12-25 +1903-05-05 12:34:56.1615,1903-12-25 +1903-05-05 12:34:56.1616,1903-12-25 +1903-05-05 12:34:56.1617,1903-12-25 +1903-05-05 12:34:56.1618,1903-12-25 +1903-05-05 12:34:56.1619,1903-12-25 +1903-05-05 12:34:56.162,1903-12-25 +1903-05-05 12:34:56.1621,1903-12-25 +1903-05-05 12:34:56.1622,1903-12-25 +1903-05-05 12:34:56.1623,1903-12-25 +1903-05-05 12:34:56.1624,1903-12-25 +1903-05-05 12:34:56.1625,1903-12-25 +1903-05-05 12:34:56.1626,1903-12-25 +1903-05-05 12:34:56.1627,1903-12-25 +1903-05-05 12:34:56.1628,1903-12-25 +1903-05-05 12:34:56.1629,1903-12-25 +1903-05-05 12:34:56.163,1903-12-25 +1903-05-05 12:34:56.1631,1903-12-25 +1903-05-05 12:34:56.1632,1903-12-25 +1903-05-05 12:34:56.1633,1903-12-25 +1903-05-05 12:34:56.1634,1903-12-25 +1903-05-05 12:34:56.1635,1903-12-25 +1903-05-05 12:34:56.1636,1903-12-25 +1903-05-05 12:34:56.1637,1903-12-25 +1903-05-05 12:34:56.1638,1903-12-25 +1903-05-05 12:34:56.1639,1903-12-25 +1903-05-05 12:34:56.164,1903-12-25 +1903-05-05 12:34:56.1641,1903-12-25 +1903-05-05 12:34:56.1642,1903-12-25 +1903-05-05 12:34:56.1643,1903-12-25 +1903-05-05 12:34:56.1644,1903-12-25 +1903-05-05 12:34:56.1645,1903-12-25 +1903-05-05 12:34:56.1646,1903-12-25 +1903-05-05 12:34:56.1647,1903-12-25 +1903-05-05 12:34:56.1648,1903-12-25 +1903-05-05 12:34:56.1649,1903-12-25 +1903-05-05 12:34:56.165,1903-12-25 +1903-05-05 12:34:56.1651,1903-12-25 +1903-05-05 12:34:56.1652,1903-12-25 +1903-05-05 12:34:56.1653,1903-12-25 +1903-05-05 12:34:56.1654,1903-12-25 +1903-05-05 12:34:56.1655,1903-12-25 +1903-05-05 12:34:56.1656,1903-12-25 +1903-05-05 12:34:56.1657,1903-12-25 +1903-05-05 12:34:56.1658,1903-12-25 +1903-05-05 12:34:56.1659,1903-12-25 +1903-05-05 12:34:56.166,1903-12-25 +1903-05-05 12:34:56.1661,1903-12-25 +1903-05-05 12:34:56.1662,1903-12-25 +1903-05-05 12:34:56.1663,1903-12-25 +1903-05-05 12:34:56.1664,1903-12-25 +1903-05-05 12:34:56.1665,1903-12-25 +1903-05-05 12:34:56.1666,1903-12-25 +1903-05-05 12:34:56.1667,1903-12-25 +1903-05-05 12:34:56.1668,1903-12-25 +1903-05-05 12:34:56.1669,1903-12-25 +1903-05-05 12:34:56.167,1903-12-25 +1903-05-05 12:34:56.1671,1903-12-25 +1903-05-05 12:34:56.1672,1903-12-25 +1903-05-05 12:34:56.1673,1903-12-25 +1903-05-05 12:34:56.1674,1903-12-25 +1903-05-05 12:34:56.1675,1903-12-25 +1903-05-05 12:34:56.1676,1903-12-25 +1903-05-05 12:34:56.1677,1903-12-25 +1903-05-05 12:34:56.1678,1903-12-25 +1903-05-05 12:34:56.1679,1903-12-25 +1903-05-05 12:34:56.168,1903-12-25 +1903-05-05 12:34:56.1681,1903-12-25 +1903-05-05 12:34:56.1682,1903-12-25 +1903-05-05 12:34:56.1683,1903-12-25 +1903-05-05 12:34:56.1684,1903-12-25 +1903-05-05 12:34:56.1685,1903-12-25 +1903-05-05 12:34:56.1686,1903-12-25 +1903-05-05 12:34:56.1687,1903-12-25 +1903-05-05 12:34:56.1688,1903-12-25 +1903-05-05 12:34:56.1689,1903-12-25 +1903-05-05 12:34:56.169,1903-12-25 +1903-05-05 12:34:56.1691,1903-12-25 +1903-05-05 12:34:56.1692,1903-12-25 +1903-05-05 12:34:56.1693,1903-12-25 +1903-05-05 12:34:56.1694,1903-12-25 +1903-05-05 12:34:56.1695,1903-12-25 +1903-05-05 12:34:56.1696,1903-12-25 +1903-05-05 12:34:56.1697,1903-12-25 +1903-05-05 12:34:56.1698,1903-12-25 +1903-05-05 12:34:56.1699,1903-12-25 +1903-05-05 12:34:56.17,1903-12-25 +1903-05-05 12:34:56.1701,1903-12-25 +1903-05-05 12:34:56.1702,1903-12-25 +1903-05-05 12:34:56.1703,1903-12-25 +1903-05-05 12:34:56.1704,1903-12-25 +1903-05-05 12:34:56.1705,1903-12-25 +1903-05-05 12:34:56.1706,1903-12-25 +1903-05-05 12:34:56.1707,1903-12-25 +1903-05-05 12:34:56.1708,1903-12-25 +1903-05-05 12:34:56.1709,1903-12-25 +1903-05-05 12:34:56.171,1903-12-25 +1903-05-05 12:34:56.1711,1903-12-25 +1903-05-05 12:34:56.1712,1903-12-25 +1903-05-05 12:34:56.1713,1903-12-25 +1903-05-05 12:34:56.1714,1903-12-25 +1903-05-05 12:34:56.1715,1903-12-25 +1903-05-05 12:34:56.1716,1903-12-25 +1903-05-05 12:34:56.1717,1903-12-25 +1903-05-05 12:34:56.1718,1903-12-25 +1903-05-05 12:34:56.1719,1903-12-25 +1903-05-05 12:34:56.172,1903-12-25 +1903-05-05 12:34:56.1721,1903-12-25 +1903-05-05 12:34:56.1722,1903-12-25 +1903-05-05 12:34:56.1723,1903-12-25 +1903-05-05 12:34:56.1724,1903-12-25 +1903-05-05 12:34:56.1725,1903-12-25 +1903-05-05 12:34:56.1726,1903-12-25 +1903-05-05 12:34:56.1727,1903-12-25 +1903-05-05 12:34:56.1728,1903-12-25 +1903-05-05 12:34:56.1729,1903-12-25 +1903-05-05 12:34:56.173,1903-12-25 +1903-05-05 12:34:56.1731,1903-12-25 +1903-05-05 12:34:56.1732,1903-12-25 +1903-05-05 12:34:56.1733,1903-12-25 +1903-05-05 12:34:56.1734,1903-12-25 +1903-05-05 12:34:56.1735,1903-12-25 +1903-05-05 12:34:56.1736,1903-12-25 +1903-05-05 12:34:56.1737,1903-12-25 +1903-05-05 12:34:56.1738,1903-12-25 +1903-05-05 12:34:56.1739,1903-12-25 +1903-05-05 12:34:56.174,1903-12-25 +1903-05-05 12:34:56.1741,1903-12-25 +1903-05-05 12:34:56.1742,1903-12-25 +1903-05-05 12:34:56.1743,1903-12-25 +1903-05-05 12:34:56.1744,1903-12-25 +1903-05-05 12:34:56.1745,1903-12-25 +1903-05-05 12:34:56.1746,1903-12-25 +1903-05-05 12:34:56.1747,1903-12-25 +1903-05-05 12:34:56.1748,1903-12-25 +1903-05-05 12:34:56.1749,1903-12-25 +1903-05-05 12:34:56.175,1903-12-25 +1903-05-05 12:34:56.1751,1903-12-25 +1903-05-05 12:34:56.1752,1903-12-25 +1903-05-05 12:34:56.1753,1903-12-25 +1903-05-05 12:34:56.1754,1903-12-25 +1903-05-05 12:34:56.1755,1903-12-25 +1903-05-05 12:34:56.1756,1903-12-25 +1903-05-05 12:34:56.1757,1903-12-25 +1903-05-05 12:34:56.1758,1903-12-25 +1903-05-05 12:34:56.1759,1903-12-25 +1903-05-05 12:34:56.176,1903-12-25 +1903-05-05 12:34:56.1761,1903-12-25 +1903-05-05 12:34:56.1762,1903-12-25 +1903-05-05 12:34:56.1763,1903-12-25 +1903-05-05 12:34:56.1764,1903-12-25 +1903-05-05 12:34:56.1765,1903-12-25 +1903-05-05 12:34:56.1766,1903-12-25 +1903-05-05 12:34:56.1767,1903-12-25 +1903-05-05 12:34:56.1768,1903-12-25 +1903-05-05 12:34:56.1769,1903-12-25 +1903-05-05 12:34:56.177,1903-12-25 +1903-05-05 12:34:56.1771,1903-12-25 +1903-05-05 12:34:56.1772,1903-12-25 +1903-05-05 12:34:56.1773,1903-12-25 +1903-05-05 12:34:56.1774,1903-12-25 +1903-05-05 12:34:56.1775,1903-12-25 +1903-05-05 12:34:56.1776,1903-12-25 +1903-05-05 12:34:56.1777,1903-12-25 +1903-05-05 12:34:56.1778,1903-12-25 +1903-05-05 12:34:56.1779,1903-12-25 +1903-05-05 12:34:56.178,1903-12-25 +1903-05-05 12:34:56.1781,1903-12-25 +1903-05-05 12:34:56.1782,1903-12-25 +1903-05-05 12:34:56.1783,1903-12-25 +1903-05-05 12:34:56.1784,1903-12-25 +1903-05-05 12:34:56.1785,1903-12-25 +1903-05-05 12:34:56.1786,1903-12-25 +1903-05-05 12:34:56.1787,1903-12-25 +1903-05-05 12:34:56.1788,1903-12-25 +1903-05-05 12:34:56.1789,1903-12-25 +1903-05-05 12:34:56.179,1903-12-25 +1903-05-05 12:34:56.1791,1903-12-25 +1903-05-05 12:34:56.1792,1903-12-25 +1903-05-05 12:34:56.1793,1903-12-25 +1903-05-05 12:34:56.1794,1903-12-25 +1903-05-05 12:34:56.1795,1903-12-25 +1903-05-05 12:34:56.1796,1903-12-25 +1903-05-05 12:34:56.1797,1903-12-25 +1903-05-05 12:34:56.1798,1903-12-25 +1903-05-05 12:34:56.1799,1903-12-25 +1903-05-05 12:34:56.18,1903-12-25 +1903-05-05 12:34:56.1801,1903-12-25 +1903-05-05 12:34:56.1802,1903-12-25 +1903-05-05 12:34:56.1803,1903-12-25 +1903-05-05 12:34:56.1804,1903-12-25 +1903-05-05 12:34:56.1805,1903-12-25 +1903-05-05 12:34:56.1806,1903-12-25 +1903-05-05 12:34:56.1807,1903-12-25 +1903-05-05 12:34:56.1808,1903-12-25 +1903-05-05 12:34:56.1809,1903-12-25 +1903-05-05 12:34:56.181,1903-12-25 +1903-05-05 12:34:56.1811,1903-12-25 +1903-05-05 12:34:56.1812,1903-12-25 +1903-05-05 12:34:56.1813,1903-12-25 +1903-05-05 12:34:56.1814,1903-12-25 +1903-05-05 12:34:56.1815,1903-12-25 +1903-05-05 12:34:56.1816,1903-12-25 +1903-05-05 12:34:56.1817,1903-12-25 +1903-05-05 12:34:56.1818,1903-12-25 +1903-05-05 12:34:56.1819,1903-12-25 +1903-05-05 12:34:56.182,1903-12-25 +1903-05-05 12:34:56.1821,1903-12-25 +1903-05-05 12:34:56.1822,1903-12-25 +1903-05-05 12:34:56.1823,1903-12-25 +1903-05-05 12:34:56.1824,1903-12-25 +1903-05-05 12:34:56.1825,1903-12-25 +1903-05-05 12:34:56.1826,1903-12-25 +1903-05-05 12:34:56.1827,1903-12-25 +1903-05-05 12:34:56.1828,1903-12-25 +1903-05-05 12:34:56.1829,1903-12-25 +1903-05-05 12:34:56.183,1903-12-25 +1903-05-05 12:34:56.1831,1903-12-25 +1903-05-05 12:34:56.1832,1903-12-25 +1903-05-05 12:34:56.1833,1903-12-25 +1903-05-05 12:34:56.1834,1903-12-25 +1903-05-05 12:34:56.1835,1903-12-25 +1903-05-05 12:34:56.1836,1903-12-25 +1903-05-05 12:34:56.1837,1903-12-25 +1903-05-05 12:34:56.1838,1903-12-25 +1903-05-05 12:34:56.1839,1903-12-25 +1903-05-05 12:34:56.184,1903-12-25 +1903-05-05 12:34:56.1841,1903-12-25 +1903-05-05 12:34:56.1842,1903-12-25 +1903-05-05 12:34:56.1843,1903-12-25 +1903-05-05 12:34:56.1844,1903-12-25 +1903-05-05 12:34:56.1845,1903-12-25 +1903-05-05 12:34:56.1846,1903-12-25 +1903-05-05 12:34:56.1847,1903-12-25 +1903-05-05 12:34:56.1848,1903-12-25 +1903-05-05 12:34:56.1849,1903-12-25 +1903-05-05 12:34:56.185,1903-12-25 +1903-05-05 12:34:56.1851,1903-12-25 +1903-05-05 12:34:56.1852,1903-12-25 +1903-05-05 12:34:56.1853,1903-12-25 +1903-05-05 12:34:56.1854,1903-12-25 +1903-05-05 12:34:56.1855,1903-12-25 +1903-05-05 12:34:56.1856,1903-12-25 +1903-05-05 12:34:56.1857,1903-12-25 +1903-05-05 12:34:56.1858,1903-12-25 +1903-05-05 12:34:56.1859,1903-12-25 +1903-05-05 12:34:56.186,1903-12-25 +1903-05-05 12:34:56.1861,1903-12-25 +1903-05-05 12:34:56.1862,1903-12-25 +1903-05-05 12:34:56.1863,1903-12-25 +1903-05-05 12:34:56.1864,1903-12-25 +1903-05-05 12:34:56.1865,1903-12-25 +1903-05-05 12:34:56.1866,1903-12-25 +1903-05-05 12:34:56.1867,1903-12-25 +1903-05-05 12:34:56.1868,1903-12-25 +1903-05-05 12:34:56.1869,1903-12-25 +1903-05-05 12:34:56.187,1903-12-25 +1903-05-05 12:34:56.1871,1903-12-25 +1903-05-05 12:34:56.1872,1903-12-25 +1903-05-05 12:34:56.1873,1903-12-25 +1903-05-05 12:34:56.1874,1903-12-25 +1903-05-05 12:34:56.1875,1903-12-25 +1903-05-05 12:34:56.1876,1903-12-25 +1903-05-05 12:34:56.1877,1903-12-25 +1903-05-05 12:34:56.1878,1903-12-25 +1903-05-05 12:34:56.1879,1903-12-25 +1903-05-05 12:34:56.188,1903-12-25 +1903-05-05 12:34:56.1881,1903-12-25 +1903-05-05 12:34:56.1882,1903-12-25 +1903-05-05 12:34:56.1883,1903-12-25 +1903-05-05 12:34:56.1884,1903-12-25 +1903-05-05 12:34:56.1885,1903-12-25 +1903-05-05 12:34:56.1886,1903-12-25 +1903-05-05 12:34:56.1887,1903-12-25 +1903-05-05 12:34:56.1888,1903-12-25 +1903-05-05 12:34:56.1889,1903-12-25 +1903-05-05 12:34:56.189,1903-12-25 +1903-05-05 12:34:56.1891,1903-12-25 +1903-05-05 12:34:56.1892,1903-12-25 +1903-05-05 12:34:56.1893,1903-12-25 +1903-05-05 12:34:56.1894,1903-12-25 +1903-05-05 12:34:56.1895,1903-12-25 +1903-05-05 12:34:56.1896,1903-12-25 +1903-05-05 12:34:56.1897,1903-12-25 +1903-05-05 12:34:56.1898,1903-12-25 +1903-05-05 12:34:56.1899,1903-12-25 +1903-05-05 12:34:56.19,1903-12-25 +1903-05-05 12:34:56.1901,1903-12-25 +1903-05-05 12:34:56.1902,1903-12-25 +1903-05-05 12:34:56.1903,1903-12-25 +1903-05-05 12:34:56.1904,1903-12-25 +1903-05-05 12:34:56.1905,1903-12-25 +1903-05-05 12:34:56.1906,1903-12-25 +1903-05-05 12:34:56.1907,1903-12-25 +1903-05-05 12:34:56.1908,1903-12-25 +1903-05-05 12:34:56.1909,1903-12-25 +1903-05-05 12:34:56.191,1903-12-25 +1903-05-05 12:34:56.1911,1903-12-25 +1903-05-05 12:34:56.1912,1903-12-25 +1903-05-05 12:34:56.1913,1903-12-25 +1903-05-05 12:34:56.1914,1903-12-25 +1903-05-05 12:34:56.1915,1903-12-25 +1903-05-05 12:34:56.1916,1903-12-25 +1903-05-05 12:34:56.1917,1903-12-25 +1903-05-05 12:34:56.1918,1903-12-25 +1903-05-05 12:34:56.1919,1903-12-25 +1903-05-05 12:34:56.192,1903-12-25 +1903-05-05 12:34:56.1921,1903-12-25 +1903-05-05 12:34:56.1922,1903-12-25 +1903-05-05 12:34:56.1923,1903-12-25 +1903-05-05 12:34:56.1924,1903-12-25 +1903-05-05 12:34:56.1925,1903-12-25 +1903-05-05 12:34:56.1926,1903-12-25 +1903-05-05 12:34:56.1927,1903-12-25 +1903-05-05 12:34:56.1928,1903-12-25 +1903-05-05 12:34:56.1929,1903-12-25 +1903-05-05 12:34:56.193,1903-12-25 +1903-05-05 12:34:56.1931,1903-12-25 +1903-05-05 12:34:56.1932,1903-12-25 +1903-05-05 12:34:56.1933,1903-12-25 +1903-05-05 12:34:56.1934,1903-12-25 +1903-05-05 12:34:56.1935,1903-12-25 +1903-05-05 12:34:56.1936,1903-12-25 +1903-05-05 12:34:56.1937,1903-12-25 +1903-05-05 12:34:56.1938,1903-12-25 +1903-05-05 12:34:56.1939,1903-12-25 +1903-05-05 12:34:56.194,1903-12-25 +1903-05-05 12:34:56.1941,1903-12-25 +1903-05-05 12:34:56.1942,1903-12-25 +1903-05-05 12:34:56.1943,1903-12-25 +1903-05-05 12:34:56.1944,1903-12-25 +1903-05-05 12:34:56.1945,1903-12-25 +1903-05-05 12:34:56.1946,1903-12-25 +1903-05-05 12:34:56.1947,1903-12-25 +1903-05-05 12:34:56.1948,1903-12-25 +1903-05-05 12:34:56.1949,1903-12-25 +1903-05-05 12:34:56.195,1903-12-25 +1903-05-05 12:34:56.1951,1903-12-25 +1903-05-05 12:34:56.1952,1903-12-25 +1903-05-05 12:34:56.1953,1903-12-25 +1903-05-05 12:34:56.1954,1903-12-25 +1903-05-05 12:34:56.1955,1903-12-25 +1903-05-05 12:34:56.1956,1903-12-25 +1903-05-05 12:34:56.1957,1903-12-25 +1903-05-05 12:34:56.1958,1903-12-25 +1903-05-05 12:34:56.1959,1903-12-25 +1903-05-05 12:34:56.196,1903-12-25 +1903-05-05 12:34:56.1961,1903-12-25 +1903-05-05 12:34:56.1962,1903-12-25 +1903-05-05 12:34:56.1963,1903-12-25 +1903-05-05 12:34:56.1964,1903-12-25 +1903-05-05 12:34:56.1965,1903-12-25 +1903-05-05 12:34:56.1966,1903-12-25 +1903-05-05 12:34:56.1967,1903-12-25 +1903-05-05 12:34:56.1968,1903-12-25 +1903-05-05 12:34:56.1969,1903-12-25 +1903-05-05 12:34:56.197,1903-12-25 +1903-05-05 12:34:56.1971,1903-12-25 +1903-05-05 12:34:56.1972,1903-12-25 +1903-05-05 12:34:56.1973,1903-12-25 +1903-05-05 12:34:56.1974,1903-12-25 +1903-05-05 12:34:56.1975,1903-12-25 +1903-05-05 12:34:56.1976,1903-12-25 +1903-05-05 12:34:56.1977,1903-12-25 +1903-05-05 12:34:56.1978,1903-12-25 +1903-05-05 12:34:56.1979,1903-12-25 +1903-05-05 12:34:56.198,1903-12-25 +1903-05-05 12:34:56.1981,1903-12-25 +1903-05-05 12:34:56.1982,1903-12-25 +1903-05-05 12:34:56.1983,1903-12-25 +1903-05-05 12:34:56.1984,1903-12-25 +1903-05-05 12:34:56.1985,1903-12-25 +1903-05-05 12:34:56.1986,1903-12-25 +1903-05-05 12:34:56.1987,1903-12-25 +1903-05-05 12:34:56.1988,1903-12-25 +1903-05-05 12:34:56.1989,1903-12-25 +1903-05-05 12:34:56.199,1903-12-25 +1903-05-05 12:34:56.1991,1903-12-25 +1903-05-05 12:34:56.1992,1903-12-25 +1903-05-05 12:34:56.1993,1903-12-25 +1903-05-05 12:34:56.1994,1903-12-25 +1903-05-05 12:34:56.1995,1903-12-25 +1903-05-05 12:34:56.1996,1903-12-25 +1903-05-05 12:34:56.1997,1903-12-25 +1903-05-05 12:34:56.1998,1903-12-25 +1903-05-05 12:34:56.1999,1903-12-25 +1904-05-05 12:34:56.1,1904-12-25 +1904-05-05 12:34:56.1001,1904-12-25 +1904-05-05 12:34:56.1002,1904-12-25 +1904-05-05 12:34:56.1003,1904-12-25 +1904-05-05 12:34:56.1004,1904-12-25 +1904-05-05 12:34:56.1005,1904-12-25 +1904-05-05 12:34:56.1006,1904-12-25 +1904-05-05 12:34:56.1007,1904-12-25 +1904-05-05 12:34:56.1008,1904-12-25 +1904-05-05 12:34:56.1009,1904-12-25 +1904-05-05 12:34:56.101,1904-12-25 +1904-05-05 12:34:56.1011,1904-12-25 +1904-05-05 12:34:56.1012,1904-12-25 +1904-05-05 12:34:56.1013,1904-12-25 +1904-05-05 12:34:56.1014,1904-12-25 +1904-05-05 12:34:56.1015,1904-12-25 +1904-05-05 12:34:56.1016,1904-12-25 +1904-05-05 12:34:56.1017,1904-12-25 +1904-05-05 12:34:56.1018,1904-12-25 +1904-05-05 12:34:56.1019,1904-12-25 +1904-05-05 12:34:56.102,1904-12-25 +1904-05-05 12:34:56.1021,1904-12-25 +1904-05-05 12:34:56.1022,1904-12-25 +1904-05-05 12:34:56.1023,1904-12-25 +1904-05-05 12:34:56.1024,1904-12-25 +1904-05-05 12:34:56.1025,1904-12-25 +1904-05-05 12:34:56.1026,1904-12-25 +1904-05-05 12:34:56.1027,1904-12-25 +1904-05-05 12:34:56.1028,1904-12-25 +1904-05-05 12:34:56.1029,1904-12-25 +1904-05-05 12:34:56.103,1904-12-25 +1904-05-05 12:34:56.1031,1904-12-25 +1904-05-05 12:34:56.1032,1904-12-25 +1904-05-05 12:34:56.1033,1904-12-25 +1904-05-05 12:34:56.1034,1904-12-25 +1904-05-05 12:34:56.1035,1904-12-25 +1904-05-05 12:34:56.1036,1904-12-25 +1904-05-05 12:34:56.1037,1904-12-25 +1904-05-05 12:34:56.1038,1904-12-25 +1904-05-05 12:34:56.1039,1904-12-25 +1904-05-05 12:34:56.104,1904-12-25 +1904-05-05 12:34:56.1041,1904-12-25 +1904-05-05 12:34:56.1042,1904-12-25 +1904-05-05 12:34:56.1043,1904-12-25 +1904-05-05 12:34:56.1044,1904-12-25 +1904-05-05 12:34:56.1045,1904-12-25 +1904-05-05 12:34:56.1046,1904-12-25 +1904-05-05 12:34:56.1047,1904-12-25 +1904-05-05 12:34:56.1048,1904-12-25 +1904-05-05 12:34:56.1049,1904-12-25 +1904-05-05 12:34:56.105,1904-12-25 +1904-05-05 12:34:56.1051,1904-12-25 +1904-05-05 12:34:56.1052,1904-12-25 +1904-05-05 12:34:56.1053,1904-12-25 +1904-05-05 12:34:56.1054,1904-12-25 +1904-05-05 12:34:56.1055,1904-12-25 +1904-05-05 12:34:56.1056,1904-12-25 +1904-05-05 12:34:56.1057,1904-12-25 +1904-05-05 12:34:56.1058,1904-12-25 +1904-05-05 12:34:56.1059,1904-12-25 +1904-05-05 12:34:56.106,1904-12-25 +1904-05-05 12:34:56.1061,1904-12-25 +1904-05-05 12:34:56.1062,1904-12-25 +1904-05-05 12:34:56.1063,1904-12-25 +1904-05-05 12:34:56.1064,1904-12-25 +1904-05-05 12:34:56.1065,1904-12-25 +1904-05-05 12:34:56.1066,1904-12-25 +1904-05-05 12:34:56.1067,1904-12-25 +1904-05-05 12:34:56.1068,1904-12-25 +1904-05-05 12:34:56.1069,1904-12-25 +1904-05-05 12:34:56.107,1904-12-25 +1904-05-05 12:34:56.1071,1904-12-25 +1904-05-05 12:34:56.1072,1904-12-25 +1904-05-05 12:34:56.1073,1904-12-25 +1904-05-05 12:34:56.1074,1904-12-25 +1904-05-05 12:34:56.1075,1904-12-25 +1904-05-05 12:34:56.1076,1904-12-25 +1904-05-05 12:34:56.1077,1904-12-25 +1904-05-05 12:34:56.1078,1904-12-25 +1904-05-05 12:34:56.1079,1904-12-25 +1904-05-05 12:34:56.108,1904-12-25 +1904-05-05 12:34:56.1081,1904-12-25 +1904-05-05 12:34:56.1082,1904-12-25 +1904-05-05 12:34:56.1083,1904-12-25 +1904-05-05 12:34:56.1084,1904-12-25 +1904-05-05 12:34:56.1085,1904-12-25 +1904-05-05 12:34:56.1086,1904-12-25 +1904-05-05 12:34:56.1087,1904-12-25 +1904-05-05 12:34:56.1088,1904-12-25 +1904-05-05 12:34:56.1089,1904-12-25 +1904-05-05 12:34:56.109,1904-12-25 +1904-05-05 12:34:56.1091,1904-12-25 +1904-05-05 12:34:56.1092,1904-12-25 +1904-05-05 12:34:56.1093,1904-12-25 +1904-05-05 12:34:56.1094,1904-12-25 +1904-05-05 12:34:56.1095,1904-12-25 +1904-05-05 12:34:56.1096,1904-12-25 +1904-05-05 12:34:56.1097,1904-12-25 +1904-05-05 12:34:56.1098,1904-12-25 +1904-05-05 12:34:56.1099,1904-12-25 +1904-05-05 12:34:56.11,1904-12-25 +1904-05-05 12:34:56.1101,1904-12-25 +1904-05-05 12:34:56.1102,1904-12-25 +1904-05-05 12:34:56.1103,1904-12-25 +1904-05-05 12:34:56.1104,1904-12-25 +1904-05-05 12:34:56.1105,1904-12-25 +1904-05-05 12:34:56.1106,1904-12-25 +1904-05-05 12:34:56.1107,1904-12-25 +1904-05-05 12:34:56.1108,1904-12-25 +1904-05-05 12:34:56.1109,1904-12-25 +1904-05-05 12:34:56.111,1904-12-25 +1904-05-05 12:34:56.1111,1904-12-25 +1904-05-05 12:34:56.1112,1904-12-25 +1904-05-05 12:34:56.1113,1904-12-25 +1904-05-05 12:34:56.1114,1904-12-25 +1904-05-05 12:34:56.1115,1904-12-25 +1904-05-05 12:34:56.1116,1904-12-25 +1904-05-05 12:34:56.1117,1904-12-25 +1904-05-05 12:34:56.1118,1904-12-25 +1904-05-05 12:34:56.1119,1904-12-25 +1904-05-05 12:34:56.112,1904-12-25 +1904-05-05 12:34:56.1121,1904-12-25 +1904-05-05 12:34:56.1122,1904-12-25 +1904-05-05 12:34:56.1123,1904-12-25 +1904-05-05 12:34:56.1124,1904-12-25 +1904-05-05 12:34:56.1125,1904-12-25 +1904-05-05 12:34:56.1126,1904-12-25 +1904-05-05 12:34:56.1127,1904-12-25 +1904-05-05 12:34:56.1128,1904-12-25 +1904-05-05 12:34:56.1129,1904-12-25 +1904-05-05 12:34:56.113,1904-12-25 +1904-05-05 12:34:56.1131,1904-12-25 +1904-05-05 12:34:56.1132,1904-12-25 +1904-05-05 12:34:56.1133,1904-12-25 +1904-05-05 12:34:56.1134,1904-12-25 +1904-05-05 12:34:56.1135,1904-12-25 +1904-05-05 12:34:56.1136,1904-12-25 +1904-05-05 12:34:56.1137,1904-12-25 +1904-05-05 12:34:56.1138,1904-12-25 +1904-05-05 12:34:56.1139,1904-12-25 +1904-05-05 12:34:56.114,1904-12-25 +1904-05-05 12:34:56.1141,1904-12-25 +1904-05-05 12:34:56.1142,1904-12-25 +1904-05-05 12:34:56.1143,1904-12-25 +1904-05-05 12:34:56.1144,1904-12-25 +1904-05-05 12:34:56.1145,1904-12-25 +1904-05-05 12:34:56.1146,1904-12-25 +1904-05-05 12:34:56.1147,1904-12-25 +1904-05-05 12:34:56.1148,1904-12-25 +1904-05-05 12:34:56.1149,1904-12-25 +1904-05-05 12:34:56.115,1904-12-25 +1904-05-05 12:34:56.1151,1904-12-25 +1904-05-05 12:34:56.1152,1904-12-25 +1904-05-05 12:34:56.1153,1904-12-25 +1904-05-05 12:34:56.1154,1904-12-25 +1904-05-05 12:34:56.1155,1904-12-25 +1904-05-05 12:34:56.1156,1904-12-25 +1904-05-05 12:34:56.1157,1904-12-25 +1904-05-05 12:34:56.1158,1904-12-25 +1904-05-05 12:34:56.1159,1904-12-25 +1904-05-05 12:34:56.116,1904-12-25 +1904-05-05 12:34:56.1161,1904-12-25 +1904-05-05 12:34:56.1162,1904-12-25 +1904-05-05 12:34:56.1163,1904-12-25 +1904-05-05 12:34:56.1164,1904-12-25 +1904-05-05 12:34:56.1165,1904-12-25 +1904-05-05 12:34:56.1166,1904-12-25 +1904-05-05 12:34:56.1167,1904-12-25 +1904-05-05 12:34:56.1168,1904-12-25 +1904-05-05 12:34:56.1169,1904-12-25 +1904-05-05 12:34:56.117,1904-12-25 +1904-05-05 12:34:56.1171,1904-12-25 +1904-05-05 12:34:56.1172,1904-12-25 +1904-05-05 12:34:56.1173,1904-12-25 +1904-05-05 12:34:56.1174,1904-12-25 +1904-05-05 12:34:56.1175,1904-12-25 +1904-05-05 12:34:56.1176,1904-12-25 +1904-05-05 12:34:56.1177,1904-12-25 +1904-05-05 12:34:56.1178,1904-12-25 +1904-05-05 12:34:56.1179,1904-12-25 +1904-05-05 12:34:56.118,1904-12-25 +1904-05-05 12:34:56.1181,1904-12-25 +1904-05-05 12:34:56.1182,1904-12-25 +1904-05-05 12:34:56.1183,1904-12-25 +1904-05-05 12:34:56.1184,1904-12-25 +1904-05-05 12:34:56.1185,1904-12-25 +1904-05-05 12:34:56.1186,1904-12-25 +1904-05-05 12:34:56.1187,1904-12-25 +1904-05-05 12:34:56.1188,1904-12-25 +1904-05-05 12:34:56.1189,1904-12-25 +1904-05-05 12:34:56.119,1904-12-25 +1904-05-05 12:34:56.1191,1904-12-25 +1904-05-05 12:34:56.1192,1904-12-25 +1904-05-05 12:34:56.1193,1904-12-25 +1904-05-05 12:34:56.1194,1904-12-25 +1904-05-05 12:34:56.1195,1904-12-25 +1904-05-05 12:34:56.1196,1904-12-25 +1904-05-05 12:34:56.1197,1904-12-25 +1904-05-05 12:34:56.1198,1904-12-25 +1904-05-05 12:34:56.1199,1904-12-25 +1904-05-05 12:34:56.12,1904-12-25 +1904-05-05 12:34:56.1201,1904-12-25 +1904-05-05 12:34:56.1202,1904-12-25 +1904-05-05 12:34:56.1203,1904-12-25 +1904-05-05 12:34:56.1204,1904-12-25 +1904-05-05 12:34:56.1205,1904-12-25 +1904-05-05 12:34:56.1206,1904-12-25 +1904-05-05 12:34:56.1207,1904-12-25 +1904-05-05 12:34:56.1208,1904-12-25 +1904-05-05 12:34:56.1209,1904-12-25 +1904-05-05 12:34:56.121,1904-12-25 +1904-05-05 12:34:56.1211,1904-12-25 +1904-05-05 12:34:56.1212,1904-12-25 +1904-05-05 12:34:56.1213,1904-12-25 +1904-05-05 12:34:56.1214,1904-12-25 +1904-05-05 12:34:56.1215,1904-12-25 +1904-05-05 12:34:56.1216,1904-12-25 +1904-05-05 12:34:56.1217,1904-12-25 +1904-05-05 12:34:56.1218,1904-12-25 +1904-05-05 12:34:56.1219,1904-12-25 +1904-05-05 12:34:56.122,1904-12-25 +1904-05-05 12:34:56.1221,1904-12-25 +1904-05-05 12:34:56.1222,1904-12-25 +1904-05-05 12:34:56.1223,1904-12-25 +1904-05-05 12:34:56.1224,1904-12-25 +1904-05-05 12:34:56.1225,1904-12-25 +1904-05-05 12:34:56.1226,1904-12-25 +1904-05-05 12:34:56.1227,1904-12-25 +1904-05-05 12:34:56.1228,1904-12-25 +1904-05-05 12:34:56.1229,1904-12-25 +1904-05-05 12:34:56.123,1904-12-25 +1904-05-05 12:34:56.1231,1904-12-25 +1904-05-05 12:34:56.1232,1904-12-25 +1904-05-05 12:34:56.1233,1904-12-25 +1904-05-05 12:34:56.1234,1904-12-25 +1904-05-05 12:34:56.1235,1904-12-25 +1904-05-05 12:34:56.1236,1904-12-25 +1904-05-05 12:34:56.1237,1904-12-25 +1904-05-05 12:34:56.1238,1904-12-25 +1904-05-05 12:34:56.1239,1904-12-25 +1904-05-05 12:34:56.124,1904-12-25 +1904-05-05 12:34:56.1241,1904-12-25 +1904-05-05 12:34:56.1242,1904-12-25 +1904-05-05 12:34:56.1243,1904-12-25 +1904-05-05 12:34:56.1244,1904-12-25 +1904-05-05 12:34:56.1245,1904-12-25 +1904-05-05 12:34:56.1246,1904-12-25 +1904-05-05 12:34:56.1247,1904-12-25 +1904-05-05 12:34:56.1248,1904-12-25 +1904-05-05 12:34:56.1249,1904-12-25 +1904-05-05 12:34:56.125,1904-12-25 +1904-05-05 12:34:56.1251,1904-12-25 +1904-05-05 12:34:56.1252,1904-12-25 +1904-05-05 12:34:56.1253,1904-12-25 +1904-05-05 12:34:56.1254,1904-12-25 +1904-05-05 12:34:56.1255,1904-12-25 +1904-05-05 12:34:56.1256,1904-12-25 +1904-05-05 12:34:56.1257,1904-12-25 +1904-05-05 12:34:56.1258,1904-12-25 +1904-05-05 12:34:56.1259,1904-12-25 +1904-05-05 12:34:56.126,1904-12-25 +1904-05-05 12:34:56.1261,1904-12-25 +1904-05-05 12:34:56.1262,1904-12-25 +1904-05-05 12:34:56.1263,1904-12-25 +1904-05-05 12:34:56.1264,1904-12-25 +1904-05-05 12:34:56.1265,1904-12-25 +1904-05-05 12:34:56.1266,1904-12-25 +1904-05-05 12:34:56.1267,1904-12-25 +1904-05-05 12:34:56.1268,1904-12-25 +1904-05-05 12:34:56.1269,1904-12-25 +1904-05-05 12:34:56.127,1904-12-25 +1904-05-05 12:34:56.1271,1904-12-25 +1904-05-05 12:34:56.1272,1904-12-25 +1904-05-05 12:34:56.1273,1904-12-25 +1904-05-05 12:34:56.1274,1904-12-25 +1904-05-05 12:34:56.1275,1904-12-25 +1904-05-05 12:34:56.1276,1904-12-25 +1904-05-05 12:34:56.1277,1904-12-25 +1904-05-05 12:34:56.1278,1904-12-25 +1904-05-05 12:34:56.1279,1904-12-25 +1904-05-05 12:34:56.128,1904-12-25 +1904-05-05 12:34:56.1281,1904-12-25 +1904-05-05 12:34:56.1282,1904-12-25 +1904-05-05 12:34:56.1283,1904-12-25 +1904-05-05 12:34:56.1284,1904-12-25 +1904-05-05 12:34:56.1285,1904-12-25 +1904-05-05 12:34:56.1286,1904-12-25 +1904-05-05 12:34:56.1287,1904-12-25 +1904-05-05 12:34:56.1288,1904-12-25 +1904-05-05 12:34:56.1289,1904-12-25 +1904-05-05 12:34:56.129,1904-12-25 +1904-05-05 12:34:56.1291,1904-12-25 +1904-05-05 12:34:56.1292,1904-12-25 +1904-05-05 12:34:56.1293,1904-12-25 +1904-05-05 12:34:56.1294,1904-12-25 +1904-05-05 12:34:56.1295,1904-12-25 +1904-05-05 12:34:56.1296,1904-12-25 +1904-05-05 12:34:56.1297,1904-12-25 +1904-05-05 12:34:56.1298,1904-12-25 +1904-05-05 12:34:56.1299,1904-12-25 +1904-05-05 12:34:56.13,1904-12-25 +1904-05-05 12:34:56.1301,1904-12-25 +1904-05-05 12:34:56.1302,1904-12-25 +1904-05-05 12:34:56.1303,1904-12-25 +1904-05-05 12:34:56.1304,1904-12-25 +1904-05-05 12:34:56.1305,1904-12-25 +1904-05-05 12:34:56.1306,1904-12-25 +1904-05-05 12:34:56.1307,1904-12-25 +1904-05-05 12:34:56.1308,1904-12-25 +1904-05-05 12:34:56.1309,1904-12-25 +1904-05-05 12:34:56.131,1904-12-25 +1904-05-05 12:34:56.1311,1904-12-25 +1904-05-05 12:34:56.1312,1904-12-25 +1904-05-05 12:34:56.1313,1904-12-25 +1904-05-05 12:34:56.1314,1904-12-25 +1904-05-05 12:34:56.1315,1904-12-25 +1904-05-05 12:34:56.1316,1904-12-25 +1904-05-05 12:34:56.1317,1904-12-25 +1904-05-05 12:34:56.1318,1904-12-25 +1904-05-05 12:34:56.1319,1904-12-25 +1904-05-05 12:34:56.132,1904-12-25 +1904-05-05 12:34:56.1321,1904-12-25 +1904-05-05 12:34:56.1322,1904-12-25 +1904-05-05 12:34:56.1323,1904-12-25 +1904-05-05 12:34:56.1324,1904-12-25 +1904-05-05 12:34:56.1325,1904-12-25 +1904-05-05 12:34:56.1326,1904-12-25 +1904-05-05 12:34:56.1327,1904-12-25 +1904-05-05 12:34:56.1328,1904-12-25 +1904-05-05 12:34:56.1329,1904-12-25 +1904-05-05 12:34:56.133,1904-12-25 +1904-05-05 12:34:56.1331,1904-12-25 +1904-05-05 12:34:56.1332,1904-12-25 +1904-05-05 12:34:56.1333,1904-12-25 +1904-05-05 12:34:56.1334,1904-12-25 +1904-05-05 12:34:56.1335,1904-12-25 +1904-05-05 12:34:56.1336,1904-12-25 +1904-05-05 12:34:56.1337,1904-12-25 +1904-05-05 12:34:56.1338,1904-12-25 +1904-05-05 12:34:56.1339,1904-12-25 +1904-05-05 12:34:56.134,1904-12-25 +1904-05-05 12:34:56.1341,1904-12-25 +1904-05-05 12:34:56.1342,1904-12-25 +1904-05-05 12:34:56.1343,1904-12-25 +1904-05-05 12:34:56.1344,1904-12-25 +1904-05-05 12:34:56.1345,1904-12-25 +1904-05-05 12:34:56.1346,1904-12-25 +1904-05-05 12:34:56.1347,1904-12-25 +1904-05-05 12:34:56.1348,1904-12-25 +1904-05-05 12:34:56.1349,1904-12-25 +1904-05-05 12:34:56.135,1904-12-25 +1904-05-05 12:34:56.1351,1904-12-25 +1904-05-05 12:34:56.1352,1904-12-25 +1904-05-05 12:34:56.1353,1904-12-25 +1904-05-05 12:34:56.1354,1904-12-25 +1904-05-05 12:34:56.1355,1904-12-25 +1904-05-05 12:34:56.1356,1904-12-25 +1904-05-05 12:34:56.1357,1904-12-25 +1904-05-05 12:34:56.1358,1904-12-25 +1904-05-05 12:34:56.1359,1904-12-25 +1904-05-05 12:34:56.136,1904-12-25 +1904-05-05 12:34:56.1361,1904-12-25 +1904-05-05 12:34:56.1362,1904-12-25 +1904-05-05 12:34:56.1363,1904-12-25 +1904-05-05 12:34:56.1364,1904-12-25 +1904-05-05 12:34:56.1365,1904-12-25 +1904-05-05 12:34:56.1366,1904-12-25 +1904-05-05 12:34:56.1367,1904-12-25 +1904-05-05 12:34:56.1368,1904-12-25 +1904-05-05 12:34:56.1369,1904-12-25 +1904-05-05 12:34:56.137,1904-12-25 +1904-05-05 12:34:56.1371,1904-12-25 +1904-05-05 12:34:56.1372,1904-12-25 +1904-05-05 12:34:56.1373,1904-12-25 +1904-05-05 12:34:56.1374,1904-12-25 +1904-05-05 12:34:56.1375,1904-12-25 +1904-05-05 12:34:56.1376,1904-12-25 +1904-05-05 12:34:56.1377,1904-12-25 +1904-05-05 12:34:56.1378,1904-12-25 +1904-05-05 12:34:56.1379,1904-12-25 +1904-05-05 12:34:56.138,1904-12-25 +1904-05-05 12:34:56.1381,1904-12-25 +1904-05-05 12:34:56.1382,1904-12-25 +1904-05-05 12:34:56.1383,1904-12-25 +1904-05-05 12:34:56.1384,1904-12-25 +1904-05-05 12:34:56.1385,1904-12-25 +1904-05-05 12:34:56.1386,1904-12-25 +1904-05-05 12:34:56.1387,1904-12-25 +1904-05-05 12:34:56.1388,1904-12-25 +1904-05-05 12:34:56.1389,1904-12-25 +1904-05-05 12:34:56.139,1904-12-25 +1904-05-05 12:34:56.1391,1904-12-25 +1904-05-05 12:34:56.1392,1904-12-25 +1904-05-05 12:34:56.1393,1904-12-25 +1904-05-05 12:34:56.1394,1904-12-25 +1904-05-05 12:34:56.1395,1904-12-25 +1904-05-05 12:34:56.1396,1904-12-25 +1904-05-05 12:34:56.1397,1904-12-25 +1904-05-05 12:34:56.1398,1904-12-25 +1904-05-05 12:34:56.1399,1904-12-25 +1904-05-05 12:34:56.14,1904-12-25 +1904-05-05 12:34:56.1401,1904-12-25 +1904-05-05 12:34:56.1402,1904-12-25 +1904-05-05 12:34:56.1403,1904-12-25 +1904-05-05 12:34:56.1404,1904-12-25 +1904-05-05 12:34:56.1405,1904-12-25 +1904-05-05 12:34:56.1406,1904-12-25 +1904-05-05 12:34:56.1407,1904-12-25 +1904-05-05 12:34:56.1408,1904-12-25 +1904-05-05 12:34:56.1409,1904-12-25 +1904-05-05 12:34:56.141,1904-12-25 +1904-05-05 12:34:56.1411,1904-12-25 +1904-05-05 12:34:56.1412,1904-12-25 +1904-05-05 12:34:56.1413,1904-12-25 +1904-05-05 12:34:56.1414,1904-12-25 +1904-05-05 12:34:56.1415,1904-12-25 +1904-05-05 12:34:56.1416,1904-12-25 +1904-05-05 12:34:56.1417,1904-12-25 +1904-05-05 12:34:56.1418,1904-12-25 +1904-05-05 12:34:56.1419,1904-12-25 +1904-05-05 12:34:56.142,1904-12-25 +1904-05-05 12:34:56.1421,1904-12-25 +1904-05-05 12:34:56.1422,1904-12-25 +1904-05-05 12:34:56.1423,1904-12-25 +1904-05-05 12:34:56.1424,1904-12-25 +1904-05-05 12:34:56.1425,1904-12-25 +1904-05-05 12:34:56.1426,1904-12-25 +1904-05-05 12:34:56.1427,1904-12-25 +1904-05-05 12:34:56.1428,1904-12-25 +1904-05-05 12:34:56.1429,1904-12-25 +1904-05-05 12:34:56.143,1904-12-25 +1904-05-05 12:34:56.1431,1904-12-25 +1904-05-05 12:34:56.1432,1904-12-25 +1904-05-05 12:34:56.1433,1904-12-25 +1904-05-05 12:34:56.1434,1904-12-25 +1904-05-05 12:34:56.1435,1904-12-25 +1904-05-05 12:34:56.1436,1904-12-25 +1904-05-05 12:34:56.1437,1904-12-25 +1904-05-05 12:34:56.1438,1904-12-25 +1904-05-05 12:34:56.1439,1904-12-25 +1904-05-05 12:34:56.144,1904-12-25 +1904-05-05 12:34:56.1441,1904-12-25 +1904-05-05 12:34:56.1442,1904-12-25 +1904-05-05 12:34:56.1443,1904-12-25 +1904-05-05 12:34:56.1444,1904-12-25 +1904-05-05 12:34:56.1445,1904-12-25 +1904-05-05 12:34:56.1446,1904-12-25 +1904-05-05 12:34:56.1447,1904-12-25 +1904-05-05 12:34:56.1448,1904-12-25 +1904-05-05 12:34:56.1449,1904-12-25 +1904-05-05 12:34:56.145,1904-12-25 +1904-05-05 12:34:56.1451,1904-12-25 +1904-05-05 12:34:56.1452,1904-12-25 +1904-05-05 12:34:56.1453,1904-12-25 +1904-05-05 12:34:56.1454,1904-12-25 +1904-05-05 12:34:56.1455,1904-12-25 +1904-05-05 12:34:56.1456,1904-12-25 +1904-05-05 12:34:56.1457,1904-12-25 +1904-05-05 12:34:56.1458,1904-12-25 +1904-05-05 12:34:56.1459,1904-12-25 +1904-05-05 12:34:56.146,1904-12-25 +1904-05-05 12:34:56.1461,1904-12-25 +1904-05-05 12:34:56.1462,1904-12-25 +1904-05-05 12:34:56.1463,1904-12-25 +1904-05-05 12:34:56.1464,1904-12-25 +1904-05-05 12:34:56.1465,1904-12-25 +1904-05-05 12:34:56.1466,1904-12-25 +1904-05-05 12:34:56.1467,1904-12-25 +1904-05-05 12:34:56.1468,1904-12-25 +1904-05-05 12:34:56.1469,1904-12-25 +1904-05-05 12:34:56.147,1904-12-25 +1904-05-05 12:34:56.1471,1904-12-25 +1904-05-05 12:34:56.1472,1904-12-25 +1904-05-05 12:34:56.1473,1904-12-25 +1904-05-05 12:34:56.1474,1904-12-25 +1904-05-05 12:34:56.1475,1904-12-25 +1904-05-05 12:34:56.1476,1904-12-25 +1904-05-05 12:34:56.1477,1904-12-25 +1904-05-05 12:34:56.1478,1904-12-25 +1904-05-05 12:34:56.1479,1904-12-25 +1904-05-05 12:34:56.148,1904-12-25 +1904-05-05 12:34:56.1481,1904-12-25 +1904-05-05 12:34:56.1482,1904-12-25 +1904-05-05 12:34:56.1483,1904-12-25 +1904-05-05 12:34:56.1484,1904-12-25 +1904-05-05 12:34:56.1485,1904-12-25 +1904-05-05 12:34:56.1486,1904-12-25 +1904-05-05 12:34:56.1487,1904-12-25 +1904-05-05 12:34:56.1488,1904-12-25 +1904-05-05 12:34:56.1489,1904-12-25 +1904-05-05 12:34:56.149,1904-12-25 +1904-05-05 12:34:56.1491,1904-12-25 +1904-05-05 12:34:56.1492,1904-12-25 +1904-05-05 12:34:56.1493,1904-12-25 +1904-05-05 12:34:56.1494,1904-12-25 +1904-05-05 12:34:56.1495,1904-12-25 +1904-05-05 12:34:56.1496,1904-12-25 +1904-05-05 12:34:56.1497,1904-12-25 +1904-05-05 12:34:56.1498,1904-12-25 +1904-05-05 12:34:56.1499,1904-12-25 +1904-05-05 12:34:56.15,1904-12-25 +1904-05-05 12:34:56.1501,1904-12-25 +1904-05-05 12:34:56.1502,1904-12-25 +1904-05-05 12:34:56.1503,1904-12-25 +1904-05-05 12:34:56.1504,1904-12-25 +1904-05-05 12:34:56.1505,1904-12-25 +1904-05-05 12:34:56.1506,1904-12-25 +1904-05-05 12:34:56.1507,1904-12-25 +1904-05-05 12:34:56.1508,1904-12-25 +1904-05-05 12:34:56.1509,1904-12-25 +1904-05-05 12:34:56.151,1904-12-25 +1904-05-05 12:34:56.1511,1904-12-25 +1904-05-05 12:34:56.1512,1904-12-25 +1904-05-05 12:34:56.1513,1904-12-25 +1904-05-05 12:34:56.1514,1904-12-25 +1904-05-05 12:34:56.1515,1904-12-25 +1904-05-05 12:34:56.1516,1904-12-25 +1904-05-05 12:34:56.1517,1904-12-25 +1904-05-05 12:34:56.1518,1904-12-25 +1904-05-05 12:34:56.1519,1904-12-25 +1904-05-05 12:34:56.152,1904-12-25 +1904-05-05 12:34:56.1521,1904-12-25 +1904-05-05 12:34:56.1522,1904-12-25 +1904-05-05 12:34:56.1523,1904-12-25 +1904-05-05 12:34:56.1524,1904-12-25 +1904-05-05 12:34:56.1525,1904-12-25 +1904-05-05 12:34:56.1526,1904-12-25 +1904-05-05 12:34:56.1527,1904-12-25 +1904-05-05 12:34:56.1528,1904-12-25 +1904-05-05 12:34:56.1529,1904-12-25 +1904-05-05 12:34:56.153,1904-12-25 +1904-05-05 12:34:56.1531,1904-12-25 +1904-05-05 12:34:56.1532,1904-12-25 +1904-05-05 12:34:56.1533,1904-12-25 +1904-05-05 12:34:56.1534,1904-12-25 +1904-05-05 12:34:56.1535,1904-12-25 +1904-05-05 12:34:56.1536,1904-12-25 +1904-05-05 12:34:56.1537,1904-12-25 +1904-05-05 12:34:56.1538,1904-12-25 +1904-05-05 12:34:56.1539,1904-12-25 +1904-05-05 12:34:56.154,1904-12-25 +1904-05-05 12:34:56.1541,1904-12-25 +1904-05-05 12:34:56.1542,1904-12-25 +1904-05-05 12:34:56.1543,1904-12-25 +1904-05-05 12:34:56.1544,1904-12-25 +1904-05-05 12:34:56.1545,1904-12-25 +1904-05-05 12:34:56.1546,1904-12-25 +1904-05-05 12:34:56.1547,1904-12-25 +1904-05-05 12:34:56.1548,1904-12-25 +1904-05-05 12:34:56.1549,1904-12-25 +1904-05-05 12:34:56.155,1904-12-25 +1904-05-05 12:34:56.1551,1904-12-25 +1904-05-05 12:34:56.1552,1904-12-25 +1904-05-05 12:34:56.1553,1904-12-25 +1904-05-05 12:34:56.1554,1904-12-25 +1904-05-05 12:34:56.1555,1904-12-25 +1904-05-05 12:34:56.1556,1904-12-25 +1904-05-05 12:34:56.1557,1904-12-25 +1904-05-05 12:34:56.1558,1904-12-25 +1904-05-05 12:34:56.1559,1904-12-25 +1904-05-05 12:34:56.156,1904-12-25 +1904-05-05 12:34:56.1561,1904-12-25 +1904-05-05 12:34:56.1562,1904-12-25 +1904-05-05 12:34:56.1563,1904-12-25 +1904-05-05 12:34:56.1564,1904-12-25 +1904-05-05 12:34:56.1565,1904-12-25 +1904-05-05 12:34:56.1566,1904-12-25 +1904-05-05 12:34:56.1567,1904-12-25 +1904-05-05 12:34:56.1568,1904-12-25 +1904-05-05 12:34:56.1569,1904-12-25 +1904-05-05 12:34:56.157,1904-12-25 +1904-05-05 12:34:56.1571,1904-12-25 +1904-05-05 12:34:56.1572,1904-12-25 +1904-05-05 12:34:56.1573,1904-12-25 +1904-05-05 12:34:56.1574,1904-12-25 +1904-05-05 12:34:56.1575,1904-12-25 +1904-05-05 12:34:56.1576,1904-12-25 +1904-05-05 12:34:56.1577,1904-12-25 +1904-05-05 12:34:56.1578,1904-12-25 +1904-05-05 12:34:56.1579,1904-12-25 +1904-05-05 12:34:56.158,1904-12-25 +1904-05-05 12:34:56.1581,1904-12-25 +1904-05-05 12:34:56.1582,1904-12-25 +1904-05-05 12:34:56.1583,1904-12-25 +1904-05-05 12:34:56.1584,1904-12-25 +1904-05-05 12:34:56.1585,1904-12-25 +1904-05-05 12:34:56.1586,1904-12-25 +1904-05-05 12:34:56.1587,1904-12-25 +1904-05-05 12:34:56.1588,1904-12-25 +1904-05-05 12:34:56.1589,1904-12-25 +1904-05-05 12:34:56.159,1904-12-25 +1904-05-05 12:34:56.1591,1904-12-25 +1904-05-05 12:34:56.1592,1904-12-25 +1904-05-05 12:34:56.1593,1904-12-25 +1904-05-05 12:34:56.1594,1904-12-25 +1904-05-05 12:34:56.1595,1904-12-25 +1904-05-05 12:34:56.1596,1904-12-25 +1904-05-05 12:34:56.1597,1904-12-25 +1904-05-05 12:34:56.1598,1904-12-25 +1904-05-05 12:34:56.1599,1904-12-25 +1904-05-05 12:34:56.16,1904-12-25 +1904-05-05 12:34:56.1601,1904-12-25 +1904-05-05 12:34:56.1602,1904-12-25 +1904-05-05 12:34:56.1603,1904-12-25 +1904-05-05 12:34:56.1604,1904-12-25 +1904-05-05 12:34:56.1605,1904-12-25 +1904-05-05 12:34:56.1606,1904-12-25 +1904-05-05 12:34:56.1607,1904-12-25 +1904-05-05 12:34:56.1608,1904-12-25 +1904-05-05 12:34:56.1609,1904-12-25 +1904-05-05 12:34:56.161,1904-12-25 +1904-05-05 12:34:56.1611,1904-12-25 +1904-05-05 12:34:56.1612,1904-12-25 +1904-05-05 12:34:56.1613,1904-12-25 +1904-05-05 12:34:56.1614,1904-12-25 +1904-05-05 12:34:56.1615,1904-12-25 +1904-05-05 12:34:56.1616,1904-12-25 +1904-05-05 12:34:56.1617,1904-12-25 +1904-05-05 12:34:56.1618,1904-12-25 +1904-05-05 12:34:56.1619,1904-12-25 +1904-05-05 12:34:56.162,1904-12-25 +1904-05-05 12:34:56.1621,1904-12-25 +1904-05-05 12:34:56.1622,1904-12-25 +1904-05-05 12:34:56.1623,1904-12-25 +1904-05-05 12:34:56.1624,1904-12-25 +1904-05-05 12:34:56.1625,1904-12-25 +1904-05-05 12:34:56.1626,1904-12-25 +1904-05-05 12:34:56.1627,1904-12-25 +1904-05-05 12:34:56.1628,1904-12-25 +1904-05-05 12:34:56.1629,1904-12-25 +1904-05-05 12:34:56.163,1904-12-25 +1904-05-05 12:34:56.1631,1904-12-25 +1904-05-05 12:34:56.1632,1904-12-25 +1904-05-05 12:34:56.1633,1904-12-25 +1904-05-05 12:34:56.1634,1904-12-25 +1904-05-05 12:34:56.1635,1904-12-25 +1904-05-05 12:34:56.1636,1904-12-25 +1904-05-05 12:34:56.1637,1904-12-25 +1904-05-05 12:34:56.1638,1904-12-25 +1904-05-05 12:34:56.1639,1904-12-25 +1904-05-05 12:34:56.164,1904-12-25 +1904-05-05 12:34:56.1641,1904-12-25 +1904-05-05 12:34:56.1642,1904-12-25 +1904-05-05 12:34:56.1643,1904-12-25 +1904-05-05 12:34:56.1644,1904-12-25 +1904-05-05 12:34:56.1645,1904-12-25 +1904-05-05 12:34:56.1646,1904-12-25 +1904-05-05 12:34:56.1647,1904-12-25 +1904-05-05 12:34:56.1648,1904-12-25 +1904-05-05 12:34:56.1649,1904-12-25 +1904-05-05 12:34:56.165,1904-12-25 +1904-05-05 12:34:56.1651,1904-12-25 +1904-05-05 12:34:56.1652,1904-12-25 +1904-05-05 12:34:56.1653,1904-12-25 +1904-05-05 12:34:56.1654,1904-12-25 +1904-05-05 12:34:56.1655,1904-12-25 +1904-05-05 12:34:56.1656,1904-12-25 +1904-05-05 12:34:56.1657,1904-12-25 +1904-05-05 12:34:56.1658,1904-12-25 +1904-05-05 12:34:56.1659,1904-12-25 +1904-05-05 12:34:56.166,1904-12-25 +1904-05-05 12:34:56.1661,1904-12-25 +1904-05-05 12:34:56.1662,1904-12-25 +1904-05-05 12:34:56.1663,1904-12-25 +1904-05-05 12:34:56.1664,1904-12-25 +1904-05-05 12:34:56.1665,1904-12-25 +1904-05-05 12:34:56.1666,1904-12-25 +1904-05-05 12:34:56.1667,1904-12-25 +1904-05-05 12:34:56.1668,1904-12-25 +1904-05-05 12:34:56.1669,1904-12-25 +1904-05-05 12:34:56.167,1904-12-25 +1904-05-05 12:34:56.1671,1904-12-25 +1904-05-05 12:34:56.1672,1904-12-25 +1904-05-05 12:34:56.1673,1904-12-25 +1904-05-05 12:34:56.1674,1904-12-25 +1904-05-05 12:34:56.1675,1904-12-25 +1904-05-05 12:34:56.1676,1904-12-25 +1904-05-05 12:34:56.1677,1904-12-25 +1904-05-05 12:34:56.1678,1904-12-25 +1904-05-05 12:34:56.1679,1904-12-25 +1904-05-05 12:34:56.168,1904-12-25 +1904-05-05 12:34:56.1681,1904-12-25 +1904-05-05 12:34:56.1682,1904-12-25 +1904-05-05 12:34:56.1683,1904-12-25 +1904-05-05 12:34:56.1684,1904-12-25 +1904-05-05 12:34:56.1685,1904-12-25 +1904-05-05 12:34:56.1686,1904-12-25 +1904-05-05 12:34:56.1687,1904-12-25 +1904-05-05 12:34:56.1688,1904-12-25 +1904-05-05 12:34:56.1689,1904-12-25 +1904-05-05 12:34:56.169,1904-12-25 +1904-05-05 12:34:56.1691,1904-12-25 +1904-05-05 12:34:56.1692,1904-12-25 +1904-05-05 12:34:56.1693,1904-12-25 +1904-05-05 12:34:56.1694,1904-12-25 +1904-05-05 12:34:56.1695,1904-12-25 +1904-05-05 12:34:56.1696,1904-12-25 +1904-05-05 12:34:56.1697,1904-12-25 +1904-05-05 12:34:56.1698,1904-12-25 +1904-05-05 12:34:56.1699,1904-12-25 +1904-05-05 12:34:56.17,1904-12-25 +1904-05-05 12:34:56.1701,1904-12-25 +1904-05-05 12:34:56.1702,1904-12-25 +1904-05-05 12:34:56.1703,1904-12-25 +1904-05-05 12:34:56.1704,1904-12-25 +1904-05-05 12:34:56.1705,1904-12-25 +1904-05-05 12:34:56.1706,1904-12-25 +1904-05-05 12:34:56.1707,1904-12-25 +1904-05-05 12:34:56.1708,1904-12-25 +1904-05-05 12:34:56.1709,1904-12-25 +1904-05-05 12:34:56.171,1904-12-25 +1904-05-05 12:34:56.1711,1904-12-25 +1904-05-05 12:34:56.1712,1904-12-25 +1904-05-05 12:34:56.1713,1904-12-25 +1904-05-05 12:34:56.1714,1904-12-25 +1904-05-05 12:34:56.1715,1904-12-25 +1904-05-05 12:34:56.1716,1904-12-25 +1904-05-05 12:34:56.1717,1904-12-25 +1904-05-05 12:34:56.1718,1904-12-25 +1904-05-05 12:34:56.1719,1904-12-25 +1904-05-05 12:34:56.172,1904-12-25 +1904-05-05 12:34:56.1721,1904-12-25 +1904-05-05 12:34:56.1722,1904-12-25 +1904-05-05 12:34:56.1723,1904-12-25 +1904-05-05 12:34:56.1724,1904-12-25 +1904-05-05 12:34:56.1725,1904-12-25 +1904-05-05 12:34:56.1726,1904-12-25 +1904-05-05 12:34:56.1727,1904-12-25 +1904-05-05 12:34:56.1728,1904-12-25 +1904-05-05 12:34:56.1729,1904-12-25 +1904-05-05 12:34:56.173,1904-12-25 +1904-05-05 12:34:56.1731,1904-12-25 +1904-05-05 12:34:56.1732,1904-12-25 +1904-05-05 12:34:56.1733,1904-12-25 +1904-05-05 12:34:56.1734,1904-12-25 +1904-05-05 12:34:56.1735,1904-12-25 +1904-05-05 12:34:56.1736,1904-12-25 +1904-05-05 12:34:56.1737,1904-12-25 +1904-05-05 12:34:56.1738,1904-12-25 +1904-05-05 12:34:56.1739,1904-12-25 +1904-05-05 12:34:56.174,1904-12-25 +1904-05-05 12:34:56.1741,1904-12-25 +1904-05-05 12:34:56.1742,1904-12-25 +1904-05-05 12:34:56.1743,1904-12-25 +1904-05-05 12:34:56.1744,1904-12-25 +1904-05-05 12:34:56.1745,1904-12-25 +1904-05-05 12:34:56.1746,1904-12-25 +1904-05-05 12:34:56.1747,1904-12-25 +1904-05-05 12:34:56.1748,1904-12-25 +1904-05-05 12:34:56.1749,1904-12-25 +1904-05-05 12:34:56.175,1904-12-25 +1904-05-05 12:34:56.1751,1904-12-25 +1904-05-05 12:34:56.1752,1904-12-25 +1904-05-05 12:34:56.1753,1904-12-25 +1904-05-05 12:34:56.1754,1904-12-25 +1904-05-05 12:34:56.1755,1904-12-25 +1904-05-05 12:34:56.1756,1904-12-25 +1904-05-05 12:34:56.1757,1904-12-25 +1904-05-05 12:34:56.1758,1904-12-25 +1904-05-05 12:34:56.1759,1904-12-25 +1904-05-05 12:34:56.176,1904-12-25 +1904-05-05 12:34:56.1761,1904-12-25 +1904-05-05 12:34:56.1762,1904-12-25 +1904-05-05 12:34:56.1763,1904-12-25 +1904-05-05 12:34:56.1764,1904-12-25 +1904-05-05 12:34:56.1765,1904-12-25 +1904-05-05 12:34:56.1766,1904-12-25 +1904-05-05 12:34:56.1767,1904-12-25 +1904-05-05 12:34:56.1768,1904-12-25 +1904-05-05 12:34:56.1769,1904-12-25 +1904-05-05 12:34:56.177,1904-12-25 +1904-05-05 12:34:56.1771,1904-12-25 +1904-05-05 12:34:56.1772,1904-12-25 +1904-05-05 12:34:56.1773,1904-12-25 +1904-05-05 12:34:56.1774,1904-12-25 +1904-05-05 12:34:56.1775,1904-12-25 +1904-05-05 12:34:56.1776,1904-12-25 +1904-05-05 12:34:56.1777,1904-12-25 +1904-05-05 12:34:56.1778,1904-12-25 +1904-05-05 12:34:56.1779,1904-12-25 +1904-05-05 12:34:56.178,1904-12-25 +1904-05-05 12:34:56.1781,1904-12-25 +1904-05-05 12:34:56.1782,1904-12-25 +1904-05-05 12:34:56.1783,1904-12-25 +1904-05-05 12:34:56.1784,1904-12-25 +1904-05-05 12:34:56.1785,1904-12-25 +1904-05-05 12:34:56.1786,1904-12-25 +1904-05-05 12:34:56.1787,1904-12-25 +1904-05-05 12:34:56.1788,1904-12-25 +1904-05-05 12:34:56.1789,1904-12-25 +1904-05-05 12:34:56.179,1904-12-25 +1904-05-05 12:34:56.1791,1904-12-25 +1904-05-05 12:34:56.1792,1904-12-25 +1904-05-05 12:34:56.1793,1904-12-25 +1904-05-05 12:34:56.1794,1904-12-25 +1904-05-05 12:34:56.1795,1904-12-25 +1904-05-05 12:34:56.1796,1904-12-25 +1904-05-05 12:34:56.1797,1904-12-25 +1904-05-05 12:34:56.1798,1904-12-25 +1904-05-05 12:34:56.1799,1904-12-25 +1904-05-05 12:34:56.18,1904-12-25 +1904-05-05 12:34:56.1801,1904-12-25 +1904-05-05 12:34:56.1802,1904-12-25 +1904-05-05 12:34:56.1803,1904-12-25 +1904-05-05 12:34:56.1804,1904-12-25 +1904-05-05 12:34:56.1805,1904-12-25 +1904-05-05 12:34:56.1806,1904-12-25 +1904-05-05 12:34:56.1807,1904-12-25 +1904-05-05 12:34:56.1808,1904-12-25 +1904-05-05 12:34:56.1809,1904-12-25 +1904-05-05 12:34:56.181,1904-12-25 +1904-05-05 12:34:56.1811,1904-12-25 +1904-05-05 12:34:56.1812,1904-12-25 +1904-05-05 12:34:56.1813,1904-12-25 +1904-05-05 12:34:56.1814,1904-12-25 +1904-05-05 12:34:56.1815,1904-12-25 +1904-05-05 12:34:56.1816,1904-12-25 +1904-05-05 12:34:56.1817,1904-12-25 +1904-05-05 12:34:56.1818,1904-12-25 +1904-05-05 12:34:56.1819,1904-12-25 +1904-05-05 12:34:56.182,1904-12-25 +1904-05-05 12:34:56.1821,1904-12-25 +1904-05-05 12:34:56.1822,1904-12-25 +1904-05-05 12:34:56.1823,1904-12-25 +1904-05-05 12:34:56.1824,1904-12-25 +1904-05-05 12:34:56.1825,1904-12-25 +1904-05-05 12:34:56.1826,1904-12-25 +1904-05-05 12:34:56.1827,1904-12-25 +1904-05-05 12:34:56.1828,1904-12-25 +1904-05-05 12:34:56.1829,1904-12-25 +1904-05-05 12:34:56.183,1904-12-25 +1904-05-05 12:34:56.1831,1904-12-25 +1904-05-05 12:34:56.1832,1904-12-25 +1904-05-05 12:34:56.1833,1904-12-25 +1904-05-05 12:34:56.1834,1904-12-25 +1904-05-05 12:34:56.1835,1904-12-25 +1904-05-05 12:34:56.1836,1904-12-25 +1904-05-05 12:34:56.1837,1904-12-25 +1904-05-05 12:34:56.1838,1904-12-25 +1904-05-05 12:34:56.1839,1904-12-25 +1904-05-05 12:34:56.184,1904-12-25 +1904-05-05 12:34:56.1841,1904-12-25 +1904-05-05 12:34:56.1842,1904-12-25 +1904-05-05 12:34:56.1843,1904-12-25 +1904-05-05 12:34:56.1844,1904-12-25 +1904-05-05 12:34:56.1845,1904-12-25 +1904-05-05 12:34:56.1846,1904-12-25 +1904-05-05 12:34:56.1847,1904-12-25 +1904-05-05 12:34:56.1848,1904-12-25 +1904-05-05 12:34:56.1849,1904-12-25 +1904-05-05 12:34:56.185,1904-12-25 +1904-05-05 12:34:56.1851,1904-12-25 +1904-05-05 12:34:56.1852,1904-12-25 +1904-05-05 12:34:56.1853,1904-12-25 +1904-05-05 12:34:56.1854,1904-12-25 +1904-05-05 12:34:56.1855,1904-12-25 +1904-05-05 12:34:56.1856,1904-12-25 +1904-05-05 12:34:56.1857,1904-12-25 +1904-05-05 12:34:56.1858,1904-12-25 +1904-05-05 12:34:56.1859,1904-12-25 +1904-05-05 12:34:56.186,1904-12-25 +1904-05-05 12:34:56.1861,1904-12-25 +1904-05-05 12:34:56.1862,1904-12-25 +1904-05-05 12:34:56.1863,1904-12-25 +1904-05-05 12:34:56.1864,1904-12-25 +1904-05-05 12:34:56.1865,1904-12-25 +1904-05-05 12:34:56.1866,1904-12-25 +1904-05-05 12:34:56.1867,1904-12-25 +1904-05-05 12:34:56.1868,1904-12-25 +1904-05-05 12:34:56.1869,1904-12-25 +1904-05-05 12:34:56.187,1904-12-25 +1904-05-05 12:34:56.1871,1904-12-25 +1904-05-05 12:34:56.1872,1904-12-25 +1904-05-05 12:34:56.1873,1904-12-25 +1904-05-05 12:34:56.1874,1904-12-25 +1904-05-05 12:34:56.1875,1904-12-25 +1904-05-05 12:34:56.1876,1904-12-25 +1904-05-05 12:34:56.1877,1904-12-25 +1904-05-05 12:34:56.1878,1904-12-25 +1904-05-05 12:34:56.1879,1904-12-25 +1904-05-05 12:34:56.188,1904-12-25 +1904-05-05 12:34:56.1881,1904-12-25 +1904-05-05 12:34:56.1882,1904-12-25 +1904-05-05 12:34:56.1883,1904-12-25 +1904-05-05 12:34:56.1884,1904-12-25 +1904-05-05 12:34:56.1885,1904-12-25 +1904-05-05 12:34:56.1886,1904-12-25 +1904-05-05 12:34:56.1887,1904-12-25 +1904-05-05 12:34:56.1888,1904-12-25 +1904-05-05 12:34:56.1889,1904-12-25 +1904-05-05 12:34:56.189,1904-12-25 +1904-05-05 12:34:56.1891,1904-12-25 +1904-05-05 12:34:56.1892,1904-12-25 +1904-05-05 12:34:56.1893,1904-12-25 +1904-05-05 12:34:56.1894,1904-12-25 +1904-05-05 12:34:56.1895,1904-12-25 +1904-05-05 12:34:56.1896,1904-12-25 +1904-05-05 12:34:56.1897,1904-12-25 +1904-05-05 12:34:56.1898,1904-12-25 +1904-05-05 12:34:56.1899,1904-12-25 +1904-05-05 12:34:56.19,1904-12-25 +1904-05-05 12:34:56.1901,1904-12-25 +1904-05-05 12:34:56.1902,1904-12-25 +1904-05-05 12:34:56.1903,1904-12-25 +1904-05-05 12:34:56.1904,1904-12-25 +1904-05-05 12:34:56.1905,1904-12-25 +1904-05-05 12:34:56.1906,1904-12-25 +1904-05-05 12:34:56.1907,1904-12-25 +1904-05-05 12:34:56.1908,1904-12-25 +1904-05-05 12:34:56.1909,1904-12-25 +1904-05-05 12:34:56.191,1904-12-25 +1904-05-05 12:34:56.1911,1904-12-25 +1904-05-05 12:34:56.1912,1904-12-25 +1904-05-05 12:34:56.1913,1904-12-25 +1904-05-05 12:34:56.1914,1904-12-25 +1904-05-05 12:34:56.1915,1904-12-25 +1904-05-05 12:34:56.1916,1904-12-25 +1904-05-05 12:34:56.1917,1904-12-25 +1904-05-05 12:34:56.1918,1904-12-25 +1904-05-05 12:34:56.1919,1904-12-25 +1904-05-05 12:34:56.192,1904-12-25 +1904-05-05 12:34:56.1921,1904-12-25 +1904-05-05 12:34:56.1922,1904-12-25 +1904-05-05 12:34:56.1923,1904-12-25 +1904-05-05 12:34:56.1924,1904-12-25 +1904-05-05 12:34:56.1925,1904-12-25 +1904-05-05 12:34:56.1926,1904-12-25 +1904-05-05 12:34:56.1927,1904-12-25 +1904-05-05 12:34:56.1928,1904-12-25 +1904-05-05 12:34:56.1929,1904-12-25 +1904-05-05 12:34:56.193,1904-12-25 +1904-05-05 12:34:56.1931,1904-12-25 +1904-05-05 12:34:56.1932,1904-12-25 +1904-05-05 12:34:56.1933,1904-12-25 +1904-05-05 12:34:56.1934,1904-12-25 +1904-05-05 12:34:56.1935,1904-12-25 +1904-05-05 12:34:56.1936,1904-12-25 +1904-05-05 12:34:56.1937,1904-12-25 +1904-05-05 12:34:56.1938,1904-12-25 +1904-05-05 12:34:56.1939,1904-12-25 +1904-05-05 12:34:56.194,1904-12-25 +1904-05-05 12:34:56.1941,1904-12-25 +1904-05-05 12:34:56.1942,1904-12-25 +1904-05-05 12:34:56.1943,1904-12-25 +1904-05-05 12:34:56.1944,1904-12-25 +1904-05-05 12:34:56.1945,1904-12-25 +1904-05-05 12:34:56.1946,1904-12-25 +1904-05-05 12:34:56.1947,1904-12-25 +1904-05-05 12:34:56.1948,1904-12-25 +1904-05-05 12:34:56.1949,1904-12-25 +1904-05-05 12:34:56.195,1904-12-25 +1904-05-05 12:34:56.1951,1904-12-25 +1904-05-05 12:34:56.1952,1904-12-25 +1904-05-05 12:34:56.1953,1904-12-25 +1904-05-05 12:34:56.1954,1904-12-25 +1904-05-05 12:34:56.1955,1904-12-25 +1904-05-05 12:34:56.1956,1904-12-25 +1904-05-05 12:34:56.1957,1904-12-25 +1904-05-05 12:34:56.1958,1904-12-25 +1904-05-05 12:34:56.1959,1904-12-25 +1904-05-05 12:34:56.196,1904-12-25 +1904-05-05 12:34:56.1961,1904-12-25 +1904-05-05 12:34:56.1962,1904-12-25 +1904-05-05 12:34:56.1963,1904-12-25 +1904-05-05 12:34:56.1964,1904-12-25 +1904-05-05 12:34:56.1965,1904-12-25 +1904-05-05 12:34:56.1966,1904-12-25 +1904-05-05 12:34:56.1967,1904-12-25 +1904-05-05 12:34:56.1968,1904-12-25 +1904-05-05 12:34:56.1969,1904-12-25 +1904-05-05 12:34:56.197,1904-12-25 +1904-05-05 12:34:56.1971,1904-12-25 +1904-05-05 12:34:56.1972,1904-12-25 +1904-05-05 12:34:56.1973,1904-12-25 +1904-05-05 12:34:56.1974,1904-12-25 +1904-05-05 12:34:56.1975,1904-12-25 +1904-05-05 12:34:56.1976,1904-12-25 +1904-05-05 12:34:56.1977,1904-12-25 +1904-05-05 12:34:56.1978,1904-12-25 +1904-05-05 12:34:56.1979,1904-12-25 +1904-05-05 12:34:56.198,1904-12-25 +1904-05-05 12:34:56.1981,1904-12-25 +1904-05-05 12:34:56.1982,1904-12-25 +1904-05-05 12:34:56.1983,1904-12-25 +1904-05-05 12:34:56.1984,1904-12-25 +1904-05-05 12:34:56.1985,1904-12-25 +1904-05-05 12:34:56.1986,1904-12-25 +1904-05-05 12:34:56.1987,1904-12-25 +1904-05-05 12:34:56.1988,1904-12-25 +1904-05-05 12:34:56.1989,1904-12-25 +1904-05-05 12:34:56.199,1904-12-25 +1904-05-05 12:34:56.1991,1904-12-25 +1904-05-05 12:34:56.1992,1904-12-25 +1904-05-05 12:34:56.1993,1904-12-25 +1904-05-05 12:34:56.1994,1904-12-25 +1904-05-05 12:34:56.1995,1904-12-25 +1904-05-05 12:34:56.1996,1904-12-25 +1904-05-05 12:34:56.1997,1904-12-25 +1904-05-05 12:34:56.1998,1904-12-25 +1904-05-05 12:34:56.1999,1904-12-25 +1905-05-05 12:34:56.1,1905-12-25 +1905-05-05 12:34:56.1001,1905-12-25 +1905-05-05 12:34:56.1002,1905-12-25 +1905-05-05 12:34:56.1003,1905-12-25 +1905-05-05 12:34:56.1004,1905-12-25 +1905-05-05 12:34:56.1005,1905-12-25 +1905-05-05 12:34:56.1006,1905-12-25 +1905-05-05 12:34:56.1007,1905-12-25 +1905-05-05 12:34:56.1008,1905-12-25 +1905-05-05 12:34:56.1009,1905-12-25 +1905-05-05 12:34:56.101,1905-12-25 +1905-05-05 12:34:56.1011,1905-12-25 +1905-05-05 12:34:56.1012,1905-12-25 +1905-05-05 12:34:56.1013,1905-12-25 +1905-05-05 12:34:56.1014,1905-12-25 +1905-05-05 12:34:56.1015,1905-12-25 +1905-05-05 12:34:56.1016,1905-12-25 +1905-05-05 12:34:56.1017,1905-12-25 +1905-05-05 12:34:56.1018,1905-12-25 +1905-05-05 12:34:56.1019,1905-12-25 +1905-05-05 12:34:56.102,1905-12-25 +1905-05-05 12:34:56.1021,1905-12-25 +1905-05-05 12:34:56.1022,1905-12-25 +1905-05-05 12:34:56.1023,1905-12-25 +1905-05-05 12:34:56.1024,1905-12-25 +1905-05-05 12:34:56.1025,1905-12-25 +1905-05-05 12:34:56.1026,1905-12-25 +1905-05-05 12:34:56.1027,1905-12-25 +1905-05-05 12:34:56.1028,1905-12-25 +1905-05-05 12:34:56.1029,1905-12-25 +1905-05-05 12:34:56.103,1905-12-25 +1905-05-05 12:34:56.1031,1905-12-25 +1905-05-05 12:34:56.1032,1905-12-25 +1905-05-05 12:34:56.1033,1905-12-25 +1905-05-05 12:34:56.1034,1905-12-25 +1905-05-05 12:34:56.1035,1905-12-25 +1905-05-05 12:34:56.1036,1905-12-25 +1905-05-05 12:34:56.1037,1905-12-25 +1905-05-05 12:34:56.1038,1905-12-25 +1905-05-05 12:34:56.1039,1905-12-25 +1905-05-05 12:34:56.104,1905-12-25 +1905-05-05 12:34:56.1041,1905-12-25 +1905-05-05 12:34:56.1042,1905-12-25 +1905-05-05 12:34:56.1043,1905-12-25 +1905-05-05 12:34:56.1044,1905-12-25 +1905-05-05 12:34:56.1045,1905-12-25 +1905-05-05 12:34:56.1046,1905-12-25 +1905-05-05 12:34:56.1047,1905-12-25 +1905-05-05 12:34:56.1048,1905-12-25 +1905-05-05 12:34:56.1049,1905-12-25 +1905-05-05 12:34:56.105,1905-12-25 +1905-05-05 12:34:56.1051,1905-12-25 +1905-05-05 12:34:56.1052,1905-12-25 +1905-05-05 12:34:56.1053,1905-12-25 +1905-05-05 12:34:56.1054,1905-12-25 +1905-05-05 12:34:56.1055,1905-12-25 +1905-05-05 12:34:56.1056,1905-12-25 +1905-05-05 12:34:56.1057,1905-12-25 +1905-05-05 12:34:56.1058,1905-12-25 +1905-05-05 12:34:56.1059,1905-12-25 +1905-05-05 12:34:56.106,1905-12-25 +1905-05-05 12:34:56.1061,1905-12-25 +1905-05-05 12:34:56.1062,1905-12-25 +1905-05-05 12:34:56.1063,1905-12-25 +1905-05-05 12:34:56.1064,1905-12-25 +1905-05-05 12:34:56.1065,1905-12-25 +1905-05-05 12:34:56.1066,1905-12-25 +1905-05-05 12:34:56.1067,1905-12-25 +1905-05-05 12:34:56.1068,1905-12-25 +1905-05-05 12:34:56.1069,1905-12-25 +1905-05-05 12:34:56.107,1905-12-25 +1905-05-05 12:34:56.1071,1905-12-25 +1905-05-05 12:34:56.1072,1905-12-25 +1905-05-05 12:34:56.1073,1905-12-25 +1905-05-05 12:34:56.1074,1905-12-25 +1905-05-05 12:34:56.1075,1905-12-25 +1905-05-05 12:34:56.1076,1905-12-25 +1905-05-05 12:34:56.1077,1905-12-25 +1905-05-05 12:34:56.1078,1905-12-25 +1905-05-05 12:34:56.1079,1905-12-25 +1905-05-05 12:34:56.108,1905-12-25 +1905-05-05 12:34:56.1081,1905-12-25 +1905-05-05 12:34:56.1082,1905-12-25 +1905-05-05 12:34:56.1083,1905-12-25 +1905-05-05 12:34:56.1084,1905-12-25 +1905-05-05 12:34:56.1085,1905-12-25 +1905-05-05 12:34:56.1086,1905-12-25 +1905-05-05 12:34:56.1087,1905-12-25 +1905-05-05 12:34:56.1088,1905-12-25 +1905-05-05 12:34:56.1089,1905-12-25 +1905-05-05 12:34:56.109,1905-12-25 +1905-05-05 12:34:56.1091,1905-12-25 +1905-05-05 12:34:56.1092,1905-12-25 +1905-05-05 12:34:56.1093,1905-12-25 +1905-05-05 12:34:56.1094,1905-12-25 +1905-05-05 12:34:56.1095,1905-12-25 +1905-05-05 12:34:56.1096,1905-12-25 +1905-05-05 12:34:56.1097,1905-12-25 +1905-05-05 12:34:56.1098,1905-12-25 +1905-05-05 12:34:56.1099,1905-12-25 +1905-05-05 12:34:56.11,1905-12-25 +1905-05-05 12:34:56.1101,1905-12-25 +1905-05-05 12:34:56.1102,1905-12-25 +1905-05-05 12:34:56.1103,1905-12-25 +1905-05-05 12:34:56.1104,1905-12-25 +1905-05-05 12:34:56.1105,1905-12-25 +1905-05-05 12:34:56.1106,1905-12-25 +1905-05-05 12:34:56.1107,1905-12-25 +1905-05-05 12:34:56.1108,1905-12-25 +1905-05-05 12:34:56.1109,1905-12-25 +1905-05-05 12:34:56.111,1905-12-25 +1905-05-05 12:34:56.1111,1905-12-25 +1905-05-05 12:34:56.1112,1905-12-25 +1905-05-05 12:34:56.1113,1905-12-25 +1905-05-05 12:34:56.1114,1905-12-25 +1905-05-05 12:34:56.1115,1905-12-25 +1905-05-05 12:34:56.1116,1905-12-25 +1905-05-05 12:34:56.1117,1905-12-25 +1905-05-05 12:34:56.1118,1905-12-25 +1905-05-05 12:34:56.1119,1905-12-25 +1905-05-05 12:34:56.112,1905-12-25 +1905-05-05 12:34:56.1121,1905-12-25 +1905-05-05 12:34:56.1122,1905-12-25 +1905-05-05 12:34:56.1123,1905-12-25 +1905-05-05 12:34:56.1124,1905-12-25 +1905-05-05 12:34:56.1125,1905-12-25 +1905-05-05 12:34:56.1126,1905-12-25 +1905-05-05 12:34:56.1127,1905-12-25 +1905-05-05 12:34:56.1128,1905-12-25 +1905-05-05 12:34:56.1129,1905-12-25 +1905-05-05 12:34:56.113,1905-12-25 +1905-05-05 12:34:56.1131,1905-12-25 +1905-05-05 12:34:56.1132,1905-12-25 +1905-05-05 12:34:56.1133,1905-12-25 +1905-05-05 12:34:56.1134,1905-12-25 +1905-05-05 12:34:56.1135,1905-12-25 +1905-05-05 12:34:56.1136,1905-12-25 +1905-05-05 12:34:56.1137,1905-12-25 +1905-05-05 12:34:56.1138,1905-12-25 +1905-05-05 12:34:56.1139,1905-12-25 +1905-05-05 12:34:56.114,1905-12-25 +1905-05-05 12:34:56.1141,1905-12-25 +1905-05-05 12:34:56.1142,1905-12-25 +1905-05-05 12:34:56.1143,1905-12-25 +1905-05-05 12:34:56.1144,1905-12-25 +1905-05-05 12:34:56.1145,1905-12-25 +1905-05-05 12:34:56.1146,1905-12-25 +1905-05-05 12:34:56.1147,1905-12-25 +1905-05-05 12:34:56.1148,1905-12-25 +1905-05-05 12:34:56.1149,1905-12-25 +1905-05-05 12:34:56.115,1905-12-25 +1905-05-05 12:34:56.1151,1905-12-25 +1905-05-05 12:34:56.1152,1905-12-25 +1905-05-05 12:34:56.1153,1905-12-25 +1905-05-05 12:34:56.1154,1905-12-25 +1905-05-05 12:34:56.1155,1905-12-25 +1905-05-05 12:34:56.1156,1905-12-25 +1905-05-05 12:34:56.1157,1905-12-25 +1905-05-05 12:34:56.1158,1905-12-25 +1905-05-05 12:34:56.1159,1905-12-25 +1905-05-05 12:34:56.116,1905-12-25 +1905-05-05 12:34:56.1161,1905-12-25 +1905-05-05 12:34:56.1162,1905-12-25 +1905-05-05 12:34:56.1163,1905-12-25 +1905-05-05 12:34:56.1164,1905-12-25 +1905-05-05 12:34:56.1165,1905-12-25 +1905-05-05 12:34:56.1166,1905-12-25 +1905-05-05 12:34:56.1167,1905-12-25 +1905-05-05 12:34:56.1168,1905-12-25 +1905-05-05 12:34:56.1169,1905-12-25 +1905-05-05 12:34:56.117,1905-12-25 +1905-05-05 12:34:56.1171,1905-12-25 +1905-05-05 12:34:56.1172,1905-12-25 +1905-05-05 12:34:56.1173,1905-12-25 +1905-05-05 12:34:56.1174,1905-12-25 +1905-05-05 12:34:56.1175,1905-12-25 +1905-05-05 12:34:56.1176,1905-12-25 +1905-05-05 12:34:56.1177,1905-12-25 +1905-05-05 12:34:56.1178,1905-12-25 +1905-05-05 12:34:56.1179,1905-12-25 +1905-05-05 12:34:56.118,1905-12-25 +1905-05-05 12:34:56.1181,1905-12-25 +1905-05-05 12:34:56.1182,1905-12-25 +1905-05-05 12:34:56.1183,1905-12-25 +1905-05-05 12:34:56.1184,1905-12-25 +1905-05-05 12:34:56.1185,1905-12-25 +1905-05-05 12:34:56.1186,1905-12-25 +1905-05-05 12:34:56.1187,1905-12-25 +1905-05-05 12:34:56.1188,1905-12-25 +1905-05-05 12:34:56.1189,1905-12-25 +1905-05-05 12:34:56.119,1905-12-25 +1905-05-05 12:34:56.1191,1905-12-25 +1905-05-05 12:34:56.1192,1905-12-25 +1905-05-05 12:34:56.1193,1905-12-25 +1905-05-05 12:34:56.1194,1905-12-25 +1905-05-05 12:34:56.1195,1905-12-25 +1905-05-05 12:34:56.1196,1905-12-25 +1905-05-05 12:34:56.1197,1905-12-25 +1905-05-05 12:34:56.1198,1905-12-25 +1905-05-05 12:34:56.1199,1905-12-25 +1905-05-05 12:34:56.12,1905-12-25 +1905-05-05 12:34:56.1201,1905-12-25 +1905-05-05 12:34:56.1202,1905-12-25 +1905-05-05 12:34:56.1203,1905-12-25 +1905-05-05 12:34:56.1204,1905-12-25 +1905-05-05 12:34:56.1205,1905-12-25 +1905-05-05 12:34:56.1206,1905-12-25 +1905-05-05 12:34:56.1207,1905-12-25 +1905-05-05 12:34:56.1208,1905-12-25 +1905-05-05 12:34:56.1209,1905-12-25 +1905-05-05 12:34:56.121,1905-12-25 +1905-05-05 12:34:56.1211,1905-12-25 +1905-05-05 12:34:56.1212,1905-12-25 +1905-05-05 12:34:56.1213,1905-12-25 +1905-05-05 12:34:56.1214,1905-12-25 +1905-05-05 12:34:56.1215,1905-12-25 +1905-05-05 12:34:56.1216,1905-12-25 +1905-05-05 12:34:56.1217,1905-12-25 +1905-05-05 12:34:56.1218,1905-12-25 +1905-05-05 12:34:56.1219,1905-12-25 +1905-05-05 12:34:56.122,1905-12-25 +1905-05-05 12:34:56.1221,1905-12-25 +1905-05-05 12:34:56.1222,1905-12-25 +1905-05-05 12:34:56.1223,1905-12-25 +1905-05-05 12:34:56.1224,1905-12-25 +1905-05-05 12:34:56.1225,1905-12-25 +1905-05-05 12:34:56.1226,1905-12-25 +1905-05-05 12:34:56.1227,1905-12-25 +1905-05-05 12:34:56.1228,1905-12-25 +1905-05-05 12:34:56.1229,1905-12-25 +1905-05-05 12:34:56.123,1905-12-25 +1905-05-05 12:34:56.1231,1905-12-25 +1905-05-05 12:34:56.1232,1905-12-25 +1905-05-05 12:34:56.1233,1905-12-25 +1905-05-05 12:34:56.1234,1905-12-25 +1905-05-05 12:34:56.1235,1905-12-25 +1905-05-05 12:34:56.1236,1905-12-25 +1905-05-05 12:34:56.1237,1905-12-25 +1905-05-05 12:34:56.1238,1905-12-25 +1905-05-05 12:34:56.1239,1905-12-25 +1905-05-05 12:34:56.124,1905-12-25 +1905-05-05 12:34:56.1241,1905-12-25 +1905-05-05 12:34:56.1242,1905-12-25 +1905-05-05 12:34:56.1243,1905-12-25 +1905-05-05 12:34:56.1244,1905-12-25 +1905-05-05 12:34:56.1245,1905-12-25 +1905-05-05 12:34:56.1246,1905-12-25 +1905-05-05 12:34:56.1247,1905-12-25 +1905-05-05 12:34:56.1248,1905-12-25 +1905-05-05 12:34:56.1249,1905-12-25 +1905-05-05 12:34:56.125,1905-12-25 +1905-05-05 12:34:56.1251,1905-12-25 +1905-05-05 12:34:56.1252,1905-12-25 +1905-05-05 12:34:56.1253,1905-12-25 +1905-05-05 12:34:56.1254,1905-12-25 +1905-05-05 12:34:56.1255,1905-12-25 +1905-05-05 12:34:56.1256,1905-12-25 +1905-05-05 12:34:56.1257,1905-12-25 +1905-05-05 12:34:56.1258,1905-12-25 +1905-05-05 12:34:56.1259,1905-12-25 +1905-05-05 12:34:56.126,1905-12-25 +1905-05-05 12:34:56.1261,1905-12-25 +1905-05-05 12:34:56.1262,1905-12-25 +1905-05-05 12:34:56.1263,1905-12-25 +1905-05-05 12:34:56.1264,1905-12-25 +1905-05-05 12:34:56.1265,1905-12-25 +1905-05-05 12:34:56.1266,1905-12-25 +1905-05-05 12:34:56.1267,1905-12-25 +1905-05-05 12:34:56.1268,1905-12-25 +1905-05-05 12:34:56.1269,1905-12-25 +1905-05-05 12:34:56.127,1905-12-25 +1905-05-05 12:34:56.1271,1905-12-25 +1905-05-05 12:34:56.1272,1905-12-25 +1905-05-05 12:34:56.1273,1905-12-25 +1905-05-05 12:34:56.1274,1905-12-25 +1905-05-05 12:34:56.1275,1905-12-25 +1905-05-05 12:34:56.1276,1905-12-25 +1905-05-05 12:34:56.1277,1905-12-25 +1905-05-05 12:34:56.1278,1905-12-25 +1905-05-05 12:34:56.1279,1905-12-25 +1905-05-05 12:34:56.128,1905-12-25 +1905-05-05 12:34:56.1281,1905-12-25 +1905-05-05 12:34:56.1282,1905-12-25 +1905-05-05 12:34:56.1283,1905-12-25 +1905-05-05 12:34:56.1284,1905-12-25 +1905-05-05 12:34:56.1285,1905-12-25 +1905-05-05 12:34:56.1286,1905-12-25 +1905-05-05 12:34:56.1287,1905-12-25 +1905-05-05 12:34:56.1288,1905-12-25 +1905-05-05 12:34:56.1289,1905-12-25 +1905-05-05 12:34:56.129,1905-12-25 +1905-05-05 12:34:56.1291,1905-12-25 +1905-05-05 12:34:56.1292,1905-12-25 +1905-05-05 12:34:56.1293,1905-12-25 +1905-05-05 12:34:56.1294,1905-12-25 +1905-05-05 12:34:56.1295,1905-12-25 +1905-05-05 12:34:56.1296,1905-12-25 +1905-05-05 12:34:56.1297,1905-12-25 +1905-05-05 12:34:56.1298,1905-12-25 +1905-05-05 12:34:56.1299,1905-12-25 +1905-05-05 12:34:56.13,1905-12-25 +1905-05-05 12:34:56.1301,1905-12-25 +1905-05-05 12:34:56.1302,1905-12-25 +1905-05-05 12:34:56.1303,1905-12-25 +1905-05-05 12:34:56.1304,1905-12-25 +1905-05-05 12:34:56.1305,1905-12-25 +1905-05-05 12:34:56.1306,1905-12-25 +1905-05-05 12:34:56.1307,1905-12-25 +1905-05-05 12:34:56.1308,1905-12-25 +1905-05-05 12:34:56.1309,1905-12-25 +1905-05-05 12:34:56.131,1905-12-25 +1905-05-05 12:34:56.1311,1905-12-25 +1905-05-05 12:34:56.1312,1905-12-25 +1905-05-05 12:34:56.1313,1905-12-25 +1905-05-05 12:34:56.1314,1905-12-25 +1905-05-05 12:34:56.1315,1905-12-25 +1905-05-05 12:34:56.1316,1905-12-25 +1905-05-05 12:34:56.1317,1905-12-25 +1905-05-05 12:34:56.1318,1905-12-25 +1905-05-05 12:34:56.1319,1905-12-25 +1905-05-05 12:34:56.132,1905-12-25 +1905-05-05 12:34:56.1321,1905-12-25 +1905-05-05 12:34:56.1322,1905-12-25 +1905-05-05 12:34:56.1323,1905-12-25 +1905-05-05 12:34:56.1324,1905-12-25 +1905-05-05 12:34:56.1325,1905-12-25 +1905-05-05 12:34:56.1326,1905-12-25 +1905-05-05 12:34:56.1327,1905-12-25 +1905-05-05 12:34:56.1328,1905-12-25 +1905-05-05 12:34:56.1329,1905-12-25 +1905-05-05 12:34:56.133,1905-12-25 +1905-05-05 12:34:56.1331,1905-12-25 +1905-05-05 12:34:56.1332,1905-12-25 +1905-05-05 12:34:56.1333,1905-12-25 +1905-05-05 12:34:56.1334,1905-12-25 +1905-05-05 12:34:56.1335,1905-12-25 +1905-05-05 12:34:56.1336,1905-12-25 +1905-05-05 12:34:56.1337,1905-12-25 +1905-05-05 12:34:56.1338,1905-12-25 +1905-05-05 12:34:56.1339,1905-12-25 +1905-05-05 12:34:56.134,1905-12-25 +1905-05-05 12:34:56.1341,1905-12-25 +1905-05-05 12:34:56.1342,1905-12-25 +1905-05-05 12:34:56.1343,1905-12-25 +1905-05-05 12:34:56.1344,1905-12-25 +1905-05-05 12:34:56.1345,1905-12-25 +1905-05-05 12:34:56.1346,1905-12-25 +1905-05-05 12:34:56.1347,1905-12-25 +1905-05-05 12:34:56.1348,1905-12-25 +1905-05-05 12:34:56.1349,1905-12-25 +1905-05-05 12:34:56.135,1905-12-25 +1905-05-05 12:34:56.1351,1905-12-25 +1905-05-05 12:34:56.1352,1905-12-25 +1905-05-05 12:34:56.1353,1905-12-25 +1905-05-05 12:34:56.1354,1905-12-25 +1905-05-05 12:34:56.1355,1905-12-25 +1905-05-05 12:34:56.1356,1905-12-25 +1905-05-05 12:34:56.1357,1905-12-25 +1905-05-05 12:34:56.1358,1905-12-25 +1905-05-05 12:34:56.1359,1905-12-25 +1905-05-05 12:34:56.136,1905-12-25 +1905-05-05 12:34:56.1361,1905-12-25 +1905-05-05 12:34:56.1362,1905-12-25 +1905-05-05 12:34:56.1363,1905-12-25 +1905-05-05 12:34:56.1364,1905-12-25 +1905-05-05 12:34:56.1365,1905-12-25 +1905-05-05 12:34:56.1366,1905-12-25 +1905-05-05 12:34:56.1367,1905-12-25 +1905-05-05 12:34:56.1368,1905-12-25 +1905-05-05 12:34:56.1369,1905-12-25 +1905-05-05 12:34:56.137,1905-12-25 +1905-05-05 12:34:56.1371,1905-12-25 +1905-05-05 12:34:56.1372,1905-12-25 +1905-05-05 12:34:56.1373,1905-12-25 +1905-05-05 12:34:56.1374,1905-12-25 +1905-05-05 12:34:56.1375,1905-12-25 +1905-05-05 12:34:56.1376,1905-12-25 +1905-05-05 12:34:56.1377,1905-12-25 +1905-05-05 12:34:56.1378,1905-12-25 +1905-05-05 12:34:56.1379,1905-12-25 +1905-05-05 12:34:56.138,1905-12-25 +1905-05-05 12:34:56.1381,1905-12-25 +1905-05-05 12:34:56.1382,1905-12-25 +1905-05-05 12:34:56.1383,1905-12-25 +1905-05-05 12:34:56.1384,1905-12-25 +1905-05-05 12:34:56.1385,1905-12-25 +1905-05-05 12:34:56.1386,1905-12-25 +1905-05-05 12:34:56.1387,1905-12-25 +1905-05-05 12:34:56.1388,1905-12-25 +1905-05-05 12:34:56.1389,1905-12-25 +1905-05-05 12:34:56.139,1905-12-25 +1905-05-05 12:34:56.1391,1905-12-25 +1905-05-05 12:34:56.1392,1905-12-25 +1905-05-05 12:34:56.1393,1905-12-25 +1905-05-05 12:34:56.1394,1905-12-25 +1905-05-05 12:34:56.1395,1905-12-25 +1905-05-05 12:34:56.1396,1905-12-25 +1905-05-05 12:34:56.1397,1905-12-25 +1905-05-05 12:34:56.1398,1905-12-25 +1905-05-05 12:34:56.1399,1905-12-25 +1905-05-05 12:34:56.14,1905-12-25 +1905-05-05 12:34:56.1401,1905-12-25 +1905-05-05 12:34:56.1402,1905-12-25 +1905-05-05 12:34:56.1403,1905-12-25 +1905-05-05 12:34:56.1404,1905-12-25 +1905-05-05 12:34:56.1405,1905-12-25 +1905-05-05 12:34:56.1406,1905-12-25 +1905-05-05 12:34:56.1407,1905-12-25 +1905-05-05 12:34:56.1408,1905-12-25 +1905-05-05 12:34:56.1409,1905-12-25 +1905-05-05 12:34:56.141,1905-12-25 +1905-05-05 12:34:56.1411,1905-12-25 +1905-05-05 12:34:56.1412,1905-12-25 +1905-05-05 12:34:56.1413,1905-12-25 +1905-05-05 12:34:56.1414,1905-12-25 +1905-05-05 12:34:56.1415,1905-12-25 +1905-05-05 12:34:56.1416,1905-12-25 +1905-05-05 12:34:56.1417,1905-12-25 +1905-05-05 12:34:56.1418,1905-12-25 +1905-05-05 12:34:56.1419,1905-12-25 +1905-05-05 12:34:56.142,1905-12-25 +1905-05-05 12:34:56.1421,1905-12-25 +1905-05-05 12:34:56.1422,1905-12-25 +1905-05-05 12:34:56.1423,1905-12-25 +1905-05-05 12:34:56.1424,1905-12-25 +1905-05-05 12:34:56.1425,1905-12-25 +1905-05-05 12:34:56.1426,1905-12-25 +1905-05-05 12:34:56.1427,1905-12-25 +1905-05-05 12:34:56.1428,1905-12-25 +1905-05-05 12:34:56.1429,1905-12-25 +1905-05-05 12:34:56.143,1905-12-25 +1905-05-05 12:34:56.1431,1905-12-25 +1905-05-05 12:34:56.1432,1905-12-25 +1905-05-05 12:34:56.1433,1905-12-25 +1905-05-05 12:34:56.1434,1905-12-25 +1905-05-05 12:34:56.1435,1905-12-25 +1905-05-05 12:34:56.1436,1905-12-25 +1905-05-05 12:34:56.1437,1905-12-25 +1905-05-05 12:34:56.1438,1905-12-25 +1905-05-05 12:34:56.1439,1905-12-25 +1905-05-05 12:34:56.144,1905-12-25 +1905-05-05 12:34:56.1441,1905-12-25 +1905-05-05 12:34:56.1442,1905-12-25 +1905-05-05 12:34:56.1443,1905-12-25 +1905-05-05 12:34:56.1444,1905-12-25 +1905-05-05 12:34:56.1445,1905-12-25 +1905-05-05 12:34:56.1446,1905-12-25 +1905-05-05 12:34:56.1447,1905-12-25 +1905-05-05 12:34:56.1448,1905-12-25 +1905-05-05 12:34:56.1449,1905-12-25 +1905-05-05 12:34:56.145,1905-12-25 +1905-05-05 12:34:56.1451,1905-12-25 +1905-05-05 12:34:56.1452,1905-12-25 +1905-05-05 12:34:56.1453,1905-12-25 +1905-05-05 12:34:56.1454,1905-12-25 +1905-05-05 12:34:56.1455,1905-12-25 +1905-05-05 12:34:56.1456,1905-12-25 +1905-05-05 12:34:56.1457,1905-12-25 +1905-05-05 12:34:56.1458,1905-12-25 +1905-05-05 12:34:56.1459,1905-12-25 +1905-05-05 12:34:56.146,1905-12-25 +1905-05-05 12:34:56.1461,1905-12-25 +1905-05-05 12:34:56.1462,1905-12-25 +1905-05-05 12:34:56.1463,1905-12-25 +1905-05-05 12:34:56.1464,1905-12-25 +1905-05-05 12:34:56.1465,1905-12-25 +1905-05-05 12:34:56.1466,1905-12-25 +1905-05-05 12:34:56.1467,1905-12-25 +1905-05-05 12:34:56.1468,1905-12-25 +1905-05-05 12:34:56.1469,1905-12-25 +1905-05-05 12:34:56.147,1905-12-25 +1905-05-05 12:34:56.1471,1905-12-25 +1905-05-05 12:34:56.1472,1905-12-25 +1905-05-05 12:34:56.1473,1905-12-25 +1905-05-05 12:34:56.1474,1905-12-25 +1905-05-05 12:34:56.1475,1905-12-25 +1905-05-05 12:34:56.1476,1905-12-25 +1905-05-05 12:34:56.1477,1905-12-25 +1905-05-05 12:34:56.1478,1905-12-25 +1905-05-05 12:34:56.1479,1905-12-25 +1905-05-05 12:34:56.148,1905-12-25 +1905-05-05 12:34:56.1481,1905-12-25 +1905-05-05 12:34:56.1482,1905-12-25 +1905-05-05 12:34:56.1483,1905-12-25 +1905-05-05 12:34:56.1484,1905-12-25 +1905-05-05 12:34:56.1485,1905-12-25 +1905-05-05 12:34:56.1486,1905-12-25 +1905-05-05 12:34:56.1487,1905-12-25 +1905-05-05 12:34:56.1488,1905-12-25 +1905-05-05 12:34:56.1489,1905-12-25 +1905-05-05 12:34:56.149,1905-12-25 +1905-05-05 12:34:56.1491,1905-12-25 +1905-05-05 12:34:56.1492,1905-12-25 +1905-05-05 12:34:56.1493,1905-12-25 +1905-05-05 12:34:56.1494,1905-12-25 +1905-05-05 12:34:56.1495,1905-12-25 +1905-05-05 12:34:56.1496,1905-12-25 +1905-05-05 12:34:56.1497,1905-12-25 +1905-05-05 12:34:56.1498,1905-12-25 +1905-05-05 12:34:56.1499,1905-12-25 +1905-05-05 12:34:56.15,1905-12-25 +1905-05-05 12:34:56.1501,1905-12-25 +1905-05-05 12:34:56.1502,1905-12-25 +1905-05-05 12:34:56.1503,1905-12-25 +1905-05-05 12:34:56.1504,1905-12-25 +1905-05-05 12:34:56.1505,1905-12-25 +1905-05-05 12:34:56.1506,1905-12-25 +1905-05-05 12:34:56.1507,1905-12-25 +1905-05-05 12:34:56.1508,1905-12-25 +1905-05-05 12:34:56.1509,1905-12-25 +1905-05-05 12:34:56.151,1905-12-25 +1905-05-05 12:34:56.1511,1905-12-25 +1905-05-05 12:34:56.1512,1905-12-25 +1905-05-05 12:34:56.1513,1905-12-25 +1905-05-05 12:34:56.1514,1905-12-25 +1905-05-05 12:34:56.1515,1905-12-25 +1905-05-05 12:34:56.1516,1905-12-25 +1905-05-05 12:34:56.1517,1905-12-25 +1905-05-05 12:34:56.1518,1905-12-25 +1905-05-05 12:34:56.1519,1905-12-25 +1905-05-05 12:34:56.152,1905-12-25 +1905-05-05 12:34:56.1521,1905-12-25 +1905-05-05 12:34:56.1522,1905-12-25 +1905-05-05 12:34:56.1523,1905-12-25 +1905-05-05 12:34:56.1524,1905-12-25 +1905-05-05 12:34:56.1525,1905-12-25 +1905-05-05 12:34:56.1526,1905-12-25 +1905-05-05 12:34:56.1527,1905-12-25 +1905-05-05 12:34:56.1528,1905-12-25 +1905-05-05 12:34:56.1529,1905-12-25 +1905-05-05 12:34:56.153,1905-12-25 +1905-05-05 12:34:56.1531,1905-12-25 +1905-05-05 12:34:56.1532,1905-12-25 +1905-05-05 12:34:56.1533,1905-12-25 +1905-05-05 12:34:56.1534,1905-12-25 +1905-05-05 12:34:56.1535,1905-12-25 +1905-05-05 12:34:56.1536,1905-12-25 +1905-05-05 12:34:56.1537,1905-12-25 +1905-05-05 12:34:56.1538,1905-12-25 +1905-05-05 12:34:56.1539,1905-12-25 +1905-05-05 12:34:56.154,1905-12-25 +1905-05-05 12:34:56.1541,1905-12-25 +1905-05-05 12:34:56.1542,1905-12-25 +1905-05-05 12:34:56.1543,1905-12-25 +1905-05-05 12:34:56.1544,1905-12-25 +1905-05-05 12:34:56.1545,1905-12-25 +1905-05-05 12:34:56.1546,1905-12-25 +1905-05-05 12:34:56.1547,1905-12-25 +1905-05-05 12:34:56.1548,1905-12-25 +1905-05-05 12:34:56.1549,1905-12-25 +1905-05-05 12:34:56.155,1905-12-25 +1905-05-05 12:34:56.1551,1905-12-25 +1905-05-05 12:34:56.1552,1905-12-25 +1905-05-05 12:34:56.1553,1905-12-25 +1905-05-05 12:34:56.1554,1905-12-25 +1905-05-05 12:34:56.1555,1905-12-25 +1905-05-05 12:34:56.1556,1905-12-25 +1905-05-05 12:34:56.1557,1905-12-25 +1905-05-05 12:34:56.1558,1905-12-25 +1905-05-05 12:34:56.1559,1905-12-25 +1905-05-05 12:34:56.156,1905-12-25 +1905-05-05 12:34:56.1561,1905-12-25 +1905-05-05 12:34:56.1562,1905-12-25 +1905-05-05 12:34:56.1563,1905-12-25 +1905-05-05 12:34:56.1564,1905-12-25 +1905-05-05 12:34:56.1565,1905-12-25 +1905-05-05 12:34:56.1566,1905-12-25 +1905-05-05 12:34:56.1567,1905-12-25 +1905-05-05 12:34:56.1568,1905-12-25 +1905-05-05 12:34:56.1569,1905-12-25 +1905-05-05 12:34:56.157,1905-12-25 +1905-05-05 12:34:56.1571,1905-12-25 +1905-05-05 12:34:56.1572,1905-12-25 +1905-05-05 12:34:56.1573,1905-12-25 +1905-05-05 12:34:56.1574,1905-12-25 +1905-05-05 12:34:56.1575,1905-12-25 +1905-05-05 12:34:56.1576,1905-12-25 +1905-05-05 12:34:56.1577,1905-12-25 +1905-05-05 12:34:56.1578,1905-12-25 +1905-05-05 12:34:56.1579,1905-12-25 +1905-05-05 12:34:56.158,1905-12-25 +1905-05-05 12:34:56.1581,1905-12-25 +1905-05-05 12:34:56.1582,1905-12-25 +1905-05-05 12:34:56.1583,1905-12-25 +1905-05-05 12:34:56.1584,1905-12-25 +1905-05-05 12:34:56.1585,1905-12-25 +1905-05-05 12:34:56.1586,1905-12-25 +1905-05-05 12:34:56.1587,1905-12-25 +1905-05-05 12:34:56.1588,1905-12-25 +1905-05-05 12:34:56.1589,1905-12-25 +1905-05-05 12:34:56.159,1905-12-25 +1905-05-05 12:34:56.1591,1905-12-25 +1905-05-05 12:34:56.1592,1905-12-25 +1905-05-05 12:34:56.1593,1905-12-25 +1905-05-05 12:34:56.1594,1905-12-25 +1905-05-05 12:34:56.1595,1905-12-25 +1905-05-05 12:34:56.1596,1905-12-25 +1905-05-05 12:34:56.1597,1905-12-25 +1905-05-05 12:34:56.1598,1905-12-25 +1905-05-05 12:34:56.1599,1905-12-25 +1905-05-05 12:34:56.16,1905-12-25 +1905-05-05 12:34:56.1601,1905-12-25 +1905-05-05 12:34:56.1602,1905-12-25 +1905-05-05 12:34:56.1603,1905-12-25 +1905-05-05 12:34:56.1604,1905-12-25 +1905-05-05 12:34:56.1605,1905-12-25 +1905-05-05 12:34:56.1606,1905-12-25 +1905-05-05 12:34:56.1607,1905-12-25 +1905-05-05 12:34:56.1608,1905-12-25 +1905-05-05 12:34:56.1609,1905-12-25 +1905-05-05 12:34:56.161,1905-12-25 +1905-05-05 12:34:56.1611,1905-12-25 +1905-05-05 12:34:56.1612,1905-12-25 +1905-05-05 12:34:56.1613,1905-12-25 +1905-05-05 12:34:56.1614,1905-12-25 +1905-05-05 12:34:56.1615,1905-12-25 +1905-05-05 12:34:56.1616,1905-12-25 +1905-05-05 12:34:56.1617,1905-12-25 +1905-05-05 12:34:56.1618,1905-12-25 +1905-05-05 12:34:56.1619,1905-12-25 +1905-05-05 12:34:56.162,1905-12-25 +1905-05-05 12:34:56.1621,1905-12-25 +1905-05-05 12:34:56.1622,1905-12-25 +1905-05-05 12:34:56.1623,1905-12-25 +1905-05-05 12:34:56.1624,1905-12-25 +1905-05-05 12:34:56.1625,1905-12-25 +1905-05-05 12:34:56.1626,1905-12-25 +1905-05-05 12:34:56.1627,1905-12-25 +1905-05-05 12:34:56.1628,1905-12-25 +1905-05-05 12:34:56.1629,1905-12-25 +1905-05-05 12:34:56.163,1905-12-25 +1905-05-05 12:34:56.1631,1905-12-25 +1905-05-05 12:34:56.1632,1905-12-25 +1905-05-05 12:34:56.1633,1905-12-25 +1905-05-05 12:34:56.1634,1905-12-25 +1905-05-05 12:34:56.1635,1905-12-25 +1905-05-05 12:34:56.1636,1905-12-25 +1905-05-05 12:34:56.1637,1905-12-25 +1905-05-05 12:34:56.1638,1905-12-25 +1905-05-05 12:34:56.1639,1905-12-25 +1905-05-05 12:34:56.164,1905-12-25 +1905-05-05 12:34:56.1641,1905-12-25 +1905-05-05 12:34:56.1642,1905-12-25 +1905-05-05 12:34:56.1643,1905-12-25 +1905-05-05 12:34:56.1644,1905-12-25 +1905-05-05 12:34:56.1645,1905-12-25 +1905-05-05 12:34:56.1646,1905-12-25 +1905-05-05 12:34:56.1647,1905-12-25 +1905-05-05 12:34:56.1648,1905-12-25 +1905-05-05 12:34:56.1649,1905-12-25 +1905-05-05 12:34:56.165,1905-12-25 +1905-05-05 12:34:56.1651,1905-12-25 +1905-05-05 12:34:56.1652,1905-12-25 +1905-05-05 12:34:56.1653,1905-12-25 +1905-05-05 12:34:56.1654,1905-12-25 +1905-05-05 12:34:56.1655,1905-12-25 +1905-05-05 12:34:56.1656,1905-12-25 +1905-05-05 12:34:56.1657,1905-12-25 +1905-05-05 12:34:56.1658,1905-12-25 +1905-05-05 12:34:56.1659,1905-12-25 +1905-05-05 12:34:56.166,1905-12-25 +1905-05-05 12:34:56.1661,1905-12-25 +1905-05-05 12:34:56.1662,1905-12-25 +1905-05-05 12:34:56.1663,1905-12-25 +1905-05-05 12:34:56.1664,1905-12-25 +1905-05-05 12:34:56.1665,1905-12-25 +1905-05-05 12:34:56.1666,1905-12-25 +1905-05-05 12:34:56.1667,1905-12-25 +1905-05-05 12:34:56.1668,1905-12-25 +1905-05-05 12:34:56.1669,1905-12-25 +1905-05-05 12:34:56.167,1905-12-25 +1905-05-05 12:34:56.1671,1905-12-25 +1905-05-05 12:34:56.1672,1905-12-25 +1905-05-05 12:34:56.1673,1905-12-25 +1905-05-05 12:34:56.1674,1905-12-25 +1905-05-05 12:34:56.1675,1905-12-25 +1905-05-05 12:34:56.1676,1905-12-25 +1905-05-05 12:34:56.1677,1905-12-25 +1905-05-05 12:34:56.1678,1905-12-25 +1905-05-05 12:34:56.1679,1905-12-25 +1905-05-05 12:34:56.168,1905-12-25 +1905-05-05 12:34:56.1681,1905-12-25 +1905-05-05 12:34:56.1682,1905-12-25 +1905-05-05 12:34:56.1683,1905-12-25 +1905-05-05 12:34:56.1684,1905-12-25 +1905-05-05 12:34:56.1685,1905-12-25 +1905-05-05 12:34:56.1686,1905-12-25 +1905-05-05 12:34:56.1687,1905-12-25 +1905-05-05 12:34:56.1688,1905-12-25 +1905-05-05 12:34:56.1689,1905-12-25 +1905-05-05 12:34:56.169,1905-12-25 +1905-05-05 12:34:56.1691,1905-12-25 +1905-05-05 12:34:56.1692,1905-12-25 +1905-05-05 12:34:56.1693,1905-12-25 +1905-05-05 12:34:56.1694,1905-12-25 +1905-05-05 12:34:56.1695,1905-12-25 +1905-05-05 12:34:56.1696,1905-12-25 +1905-05-05 12:34:56.1697,1905-12-25 +1905-05-05 12:34:56.1698,1905-12-25 +1905-05-05 12:34:56.1699,1905-12-25 +1905-05-05 12:34:56.17,1905-12-25 +1905-05-05 12:34:56.1701,1905-12-25 +1905-05-05 12:34:56.1702,1905-12-25 +1905-05-05 12:34:56.1703,1905-12-25 +1905-05-05 12:34:56.1704,1905-12-25 +1905-05-05 12:34:56.1705,1905-12-25 +1905-05-05 12:34:56.1706,1905-12-25 +1905-05-05 12:34:56.1707,1905-12-25 +1905-05-05 12:34:56.1708,1905-12-25 +1905-05-05 12:34:56.1709,1905-12-25 +1905-05-05 12:34:56.171,1905-12-25 +1905-05-05 12:34:56.1711,1905-12-25 +1905-05-05 12:34:56.1712,1905-12-25 +1905-05-05 12:34:56.1713,1905-12-25 +1905-05-05 12:34:56.1714,1905-12-25 +1905-05-05 12:34:56.1715,1905-12-25 +1905-05-05 12:34:56.1716,1905-12-25 +1905-05-05 12:34:56.1717,1905-12-25 +1905-05-05 12:34:56.1718,1905-12-25 +1905-05-05 12:34:56.1719,1905-12-25 +1905-05-05 12:34:56.172,1905-12-25 +1905-05-05 12:34:56.1721,1905-12-25 +1905-05-05 12:34:56.1722,1905-12-25 +1905-05-05 12:34:56.1723,1905-12-25 +1905-05-05 12:34:56.1724,1905-12-25 +1905-05-05 12:34:56.1725,1905-12-25 +1905-05-05 12:34:56.1726,1905-12-25 +1905-05-05 12:34:56.1727,1905-12-25 +1905-05-05 12:34:56.1728,1905-12-25 +1905-05-05 12:34:56.1729,1905-12-25 +1905-05-05 12:34:56.173,1905-12-25 +1905-05-05 12:34:56.1731,1905-12-25 +1905-05-05 12:34:56.1732,1905-12-25 +1905-05-05 12:34:56.1733,1905-12-25 +1905-05-05 12:34:56.1734,1905-12-25 +1905-05-05 12:34:56.1735,1905-12-25 +1905-05-05 12:34:56.1736,1905-12-25 +1905-05-05 12:34:56.1737,1905-12-25 +1905-05-05 12:34:56.1738,1905-12-25 +1905-05-05 12:34:56.1739,1905-12-25 +1905-05-05 12:34:56.174,1905-12-25 +1905-05-05 12:34:56.1741,1905-12-25 +1905-05-05 12:34:56.1742,1905-12-25 +1905-05-05 12:34:56.1743,1905-12-25 +1905-05-05 12:34:56.1744,1905-12-25 +1905-05-05 12:34:56.1745,1905-12-25 +1905-05-05 12:34:56.1746,1905-12-25 +1905-05-05 12:34:56.1747,1905-12-25 +1905-05-05 12:34:56.1748,1905-12-25 +1905-05-05 12:34:56.1749,1905-12-25 +1905-05-05 12:34:56.175,1905-12-25 +1905-05-05 12:34:56.1751,1905-12-25 +1905-05-05 12:34:56.1752,1905-12-25 +1905-05-05 12:34:56.1753,1905-12-25 +1905-05-05 12:34:56.1754,1905-12-25 +1905-05-05 12:34:56.1755,1905-12-25 +1905-05-05 12:34:56.1756,1905-12-25 +1905-05-05 12:34:56.1757,1905-12-25 +1905-05-05 12:34:56.1758,1905-12-25 +1905-05-05 12:34:56.1759,1905-12-25 +1905-05-05 12:34:56.176,1905-12-25 +1905-05-05 12:34:56.1761,1905-12-25 +1905-05-05 12:34:56.1762,1905-12-25 +1905-05-05 12:34:56.1763,1905-12-25 +1905-05-05 12:34:56.1764,1905-12-25 +1905-05-05 12:34:56.1765,1905-12-25 +1905-05-05 12:34:56.1766,1905-12-25 +1905-05-05 12:34:56.1767,1905-12-25 +1905-05-05 12:34:56.1768,1905-12-25 +1905-05-05 12:34:56.1769,1905-12-25 +1905-05-05 12:34:56.177,1905-12-25 +1905-05-05 12:34:56.1771,1905-12-25 +1905-05-05 12:34:56.1772,1905-12-25 +1905-05-05 12:34:56.1773,1905-12-25 +1905-05-05 12:34:56.1774,1905-12-25 +1905-05-05 12:34:56.1775,1905-12-25 +1905-05-05 12:34:56.1776,1905-12-25 +1905-05-05 12:34:56.1777,1905-12-25 +1905-05-05 12:34:56.1778,1905-12-25 +1905-05-05 12:34:56.1779,1905-12-25 +1905-05-05 12:34:56.178,1905-12-25 +1905-05-05 12:34:56.1781,1905-12-25 +1905-05-05 12:34:56.1782,1905-12-25 +1905-05-05 12:34:56.1783,1905-12-25 +1905-05-05 12:34:56.1784,1905-12-25 +1905-05-05 12:34:56.1785,1905-12-25 +1905-05-05 12:34:56.1786,1905-12-25 +1905-05-05 12:34:56.1787,1905-12-25 +1905-05-05 12:34:56.1788,1905-12-25 +1905-05-05 12:34:56.1789,1905-12-25 +1905-05-05 12:34:56.179,1905-12-25 +1905-05-05 12:34:56.1791,1905-12-25 +1905-05-05 12:34:56.1792,1905-12-25 +1905-05-05 12:34:56.1793,1905-12-25 +1905-05-05 12:34:56.1794,1905-12-25 +1905-05-05 12:34:56.1795,1905-12-25 +1905-05-05 12:34:56.1796,1905-12-25 +1905-05-05 12:34:56.1797,1905-12-25 +1905-05-05 12:34:56.1798,1905-12-25 +1905-05-05 12:34:56.1799,1905-12-25 +1905-05-05 12:34:56.18,1905-12-25 +1905-05-05 12:34:56.1801,1905-12-25 +1905-05-05 12:34:56.1802,1905-12-25 +1905-05-05 12:34:56.1803,1905-12-25 +1905-05-05 12:34:56.1804,1905-12-25 +1905-05-05 12:34:56.1805,1905-12-25 +1905-05-05 12:34:56.1806,1905-12-25 +1905-05-05 12:34:56.1807,1905-12-25 +1905-05-05 12:34:56.1808,1905-12-25 +1905-05-05 12:34:56.1809,1905-12-25 +1905-05-05 12:34:56.181,1905-12-25 +1905-05-05 12:34:56.1811,1905-12-25 +1905-05-05 12:34:56.1812,1905-12-25 +1905-05-05 12:34:56.1813,1905-12-25 +1905-05-05 12:34:56.1814,1905-12-25 +1905-05-05 12:34:56.1815,1905-12-25 +1905-05-05 12:34:56.1816,1905-12-25 +1905-05-05 12:34:56.1817,1905-12-25 +1905-05-05 12:34:56.1818,1905-12-25 +1905-05-05 12:34:56.1819,1905-12-25 +1905-05-05 12:34:56.182,1905-12-25 +1905-05-05 12:34:56.1821,1905-12-25 +1905-05-05 12:34:56.1822,1905-12-25 +1905-05-05 12:34:56.1823,1905-12-25 +1905-05-05 12:34:56.1824,1905-12-25 +1905-05-05 12:34:56.1825,1905-12-25 +1905-05-05 12:34:56.1826,1905-12-25 +1905-05-05 12:34:56.1827,1905-12-25 +1905-05-05 12:34:56.1828,1905-12-25 +1905-05-05 12:34:56.1829,1905-12-25 +1905-05-05 12:34:56.183,1905-12-25 +1905-05-05 12:34:56.1831,1905-12-25 +1905-05-05 12:34:56.1832,1905-12-25 +1905-05-05 12:34:56.1833,1905-12-25 +1905-05-05 12:34:56.1834,1905-12-25 +1905-05-05 12:34:56.1835,1905-12-25 +1905-05-05 12:34:56.1836,1905-12-25 +1905-05-05 12:34:56.1837,1905-12-25 +1905-05-05 12:34:56.1838,1905-12-25 +1905-05-05 12:34:56.1839,1905-12-25 +1905-05-05 12:34:56.184,1905-12-25 +1905-05-05 12:34:56.1841,1905-12-25 +1905-05-05 12:34:56.1842,1905-12-25 +1905-05-05 12:34:56.1843,1905-12-25 +1905-05-05 12:34:56.1844,1905-12-25 +1905-05-05 12:34:56.1845,1905-12-25 +1905-05-05 12:34:56.1846,1905-12-25 +1905-05-05 12:34:56.1847,1905-12-25 +1905-05-05 12:34:56.1848,1905-12-25 +1905-05-05 12:34:56.1849,1905-12-25 +1905-05-05 12:34:56.185,1905-12-25 +1905-05-05 12:34:56.1851,1905-12-25 +1905-05-05 12:34:56.1852,1905-12-25 +1905-05-05 12:34:56.1853,1905-12-25 +1905-05-05 12:34:56.1854,1905-12-25 +1905-05-05 12:34:56.1855,1905-12-25 +1905-05-05 12:34:56.1856,1905-12-25 +1905-05-05 12:34:56.1857,1905-12-25 +1905-05-05 12:34:56.1858,1905-12-25 +1905-05-05 12:34:56.1859,1905-12-25 +1905-05-05 12:34:56.186,1905-12-25 +1905-05-05 12:34:56.1861,1905-12-25 +1905-05-05 12:34:56.1862,1905-12-25 +1905-05-05 12:34:56.1863,1905-12-25 +1905-05-05 12:34:56.1864,1905-12-25 +1905-05-05 12:34:56.1865,1905-12-25 +1905-05-05 12:34:56.1866,1905-12-25 +1905-05-05 12:34:56.1867,1905-12-25 +1905-05-05 12:34:56.1868,1905-12-25 +1905-05-05 12:34:56.1869,1905-12-25 +1905-05-05 12:34:56.187,1905-12-25 +1905-05-05 12:34:56.1871,1905-12-25 +1905-05-05 12:34:56.1872,1905-12-25 +1905-05-05 12:34:56.1873,1905-12-25 +1905-05-05 12:34:56.1874,1905-12-25 +1905-05-05 12:34:56.1875,1905-12-25 +1905-05-05 12:34:56.1876,1905-12-25 +1905-05-05 12:34:56.1877,1905-12-25 +1905-05-05 12:34:56.1878,1905-12-25 +1905-05-05 12:34:56.1879,1905-12-25 +1905-05-05 12:34:56.188,1905-12-25 +1905-05-05 12:34:56.1881,1905-12-25 +1905-05-05 12:34:56.1882,1905-12-25 +1905-05-05 12:34:56.1883,1905-12-25 +1905-05-05 12:34:56.1884,1905-12-25 +1905-05-05 12:34:56.1885,1905-12-25 +1905-05-05 12:34:56.1886,1905-12-25 +1905-05-05 12:34:56.1887,1905-12-25 +1905-05-05 12:34:56.1888,1905-12-25 +1905-05-05 12:34:56.1889,1905-12-25 +1905-05-05 12:34:56.189,1905-12-25 +1905-05-05 12:34:56.1891,1905-12-25 +1905-05-05 12:34:56.1892,1905-12-25 +1905-05-05 12:34:56.1893,1905-12-25 +1905-05-05 12:34:56.1894,1905-12-25 +1905-05-05 12:34:56.1895,1905-12-25 +1905-05-05 12:34:56.1896,1905-12-25 +1905-05-05 12:34:56.1897,1905-12-25 +1905-05-05 12:34:56.1898,1905-12-25 +1905-05-05 12:34:56.1899,1905-12-25 +1905-05-05 12:34:56.19,1905-12-25 +1905-05-05 12:34:56.1901,1905-12-25 +1905-05-05 12:34:56.1902,1905-12-25 +1905-05-05 12:34:56.1903,1905-12-25 +1905-05-05 12:34:56.1904,1905-12-25 +1905-05-05 12:34:56.1905,1905-12-25 +1905-05-05 12:34:56.1906,1905-12-25 +1905-05-05 12:34:56.1907,1905-12-25 +1905-05-05 12:34:56.1908,1905-12-25 +1905-05-05 12:34:56.1909,1905-12-25 +1905-05-05 12:34:56.191,1905-12-25 +1905-05-05 12:34:56.1911,1905-12-25 +1905-05-05 12:34:56.1912,1905-12-25 +1905-05-05 12:34:56.1913,1905-12-25 +1905-05-05 12:34:56.1914,1905-12-25 +1905-05-05 12:34:56.1915,1905-12-25 +1905-05-05 12:34:56.1916,1905-12-25 +1905-05-05 12:34:56.1917,1905-12-25 +1905-05-05 12:34:56.1918,1905-12-25 +1905-05-05 12:34:56.1919,1905-12-25 +1905-05-05 12:34:56.192,1905-12-25 +1905-05-05 12:34:56.1921,1905-12-25 +1905-05-05 12:34:56.1922,1905-12-25 +1905-05-05 12:34:56.1923,1905-12-25 +1905-05-05 12:34:56.1924,1905-12-25 +1905-05-05 12:34:56.1925,1905-12-25 +1905-05-05 12:34:56.1926,1905-12-25 +1905-05-05 12:34:56.1927,1905-12-25 +1905-05-05 12:34:56.1928,1905-12-25 +1905-05-05 12:34:56.1929,1905-12-25 +1905-05-05 12:34:56.193,1905-12-25 +1905-05-05 12:34:56.1931,1905-12-25 +1905-05-05 12:34:56.1932,1905-12-25 +1905-05-05 12:34:56.1933,1905-12-25 +1905-05-05 12:34:56.1934,1905-12-25 +1905-05-05 12:34:56.1935,1905-12-25 +1905-05-05 12:34:56.1936,1905-12-25 +1905-05-05 12:34:56.1937,1905-12-25 +1905-05-05 12:34:56.1938,1905-12-25 +1905-05-05 12:34:56.1939,1905-12-25 +1905-05-05 12:34:56.194,1905-12-25 +1905-05-05 12:34:56.1941,1905-12-25 +1905-05-05 12:34:56.1942,1905-12-25 +1905-05-05 12:34:56.1943,1905-12-25 +1905-05-05 12:34:56.1944,1905-12-25 +1905-05-05 12:34:56.1945,1905-12-25 +1905-05-05 12:34:56.1946,1905-12-25 +1905-05-05 12:34:56.1947,1905-12-25 +1905-05-05 12:34:56.1948,1905-12-25 +1905-05-05 12:34:56.1949,1905-12-25 +1905-05-05 12:34:56.195,1905-12-25 +1905-05-05 12:34:56.1951,1905-12-25 +1905-05-05 12:34:56.1952,1905-12-25 +1905-05-05 12:34:56.1953,1905-12-25 +1905-05-05 12:34:56.1954,1905-12-25 +1905-05-05 12:34:56.1955,1905-12-25 +1905-05-05 12:34:56.1956,1905-12-25 +1905-05-05 12:34:56.1957,1905-12-25 +1905-05-05 12:34:56.1958,1905-12-25 +1905-05-05 12:34:56.1959,1905-12-25 +1905-05-05 12:34:56.196,1905-12-25 +1905-05-05 12:34:56.1961,1905-12-25 +1905-05-05 12:34:56.1962,1905-12-25 +1905-05-05 12:34:56.1963,1905-12-25 +1905-05-05 12:34:56.1964,1905-12-25 +1905-05-05 12:34:56.1965,1905-12-25 +1905-05-05 12:34:56.1966,1905-12-25 +1905-05-05 12:34:56.1967,1905-12-25 +1905-05-05 12:34:56.1968,1905-12-25 +1905-05-05 12:34:56.1969,1905-12-25 +1905-05-05 12:34:56.197,1905-12-25 +1905-05-05 12:34:56.1971,1905-12-25 +1905-05-05 12:34:56.1972,1905-12-25 +1905-05-05 12:34:56.1973,1905-12-25 +1905-05-05 12:34:56.1974,1905-12-25 +1905-05-05 12:34:56.1975,1905-12-25 +1905-05-05 12:34:56.1976,1905-12-25 +1905-05-05 12:34:56.1977,1905-12-25 +1905-05-05 12:34:56.1978,1905-12-25 +1905-05-05 12:34:56.1979,1905-12-25 +1905-05-05 12:34:56.198,1905-12-25 +1905-05-05 12:34:56.1981,1905-12-25 +1905-05-05 12:34:56.1982,1905-12-25 +1905-05-05 12:34:56.1983,1905-12-25 +1905-05-05 12:34:56.1984,1905-12-25 +1905-05-05 12:34:56.1985,1905-12-25 +1905-05-05 12:34:56.1986,1905-12-25 +1905-05-05 12:34:56.1987,1905-12-25 +1905-05-05 12:34:56.1988,1905-12-25 +1905-05-05 12:34:56.1989,1905-12-25 +1905-05-05 12:34:56.199,1905-12-25 +1905-05-05 12:34:56.1991,1905-12-25 +1905-05-05 12:34:56.1992,1905-12-25 +1905-05-05 12:34:56.1993,1905-12-25 +1905-05-05 12:34:56.1994,1905-12-25 +1905-05-05 12:34:56.1995,1905-12-25 +1905-05-05 12:34:56.1996,1905-12-25 +1905-05-05 12:34:56.1997,1905-12-25 +1905-05-05 12:34:56.1998,1905-12-25 +1905-05-05 12:34:56.1999,1905-12-25 +1906-05-05 12:34:56.1,1906-12-25 +1906-05-05 12:34:56.1001,1906-12-25 +1906-05-05 12:34:56.1002,1906-12-25 +1906-05-05 12:34:56.1003,1906-12-25 +1906-05-05 12:34:56.1004,1906-12-25 +1906-05-05 12:34:56.1005,1906-12-25 +1906-05-05 12:34:56.1006,1906-12-25 +1906-05-05 12:34:56.1007,1906-12-25 +1906-05-05 12:34:56.1008,1906-12-25 +1906-05-05 12:34:56.1009,1906-12-25 +1906-05-05 12:34:56.101,1906-12-25 +1906-05-05 12:34:56.1011,1906-12-25 +1906-05-05 12:34:56.1012,1906-12-25 +1906-05-05 12:34:56.1013,1906-12-25 +1906-05-05 12:34:56.1014,1906-12-25 +1906-05-05 12:34:56.1015,1906-12-25 +1906-05-05 12:34:56.1016,1906-12-25 +1906-05-05 12:34:56.1017,1906-12-25 +1906-05-05 12:34:56.1018,1906-12-25 +1906-05-05 12:34:56.1019,1906-12-25 +1906-05-05 12:34:56.102,1906-12-25 +1906-05-05 12:34:56.1021,1906-12-25 +1906-05-05 12:34:56.1022,1906-12-25 +1906-05-05 12:34:56.1023,1906-12-25 +1906-05-05 12:34:56.1024,1906-12-25 +1906-05-05 12:34:56.1025,1906-12-25 +1906-05-05 12:34:56.1026,1906-12-25 +1906-05-05 12:34:56.1027,1906-12-25 +1906-05-05 12:34:56.1028,1906-12-25 +1906-05-05 12:34:56.1029,1906-12-25 +1906-05-05 12:34:56.103,1906-12-25 +1906-05-05 12:34:56.1031,1906-12-25 +1906-05-05 12:34:56.1032,1906-12-25 +1906-05-05 12:34:56.1033,1906-12-25 +1906-05-05 12:34:56.1034,1906-12-25 +1906-05-05 12:34:56.1035,1906-12-25 +1906-05-05 12:34:56.1036,1906-12-25 +1906-05-05 12:34:56.1037,1906-12-25 +1906-05-05 12:34:56.1038,1906-12-25 +1906-05-05 12:34:56.1039,1906-12-25 +1906-05-05 12:34:56.104,1906-12-25 +1906-05-05 12:34:56.1041,1906-12-25 +1906-05-05 12:34:56.1042,1906-12-25 +1906-05-05 12:34:56.1043,1906-12-25 +1906-05-05 12:34:56.1044,1906-12-25 +1906-05-05 12:34:56.1045,1906-12-25 +1906-05-05 12:34:56.1046,1906-12-25 +1906-05-05 12:34:56.1047,1906-12-25 +1906-05-05 12:34:56.1048,1906-12-25 +1906-05-05 12:34:56.1049,1906-12-25 +1906-05-05 12:34:56.105,1906-12-25 +1906-05-05 12:34:56.1051,1906-12-25 +1906-05-05 12:34:56.1052,1906-12-25 +1906-05-05 12:34:56.1053,1906-12-25 +1906-05-05 12:34:56.1054,1906-12-25 +1906-05-05 12:34:56.1055,1906-12-25 +1906-05-05 12:34:56.1056,1906-12-25 +1906-05-05 12:34:56.1057,1906-12-25 +1906-05-05 12:34:56.1058,1906-12-25 +1906-05-05 12:34:56.1059,1906-12-25 +1906-05-05 12:34:56.106,1906-12-25 +1906-05-05 12:34:56.1061,1906-12-25 +1906-05-05 12:34:56.1062,1906-12-25 +1906-05-05 12:34:56.1063,1906-12-25 +1906-05-05 12:34:56.1064,1906-12-25 +1906-05-05 12:34:56.1065,1906-12-25 +1906-05-05 12:34:56.1066,1906-12-25 +1906-05-05 12:34:56.1067,1906-12-25 +1906-05-05 12:34:56.1068,1906-12-25 +1906-05-05 12:34:56.1069,1906-12-25 +1906-05-05 12:34:56.107,1906-12-25 +1906-05-05 12:34:56.1071,1906-12-25 +1906-05-05 12:34:56.1072,1906-12-25 +1906-05-05 12:34:56.1073,1906-12-25 +1906-05-05 12:34:56.1074,1906-12-25 +1906-05-05 12:34:56.1075,1906-12-25 +1906-05-05 12:34:56.1076,1906-12-25 +1906-05-05 12:34:56.1077,1906-12-25 +1906-05-05 12:34:56.1078,1906-12-25 +1906-05-05 12:34:56.1079,1906-12-25 +1906-05-05 12:34:56.108,1906-12-25 +1906-05-05 12:34:56.1081,1906-12-25 +1906-05-05 12:34:56.1082,1906-12-25 +1906-05-05 12:34:56.1083,1906-12-25 +1906-05-05 12:34:56.1084,1906-12-25 +1906-05-05 12:34:56.1085,1906-12-25 +1906-05-05 12:34:56.1086,1906-12-25 +1906-05-05 12:34:56.1087,1906-12-25 +1906-05-05 12:34:56.1088,1906-12-25 +1906-05-05 12:34:56.1089,1906-12-25 +1906-05-05 12:34:56.109,1906-12-25 +1906-05-05 12:34:56.1091,1906-12-25 +1906-05-05 12:34:56.1092,1906-12-25 +1906-05-05 12:34:56.1093,1906-12-25 +1906-05-05 12:34:56.1094,1906-12-25 +1906-05-05 12:34:56.1095,1906-12-25 +1906-05-05 12:34:56.1096,1906-12-25 +1906-05-05 12:34:56.1097,1906-12-25 +1906-05-05 12:34:56.1098,1906-12-25 +1906-05-05 12:34:56.1099,1906-12-25 +1906-05-05 12:34:56.11,1906-12-25 +1906-05-05 12:34:56.1101,1906-12-25 +1906-05-05 12:34:56.1102,1906-12-25 +1906-05-05 12:34:56.1103,1906-12-25 +1906-05-05 12:34:56.1104,1906-12-25 +1906-05-05 12:34:56.1105,1906-12-25 +1906-05-05 12:34:56.1106,1906-12-25 +1906-05-05 12:34:56.1107,1906-12-25 +1906-05-05 12:34:56.1108,1906-12-25 +1906-05-05 12:34:56.1109,1906-12-25 +1906-05-05 12:34:56.111,1906-12-25 +1906-05-05 12:34:56.1111,1906-12-25 +1906-05-05 12:34:56.1112,1906-12-25 +1906-05-05 12:34:56.1113,1906-12-25 +1906-05-05 12:34:56.1114,1906-12-25 +1906-05-05 12:34:56.1115,1906-12-25 +1906-05-05 12:34:56.1116,1906-12-25 +1906-05-05 12:34:56.1117,1906-12-25 +1906-05-05 12:34:56.1118,1906-12-25 +1906-05-05 12:34:56.1119,1906-12-25 +1906-05-05 12:34:56.112,1906-12-25 +1906-05-05 12:34:56.1121,1906-12-25 +1906-05-05 12:34:56.1122,1906-12-25 +1906-05-05 12:34:56.1123,1906-12-25 +1906-05-05 12:34:56.1124,1906-12-25 +1906-05-05 12:34:56.1125,1906-12-25 +1906-05-05 12:34:56.1126,1906-12-25 +1906-05-05 12:34:56.1127,1906-12-25 +1906-05-05 12:34:56.1128,1906-12-25 +1906-05-05 12:34:56.1129,1906-12-25 +1906-05-05 12:34:56.113,1906-12-25 +1906-05-05 12:34:56.1131,1906-12-25 +1906-05-05 12:34:56.1132,1906-12-25 +1906-05-05 12:34:56.1133,1906-12-25 +1906-05-05 12:34:56.1134,1906-12-25 +1906-05-05 12:34:56.1135,1906-12-25 +1906-05-05 12:34:56.1136,1906-12-25 +1906-05-05 12:34:56.1137,1906-12-25 +1906-05-05 12:34:56.1138,1906-12-25 +1906-05-05 12:34:56.1139,1906-12-25 +1906-05-05 12:34:56.114,1906-12-25 +1906-05-05 12:34:56.1141,1906-12-25 +1906-05-05 12:34:56.1142,1906-12-25 +1906-05-05 12:34:56.1143,1906-12-25 +1906-05-05 12:34:56.1144,1906-12-25 +1906-05-05 12:34:56.1145,1906-12-25 +1906-05-05 12:34:56.1146,1906-12-25 +1906-05-05 12:34:56.1147,1906-12-25 +1906-05-05 12:34:56.1148,1906-12-25 +1906-05-05 12:34:56.1149,1906-12-25 +1906-05-05 12:34:56.115,1906-12-25 +1906-05-05 12:34:56.1151,1906-12-25 +1906-05-05 12:34:56.1152,1906-12-25 +1906-05-05 12:34:56.1153,1906-12-25 +1906-05-05 12:34:56.1154,1906-12-25 +1906-05-05 12:34:56.1155,1906-12-25 +1906-05-05 12:34:56.1156,1906-12-25 +1906-05-05 12:34:56.1157,1906-12-25 +1906-05-05 12:34:56.1158,1906-12-25 +1906-05-05 12:34:56.1159,1906-12-25 +1906-05-05 12:34:56.116,1906-12-25 +1906-05-05 12:34:56.1161,1906-12-25 +1906-05-05 12:34:56.1162,1906-12-25 +1906-05-05 12:34:56.1163,1906-12-25 +1906-05-05 12:34:56.1164,1906-12-25 +1906-05-05 12:34:56.1165,1906-12-25 +1906-05-05 12:34:56.1166,1906-12-25 +1906-05-05 12:34:56.1167,1906-12-25 +1906-05-05 12:34:56.1168,1906-12-25 +1906-05-05 12:34:56.1169,1906-12-25 +1906-05-05 12:34:56.117,1906-12-25 +1906-05-05 12:34:56.1171,1906-12-25 +1906-05-05 12:34:56.1172,1906-12-25 +1906-05-05 12:34:56.1173,1906-12-25 +1906-05-05 12:34:56.1174,1906-12-25 +1906-05-05 12:34:56.1175,1906-12-25 +1906-05-05 12:34:56.1176,1906-12-25 +1906-05-05 12:34:56.1177,1906-12-25 +1906-05-05 12:34:56.1178,1906-12-25 +1906-05-05 12:34:56.1179,1906-12-25 +1906-05-05 12:34:56.118,1906-12-25 +1906-05-05 12:34:56.1181,1906-12-25 +1906-05-05 12:34:56.1182,1906-12-25 +1906-05-05 12:34:56.1183,1906-12-25 +1906-05-05 12:34:56.1184,1906-12-25 +1906-05-05 12:34:56.1185,1906-12-25 +1906-05-05 12:34:56.1186,1906-12-25 +1906-05-05 12:34:56.1187,1906-12-25 +1906-05-05 12:34:56.1188,1906-12-25 +1906-05-05 12:34:56.1189,1906-12-25 +1906-05-05 12:34:56.119,1906-12-25 +1906-05-05 12:34:56.1191,1906-12-25 +1906-05-05 12:34:56.1192,1906-12-25 +1906-05-05 12:34:56.1193,1906-12-25 +1906-05-05 12:34:56.1194,1906-12-25 +1906-05-05 12:34:56.1195,1906-12-25 +1906-05-05 12:34:56.1196,1906-12-25 +1906-05-05 12:34:56.1197,1906-12-25 +1906-05-05 12:34:56.1198,1906-12-25 +1906-05-05 12:34:56.1199,1906-12-25 +1906-05-05 12:34:56.12,1906-12-25 +1906-05-05 12:34:56.1201,1906-12-25 +1906-05-05 12:34:56.1202,1906-12-25 +1906-05-05 12:34:56.1203,1906-12-25 +1906-05-05 12:34:56.1204,1906-12-25 +1906-05-05 12:34:56.1205,1906-12-25 +1906-05-05 12:34:56.1206,1906-12-25 +1906-05-05 12:34:56.1207,1906-12-25 +1906-05-05 12:34:56.1208,1906-12-25 +1906-05-05 12:34:56.1209,1906-12-25 +1906-05-05 12:34:56.121,1906-12-25 +1906-05-05 12:34:56.1211,1906-12-25 +1906-05-05 12:34:56.1212,1906-12-25 +1906-05-05 12:34:56.1213,1906-12-25 +1906-05-05 12:34:56.1214,1906-12-25 +1906-05-05 12:34:56.1215,1906-12-25 +1906-05-05 12:34:56.1216,1906-12-25 +1906-05-05 12:34:56.1217,1906-12-25 +1906-05-05 12:34:56.1218,1906-12-25 +1906-05-05 12:34:56.1219,1906-12-25 +1906-05-05 12:34:56.122,1906-12-25 +1906-05-05 12:34:56.1221,1906-12-25 +1906-05-05 12:34:56.1222,1906-12-25 +1906-05-05 12:34:56.1223,1906-12-25 +1906-05-05 12:34:56.1224,1906-12-25 +1906-05-05 12:34:56.1225,1906-12-25 +1906-05-05 12:34:56.1226,1906-12-25 +1906-05-05 12:34:56.1227,1906-12-25 +1906-05-05 12:34:56.1228,1906-12-25 +1906-05-05 12:34:56.1229,1906-12-25 +1906-05-05 12:34:56.123,1906-12-25 +1906-05-05 12:34:56.1231,1906-12-25 +1906-05-05 12:34:56.1232,1906-12-25 +1906-05-05 12:34:56.1233,1906-12-25 +1906-05-05 12:34:56.1234,1906-12-25 +1906-05-05 12:34:56.1235,1906-12-25 +1906-05-05 12:34:56.1236,1906-12-25 +1906-05-05 12:34:56.1237,1906-12-25 +1906-05-05 12:34:56.1238,1906-12-25 +1906-05-05 12:34:56.1239,1906-12-25 +1906-05-05 12:34:56.124,1906-12-25 +1906-05-05 12:34:56.1241,1906-12-25 +1906-05-05 12:34:56.1242,1906-12-25 +1906-05-05 12:34:56.1243,1906-12-25 +1906-05-05 12:34:56.1244,1906-12-25 +1906-05-05 12:34:56.1245,1906-12-25 +1906-05-05 12:34:56.1246,1906-12-25 +1906-05-05 12:34:56.1247,1906-12-25 +1906-05-05 12:34:56.1248,1906-12-25 +1906-05-05 12:34:56.1249,1906-12-25 +1906-05-05 12:34:56.125,1906-12-25 +1906-05-05 12:34:56.1251,1906-12-25 +1906-05-05 12:34:56.1252,1906-12-25 +1906-05-05 12:34:56.1253,1906-12-25 +1906-05-05 12:34:56.1254,1906-12-25 +1906-05-05 12:34:56.1255,1906-12-25 +1906-05-05 12:34:56.1256,1906-12-25 +1906-05-05 12:34:56.1257,1906-12-25 +1906-05-05 12:34:56.1258,1906-12-25 +1906-05-05 12:34:56.1259,1906-12-25 +1906-05-05 12:34:56.126,1906-12-25 +1906-05-05 12:34:56.1261,1906-12-25 +1906-05-05 12:34:56.1262,1906-12-25 +1906-05-05 12:34:56.1263,1906-12-25 +1906-05-05 12:34:56.1264,1906-12-25 +1906-05-05 12:34:56.1265,1906-12-25 +1906-05-05 12:34:56.1266,1906-12-25 +1906-05-05 12:34:56.1267,1906-12-25 +1906-05-05 12:34:56.1268,1906-12-25 +1906-05-05 12:34:56.1269,1906-12-25 +1906-05-05 12:34:56.127,1906-12-25 +1906-05-05 12:34:56.1271,1906-12-25 +1906-05-05 12:34:56.1272,1906-12-25 +1906-05-05 12:34:56.1273,1906-12-25 +1906-05-05 12:34:56.1274,1906-12-25 +1906-05-05 12:34:56.1275,1906-12-25 +1906-05-05 12:34:56.1276,1906-12-25 +1906-05-05 12:34:56.1277,1906-12-25 +1906-05-05 12:34:56.1278,1906-12-25 +1906-05-05 12:34:56.1279,1906-12-25 +1906-05-05 12:34:56.128,1906-12-25 +1906-05-05 12:34:56.1281,1906-12-25 +1906-05-05 12:34:56.1282,1906-12-25 +1906-05-05 12:34:56.1283,1906-12-25 +1906-05-05 12:34:56.1284,1906-12-25 +1906-05-05 12:34:56.1285,1906-12-25 +1906-05-05 12:34:56.1286,1906-12-25 +1906-05-05 12:34:56.1287,1906-12-25 +1906-05-05 12:34:56.1288,1906-12-25 +1906-05-05 12:34:56.1289,1906-12-25 +1906-05-05 12:34:56.129,1906-12-25 +1906-05-05 12:34:56.1291,1906-12-25 +1906-05-05 12:34:56.1292,1906-12-25 +1906-05-05 12:34:56.1293,1906-12-25 +1906-05-05 12:34:56.1294,1906-12-25 +1906-05-05 12:34:56.1295,1906-12-25 +1906-05-05 12:34:56.1296,1906-12-25 +1906-05-05 12:34:56.1297,1906-12-25 +1906-05-05 12:34:56.1298,1906-12-25 +1906-05-05 12:34:56.1299,1906-12-25 +1906-05-05 12:34:56.13,1906-12-25 +1906-05-05 12:34:56.1301,1906-12-25 +1906-05-05 12:34:56.1302,1906-12-25 +1906-05-05 12:34:56.1303,1906-12-25 +1906-05-05 12:34:56.1304,1906-12-25 +1906-05-05 12:34:56.1305,1906-12-25 +1906-05-05 12:34:56.1306,1906-12-25 +1906-05-05 12:34:56.1307,1906-12-25 +1906-05-05 12:34:56.1308,1906-12-25 +1906-05-05 12:34:56.1309,1906-12-25 +1906-05-05 12:34:56.131,1906-12-25 +1906-05-05 12:34:56.1311,1906-12-25 +1906-05-05 12:34:56.1312,1906-12-25 +1906-05-05 12:34:56.1313,1906-12-25 +1906-05-05 12:34:56.1314,1906-12-25 +1906-05-05 12:34:56.1315,1906-12-25 +1906-05-05 12:34:56.1316,1906-12-25 +1906-05-05 12:34:56.1317,1906-12-25 +1906-05-05 12:34:56.1318,1906-12-25 +1906-05-05 12:34:56.1319,1906-12-25 +1906-05-05 12:34:56.132,1906-12-25 +1906-05-05 12:34:56.1321,1906-12-25 +1906-05-05 12:34:56.1322,1906-12-25 +1906-05-05 12:34:56.1323,1906-12-25 +1906-05-05 12:34:56.1324,1906-12-25 +1906-05-05 12:34:56.1325,1906-12-25 +1906-05-05 12:34:56.1326,1906-12-25 +1906-05-05 12:34:56.1327,1906-12-25 +1906-05-05 12:34:56.1328,1906-12-25 +1906-05-05 12:34:56.1329,1906-12-25 +1906-05-05 12:34:56.133,1906-12-25 +1906-05-05 12:34:56.1331,1906-12-25 +1906-05-05 12:34:56.1332,1906-12-25 +1906-05-05 12:34:56.1333,1906-12-25 +1906-05-05 12:34:56.1334,1906-12-25 +1906-05-05 12:34:56.1335,1906-12-25 +1906-05-05 12:34:56.1336,1906-12-25 +1906-05-05 12:34:56.1337,1906-12-25 +1906-05-05 12:34:56.1338,1906-12-25 +1906-05-05 12:34:56.1339,1906-12-25 +1906-05-05 12:34:56.134,1906-12-25 +1906-05-05 12:34:56.1341,1906-12-25 +1906-05-05 12:34:56.1342,1906-12-25 +1906-05-05 12:34:56.1343,1906-12-25 +1906-05-05 12:34:56.1344,1906-12-25 +1906-05-05 12:34:56.1345,1906-12-25 +1906-05-05 12:34:56.1346,1906-12-25 +1906-05-05 12:34:56.1347,1906-12-25 +1906-05-05 12:34:56.1348,1906-12-25 +1906-05-05 12:34:56.1349,1906-12-25 +1906-05-05 12:34:56.135,1906-12-25 +1906-05-05 12:34:56.1351,1906-12-25 +1906-05-05 12:34:56.1352,1906-12-25 +1906-05-05 12:34:56.1353,1906-12-25 +1906-05-05 12:34:56.1354,1906-12-25 +1906-05-05 12:34:56.1355,1906-12-25 +1906-05-05 12:34:56.1356,1906-12-25 +1906-05-05 12:34:56.1357,1906-12-25 +1906-05-05 12:34:56.1358,1906-12-25 +1906-05-05 12:34:56.1359,1906-12-25 +1906-05-05 12:34:56.136,1906-12-25 +1906-05-05 12:34:56.1361,1906-12-25 +1906-05-05 12:34:56.1362,1906-12-25 +1906-05-05 12:34:56.1363,1906-12-25 +1906-05-05 12:34:56.1364,1906-12-25 +1906-05-05 12:34:56.1365,1906-12-25 +1906-05-05 12:34:56.1366,1906-12-25 +1906-05-05 12:34:56.1367,1906-12-25 +1906-05-05 12:34:56.1368,1906-12-25 +1906-05-05 12:34:56.1369,1906-12-25 +1906-05-05 12:34:56.137,1906-12-25 +1906-05-05 12:34:56.1371,1906-12-25 +1906-05-05 12:34:56.1372,1906-12-25 +1906-05-05 12:34:56.1373,1906-12-25 +1906-05-05 12:34:56.1374,1906-12-25 +1906-05-05 12:34:56.1375,1906-12-25 +1906-05-05 12:34:56.1376,1906-12-25 +1906-05-05 12:34:56.1377,1906-12-25 +1906-05-05 12:34:56.1378,1906-12-25 +1906-05-05 12:34:56.1379,1906-12-25 +1906-05-05 12:34:56.138,1906-12-25 +1906-05-05 12:34:56.1381,1906-12-25 +1906-05-05 12:34:56.1382,1906-12-25 +1906-05-05 12:34:56.1383,1906-12-25 +1906-05-05 12:34:56.1384,1906-12-25 +1906-05-05 12:34:56.1385,1906-12-25 +1906-05-05 12:34:56.1386,1906-12-25 +1906-05-05 12:34:56.1387,1906-12-25 +1906-05-05 12:34:56.1388,1906-12-25 +1906-05-05 12:34:56.1389,1906-12-25 +1906-05-05 12:34:56.139,1906-12-25 +1906-05-05 12:34:56.1391,1906-12-25 +1906-05-05 12:34:56.1392,1906-12-25 +1906-05-05 12:34:56.1393,1906-12-25 +1906-05-05 12:34:56.1394,1906-12-25 +1906-05-05 12:34:56.1395,1906-12-25 +1906-05-05 12:34:56.1396,1906-12-25 +1906-05-05 12:34:56.1397,1906-12-25 +1906-05-05 12:34:56.1398,1906-12-25 +1906-05-05 12:34:56.1399,1906-12-25 +1906-05-05 12:34:56.14,1906-12-25 +1906-05-05 12:34:56.1401,1906-12-25 +1906-05-05 12:34:56.1402,1906-12-25 +1906-05-05 12:34:56.1403,1906-12-25 +1906-05-05 12:34:56.1404,1906-12-25 +1906-05-05 12:34:56.1405,1906-12-25 +1906-05-05 12:34:56.1406,1906-12-25 +1906-05-05 12:34:56.1407,1906-12-25 +1906-05-05 12:34:56.1408,1906-12-25 +1906-05-05 12:34:56.1409,1906-12-25 +1906-05-05 12:34:56.141,1906-12-25 +1906-05-05 12:34:56.1411,1906-12-25 +1906-05-05 12:34:56.1412,1906-12-25 +1906-05-05 12:34:56.1413,1906-12-25 +1906-05-05 12:34:56.1414,1906-12-25 +1906-05-05 12:34:56.1415,1906-12-25 +1906-05-05 12:34:56.1416,1906-12-25 +1906-05-05 12:34:56.1417,1906-12-25 +1906-05-05 12:34:56.1418,1906-12-25 +1906-05-05 12:34:56.1419,1906-12-25 +1906-05-05 12:34:56.142,1906-12-25 +1906-05-05 12:34:56.1421,1906-12-25 +1906-05-05 12:34:56.1422,1906-12-25 +1906-05-05 12:34:56.1423,1906-12-25 +1906-05-05 12:34:56.1424,1906-12-25 +1906-05-05 12:34:56.1425,1906-12-25 +1906-05-05 12:34:56.1426,1906-12-25 +1906-05-05 12:34:56.1427,1906-12-25 +1906-05-05 12:34:56.1428,1906-12-25 +1906-05-05 12:34:56.1429,1906-12-25 +1906-05-05 12:34:56.143,1906-12-25 +1906-05-05 12:34:56.1431,1906-12-25 +1906-05-05 12:34:56.1432,1906-12-25 +1906-05-05 12:34:56.1433,1906-12-25 +1906-05-05 12:34:56.1434,1906-12-25 +1906-05-05 12:34:56.1435,1906-12-25 +1906-05-05 12:34:56.1436,1906-12-25 +1906-05-05 12:34:56.1437,1906-12-25 +1906-05-05 12:34:56.1438,1906-12-25 +1906-05-05 12:34:56.1439,1906-12-25 +1906-05-05 12:34:56.144,1906-12-25 +1906-05-05 12:34:56.1441,1906-12-25 +1906-05-05 12:34:56.1442,1906-12-25 +1906-05-05 12:34:56.1443,1906-12-25 +1906-05-05 12:34:56.1444,1906-12-25 +1906-05-05 12:34:56.1445,1906-12-25 +1906-05-05 12:34:56.1446,1906-12-25 +1906-05-05 12:34:56.1447,1906-12-25 +1906-05-05 12:34:56.1448,1906-12-25 +1906-05-05 12:34:56.1449,1906-12-25 +1906-05-05 12:34:56.145,1906-12-25 +1906-05-05 12:34:56.1451,1906-12-25 +1906-05-05 12:34:56.1452,1906-12-25 +1906-05-05 12:34:56.1453,1906-12-25 +1906-05-05 12:34:56.1454,1906-12-25 +1906-05-05 12:34:56.1455,1906-12-25 +1906-05-05 12:34:56.1456,1906-12-25 +1906-05-05 12:34:56.1457,1906-12-25 +1906-05-05 12:34:56.1458,1906-12-25 +1906-05-05 12:34:56.1459,1906-12-25 +1906-05-05 12:34:56.146,1906-12-25 +1906-05-05 12:34:56.1461,1906-12-25 +1906-05-05 12:34:56.1462,1906-12-25 +1906-05-05 12:34:56.1463,1906-12-25 +1906-05-05 12:34:56.1464,1906-12-25 +1906-05-05 12:34:56.1465,1906-12-25 +1906-05-05 12:34:56.1466,1906-12-25 +1906-05-05 12:34:56.1467,1906-12-25 +1906-05-05 12:34:56.1468,1906-12-25 +1906-05-05 12:34:56.1469,1906-12-25 +1906-05-05 12:34:56.147,1906-12-25 +1906-05-05 12:34:56.1471,1906-12-25 +1906-05-05 12:34:56.1472,1906-12-25 +1906-05-05 12:34:56.1473,1906-12-25 +1906-05-05 12:34:56.1474,1906-12-25 +1906-05-05 12:34:56.1475,1906-12-25 +1906-05-05 12:34:56.1476,1906-12-25 +1906-05-05 12:34:56.1477,1906-12-25 +1906-05-05 12:34:56.1478,1906-12-25 +1906-05-05 12:34:56.1479,1906-12-25 +1906-05-05 12:34:56.148,1906-12-25 +1906-05-05 12:34:56.1481,1906-12-25 +1906-05-05 12:34:56.1482,1906-12-25 +1906-05-05 12:34:56.1483,1906-12-25 +1906-05-05 12:34:56.1484,1906-12-25 +1906-05-05 12:34:56.1485,1906-12-25 +1906-05-05 12:34:56.1486,1906-12-25 +1906-05-05 12:34:56.1487,1906-12-25 +1906-05-05 12:34:56.1488,1906-12-25 +1906-05-05 12:34:56.1489,1906-12-25 +1906-05-05 12:34:56.149,1906-12-25 +1906-05-05 12:34:56.1491,1906-12-25 +1906-05-05 12:34:56.1492,1906-12-25 +1906-05-05 12:34:56.1493,1906-12-25 +1906-05-05 12:34:56.1494,1906-12-25 +1906-05-05 12:34:56.1495,1906-12-25 +1906-05-05 12:34:56.1496,1906-12-25 +1906-05-05 12:34:56.1497,1906-12-25 +1906-05-05 12:34:56.1498,1906-12-25 +1906-05-05 12:34:56.1499,1906-12-25 +1906-05-05 12:34:56.15,1906-12-25 +1906-05-05 12:34:56.1501,1906-12-25 +1906-05-05 12:34:56.1502,1906-12-25 +1906-05-05 12:34:56.1503,1906-12-25 +1906-05-05 12:34:56.1504,1906-12-25 +1906-05-05 12:34:56.1505,1906-12-25 +1906-05-05 12:34:56.1506,1906-12-25 +1906-05-05 12:34:56.1507,1906-12-25 +1906-05-05 12:34:56.1508,1906-12-25 +1906-05-05 12:34:56.1509,1906-12-25 +1906-05-05 12:34:56.151,1906-12-25 +1906-05-05 12:34:56.1511,1906-12-25 +1906-05-05 12:34:56.1512,1906-12-25 +1906-05-05 12:34:56.1513,1906-12-25 +1906-05-05 12:34:56.1514,1906-12-25 +1906-05-05 12:34:56.1515,1906-12-25 +1906-05-05 12:34:56.1516,1906-12-25 +1906-05-05 12:34:56.1517,1906-12-25 +1906-05-05 12:34:56.1518,1906-12-25 +1906-05-05 12:34:56.1519,1906-12-25 +1906-05-05 12:34:56.152,1906-12-25 +1906-05-05 12:34:56.1521,1906-12-25 +1906-05-05 12:34:56.1522,1906-12-25 +1906-05-05 12:34:56.1523,1906-12-25 +1906-05-05 12:34:56.1524,1906-12-25 +1906-05-05 12:34:56.1525,1906-12-25 +1906-05-05 12:34:56.1526,1906-12-25 +1906-05-05 12:34:56.1527,1906-12-25 +1906-05-05 12:34:56.1528,1906-12-25 +1906-05-05 12:34:56.1529,1906-12-25 +1906-05-05 12:34:56.153,1906-12-25 +1906-05-05 12:34:56.1531,1906-12-25 +1906-05-05 12:34:56.1532,1906-12-25 +1906-05-05 12:34:56.1533,1906-12-25 +1906-05-05 12:34:56.1534,1906-12-25 +1906-05-05 12:34:56.1535,1906-12-25 +1906-05-05 12:34:56.1536,1906-12-25 +1906-05-05 12:34:56.1537,1906-12-25 +1906-05-05 12:34:56.1538,1906-12-25 +1906-05-05 12:34:56.1539,1906-12-25 +1906-05-05 12:34:56.154,1906-12-25 +1906-05-05 12:34:56.1541,1906-12-25 +1906-05-05 12:34:56.1542,1906-12-25 +1906-05-05 12:34:56.1543,1906-12-25 +1906-05-05 12:34:56.1544,1906-12-25 +1906-05-05 12:34:56.1545,1906-12-25 +1906-05-05 12:34:56.1546,1906-12-25 +1906-05-05 12:34:56.1547,1906-12-25 +1906-05-05 12:34:56.1548,1906-12-25 +1906-05-05 12:34:56.1549,1906-12-25 +1906-05-05 12:34:56.155,1906-12-25 +1906-05-05 12:34:56.1551,1906-12-25 +1906-05-05 12:34:56.1552,1906-12-25 +1906-05-05 12:34:56.1553,1906-12-25 +1906-05-05 12:34:56.1554,1906-12-25 +1906-05-05 12:34:56.1555,1906-12-25 +1906-05-05 12:34:56.1556,1906-12-25 +1906-05-05 12:34:56.1557,1906-12-25 +1906-05-05 12:34:56.1558,1906-12-25 +1906-05-05 12:34:56.1559,1906-12-25 +1906-05-05 12:34:56.156,1906-12-25 +1906-05-05 12:34:56.1561,1906-12-25 +1906-05-05 12:34:56.1562,1906-12-25 +1906-05-05 12:34:56.1563,1906-12-25 +1906-05-05 12:34:56.1564,1906-12-25 +1906-05-05 12:34:56.1565,1906-12-25 +1906-05-05 12:34:56.1566,1906-12-25 +1906-05-05 12:34:56.1567,1906-12-25 +1906-05-05 12:34:56.1568,1906-12-25 +1906-05-05 12:34:56.1569,1906-12-25 +1906-05-05 12:34:56.157,1906-12-25 +1906-05-05 12:34:56.1571,1906-12-25 +1906-05-05 12:34:56.1572,1906-12-25 +1906-05-05 12:34:56.1573,1906-12-25 +1906-05-05 12:34:56.1574,1906-12-25 +1906-05-05 12:34:56.1575,1906-12-25 +1906-05-05 12:34:56.1576,1906-12-25 +1906-05-05 12:34:56.1577,1906-12-25 +1906-05-05 12:34:56.1578,1906-12-25 +1906-05-05 12:34:56.1579,1906-12-25 +1906-05-05 12:34:56.158,1906-12-25 +1906-05-05 12:34:56.1581,1906-12-25 +1906-05-05 12:34:56.1582,1906-12-25 +1906-05-05 12:34:56.1583,1906-12-25 +1906-05-05 12:34:56.1584,1906-12-25 +1906-05-05 12:34:56.1585,1906-12-25 +1906-05-05 12:34:56.1586,1906-12-25 +1906-05-05 12:34:56.1587,1906-12-25 +1906-05-05 12:34:56.1588,1906-12-25 +1906-05-05 12:34:56.1589,1906-12-25 +1906-05-05 12:34:56.159,1906-12-25 +1906-05-05 12:34:56.1591,1906-12-25 +1906-05-05 12:34:56.1592,1906-12-25 +1906-05-05 12:34:56.1593,1906-12-25 +1906-05-05 12:34:56.1594,1906-12-25 +1906-05-05 12:34:56.1595,1906-12-25 +1906-05-05 12:34:56.1596,1906-12-25 +1906-05-05 12:34:56.1597,1906-12-25 +1906-05-05 12:34:56.1598,1906-12-25 +1906-05-05 12:34:56.1599,1906-12-25 +1906-05-05 12:34:56.16,1906-12-25 +1906-05-05 12:34:56.1601,1906-12-25 +1906-05-05 12:34:56.1602,1906-12-25 +1906-05-05 12:34:56.1603,1906-12-25 +1906-05-05 12:34:56.1604,1906-12-25 +1906-05-05 12:34:56.1605,1906-12-25 +1906-05-05 12:34:56.1606,1906-12-25 +1906-05-05 12:34:56.1607,1906-12-25 +1906-05-05 12:34:56.1608,1906-12-25 +1906-05-05 12:34:56.1609,1906-12-25 +1906-05-05 12:34:56.161,1906-12-25 +1906-05-05 12:34:56.1611,1906-12-25 +1906-05-05 12:34:56.1612,1906-12-25 +1906-05-05 12:34:56.1613,1906-12-25 +1906-05-05 12:34:56.1614,1906-12-25 +1906-05-05 12:34:56.1615,1906-12-25 +1906-05-05 12:34:56.1616,1906-12-25 +1906-05-05 12:34:56.1617,1906-12-25 +1906-05-05 12:34:56.1618,1906-12-25 +1906-05-05 12:34:56.1619,1906-12-25 +1906-05-05 12:34:56.162,1906-12-25 +1906-05-05 12:34:56.1621,1906-12-25 +1906-05-05 12:34:56.1622,1906-12-25 +1906-05-05 12:34:56.1623,1906-12-25 +1906-05-05 12:34:56.1624,1906-12-25 +1906-05-05 12:34:56.1625,1906-12-25 +1906-05-05 12:34:56.1626,1906-12-25 +1906-05-05 12:34:56.1627,1906-12-25 +1906-05-05 12:34:56.1628,1906-12-25 +1906-05-05 12:34:56.1629,1906-12-25 +1906-05-05 12:34:56.163,1906-12-25 +1906-05-05 12:34:56.1631,1906-12-25 +1906-05-05 12:34:56.1632,1906-12-25 +1906-05-05 12:34:56.1633,1906-12-25 +1906-05-05 12:34:56.1634,1906-12-25 +1906-05-05 12:34:56.1635,1906-12-25 +1906-05-05 12:34:56.1636,1906-12-25 +1906-05-05 12:34:56.1637,1906-12-25 +1906-05-05 12:34:56.1638,1906-12-25 +1906-05-05 12:34:56.1639,1906-12-25 +1906-05-05 12:34:56.164,1906-12-25 +1906-05-05 12:34:56.1641,1906-12-25 +1906-05-05 12:34:56.1642,1906-12-25 +1906-05-05 12:34:56.1643,1906-12-25 +1906-05-05 12:34:56.1644,1906-12-25 +1906-05-05 12:34:56.1645,1906-12-25 +1906-05-05 12:34:56.1646,1906-12-25 +1906-05-05 12:34:56.1647,1906-12-25 +1906-05-05 12:34:56.1648,1906-12-25 +1906-05-05 12:34:56.1649,1906-12-25 +1906-05-05 12:34:56.165,1906-12-25 +1906-05-05 12:34:56.1651,1906-12-25 +1906-05-05 12:34:56.1652,1906-12-25 +1906-05-05 12:34:56.1653,1906-12-25 +1906-05-05 12:34:56.1654,1906-12-25 +1906-05-05 12:34:56.1655,1906-12-25 +1906-05-05 12:34:56.1656,1906-12-25 +1906-05-05 12:34:56.1657,1906-12-25 +1906-05-05 12:34:56.1658,1906-12-25 +1906-05-05 12:34:56.1659,1906-12-25 +1906-05-05 12:34:56.166,1906-12-25 +1906-05-05 12:34:56.1661,1906-12-25 +1906-05-05 12:34:56.1662,1906-12-25 +1906-05-05 12:34:56.1663,1906-12-25 +1906-05-05 12:34:56.1664,1906-12-25 +1906-05-05 12:34:56.1665,1906-12-25 +1906-05-05 12:34:56.1666,1906-12-25 +1906-05-05 12:34:56.1667,1906-12-25 +1906-05-05 12:34:56.1668,1906-12-25 +1906-05-05 12:34:56.1669,1906-12-25 +1906-05-05 12:34:56.167,1906-12-25 +1906-05-05 12:34:56.1671,1906-12-25 +1906-05-05 12:34:56.1672,1906-12-25 +1906-05-05 12:34:56.1673,1906-12-25 +1906-05-05 12:34:56.1674,1906-12-25 +1906-05-05 12:34:56.1675,1906-12-25 +1906-05-05 12:34:56.1676,1906-12-25 +1906-05-05 12:34:56.1677,1906-12-25 +1906-05-05 12:34:56.1678,1906-12-25 +1906-05-05 12:34:56.1679,1906-12-25 +1906-05-05 12:34:56.168,1906-12-25 +1906-05-05 12:34:56.1681,1906-12-25 +1906-05-05 12:34:56.1682,1906-12-25 +1906-05-05 12:34:56.1683,1906-12-25 +1906-05-05 12:34:56.1684,1906-12-25 +1906-05-05 12:34:56.1685,1906-12-25 +1906-05-05 12:34:56.1686,1906-12-25 +1906-05-05 12:34:56.1687,1906-12-25 +1906-05-05 12:34:56.1688,1906-12-25 +1906-05-05 12:34:56.1689,1906-12-25 +1906-05-05 12:34:56.169,1906-12-25 +1906-05-05 12:34:56.1691,1906-12-25 +1906-05-05 12:34:56.1692,1906-12-25 +1906-05-05 12:34:56.1693,1906-12-25 +1906-05-05 12:34:56.1694,1906-12-25 +1906-05-05 12:34:56.1695,1906-12-25 +1906-05-05 12:34:56.1696,1906-12-25 +1906-05-05 12:34:56.1697,1906-12-25 +1906-05-05 12:34:56.1698,1906-12-25 +1906-05-05 12:34:56.1699,1906-12-25 +1906-05-05 12:34:56.17,1906-12-25 +1906-05-05 12:34:56.1701,1906-12-25 +1906-05-05 12:34:56.1702,1906-12-25 +1906-05-05 12:34:56.1703,1906-12-25 +1906-05-05 12:34:56.1704,1906-12-25 +1906-05-05 12:34:56.1705,1906-12-25 +1906-05-05 12:34:56.1706,1906-12-25 +1906-05-05 12:34:56.1707,1906-12-25 +1906-05-05 12:34:56.1708,1906-12-25 +1906-05-05 12:34:56.1709,1906-12-25 +1906-05-05 12:34:56.171,1906-12-25 +1906-05-05 12:34:56.1711,1906-12-25 +1906-05-05 12:34:56.1712,1906-12-25 +1906-05-05 12:34:56.1713,1906-12-25 +1906-05-05 12:34:56.1714,1906-12-25 +1906-05-05 12:34:56.1715,1906-12-25 +1906-05-05 12:34:56.1716,1906-12-25 +1906-05-05 12:34:56.1717,1906-12-25 +1906-05-05 12:34:56.1718,1906-12-25 +1906-05-05 12:34:56.1719,1906-12-25 +1906-05-05 12:34:56.172,1906-12-25 +1906-05-05 12:34:56.1721,1906-12-25 +1906-05-05 12:34:56.1722,1906-12-25 +1906-05-05 12:34:56.1723,1906-12-25 +1906-05-05 12:34:56.1724,1906-12-25 +1906-05-05 12:34:56.1725,1906-12-25 +1906-05-05 12:34:56.1726,1906-12-25 +1906-05-05 12:34:56.1727,1906-12-25 +1906-05-05 12:34:56.1728,1906-12-25 +1906-05-05 12:34:56.1729,1906-12-25 +1906-05-05 12:34:56.173,1906-12-25 +1906-05-05 12:34:56.1731,1906-12-25 +1906-05-05 12:34:56.1732,1906-12-25 +1906-05-05 12:34:56.1733,1906-12-25 +1906-05-05 12:34:56.1734,1906-12-25 +1906-05-05 12:34:56.1735,1906-12-25 +1906-05-05 12:34:56.1736,1906-12-25 +1906-05-05 12:34:56.1737,1906-12-25 +1906-05-05 12:34:56.1738,1906-12-25 +1906-05-05 12:34:56.1739,1906-12-25 +1906-05-05 12:34:56.174,1906-12-25 +1906-05-05 12:34:56.1741,1906-12-25 +1906-05-05 12:34:56.1742,1906-12-25 +1906-05-05 12:34:56.1743,1906-12-25 +1906-05-05 12:34:56.1744,1906-12-25 +1906-05-05 12:34:56.1745,1906-12-25 +1906-05-05 12:34:56.1746,1906-12-25 +1906-05-05 12:34:56.1747,1906-12-25 +1906-05-05 12:34:56.1748,1906-12-25 +1906-05-05 12:34:56.1749,1906-12-25 +1906-05-05 12:34:56.175,1906-12-25 +1906-05-05 12:34:56.1751,1906-12-25 +1906-05-05 12:34:56.1752,1906-12-25 +1906-05-05 12:34:56.1753,1906-12-25 +1906-05-05 12:34:56.1754,1906-12-25 +1906-05-05 12:34:56.1755,1906-12-25 +1906-05-05 12:34:56.1756,1906-12-25 +1906-05-05 12:34:56.1757,1906-12-25 +1906-05-05 12:34:56.1758,1906-12-25 +1906-05-05 12:34:56.1759,1906-12-25 +1906-05-05 12:34:56.176,1906-12-25 +1906-05-05 12:34:56.1761,1906-12-25 +1906-05-05 12:34:56.1762,1906-12-25 +1906-05-05 12:34:56.1763,1906-12-25 +1906-05-05 12:34:56.1764,1906-12-25 +1906-05-05 12:34:56.1765,1906-12-25 +1906-05-05 12:34:56.1766,1906-12-25 +1906-05-05 12:34:56.1767,1906-12-25 +1906-05-05 12:34:56.1768,1906-12-25 +1906-05-05 12:34:56.1769,1906-12-25 +1906-05-05 12:34:56.177,1906-12-25 +1906-05-05 12:34:56.1771,1906-12-25 +1906-05-05 12:34:56.1772,1906-12-25 +1906-05-05 12:34:56.1773,1906-12-25 +1906-05-05 12:34:56.1774,1906-12-25 +1906-05-05 12:34:56.1775,1906-12-25 +1906-05-05 12:34:56.1776,1906-12-25 +1906-05-05 12:34:56.1777,1906-12-25 +1906-05-05 12:34:56.1778,1906-12-25 +1906-05-05 12:34:56.1779,1906-12-25 +1906-05-05 12:34:56.178,1906-12-25 +1906-05-05 12:34:56.1781,1906-12-25 +1906-05-05 12:34:56.1782,1906-12-25 +1906-05-05 12:34:56.1783,1906-12-25 +1906-05-05 12:34:56.1784,1906-12-25 +1906-05-05 12:34:56.1785,1906-12-25 +1906-05-05 12:34:56.1786,1906-12-25 +1906-05-05 12:34:56.1787,1906-12-25 +1906-05-05 12:34:56.1788,1906-12-25 +1906-05-05 12:34:56.1789,1906-12-25 +1906-05-05 12:34:56.179,1906-12-25 +1906-05-05 12:34:56.1791,1906-12-25 +1906-05-05 12:34:56.1792,1906-12-25 +1906-05-05 12:34:56.1793,1906-12-25 +1906-05-05 12:34:56.1794,1906-12-25 +1906-05-05 12:34:56.1795,1906-12-25 +1906-05-05 12:34:56.1796,1906-12-25 +1906-05-05 12:34:56.1797,1906-12-25 +1906-05-05 12:34:56.1798,1906-12-25 +1906-05-05 12:34:56.1799,1906-12-25 +1906-05-05 12:34:56.18,1906-12-25 +1906-05-05 12:34:56.1801,1906-12-25 +1906-05-05 12:34:56.1802,1906-12-25 +1906-05-05 12:34:56.1803,1906-12-25 +1906-05-05 12:34:56.1804,1906-12-25 +1906-05-05 12:34:56.1805,1906-12-25 +1906-05-05 12:34:56.1806,1906-12-25 +1906-05-05 12:34:56.1807,1906-12-25 +1906-05-05 12:34:56.1808,1906-12-25 +1906-05-05 12:34:56.1809,1906-12-25 +1906-05-05 12:34:56.181,1906-12-25 +1906-05-05 12:34:56.1811,1906-12-25 +1906-05-05 12:34:56.1812,1906-12-25 +1906-05-05 12:34:56.1813,1906-12-25 +1906-05-05 12:34:56.1814,1906-12-25 +1906-05-05 12:34:56.1815,1906-12-25 +1906-05-05 12:34:56.1816,1906-12-25 +1906-05-05 12:34:56.1817,1906-12-25 +1906-05-05 12:34:56.1818,1906-12-25 +1906-05-05 12:34:56.1819,1906-12-25 +1906-05-05 12:34:56.182,1906-12-25 +1906-05-05 12:34:56.1821,1906-12-25 +1906-05-05 12:34:56.1822,1906-12-25 +1906-05-05 12:34:56.1823,1906-12-25 +1906-05-05 12:34:56.1824,1906-12-25 +1906-05-05 12:34:56.1825,1906-12-25 +1906-05-05 12:34:56.1826,1906-12-25 +1906-05-05 12:34:56.1827,1906-12-25 +1906-05-05 12:34:56.1828,1906-12-25 +1906-05-05 12:34:56.1829,1906-12-25 +1906-05-05 12:34:56.183,1906-12-25 +1906-05-05 12:34:56.1831,1906-12-25 +1906-05-05 12:34:56.1832,1906-12-25 +1906-05-05 12:34:56.1833,1906-12-25 +1906-05-05 12:34:56.1834,1906-12-25 +1906-05-05 12:34:56.1835,1906-12-25 +1906-05-05 12:34:56.1836,1906-12-25 +1906-05-05 12:34:56.1837,1906-12-25 +1906-05-05 12:34:56.1838,1906-12-25 +1906-05-05 12:34:56.1839,1906-12-25 +1906-05-05 12:34:56.184,1906-12-25 +1906-05-05 12:34:56.1841,1906-12-25 +1906-05-05 12:34:56.1842,1906-12-25 +1906-05-05 12:34:56.1843,1906-12-25 +1906-05-05 12:34:56.1844,1906-12-25 +1906-05-05 12:34:56.1845,1906-12-25 +1906-05-05 12:34:56.1846,1906-12-25 +1906-05-05 12:34:56.1847,1906-12-25 +1906-05-05 12:34:56.1848,1906-12-25 +1906-05-05 12:34:56.1849,1906-12-25 +1906-05-05 12:34:56.185,1906-12-25 +1906-05-05 12:34:56.1851,1906-12-25 +1906-05-05 12:34:56.1852,1906-12-25 +1906-05-05 12:34:56.1853,1906-12-25 +1906-05-05 12:34:56.1854,1906-12-25 +1906-05-05 12:34:56.1855,1906-12-25 +1906-05-05 12:34:56.1856,1906-12-25 +1906-05-05 12:34:56.1857,1906-12-25 +1906-05-05 12:34:56.1858,1906-12-25 +1906-05-05 12:34:56.1859,1906-12-25 +1906-05-05 12:34:56.186,1906-12-25 +1906-05-05 12:34:56.1861,1906-12-25 +1906-05-05 12:34:56.1862,1906-12-25 +1906-05-05 12:34:56.1863,1906-12-25 +1906-05-05 12:34:56.1864,1906-12-25 +1906-05-05 12:34:56.1865,1906-12-25 +1906-05-05 12:34:56.1866,1906-12-25 +1906-05-05 12:34:56.1867,1906-12-25 +1906-05-05 12:34:56.1868,1906-12-25 +1906-05-05 12:34:56.1869,1906-12-25 +1906-05-05 12:34:56.187,1906-12-25 +1906-05-05 12:34:56.1871,1906-12-25 +1906-05-05 12:34:56.1872,1906-12-25 +1906-05-05 12:34:56.1873,1906-12-25 +1906-05-05 12:34:56.1874,1906-12-25 +1906-05-05 12:34:56.1875,1906-12-25 +1906-05-05 12:34:56.1876,1906-12-25 +1906-05-05 12:34:56.1877,1906-12-25 +1906-05-05 12:34:56.1878,1906-12-25 +1906-05-05 12:34:56.1879,1906-12-25 +1906-05-05 12:34:56.188,1906-12-25 +1906-05-05 12:34:56.1881,1906-12-25 +1906-05-05 12:34:56.1882,1906-12-25 +1906-05-05 12:34:56.1883,1906-12-25 +1906-05-05 12:34:56.1884,1906-12-25 +1906-05-05 12:34:56.1885,1906-12-25 +1906-05-05 12:34:56.1886,1906-12-25 +1906-05-05 12:34:56.1887,1906-12-25 +1906-05-05 12:34:56.1888,1906-12-25 +1906-05-05 12:34:56.1889,1906-12-25 +1906-05-05 12:34:56.189,1906-12-25 +1906-05-05 12:34:56.1891,1906-12-25 +1906-05-05 12:34:56.1892,1906-12-25 +1906-05-05 12:34:56.1893,1906-12-25 +1906-05-05 12:34:56.1894,1906-12-25 +1906-05-05 12:34:56.1895,1906-12-25 +1906-05-05 12:34:56.1896,1906-12-25 +1906-05-05 12:34:56.1897,1906-12-25 +1906-05-05 12:34:56.1898,1906-12-25 +1906-05-05 12:34:56.1899,1906-12-25 +1906-05-05 12:34:56.19,1906-12-25 +1906-05-05 12:34:56.1901,1906-12-25 +1906-05-05 12:34:56.1902,1906-12-25 +1906-05-05 12:34:56.1903,1906-12-25 +1906-05-05 12:34:56.1904,1906-12-25 +1906-05-05 12:34:56.1905,1906-12-25 +1906-05-05 12:34:56.1906,1906-12-25 +1906-05-05 12:34:56.1907,1906-12-25 +1906-05-05 12:34:56.1908,1906-12-25 +1906-05-05 12:34:56.1909,1906-12-25 +1906-05-05 12:34:56.191,1906-12-25 +1906-05-05 12:34:56.1911,1906-12-25 +1906-05-05 12:34:56.1912,1906-12-25 +1906-05-05 12:34:56.1913,1906-12-25 +1906-05-05 12:34:56.1914,1906-12-25 +1906-05-05 12:34:56.1915,1906-12-25 +1906-05-05 12:34:56.1916,1906-12-25 +1906-05-05 12:34:56.1917,1906-12-25 +1906-05-05 12:34:56.1918,1906-12-25 +1906-05-05 12:34:56.1919,1906-12-25 +1906-05-05 12:34:56.192,1906-12-25 +1906-05-05 12:34:56.1921,1906-12-25 +1906-05-05 12:34:56.1922,1906-12-25 +1906-05-05 12:34:56.1923,1906-12-25 +1906-05-05 12:34:56.1924,1906-12-25 +1906-05-05 12:34:56.1925,1906-12-25 +1906-05-05 12:34:56.1926,1906-12-25 +1906-05-05 12:34:56.1927,1906-12-25 +1906-05-05 12:34:56.1928,1906-12-25 +1906-05-05 12:34:56.1929,1906-12-25 +1906-05-05 12:34:56.193,1906-12-25 +1906-05-05 12:34:56.1931,1906-12-25 +1906-05-05 12:34:56.1932,1906-12-25 +1906-05-05 12:34:56.1933,1906-12-25 +1906-05-05 12:34:56.1934,1906-12-25 +1906-05-05 12:34:56.1935,1906-12-25 +1906-05-05 12:34:56.1936,1906-12-25 +1906-05-05 12:34:56.1937,1906-12-25 +1906-05-05 12:34:56.1938,1906-12-25 +1906-05-05 12:34:56.1939,1906-12-25 +1906-05-05 12:34:56.194,1906-12-25 +1906-05-05 12:34:56.1941,1906-12-25 +1906-05-05 12:34:56.1942,1906-12-25 +1906-05-05 12:34:56.1943,1906-12-25 +1906-05-05 12:34:56.1944,1906-12-25 +1906-05-05 12:34:56.1945,1906-12-25 +1906-05-05 12:34:56.1946,1906-12-25 +1906-05-05 12:34:56.1947,1906-12-25 +1906-05-05 12:34:56.1948,1906-12-25 +1906-05-05 12:34:56.1949,1906-12-25 +1906-05-05 12:34:56.195,1906-12-25 +1906-05-05 12:34:56.1951,1906-12-25 +1906-05-05 12:34:56.1952,1906-12-25 +1906-05-05 12:34:56.1953,1906-12-25 +1906-05-05 12:34:56.1954,1906-12-25 +1906-05-05 12:34:56.1955,1906-12-25 +1906-05-05 12:34:56.1956,1906-12-25 +1906-05-05 12:34:56.1957,1906-12-25 +1906-05-05 12:34:56.1958,1906-12-25 +1906-05-05 12:34:56.1959,1906-12-25 +1906-05-05 12:34:56.196,1906-12-25 +1906-05-05 12:34:56.1961,1906-12-25 +1906-05-05 12:34:56.1962,1906-12-25 +1906-05-05 12:34:56.1963,1906-12-25 +1906-05-05 12:34:56.1964,1906-12-25 +1906-05-05 12:34:56.1965,1906-12-25 +1906-05-05 12:34:56.1966,1906-12-25 +1906-05-05 12:34:56.1967,1906-12-25 +1906-05-05 12:34:56.1968,1906-12-25 +1906-05-05 12:34:56.1969,1906-12-25 +1906-05-05 12:34:56.197,1906-12-25 +1906-05-05 12:34:56.1971,1906-12-25 +1906-05-05 12:34:56.1972,1906-12-25 +1906-05-05 12:34:56.1973,1906-12-25 +1906-05-05 12:34:56.1974,1906-12-25 +1906-05-05 12:34:56.1975,1906-12-25 +1906-05-05 12:34:56.1976,1906-12-25 +1906-05-05 12:34:56.1977,1906-12-25 +1906-05-05 12:34:56.1978,1906-12-25 +1906-05-05 12:34:56.1979,1906-12-25 +1906-05-05 12:34:56.198,1906-12-25 +1906-05-05 12:34:56.1981,1906-12-25 +1906-05-05 12:34:56.1982,1906-12-25 +1906-05-05 12:34:56.1983,1906-12-25 +1906-05-05 12:34:56.1984,1906-12-25 +1906-05-05 12:34:56.1985,1906-12-25 +1906-05-05 12:34:56.1986,1906-12-25 +1906-05-05 12:34:56.1987,1906-12-25 +1906-05-05 12:34:56.1988,1906-12-25 +1906-05-05 12:34:56.1989,1906-12-25 +1906-05-05 12:34:56.199,1906-12-25 +1906-05-05 12:34:56.1991,1906-12-25 +1906-05-05 12:34:56.1992,1906-12-25 +1906-05-05 12:34:56.1993,1906-12-25 +1906-05-05 12:34:56.1994,1906-12-25 +1906-05-05 12:34:56.1995,1906-12-25 +1906-05-05 12:34:56.1996,1906-12-25 +1906-05-05 12:34:56.1997,1906-12-25 +1906-05-05 12:34:56.1998,1906-12-25 +1906-05-05 12:34:56.1999,1906-12-25 +1907-05-05 12:34:56.1,1907-12-25 +1907-05-05 12:34:56.1001,1907-12-25 +1907-05-05 12:34:56.1002,1907-12-25 +1907-05-05 12:34:56.1003,1907-12-25 +1907-05-05 12:34:56.1004,1907-12-25 +1907-05-05 12:34:56.1005,1907-12-25 +1907-05-05 12:34:56.1006,1907-12-25 +1907-05-05 12:34:56.1007,1907-12-25 +1907-05-05 12:34:56.1008,1907-12-25 +1907-05-05 12:34:56.1009,1907-12-25 +1907-05-05 12:34:56.101,1907-12-25 +1907-05-05 12:34:56.1011,1907-12-25 +1907-05-05 12:34:56.1012,1907-12-25 +1907-05-05 12:34:56.1013,1907-12-25 +1907-05-05 12:34:56.1014,1907-12-25 +1907-05-05 12:34:56.1015,1907-12-25 +1907-05-05 12:34:56.1016,1907-12-25 +1907-05-05 12:34:56.1017,1907-12-25 +1907-05-05 12:34:56.1018,1907-12-25 +1907-05-05 12:34:56.1019,1907-12-25 +1907-05-05 12:34:56.102,1907-12-25 +1907-05-05 12:34:56.1021,1907-12-25 +1907-05-05 12:34:56.1022,1907-12-25 +1907-05-05 12:34:56.1023,1907-12-25 +1907-05-05 12:34:56.1024,1907-12-25 +1907-05-05 12:34:56.1025,1907-12-25 +1907-05-05 12:34:56.1026,1907-12-25 +1907-05-05 12:34:56.1027,1907-12-25 +1907-05-05 12:34:56.1028,1907-12-25 +1907-05-05 12:34:56.1029,1907-12-25 +1907-05-05 12:34:56.103,1907-12-25 +1907-05-05 12:34:56.1031,1907-12-25 +1907-05-05 12:34:56.1032,1907-12-25 +1907-05-05 12:34:56.1033,1907-12-25 +1907-05-05 12:34:56.1034,1907-12-25 +1907-05-05 12:34:56.1035,1907-12-25 +1907-05-05 12:34:56.1036,1907-12-25 +1907-05-05 12:34:56.1037,1907-12-25 +1907-05-05 12:34:56.1038,1907-12-25 +1907-05-05 12:34:56.1039,1907-12-25 +1907-05-05 12:34:56.104,1907-12-25 +1907-05-05 12:34:56.1041,1907-12-25 +1907-05-05 12:34:56.1042,1907-12-25 +1907-05-05 12:34:56.1043,1907-12-25 +1907-05-05 12:34:56.1044,1907-12-25 +1907-05-05 12:34:56.1045,1907-12-25 +1907-05-05 12:34:56.1046,1907-12-25 +1907-05-05 12:34:56.1047,1907-12-25 +1907-05-05 12:34:56.1048,1907-12-25 +1907-05-05 12:34:56.1049,1907-12-25 +1907-05-05 12:34:56.105,1907-12-25 +1907-05-05 12:34:56.1051,1907-12-25 +1907-05-05 12:34:56.1052,1907-12-25 +1907-05-05 12:34:56.1053,1907-12-25 +1907-05-05 12:34:56.1054,1907-12-25 +1907-05-05 12:34:56.1055,1907-12-25 +1907-05-05 12:34:56.1056,1907-12-25 +1907-05-05 12:34:56.1057,1907-12-25 +1907-05-05 12:34:56.1058,1907-12-25 +1907-05-05 12:34:56.1059,1907-12-25 +1907-05-05 12:34:56.106,1907-12-25 +1907-05-05 12:34:56.1061,1907-12-25 +1907-05-05 12:34:56.1062,1907-12-25 +1907-05-05 12:34:56.1063,1907-12-25 +1907-05-05 12:34:56.1064,1907-12-25 +1907-05-05 12:34:56.1065,1907-12-25 +1907-05-05 12:34:56.1066,1907-12-25 +1907-05-05 12:34:56.1067,1907-12-25 +1907-05-05 12:34:56.1068,1907-12-25 +1907-05-05 12:34:56.1069,1907-12-25 +1907-05-05 12:34:56.107,1907-12-25 +1907-05-05 12:34:56.1071,1907-12-25 +1907-05-05 12:34:56.1072,1907-12-25 +1907-05-05 12:34:56.1073,1907-12-25 +1907-05-05 12:34:56.1074,1907-12-25 +1907-05-05 12:34:56.1075,1907-12-25 +1907-05-05 12:34:56.1076,1907-12-25 +1907-05-05 12:34:56.1077,1907-12-25 +1907-05-05 12:34:56.1078,1907-12-25 +1907-05-05 12:34:56.1079,1907-12-25 +1907-05-05 12:34:56.108,1907-12-25 +1907-05-05 12:34:56.1081,1907-12-25 +1907-05-05 12:34:56.1082,1907-12-25 +1907-05-05 12:34:56.1083,1907-12-25 +1907-05-05 12:34:56.1084,1907-12-25 +1907-05-05 12:34:56.1085,1907-12-25 +1907-05-05 12:34:56.1086,1907-12-25 +1907-05-05 12:34:56.1087,1907-12-25 +1907-05-05 12:34:56.1088,1907-12-25 +1907-05-05 12:34:56.1089,1907-12-25 +1907-05-05 12:34:56.109,1907-12-25 +1907-05-05 12:34:56.1091,1907-12-25 +1907-05-05 12:34:56.1092,1907-12-25 +1907-05-05 12:34:56.1093,1907-12-25 +1907-05-05 12:34:56.1094,1907-12-25 +1907-05-05 12:34:56.1095,1907-12-25 +1907-05-05 12:34:56.1096,1907-12-25 +1907-05-05 12:34:56.1097,1907-12-25 +1907-05-05 12:34:56.1098,1907-12-25 +1907-05-05 12:34:56.1099,1907-12-25 +1907-05-05 12:34:56.11,1907-12-25 +1907-05-05 12:34:56.1101,1907-12-25 +1907-05-05 12:34:56.1102,1907-12-25 +1907-05-05 12:34:56.1103,1907-12-25 +1907-05-05 12:34:56.1104,1907-12-25 +1907-05-05 12:34:56.1105,1907-12-25 +1907-05-05 12:34:56.1106,1907-12-25 +1907-05-05 12:34:56.1107,1907-12-25 +1907-05-05 12:34:56.1108,1907-12-25 +1907-05-05 12:34:56.1109,1907-12-25 +1907-05-05 12:34:56.111,1907-12-25 +1907-05-05 12:34:56.1111,1907-12-25 +1907-05-05 12:34:56.1112,1907-12-25 +1907-05-05 12:34:56.1113,1907-12-25 +1907-05-05 12:34:56.1114,1907-12-25 +1907-05-05 12:34:56.1115,1907-12-25 +1907-05-05 12:34:56.1116,1907-12-25 +1907-05-05 12:34:56.1117,1907-12-25 +1907-05-05 12:34:56.1118,1907-12-25 +1907-05-05 12:34:56.1119,1907-12-25 +1907-05-05 12:34:56.112,1907-12-25 +1907-05-05 12:34:56.1121,1907-12-25 +1907-05-05 12:34:56.1122,1907-12-25 +1907-05-05 12:34:56.1123,1907-12-25 +1907-05-05 12:34:56.1124,1907-12-25 +1907-05-05 12:34:56.1125,1907-12-25 +1907-05-05 12:34:56.1126,1907-12-25 +1907-05-05 12:34:56.1127,1907-12-25 +1907-05-05 12:34:56.1128,1907-12-25 +1907-05-05 12:34:56.1129,1907-12-25 +1907-05-05 12:34:56.113,1907-12-25 +1907-05-05 12:34:56.1131,1907-12-25 +1907-05-05 12:34:56.1132,1907-12-25 +1907-05-05 12:34:56.1133,1907-12-25 +1907-05-05 12:34:56.1134,1907-12-25 +1907-05-05 12:34:56.1135,1907-12-25 +1907-05-05 12:34:56.1136,1907-12-25 +1907-05-05 12:34:56.1137,1907-12-25 +1907-05-05 12:34:56.1138,1907-12-25 +1907-05-05 12:34:56.1139,1907-12-25 +1907-05-05 12:34:56.114,1907-12-25 +1907-05-05 12:34:56.1141,1907-12-25 +1907-05-05 12:34:56.1142,1907-12-25 +1907-05-05 12:34:56.1143,1907-12-25 +1907-05-05 12:34:56.1144,1907-12-25 +1907-05-05 12:34:56.1145,1907-12-25 +1907-05-05 12:34:56.1146,1907-12-25 +1907-05-05 12:34:56.1147,1907-12-25 +1907-05-05 12:34:56.1148,1907-12-25 +1907-05-05 12:34:56.1149,1907-12-25 +1907-05-05 12:34:56.115,1907-12-25 +1907-05-05 12:34:56.1151,1907-12-25 +1907-05-05 12:34:56.1152,1907-12-25 +1907-05-05 12:34:56.1153,1907-12-25 +1907-05-05 12:34:56.1154,1907-12-25 +1907-05-05 12:34:56.1155,1907-12-25 +1907-05-05 12:34:56.1156,1907-12-25 +1907-05-05 12:34:56.1157,1907-12-25 +1907-05-05 12:34:56.1158,1907-12-25 +1907-05-05 12:34:56.1159,1907-12-25 +1907-05-05 12:34:56.116,1907-12-25 +1907-05-05 12:34:56.1161,1907-12-25 +1907-05-05 12:34:56.1162,1907-12-25 +1907-05-05 12:34:56.1163,1907-12-25 +1907-05-05 12:34:56.1164,1907-12-25 +1907-05-05 12:34:56.1165,1907-12-25 +1907-05-05 12:34:56.1166,1907-12-25 +1907-05-05 12:34:56.1167,1907-12-25 +1907-05-05 12:34:56.1168,1907-12-25 +1907-05-05 12:34:56.1169,1907-12-25 +1907-05-05 12:34:56.117,1907-12-25 +1907-05-05 12:34:56.1171,1907-12-25 +1907-05-05 12:34:56.1172,1907-12-25 +1907-05-05 12:34:56.1173,1907-12-25 +1907-05-05 12:34:56.1174,1907-12-25 +1907-05-05 12:34:56.1175,1907-12-25 +1907-05-05 12:34:56.1176,1907-12-25 +1907-05-05 12:34:56.1177,1907-12-25 +1907-05-05 12:34:56.1178,1907-12-25 +1907-05-05 12:34:56.1179,1907-12-25 +1907-05-05 12:34:56.118,1907-12-25 +1907-05-05 12:34:56.1181,1907-12-25 +1907-05-05 12:34:56.1182,1907-12-25 +1907-05-05 12:34:56.1183,1907-12-25 +1907-05-05 12:34:56.1184,1907-12-25 +1907-05-05 12:34:56.1185,1907-12-25 +1907-05-05 12:34:56.1186,1907-12-25 +1907-05-05 12:34:56.1187,1907-12-25 +1907-05-05 12:34:56.1188,1907-12-25 +1907-05-05 12:34:56.1189,1907-12-25 +1907-05-05 12:34:56.119,1907-12-25 +1907-05-05 12:34:56.1191,1907-12-25 +1907-05-05 12:34:56.1192,1907-12-25 +1907-05-05 12:34:56.1193,1907-12-25 +1907-05-05 12:34:56.1194,1907-12-25 +1907-05-05 12:34:56.1195,1907-12-25 +1907-05-05 12:34:56.1196,1907-12-25 +1907-05-05 12:34:56.1197,1907-12-25 +1907-05-05 12:34:56.1198,1907-12-25 +1907-05-05 12:34:56.1199,1907-12-25 +1907-05-05 12:34:56.12,1907-12-25 +1907-05-05 12:34:56.1201,1907-12-25 +1907-05-05 12:34:56.1202,1907-12-25 +1907-05-05 12:34:56.1203,1907-12-25 +1907-05-05 12:34:56.1204,1907-12-25 +1907-05-05 12:34:56.1205,1907-12-25 +1907-05-05 12:34:56.1206,1907-12-25 +1907-05-05 12:34:56.1207,1907-12-25 +1907-05-05 12:34:56.1208,1907-12-25 +1907-05-05 12:34:56.1209,1907-12-25 +1907-05-05 12:34:56.121,1907-12-25 +1907-05-05 12:34:56.1211,1907-12-25 +1907-05-05 12:34:56.1212,1907-12-25 +1907-05-05 12:34:56.1213,1907-12-25 +1907-05-05 12:34:56.1214,1907-12-25 +1907-05-05 12:34:56.1215,1907-12-25 +1907-05-05 12:34:56.1216,1907-12-25 +1907-05-05 12:34:56.1217,1907-12-25 +1907-05-05 12:34:56.1218,1907-12-25 +1907-05-05 12:34:56.1219,1907-12-25 +1907-05-05 12:34:56.122,1907-12-25 +1907-05-05 12:34:56.1221,1907-12-25 +1907-05-05 12:34:56.1222,1907-12-25 +1907-05-05 12:34:56.1223,1907-12-25 +1907-05-05 12:34:56.1224,1907-12-25 +1907-05-05 12:34:56.1225,1907-12-25 +1907-05-05 12:34:56.1226,1907-12-25 +1907-05-05 12:34:56.1227,1907-12-25 +1907-05-05 12:34:56.1228,1907-12-25 +1907-05-05 12:34:56.1229,1907-12-25 +1907-05-05 12:34:56.123,1907-12-25 +1907-05-05 12:34:56.1231,1907-12-25 +1907-05-05 12:34:56.1232,1907-12-25 +1907-05-05 12:34:56.1233,1907-12-25 +1907-05-05 12:34:56.1234,1907-12-25 +1907-05-05 12:34:56.1235,1907-12-25 +1907-05-05 12:34:56.1236,1907-12-25 +1907-05-05 12:34:56.1237,1907-12-25 +1907-05-05 12:34:56.1238,1907-12-25 +1907-05-05 12:34:56.1239,1907-12-25 +1907-05-05 12:34:56.124,1907-12-25 +1907-05-05 12:34:56.1241,1907-12-25 +1907-05-05 12:34:56.1242,1907-12-25 +1907-05-05 12:34:56.1243,1907-12-25 +1907-05-05 12:34:56.1244,1907-12-25 +1907-05-05 12:34:56.1245,1907-12-25 +1907-05-05 12:34:56.1246,1907-12-25 +1907-05-05 12:34:56.1247,1907-12-25 +1907-05-05 12:34:56.1248,1907-12-25 +1907-05-05 12:34:56.1249,1907-12-25 +1907-05-05 12:34:56.125,1907-12-25 +1907-05-05 12:34:56.1251,1907-12-25 +1907-05-05 12:34:56.1252,1907-12-25 +1907-05-05 12:34:56.1253,1907-12-25 +1907-05-05 12:34:56.1254,1907-12-25 +1907-05-05 12:34:56.1255,1907-12-25 +1907-05-05 12:34:56.1256,1907-12-25 +1907-05-05 12:34:56.1257,1907-12-25 +1907-05-05 12:34:56.1258,1907-12-25 +1907-05-05 12:34:56.1259,1907-12-25 +1907-05-05 12:34:56.126,1907-12-25 +1907-05-05 12:34:56.1261,1907-12-25 +1907-05-05 12:34:56.1262,1907-12-25 +1907-05-05 12:34:56.1263,1907-12-25 +1907-05-05 12:34:56.1264,1907-12-25 +1907-05-05 12:34:56.1265,1907-12-25 +1907-05-05 12:34:56.1266,1907-12-25 +1907-05-05 12:34:56.1267,1907-12-25 +1907-05-05 12:34:56.1268,1907-12-25 +1907-05-05 12:34:56.1269,1907-12-25 +1907-05-05 12:34:56.127,1907-12-25 +1907-05-05 12:34:56.1271,1907-12-25 +1907-05-05 12:34:56.1272,1907-12-25 +1907-05-05 12:34:56.1273,1907-12-25 +1907-05-05 12:34:56.1274,1907-12-25 +1907-05-05 12:34:56.1275,1907-12-25 +1907-05-05 12:34:56.1276,1907-12-25 +1907-05-05 12:34:56.1277,1907-12-25 +1907-05-05 12:34:56.1278,1907-12-25 +1907-05-05 12:34:56.1279,1907-12-25 +1907-05-05 12:34:56.128,1907-12-25 +1907-05-05 12:34:56.1281,1907-12-25 +1907-05-05 12:34:56.1282,1907-12-25 +1907-05-05 12:34:56.1283,1907-12-25 +1907-05-05 12:34:56.1284,1907-12-25 +1907-05-05 12:34:56.1285,1907-12-25 +1907-05-05 12:34:56.1286,1907-12-25 +1907-05-05 12:34:56.1287,1907-12-25 +1907-05-05 12:34:56.1288,1907-12-25 +1907-05-05 12:34:56.1289,1907-12-25 +1907-05-05 12:34:56.129,1907-12-25 +1907-05-05 12:34:56.1291,1907-12-25 +1907-05-05 12:34:56.1292,1907-12-25 +1907-05-05 12:34:56.1293,1907-12-25 +1907-05-05 12:34:56.1294,1907-12-25 +1907-05-05 12:34:56.1295,1907-12-25 +1907-05-05 12:34:56.1296,1907-12-25 +1907-05-05 12:34:56.1297,1907-12-25 +1907-05-05 12:34:56.1298,1907-12-25 +1907-05-05 12:34:56.1299,1907-12-25 +1907-05-05 12:34:56.13,1907-12-25 +1907-05-05 12:34:56.1301,1907-12-25 +1907-05-05 12:34:56.1302,1907-12-25 +1907-05-05 12:34:56.1303,1907-12-25 +1907-05-05 12:34:56.1304,1907-12-25 +1907-05-05 12:34:56.1305,1907-12-25 +1907-05-05 12:34:56.1306,1907-12-25 +1907-05-05 12:34:56.1307,1907-12-25 +1907-05-05 12:34:56.1308,1907-12-25 +1907-05-05 12:34:56.1309,1907-12-25 +1907-05-05 12:34:56.131,1907-12-25 +1907-05-05 12:34:56.1311,1907-12-25 +1907-05-05 12:34:56.1312,1907-12-25 +1907-05-05 12:34:56.1313,1907-12-25 +1907-05-05 12:34:56.1314,1907-12-25 +1907-05-05 12:34:56.1315,1907-12-25 +1907-05-05 12:34:56.1316,1907-12-25 +1907-05-05 12:34:56.1317,1907-12-25 +1907-05-05 12:34:56.1318,1907-12-25 +1907-05-05 12:34:56.1319,1907-12-25 +1907-05-05 12:34:56.132,1907-12-25 +1907-05-05 12:34:56.1321,1907-12-25 +1907-05-05 12:34:56.1322,1907-12-25 +1907-05-05 12:34:56.1323,1907-12-25 +1907-05-05 12:34:56.1324,1907-12-25 +1907-05-05 12:34:56.1325,1907-12-25 +1907-05-05 12:34:56.1326,1907-12-25 +1907-05-05 12:34:56.1327,1907-12-25 +1907-05-05 12:34:56.1328,1907-12-25 +1907-05-05 12:34:56.1329,1907-12-25 +1907-05-05 12:34:56.133,1907-12-25 +1907-05-05 12:34:56.1331,1907-12-25 +1907-05-05 12:34:56.1332,1907-12-25 +1907-05-05 12:34:56.1333,1907-12-25 +1907-05-05 12:34:56.1334,1907-12-25 +1907-05-05 12:34:56.1335,1907-12-25 +1907-05-05 12:34:56.1336,1907-12-25 +1907-05-05 12:34:56.1337,1907-12-25 +1907-05-05 12:34:56.1338,1907-12-25 +1907-05-05 12:34:56.1339,1907-12-25 +1907-05-05 12:34:56.134,1907-12-25 +1907-05-05 12:34:56.1341,1907-12-25 +1907-05-05 12:34:56.1342,1907-12-25 +1907-05-05 12:34:56.1343,1907-12-25 +1907-05-05 12:34:56.1344,1907-12-25 +1907-05-05 12:34:56.1345,1907-12-25 +1907-05-05 12:34:56.1346,1907-12-25 +1907-05-05 12:34:56.1347,1907-12-25 +1907-05-05 12:34:56.1348,1907-12-25 +1907-05-05 12:34:56.1349,1907-12-25 +1907-05-05 12:34:56.135,1907-12-25 +1907-05-05 12:34:56.1351,1907-12-25 +1907-05-05 12:34:56.1352,1907-12-25 +1907-05-05 12:34:56.1353,1907-12-25 +1907-05-05 12:34:56.1354,1907-12-25 +1907-05-05 12:34:56.1355,1907-12-25 +1907-05-05 12:34:56.1356,1907-12-25 +1907-05-05 12:34:56.1357,1907-12-25 +1907-05-05 12:34:56.1358,1907-12-25 +1907-05-05 12:34:56.1359,1907-12-25 +1907-05-05 12:34:56.136,1907-12-25 +1907-05-05 12:34:56.1361,1907-12-25 +1907-05-05 12:34:56.1362,1907-12-25 +1907-05-05 12:34:56.1363,1907-12-25 +1907-05-05 12:34:56.1364,1907-12-25 +1907-05-05 12:34:56.1365,1907-12-25 +1907-05-05 12:34:56.1366,1907-12-25 +1907-05-05 12:34:56.1367,1907-12-25 +1907-05-05 12:34:56.1368,1907-12-25 +1907-05-05 12:34:56.1369,1907-12-25 +1907-05-05 12:34:56.137,1907-12-25 +1907-05-05 12:34:56.1371,1907-12-25 +1907-05-05 12:34:56.1372,1907-12-25 +1907-05-05 12:34:56.1373,1907-12-25 +1907-05-05 12:34:56.1374,1907-12-25 +1907-05-05 12:34:56.1375,1907-12-25 +1907-05-05 12:34:56.1376,1907-12-25 +1907-05-05 12:34:56.1377,1907-12-25 +1907-05-05 12:34:56.1378,1907-12-25 +1907-05-05 12:34:56.1379,1907-12-25 +1907-05-05 12:34:56.138,1907-12-25 +1907-05-05 12:34:56.1381,1907-12-25 +1907-05-05 12:34:56.1382,1907-12-25 +1907-05-05 12:34:56.1383,1907-12-25 +1907-05-05 12:34:56.1384,1907-12-25 +1907-05-05 12:34:56.1385,1907-12-25 +1907-05-05 12:34:56.1386,1907-12-25 +1907-05-05 12:34:56.1387,1907-12-25 +1907-05-05 12:34:56.1388,1907-12-25 +1907-05-05 12:34:56.1389,1907-12-25 +1907-05-05 12:34:56.139,1907-12-25 +1907-05-05 12:34:56.1391,1907-12-25 +1907-05-05 12:34:56.1392,1907-12-25 +1907-05-05 12:34:56.1393,1907-12-25 +1907-05-05 12:34:56.1394,1907-12-25 +1907-05-05 12:34:56.1395,1907-12-25 +1907-05-05 12:34:56.1396,1907-12-25 +1907-05-05 12:34:56.1397,1907-12-25 +1907-05-05 12:34:56.1398,1907-12-25 +1907-05-05 12:34:56.1399,1907-12-25 +1907-05-05 12:34:56.14,1907-12-25 +1907-05-05 12:34:56.1401,1907-12-25 +1907-05-05 12:34:56.1402,1907-12-25 +1907-05-05 12:34:56.1403,1907-12-25 +1907-05-05 12:34:56.1404,1907-12-25 +1907-05-05 12:34:56.1405,1907-12-25 +1907-05-05 12:34:56.1406,1907-12-25 +1907-05-05 12:34:56.1407,1907-12-25 +1907-05-05 12:34:56.1408,1907-12-25 +1907-05-05 12:34:56.1409,1907-12-25 +1907-05-05 12:34:56.141,1907-12-25 +1907-05-05 12:34:56.1411,1907-12-25 +1907-05-05 12:34:56.1412,1907-12-25 +1907-05-05 12:34:56.1413,1907-12-25 +1907-05-05 12:34:56.1414,1907-12-25 +1907-05-05 12:34:56.1415,1907-12-25 +1907-05-05 12:34:56.1416,1907-12-25 +1907-05-05 12:34:56.1417,1907-12-25 +1907-05-05 12:34:56.1418,1907-12-25 +1907-05-05 12:34:56.1419,1907-12-25 +1907-05-05 12:34:56.142,1907-12-25 +1907-05-05 12:34:56.1421,1907-12-25 +1907-05-05 12:34:56.1422,1907-12-25 +1907-05-05 12:34:56.1423,1907-12-25 +1907-05-05 12:34:56.1424,1907-12-25 +1907-05-05 12:34:56.1425,1907-12-25 +1907-05-05 12:34:56.1426,1907-12-25 +1907-05-05 12:34:56.1427,1907-12-25 +1907-05-05 12:34:56.1428,1907-12-25 +1907-05-05 12:34:56.1429,1907-12-25 +1907-05-05 12:34:56.143,1907-12-25 +1907-05-05 12:34:56.1431,1907-12-25 +1907-05-05 12:34:56.1432,1907-12-25 +1907-05-05 12:34:56.1433,1907-12-25 +1907-05-05 12:34:56.1434,1907-12-25 +1907-05-05 12:34:56.1435,1907-12-25 +1907-05-05 12:34:56.1436,1907-12-25 +1907-05-05 12:34:56.1437,1907-12-25 +1907-05-05 12:34:56.1438,1907-12-25 +1907-05-05 12:34:56.1439,1907-12-25 +1907-05-05 12:34:56.144,1907-12-25 +1907-05-05 12:34:56.1441,1907-12-25 +1907-05-05 12:34:56.1442,1907-12-25 +1907-05-05 12:34:56.1443,1907-12-25 +1907-05-05 12:34:56.1444,1907-12-25 +1907-05-05 12:34:56.1445,1907-12-25 +1907-05-05 12:34:56.1446,1907-12-25 +1907-05-05 12:34:56.1447,1907-12-25 +1907-05-05 12:34:56.1448,1907-12-25 +1907-05-05 12:34:56.1449,1907-12-25 +1907-05-05 12:34:56.145,1907-12-25 +1907-05-05 12:34:56.1451,1907-12-25 +1907-05-05 12:34:56.1452,1907-12-25 +1907-05-05 12:34:56.1453,1907-12-25 +1907-05-05 12:34:56.1454,1907-12-25 +1907-05-05 12:34:56.1455,1907-12-25 +1907-05-05 12:34:56.1456,1907-12-25 +1907-05-05 12:34:56.1457,1907-12-25 +1907-05-05 12:34:56.1458,1907-12-25 +1907-05-05 12:34:56.1459,1907-12-25 +1907-05-05 12:34:56.146,1907-12-25 +1907-05-05 12:34:56.1461,1907-12-25 +1907-05-05 12:34:56.1462,1907-12-25 +1907-05-05 12:34:56.1463,1907-12-25 +1907-05-05 12:34:56.1464,1907-12-25 +1907-05-05 12:34:56.1465,1907-12-25 +1907-05-05 12:34:56.1466,1907-12-25 +1907-05-05 12:34:56.1467,1907-12-25 +1907-05-05 12:34:56.1468,1907-12-25 +1907-05-05 12:34:56.1469,1907-12-25 +1907-05-05 12:34:56.147,1907-12-25 +1907-05-05 12:34:56.1471,1907-12-25 +1907-05-05 12:34:56.1472,1907-12-25 +1907-05-05 12:34:56.1473,1907-12-25 +1907-05-05 12:34:56.1474,1907-12-25 +1907-05-05 12:34:56.1475,1907-12-25 +1907-05-05 12:34:56.1476,1907-12-25 +1907-05-05 12:34:56.1477,1907-12-25 +1907-05-05 12:34:56.1478,1907-12-25 +1907-05-05 12:34:56.1479,1907-12-25 +1907-05-05 12:34:56.148,1907-12-25 +1907-05-05 12:34:56.1481,1907-12-25 +1907-05-05 12:34:56.1482,1907-12-25 +1907-05-05 12:34:56.1483,1907-12-25 +1907-05-05 12:34:56.1484,1907-12-25 +1907-05-05 12:34:56.1485,1907-12-25 +1907-05-05 12:34:56.1486,1907-12-25 +1907-05-05 12:34:56.1487,1907-12-25 +1907-05-05 12:34:56.1488,1907-12-25 +1907-05-05 12:34:56.1489,1907-12-25 +1907-05-05 12:34:56.149,1907-12-25 +1907-05-05 12:34:56.1491,1907-12-25 +1907-05-05 12:34:56.1492,1907-12-25 +1907-05-05 12:34:56.1493,1907-12-25 +1907-05-05 12:34:56.1494,1907-12-25 +1907-05-05 12:34:56.1495,1907-12-25 +1907-05-05 12:34:56.1496,1907-12-25 +1907-05-05 12:34:56.1497,1907-12-25 +1907-05-05 12:34:56.1498,1907-12-25 +1907-05-05 12:34:56.1499,1907-12-25 +1907-05-05 12:34:56.15,1907-12-25 +1907-05-05 12:34:56.1501,1907-12-25 +1907-05-05 12:34:56.1502,1907-12-25 +1907-05-05 12:34:56.1503,1907-12-25 +1907-05-05 12:34:56.1504,1907-12-25 +1907-05-05 12:34:56.1505,1907-12-25 +1907-05-05 12:34:56.1506,1907-12-25 +1907-05-05 12:34:56.1507,1907-12-25 +1907-05-05 12:34:56.1508,1907-12-25 +1907-05-05 12:34:56.1509,1907-12-25 +1907-05-05 12:34:56.151,1907-12-25 +1907-05-05 12:34:56.1511,1907-12-25 +1907-05-05 12:34:56.1512,1907-12-25 +1907-05-05 12:34:56.1513,1907-12-25 +1907-05-05 12:34:56.1514,1907-12-25 +1907-05-05 12:34:56.1515,1907-12-25 +1907-05-05 12:34:56.1516,1907-12-25 +1907-05-05 12:34:56.1517,1907-12-25 +1907-05-05 12:34:56.1518,1907-12-25 +1907-05-05 12:34:56.1519,1907-12-25 +1907-05-05 12:34:56.152,1907-12-25 +1907-05-05 12:34:56.1521,1907-12-25 +1907-05-05 12:34:56.1522,1907-12-25 +1907-05-05 12:34:56.1523,1907-12-25 +1907-05-05 12:34:56.1524,1907-12-25 +1907-05-05 12:34:56.1525,1907-12-25 +1907-05-05 12:34:56.1526,1907-12-25 +1907-05-05 12:34:56.1527,1907-12-25 +1907-05-05 12:34:56.1528,1907-12-25 +1907-05-05 12:34:56.1529,1907-12-25 +1907-05-05 12:34:56.153,1907-12-25 +1907-05-05 12:34:56.1531,1907-12-25 +1907-05-05 12:34:56.1532,1907-12-25 +1907-05-05 12:34:56.1533,1907-12-25 +1907-05-05 12:34:56.1534,1907-12-25 +1907-05-05 12:34:56.1535,1907-12-25 +1907-05-05 12:34:56.1536,1907-12-25 +1907-05-05 12:34:56.1537,1907-12-25 +1907-05-05 12:34:56.1538,1907-12-25 +1907-05-05 12:34:56.1539,1907-12-25 +1907-05-05 12:34:56.154,1907-12-25 +1907-05-05 12:34:56.1541,1907-12-25 +1907-05-05 12:34:56.1542,1907-12-25 +1907-05-05 12:34:56.1543,1907-12-25 +1907-05-05 12:34:56.1544,1907-12-25 +1907-05-05 12:34:56.1545,1907-12-25 +1907-05-05 12:34:56.1546,1907-12-25 +1907-05-05 12:34:56.1547,1907-12-25 +1907-05-05 12:34:56.1548,1907-12-25 +1907-05-05 12:34:56.1549,1907-12-25 +1907-05-05 12:34:56.155,1907-12-25 +1907-05-05 12:34:56.1551,1907-12-25 +1907-05-05 12:34:56.1552,1907-12-25 +1907-05-05 12:34:56.1553,1907-12-25 +1907-05-05 12:34:56.1554,1907-12-25 +1907-05-05 12:34:56.1555,1907-12-25 +1907-05-05 12:34:56.1556,1907-12-25 +1907-05-05 12:34:56.1557,1907-12-25 +1907-05-05 12:34:56.1558,1907-12-25 +1907-05-05 12:34:56.1559,1907-12-25 +1907-05-05 12:34:56.156,1907-12-25 +1907-05-05 12:34:56.1561,1907-12-25 +1907-05-05 12:34:56.1562,1907-12-25 +1907-05-05 12:34:56.1563,1907-12-25 +1907-05-05 12:34:56.1564,1907-12-25 +1907-05-05 12:34:56.1565,1907-12-25 +1907-05-05 12:34:56.1566,1907-12-25 +1907-05-05 12:34:56.1567,1907-12-25 +1907-05-05 12:34:56.1568,1907-12-25 +1907-05-05 12:34:56.1569,1907-12-25 +1907-05-05 12:34:56.157,1907-12-25 +1907-05-05 12:34:56.1571,1907-12-25 +1907-05-05 12:34:56.1572,1907-12-25 +1907-05-05 12:34:56.1573,1907-12-25 +1907-05-05 12:34:56.1574,1907-12-25 +1907-05-05 12:34:56.1575,1907-12-25 +1907-05-05 12:34:56.1576,1907-12-25 +1907-05-05 12:34:56.1577,1907-12-25 +1907-05-05 12:34:56.1578,1907-12-25 +1907-05-05 12:34:56.1579,1907-12-25 +1907-05-05 12:34:56.158,1907-12-25 +1907-05-05 12:34:56.1581,1907-12-25 +1907-05-05 12:34:56.1582,1907-12-25 +1907-05-05 12:34:56.1583,1907-12-25 +1907-05-05 12:34:56.1584,1907-12-25 +1907-05-05 12:34:56.1585,1907-12-25 +1907-05-05 12:34:56.1586,1907-12-25 +1907-05-05 12:34:56.1587,1907-12-25 +1907-05-05 12:34:56.1588,1907-12-25 +1907-05-05 12:34:56.1589,1907-12-25 +1907-05-05 12:34:56.159,1907-12-25 +1907-05-05 12:34:56.1591,1907-12-25 +1907-05-05 12:34:56.1592,1907-12-25 +1907-05-05 12:34:56.1593,1907-12-25 +1907-05-05 12:34:56.1594,1907-12-25 +1907-05-05 12:34:56.1595,1907-12-25 +1907-05-05 12:34:56.1596,1907-12-25 +1907-05-05 12:34:56.1597,1907-12-25 +1907-05-05 12:34:56.1598,1907-12-25 +1907-05-05 12:34:56.1599,1907-12-25 +1907-05-05 12:34:56.16,1907-12-25 +1907-05-05 12:34:56.1601,1907-12-25 +1907-05-05 12:34:56.1602,1907-12-25 +1907-05-05 12:34:56.1603,1907-12-25 +1907-05-05 12:34:56.1604,1907-12-25 +1907-05-05 12:34:56.1605,1907-12-25 +1907-05-05 12:34:56.1606,1907-12-25 +1907-05-05 12:34:56.1607,1907-12-25 +1907-05-05 12:34:56.1608,1907-12-25 +1907-05-05 12:34:56.1609,1907-12-25 +1907-05-05 12:34:56.161,1907-12-25 +1907-05-05 12:34:56.1611,1907-12-25 +1907-05-05 12:34:56.1612,1907-12-25 +1907-05-05 12:34:56.1613,1907-12-25 +1907-05-05 12:34:56.1614,1907-12-25 +1907-05-05 12:34:56.1615,1907-12-25 +1907-05-05 12:34:56.1616,1907-12-25 +1907-05-05 12:34:56.1617,1907-12-25 +1907-05-05 12:34:56.1618,1907-12-25 +1907-05-05 12:34:56.1619,1907-12-25 +1907-05-05 12:34:56.162,1907-12-25 +1907-05-05 12:34:56.1621,1907-12-25 +1907-05-05 12:34:56.1622,1907-12-25 +1907-05-05 12:34:56.1623,1907-12-25 +1907-05-05 12:34:56.1624,1907-12-25 +1907-05-05 12:34:56.1625,1907-12-25 +1907-05-05 12:34:56.1626,1907-12-25 +1907-05-05 12:34:56.1627,1907-12-25 +1907-05-05 12:34:56.1628,1907-12-25 +1907-05-05 12:34:56.1629,1907-12-25 +1907-05-05 12:34:56.163,1907-12-25 +1907-05-05 12:34:56.1631,1907-12-25 +1907-05-05 12:34:56.1632,1907-12-25 +1907-05-05 12:34:56.1633,1907-12-25 +1907-05-05 12:34:56.1634,1907-12-25 +1907-05-05 12:34:56.1635,1907-12-25 +1907-05-05 12:34:56.1636,1907-12-25 +1907-05-05 12:34:56.1637,1907-12-25 +1907-05-05 12:34:56.1638,1907-12-25 +1907-05-05 12:34:56.1639,1907-12-25 +1907-05-05 12:34:56.164,1907-12-25 +1907-05-05 12:34:56.1641,1907-12-25 +1907-05-05 12:34:56.1642,1907-12-25 +1907-05-05 12:34:56.1643,1907-12-25 +1907-05-05 12:34:56.1644,1907-12-25 +1907-05-05 12:34:56.1645,1907-12-25 +1907-05-05 12:34:56.1646,1907-12-25 +1907-05-05 12:34:56.1647,1907-12-25 +1907-05-05 12:34:56.1648,1907-12-25 +1907-05-05 12:34:56.1649,1907-12-25 +1907-05-05 12:34:56.165,1907-12-25 +1907-05-05 12:34:56.1651,1907-12-25 +1907-05-05 12:34:56.1652,1907-12-25 +1907-05-05 12:34:56.1653,1907-12-25 +1907-05-05 12:34:56.1654,1907-12-25 +1907-05-05 12:34:56.1655,1907-12-25 +1907-05-05 12:34:56.1656,1907-12-25 +1907-05-05 12:34:56.1657,1907-12-25 +1907-05-05 12:34:56.1658,1907-12-25 +1907-05-05 12:34:56.1659,1907-12-25 +1907-05-05 12:34:56.166,1907-12-25 +1907-05-05 12:34:56.1661,1907-12-25 +1907-05-05 12:34:56.1662,1907-12-25 +1907-05-05 12:34:56.1663,1907-12-25 +1907-05-05 12:34:56.1664,1907-12-25 +1907-05-05 12:34:56.1665,1907-12-25 +1907-05-05 12:34:56.1666,1907-12-25 +1907-05-05 12:34:56.1667,1907-12-25 +1907-05-05 12:34:56.1668,1907-12-25 +1907-05-05 12:34:56.1669,1907-12-25 +1907-05-05 12:34:56.167,1907-12-25 +1907-05-05 12:34:56.1671,1907-12-25 +1907-05-05 12:34:56.1672,1907-12-25 +1907-05-05 12:34:56.1673,1907-12-25 +1907-05-05 12:34:56.1674,1907-12-25 +1907-05-05 12:34:56.1675,1907-12-25 +1907-05-05 12:34:56.1676,1907-12-25 +1907-05-05 12:34:56.1677,1907-12-25 +1907-05-05 12:34:56.1678,1907-12-25 +1907-05-05 12:34:56.1679,1907-12-25 +1907-05-05 12:34:56.168,1907-12-25 +1907-05-05 12:34:56.1681,1907-12-25 +1907-05-05 12:34:56.1682,1907-12-25 +1907-05-05 12:34:56.1683,1907-12-25 +1907-05-05 12:34:56.1684,1907-12-25 +1907-05-05 12:34:56.1685,1907-12-25 +1907-05-05 12:34:56.1686,1907-12-25 +1907-05-05 12:34:56.1687,1907-12-25 +1907-05-05 12:34:56.1688,1907-12-25 +1907-05-05 12:34:56.1689,1907-12-25 +1907-05-05 12:34:56.169,1907-12-25 +1907-05-05 12:34:56.1691,1907-12-25 +1907-05-05 12:34:56.1692,1907-12-25 +1907-05-05 12:34:56.1693,1907-12-25 +1907-05-05 12:34:56.1694,1907-12-25 +1907-05-05 12:34:56.1695,1907-12-25 +1907-05-05 12:34:56.1696,1907-12-25 +1907-05-05 12:34:56.1697,1907-12-25 +1907-05-05 12:34:56.1698,1907-12-25 +1907-05-05 12:34:56.1699,1907-12-25 +1907-05-05 12:34:56.17,1907-12-25 +1907-05-05 12:34:56.1701,1907-12-25 +1907-05-05 12:34:56.1702,1907-12-25 +1907-05-05 12:34:56.1703,1907-12-25 +1907-05-05 12:34:56.1704,1907-12-25 +1907-05-05 12:34:56.1705,1907-12-25 +1907-05-05 12:34:56.1706,1907-12-25 +1907-05-05 12:34:56.1707,1907-12-25 +1907-05-05 12:34:56.1708,1907-12-25 +1907-05-05 12:34:56.1709,1907-12-25 +1907-05-05 12:34:56.171,1907-12-25 +1907-05-05 12:34:56.1711,1907-12-25 +1907-05-05 12:34:56.1712,1907-12-25 +1907-05-05 12:34:56.1713,1907-12-25 +1907-05-05 12:34:56.1714,1907-12-25 +1907-05-05 12:34:56.1715,1907-12-25 +1907-05-05 12:34:56.1716,1907-12-25 +1907-05-05 12:34:56.1717,1907-12-25 +1907-05-05 12:34:56.1718,1907-12-25 +1907-05-05 12:34:56.1719,1907-12-25 +1907-05-05 12:34:56.172,1907-12-25 +1907-05-05 12:34:56.1721,1907-12-25 +1907-05-05 12:34:56.1722,1907-12-25 +1907-05-05 12:34:56.1723,1907-12-25 +1907-05-05 12:34:56.1724,1907-12-25 +1907-05-05 12:34:56.1725,1907-12-25 +1907-05-05 12:34:56.1726,1907-12-25 +1907-05-05 12:34:56.1727,1907-12-25 +1907-05-05 12:34:56.1728,1907-12-25 +1907-05-05 12:34:56.1729,1907-12-25 +1907-05-05 12:34:56.173,1907-12-25 +1907-05-05 12:34:56.1731,1907-12-25 +1907-05-05 12:34:56.1732,1907-12-25 +1907-05-05 12:34:56.1733,1907-12-25 +1907-05-05 12:34:56.1734,1907-12-25 +1907-05-05 12:34:56.1735,1907-12-25 +1907-05-05 12:34:56.1736,1907-12-25 +1907-05-05 12:34:56.1737,1907-12-25 +1907-05-05 12:34:56.1738,1907-12-25 +1907-05-05 12:34:56.1739,1907-12-25 +1907-05-05 12:34:56.174,1907-12-25 +1907-05-05 12:34:56.1741,1907-12-25 +1907-05-05 12:34:56.1742,1907-12-25 +1907-05-05 12:34:56.1743,1907-12-25 +1907-05-05 12:34:56.1744,1907-12-25 +1907-05-05 12:34:56.1745,1907-12-25 +1907-05-05 12:34:56.1746,1907-12-25 +1907-05-05 12:34:56.1747,1907-12-25 +1907-05-05 12:34:56.1748,1907-12-25 +1907-05-05 12:34:56.1749,1907-12-25 +1907-05-05 12:34:56.175,1907-12-25 +1907-05-05 12:34:56.1751,1907-12-25 +1907-05-05 12:34:56.1752,1907-12-25 +1907-05-05 12:34:56.1753,1907-12-25 +1907-05-05 12:34:56.1754,1907-12-25 +1907-05-05 12:34:56.1755,1907-12-25 +1907-05-05 12:34:56.1756,1907-12-25 +1907-05-05 12:34:56.1757,1907-12-25 +1907-05-05 12:34:56.1758,1907-12-25 +1907-05-05 12:34:56.1759,1907-12-25 +1907-05-05 12:34:56.176,1907-12-25 +1907-05-05 12:34:56.1761,1907-12-25 +1907-05-05 12:34:56.1762,1907-12-25 +1907-05-05 12:34:56.1763,1907-12-25 +1907-05-05 12:34:56.1764,1907-12-25 +1907-05-05 12:34:56.1765,1907-12-25 +1907-05-05 12:34:56.1766,1907-12-25 +1907-05-05 12:34:56.1767,1907-12-25 +1907-05-05 12:34:56.1768,1907-12-25 +1907-05-05 12:34:56.1769,1907-12-25 +1907-05-05 12:34:56.177,1907-12-25 +1907-05-05 12:34:56.1771,1907-12-25 +1907-05-05 12:34:56.1772,1907-12-25 +1907-05-05 12:34:56.1773,1907-12-25 +1907-05-05 12:34:56.1774,1907-12-25 +1907-05-05 12:34:56.1775,1907-12-25 +1907-05-05 12:34:56.1776,1907-12-25 +1907-05-05 12:34:56.1777,1907-12-25 +1907-05-05 12:34:56.1778,1907-12-25 +1907-05-05 12:34:56.1779,1907-12-25 +1907-05-05 12:34:56.178,1907-12-25 +1907-05-05 12:34:56.1781,1907-12-25 +1907-05-05 12:34:56.1782,1907-12-25 +1907-05-05 12:34:56.1783,1907-12-25 +1907-05-05 12:34:56.1784,1907-12-25 +1907-05-05 12:34:56.1785,1907-12-25 +1907-05-05 12:34:56.1786,1907-12-25 +1907-05-05 12:34:56.1787,1907-12-25 +1907-05-05 12:34:56.1788,1907-12-25 +1907-05-05 12:34:56.1789,1907-12-25 +1907-05-05 12:34:56.179,1907-12-25 +1907-05-05 12:34:56.1791,1907-12-25 +1907-05-05 12:34:56.1792,1907-12-25 +1907-05-05 12:34:56.1793,1907-12-25 +1907-05-05 12:34:56.1794,1907-12-25 +1907-05-05 12:34:56.1795,1907-12-25 +1907-05-05 12:34:56.1796,1907-12-25 +1907-05-05 12:34:56.1797,1907-12-25 +1907-05-05 12:34:56.1798,1907-12-25 +1907-05-05 12:34:56.1799,1907-12-25 +1907-05-05 12:34:56.18,1907-12-25 +1907-05-05 12:34:56.1801,1907-12-25 +1907-05-05 12:34:56.1802,1907-12-25 +1907-05-05 12:34:56.1803,1907-12-25 +1907-05-05 12:34:56.1804,1907-12-25 +1907-05-05 12:34:56.1805,1907-12-25 +1907-05-05 12:34:56.1806,1907-12-25 +1907-05-05 12:34:56.1807,1907-12-25 +1907-05-05 12:34:56.1808,1907-12-25 +1907-05-05 12:34:56.1809,1907-12-25 +1907-05-05 12:34:56.181,1907-12-25 +1907-05-05 12:34:56.1811,1907-12-25 +1907-05-05 12:34:56.1812,1907-12-25 +1907-05-05 12:34:56.1813,1907-12-25 +1907-05-05 12:34:56.1814,1907-12-25 +1907-05-05 12:34:56.1815,1907-12-25 +1907-05-05 12:34:56.1816,1907-12-25 +1907-05-05 12:34:56.1817,1907-12-25 +1907-05-05 12:34:56.1818,1907-12-25 +1907-05-05 12:34:56.1819,1907-12-25 +1907-05-05 12:34:56.182,1907-12-25 +1907-05-05 12:34:56.1821,1907-12-25 +1907-05-05 12:34:56.1822,1907-12-25 +1907-05-05 12:34:56.1823,1907-12-25 +1907-05-05 12:34:56.1824,1907-12-25 +1907-05-05 12:34:56.1825,1907-12-25 +1907-05-05 12:34:56.1826,1907-12-25 +1907-05-05 12:34:56.1827,1907-12-25 +1907-05-05 12:34:56.1828,1907-12-25 +1907-05-05 12:34:56.1829,1907-12-25 +1907-05-05 12:34:56.183,1907-12-25 +1907-05-05 12:34:56.1831,1907-12-25 +1907-05-05 12:34:56.1832,1907-12-25 +1907-05-05 12:34:56.1833,1907-12-25 +1907-05-05 12:34:56.1834,1907-12-25 +1907-05-05 12:34:56.1835,1907-12-25 +1907-05-05 12:34:56.1836,1907-12-25 +1907-05-05 12:34:56.1837,1907-12-25 +1907-05-05 12:34:56.1838,1907-12-25 +1907-05-05 12:34:56.1839,1907-12-25 +1907-05-05 12:34:56.184,1907-12-25 +1907-05-05 12:34:56.1841,1907-12-25 +1907-05-05 12:34:56.1842,1907-12-25 +1907-05-05 12:34:56.1843,1907-12-25 +1907-05-05 12:34:56.1844,1907-12-25 +1907-05-05 12:34:56.1845,1907-12-25 +1907-05-05 12:34:56.1846,1907-12-25 +1907-05-05 12:34:56.1847,1907-12-25 +1907-05-05 12:34:56.1848,1907-12-25 +1907-05-05 12:34:56.1849,1907-12-25 +1907-05-05 12:34:56.185,1907-12-25 +1907-05-05 12:34:56.1851,1907-12-25 +1907-05-05 12:34:56.1852,1907-12-25 +1907-05-05 12:34:56.1853,1907-12-25 +1907-05-05 12:34:56.1854,1907-12-25 +1907-05-05 12:34:56.1855,1907-12-25 +1907-05-05 12:34:56.1856,1907-12-25 +1907-05-05 12:34:56.1857,1907-12-25 +1907-05-05 12:34:56.1858,1907-12-25 +1907-05-05 12:34:56.1859,1907-12-25 +1907-05-05 12:34:56.186,1907-12-25 +1907-05-05 12:34:56.1861,1907-12-25 +1907-05-05 12:34:56.1862,1907-12-25 +1907-05-05 12:34:56.1863,1907-12-25 +1907-05-05 12:34:56.1864,1907-12-25 +1907-05-05 12:34:56.1865,1907-12-25 +1907-05-05 12:34:56.1866,1907-12-25 +1907-05-05 12:34:56.1867,1907-12-25 +1907-05-05 12:34:56.1868,1907-12-25 +1907-05-05 12:34:56.1869,1907-12-25 +1907-05-05 12:34:56.187,1907-12-25 +1907-05-05 12:34:56.1871,1907-12-25 +1907-05-05 12:34:56.1872,1907-12-25 +1907-05-05 12:34:56.1873,1907-12-25 +1907-05-05 12:34:56.1874,1907-12-25 +1907-05-05 12:34:56.1875,1907-12-25 +1907-05-05 12:34:56.1876,1907-12-25 +1907-05-05 12:34:56.1877,1907-12-25 +1907-05-05 12:34:56.1878,1907-12-25 +1907-05-05 12:34:56.1879,1907-12-25 +1907-05-05 12:34:56.188,1907-12-25 +1907-05-05 12:34:56.1881,1907-12-25 +1907-05-05 12:34:56.1882,1907-12-25 +1907-05-05 12:34:56.1883,1907-12-25 +1907-05-05 12:34:56.1884,1907-12-25 +1907-05-05 12:34:56.1885,1907-12-25 +1907-05-05 12:34:56.1886,1907-12-25 +1907-05-05 12:34:56.1887,1907-12-25 +1907-05-05 12:34:56.1888,1907-12-25 +1907-05-05 12:34:56.1889,1907-12-25 +1907-05-05 12:34:56.189,1907-12-25 +1907-05-05 12:34:56.1891,1907-12-25 +1907-05-05 12:34:56.1892,1907-12-25 +1907-05-05 12:34:56.1893,1907-12-25 +1907-05-05 12:34:56.1894,1907-12-25 +1907-05-05 12:34:56.1895,1907-12-25 +1907-05-05 12:34:56.1896,1907-12-25 +1907-05-05 12:34:56.1897,1907-12-25 +1907-05-05 12:34:56.1898,1907-12-25 +1907-05-05 12:34:56.1899,1907-12-25 +1907-05-05 12:34:56.19,1907-12-25 +1907-05-05 12:34:56.1901,1907-12-25 +1907-05-05 12:34:56.1902,1907-12-25 +1907-05-05 12:34:56.1903,1907-12-25 +1907-05-05 12:34:56.1904,1907-12-25 +1907-05-05 12:34:56.1905,1907-12-25 +1907-05-05 12:34:56.1906,1907-12-25 +1907-05-05 12:34:56.1907,1907-12-25 +1907-05-05 12:34:56.1908,1907-12-25 +1907-05-05 12:34:56.1909,1907-12-25 +1907-05-05 12:34:56.191,1907-12-25 +1907-05-05 12:34:56.1911,1907-12-25 +1907-05-05 12:34:56.1912,1907-12-25 +1907-05-05 12:34:56.1913,1907-12-25 +1907-05-05 12:34:56.1914,1907-12-25 +1907-05-05 12:34:56.1915,1907-12-25 +1907-05-05 12:34:56.1916,1907-12-25 +1907-05-05 12:34:56.1917,1907-12-25 +1907-05-05 12:34:56.1918,1907-12-25 +1907-05-05 12:34:56.1919,1907-12-25 +1907-05-05 12:34:56.192,1907-12-25 +1907-05-05 12:34:56.1921,1907-12-25 +1907-05-05 12:34:56.1922,1907-12-25 +1907-05-05 12:34:56.1923,1907-12-25 +1907-05-05 12:34:56.1924,1907-12-25 +1907-05-05 12:34:56.1925,1907-12-25 +1907-05-05 12:34:56.1926,1907-12-25 +1907-05-05 12:34:56.1927,1907-12-25 +1907-05-05 12:34:56.1928,1907-12-25 +1907-05-05 12:34:56.1929,1907-12-25 +1907-05-05 12:34:56.193,1907-12-25 +1907-05-05 12:34:56.1931,1907-12-25 +1907-05-05 12:34:56.1932,1907-12-25 +1907-05-05 12:34:56.1933,1907-12-25 +1907-05-05 12:34:56.1934,1907-12-25 +1907-05-05 12:34:56.1935,1907-12-25 +1907-05-05 12:34:56.1936,1907-12-25 +1907-05-05 12:34:56.1937,1907-12-25 +1907-05-05 12:34:56.1938,1907-12-25 +1907-05-05 12:34:56.1939,1907-12-25 +1907-05-05 12:34:56.194,1907-12-25 +1907-05-05 12:34:56.1941,1907-12-25 +1907-05-05 12:34:56.1942,1907-12-25 +1907-05-05 12:34:56.1943,1907-12-25 +1907-05-05 12:34:56.1944,1907-12-25 +1907-05-05 12:34:56.1945,1907-12-25 +1907-05-05 12:34:56.1946,1907-12-25 +1907-05-05 12:34:56.1947,1907-12-25 +1907-05-05 12:34:56.1948,1907-12-25 +1907-05-05 12:34:56.1949,1907-12-25 +1907-05-05 12:34:56.195,1907-12-25 +1907-05-05 12:34:56.1951,1907-12-25 +1907-05-05 12:34:56.1952,1907-12-25 +1907-05-05 12:34:56.1953,1907-12-25 +1907-05-05 12:34:56.1954,1907-12-25 +1907-05-05 12:34:56.1955,1907-12-25 +1907-05-05 12:34:56.1956,1907-12-25 +1907-05-05 12:34:56.1957,1907-12-25 +1907-05-05 12:34:56.1958,1907-12-25 +1907-05-05 12:34:56.1959,1907-12-25 +1907-05-05 12:34:56.196,1907-12-25 +1907-05-05 12:34:56.1961,1907-12-25 +1907-05-05 12:34:56.1962,1907-12-25 +1907-05-05 12:34:56.1963,1907-12-25 +1907-05-05 12:34:56.1964,1907-12-25 +1907-05-05 12:34:56.1965,1907-12-25 +1907-05-05 12:34:56.1966,1907-12-25 +1907-05-05 12:34:56.1967,1907-12-25 +1907-05-05 12:34:56.1968,1907-12-25 +1907-05-05 12:34:56.1969,1907-12-25 +1907-05-05 12:34:56.197,1907-12-25 +1907-05-05 12:34:56.1971,1907-12-25 +1907-05-05 12:34:56.1972,1907-12-25 +1907-05-05 12:34:56.1973,1907-12-25 +1907-05-05 12:34:56.1974,1907-12-25 +1907-05-05 12:34:56.1975,1907-12-25 +1907-05-05 12:34:56.1976,1907-12-25 +1907-05-05 12:34:56.1977,1907-12-25 +1907-05-05 12:34:56.1978,1907-12-25 +1907-05-05 12:34:56.1979,1907-12-25 +1907-05-05 12:34:56.198,1907-12-25 +1907-05-05 12:34:56.1981,1907-12-25 +1907-05-05 12:34:56.1982,1907-12-25 +1907-05-05 12:34:56.1983,1907-12-25 +1907-05-05 12:34:56.1984,1907-12-25 +1907-05-05 12:34:56.1985,1907-12-25 +1907-05-05 12:34:56.1986,1907-12-25 +1907-05-05 12:34:56.1987,1907-12-25 +1907-05-05 12:34:56.1988,1907-12-25 +1907-05-05 12:34:56.1989,1907-12-25 +1907-05-05 12:34:56.199,1907-12-25 +1907-05-05 12:34:56.1991,1907-12-25 +1907-05-05 12:34:56.1992,1907-12-25 +1907-05-05 12:34:56.1993,1907-12-25 +1907-05-05 12:34:56.1994,1907-12-25 +1907-05-05 12:34:56.1995,1907-12-25 +1907-05-05 12:34:56.1996,1907-12-25 +1907-05-05 12:34:56.1997,1907-12-25 +1907-05-05 12:34:56.1998,1907-12-25 +1907-05-05 12:34:56.1999,1907-12-25 +1908-05-05 12:34:56.1,1908-12-25 +1908-05-05 12:34:56.1001,1908-12-25 +1908-05-05 12:34:56.1002,1908-12-25 +1908-05-05 12:34:56.1003,1908-12-25 +1908-05-05 12:34:56.1004,1908-12-25 +1908-05-05 12:34:56.1005,1908-12-25 +1908-05-05 12:34:56.1006,1908-12-25 +1908-05-05 12:34:56.1007,1908-12-25 +1908-05-05 12:34:56.1008,1908-12-25 +1908-05-05 12:34:56.1009,1908-12-25 +1908-05-05 12:34:56.101,1908-12-25 +1908-05-05 12:34:56.1011,1908-12-25 +1908-05-05 12:34:56.1012,1908-12-25 +1908-05-05 12:34:56.1013,1908-12-25 +1908-05-05 12:34:56.1014,1908-12-25 +1908-05-05 12:34:56.1015,1908-12-25 +1908-05-05 12:34:56.1016,1908-12-25 +1908-05-05 12:34:56.1017,1908-12-25 +1908-05-05 12:34:56.1018,1908-12-25 +1908-05-05 12:34:56.1019,1908-12-25 +1908-05-05 12:34:56.102,1908-12-25 +1908-05-05 12:34:56.1021,1908-12-25 +1908-05-05 12:34:56.1022,1908-12-25 +1908-05-05 12:34:56.1023,1908-12-25 +1908-05-05 12:34:56.1024,1908-12-25 +1908-05-05 12:34:56.1025,1908-12-25 +1908-05-05 12:34:56.1026,1908-12-25 +1908-05-05 12:34:56.1027,1908-12-25 +1908-05-05 12:34:56.1028,1908-12-25 +1908-05-05 12:34:56.1029,1908-12-25 +1908-05-05 12:34:56.103,1908-12-25 +1908-05-05 12:34:56.1031,1908-12-25 +1908-05-05 12:34:56.1032,1908-12-25 +1908-05-05 12:34:56.1033,1908-12-25 +1908-05-05 12:34:56.1034,1908-12-25 +1908-05-05 12:34:56.1035,1908-12-25 +1908-05-05 12:34:56.1036,1908-12-25 +1908-05-05 12:34:56.1037,1908-12-25 +1908-05-05 12:34:56.1038,1908-12-25 +1908-05-05 12:34:56.1039,1908-12-25 +1908-05-05 12:34:56.104,1908-12-25 +1908-05-05 12:34:56.1041,1908-12-25 +1908-05-05 12:34:56.1042,1908-12-25 +1908-05-05 12:34:56.1043,1908-12-25 +1908-05-05 12:34:56.1044,1908-12-25 +1908-05-05 12:34:56.1045,1908-12-25 +1908-05-05 12:34:56.1046,1908-12-25 +1908-05-05 12:34:56.1047,1908-12-25 +1908-05-05 12:34:56.1048,1908-12-25 +1908-05-05 12:34:56.1049,1908-12-25 +1908-05-05 12:34:56.105,1908-12-25 +1908-05-05 12:34:56.1051,1908-12-25 +1908-05-05 12:34:56.1052,1908-12-25 +1908-05-05 12:34:56.1053,1908-12-25 +1908-05-05 12:34:56.1054,1908-12-25 +1908-05-05 12:34:56.1055,1908-12-25 +1908-05-05 12:34:56.1056,1908-12-25 +1908-05-05 12:34:56.1057,1908-12-25 +1908-05-05 12:34:56.1058,1908-12-25 +1908-05-05 12:34:56.1059,1908-12-25 +1908-05-05 12:34:56.106,1908-12-25 +1908-05-05 12:34:56.1061,1908-12-25 +1908-05-05 12:34:56.1062,1908-12-25 +1908-05-05 12:34:56.1063,1908-12-25 +1908-05-05 12:34:56.1064,1908-12-25 +1908-05-05 12:34:56.1065,1908-12-25 +1908-05-05 12:34:56.1066,1908-12-25 +1908-05-05 12:34:56.1067,1908-12-25 +1908-05-05 12:34:56.1068,1908-12-25 +1908-05-05 12:34:56.1069,1908-12-25 +1908-05-05 12:34:56.107,1908-12-25 +1908-05-05 12:34:56.1071,1908-12-25 +1908-05-05 12:34:56.1072,1908-12-25 +1908-05-05 12:34:56.1073,1908-12-25 +1908-05-05 12:34:56.1074,1908-12-25 +1908-05-05 12:34:56.1075,1908-12-25 +1908-05-05 12:34:56.1076,1908-12-25 +1908-05-05 12:34:56.1077,1908-12-25 +1908-05-05 12:34:56.1078,1908-12-25 +1908-05-05 12:34:56.1079,1908-12-25 +1908-05-05 12:34:56.108,1908-12-25 +1908-05-05 12:34:56.1081,1908-12-25 +1908-05-05 12:34:56.1082,1908-12-25 +1908-05-05 12:34:56.1083,1908-12-25 +1908-05-05 12:34:56.1084,1908-12-25 +1908-05-05 12:34:56.1085,1908-12-25 +1908-05-05 12:34:56.1086,1908-12-25 +1908-05-05 12:34:56.1087,1908-12-25 +1908-05-05 12:34:56.1088,1908-12-25 +1908-05-05 12:34:56.1089,1908-12-25 +1908-05-05 12:34:56.109,1908-12-25 +1908-05-05 12:34:56.1091,1908-12-25 +1908-05-05 12:34:56.1092,1908-12-25 +1908-05-05 12:34:56.1093,1908-12-25 +1908-05-05 12:34:56.1094,1908-12-25 +1908-05-05 12:34:56.1095,1908-12-25 +1908-05-05 12:34:56.1096,1908-12-25 +1908-05-05 12:34:56.1097,1908-12-25 +1908-05-05 12:34:56.1098,1908-12-25 +1908-05-05 12:34:56.1099,1908-12-25 +1908-05-05 12:34:56.11,1908-12-25 +1908-05-05 12:34:56.1101,1908-12-25 +1908-05-05 12:34:56.1102,1908-12-25 +1908-05-05 12:34:56.1103,1908-12-25 +1908-05-05 12:34:56.1104,1908-12-25 +1908-05-05 12:34:56.1105,1908-12-25 +1908-05-05 12:34:56.1106,1908-12-25 +1908-05-05 12:34:56.1107,1908-12-25 +1908-05-05 12:34:56.1108,1908-12-25 +1908-05-05 12:34:56.1109,1908-12-25 +1908-05-05 12:34:56.111,1908-12-25 +1908-05-05 12:34:56.1111,1908-12-25 +1908-05-05 12:34:56.1112,1908-12-25 +1908-05-05 12:34:56.1113,1908-12-25 +1908-05-05 12:34:56.1114,1908-12-25 +1908-05-05 12:34:56.1115,1908-12-25 +1908-05-05 12:34:56.1116,1908-12-25 +1908-05-05 12:34:56.1117,1908-12-25 +1908-05-05 12:34:56.1118,1908-12-25 +1908-05-05 12:34:56.1119,1908-12-25 +1908-05-05 12:34:56.112,1908-12-25 +1908-05-05 12:34:56.1121,1908-12-25 +1908-05-05 12:34:56.1122,1908-12-25 +1908-05-05 12:34:56.1123,1908-12-25 +1908-05-05 12:34:56.1124,1908-12-25 +1908-05-05 12:34:56.1125,1908-12-25 +1908-05-05 12:34:56.1126,1908-12-25 +1908-05-05 12:34:56.1127,1908-12-25 +1908-05-05 12:34:56.1128,1908-12-25 +1908-05-05 12:34:56.1129,1908-12-25 +1908-05-05 12:34:56.113,1908-12-25 +1908-05-05 12:34:56.1131,1908-12-25 +1908-05-05 12:34:56.1132,1908-12-25 +1908-05-05 12:34:56.1133,1908-12-25 +1908-05-05 12:34:56.1134,1908-12-25 +1908-05-05 12:34:56.1135,1908-12-25 +1908-05-05 12:34:56.1136,1908-12-25 +1908-05-05 12:34:56.1137,1908-12-25 +1908-05-05 12:34:56.1138,1908-12-25 +1908-05-05 12:34:56.1139,1908-12-25 +1908-05-05 12:34:56.114,1908-12-25 +1908-05-05 12:34:56.1141,1908-12-25 +1908-05-05 12:34:56.1142,1908-12-25 +1908-05-05 12:34:56.1143,1908-12-25 +1908-05-05 12:34:56.1144,1908-12-25 +1908-05-05 12:34:56.1145,1908-12-25 +1908-05-05 12:34:56.1146,1908-12-25 +1908-05-05 12:34:56.1147,1908-12-25 +1908-05-05 12:34:56.1148,1908-12-25 +1908-05-05 12:34:56.1149,1908-12-25 +1908-05-05 12:34:56.115,1908-12-25 +1908-05-05 12:34:56.1151,1908-12-25 +1908-05-05 12:34:56.1152,1908-12-25 +1908-05-05 12:34:56.1153,1908-12-25 +1908-05-05 12:34:56.1154,1908-12-25 +1908-05-05 12:34:56.1155,1908-12-25 +1908-05-05 12:34:56.1156,1908-12-25 +1908-05-05 12:34:56.1157,1908-12-25 +1908-05-05 12:34:56.1158,1908-12-25 +1908-05-05 12:34:56.1159,1908-12-25 +1908-05-05 12:34:56.116,1908-12-25 +1908-05-05 12:34:56.1161,1908-12-25 +1908-05-05 12:34:56.1162,1908-12-25 +1908-05-05 12:34:56.1163,1908-12-25 +1908-05-05 12:34:56.1164,1908-12-25 +1908-05-05 12:34:56.1165,1908-12-25 +1908-05-05 12:34:56.1166,1908-12-25 +1908-05-05 12:34:56.1167,1908-12-25 +1908-05-05 12:34:56.1168,1908-12-25 +1908-05-05 12:34:56.1169,1908-12-25 +1908-05-05 12:34:56.117,1908-12-25 +1908-05-05 12:34:56.1171,1908-12-25 +1908-05-05 12:34:56.1172,1908-12-25 +1908-05-05 12:34:56.1173,1908-12-25 +1908-05-05 12:34:56.1174,1908-12-25 +1908-05-05 12:34:56.1175,1908-12-25 +1908-05-05 12:34:56.1176,1908-12-25 +1908-05-05 12:34:56.1177,1908-12-25 +1908-05-05 12:34:56.1178,1908-12-25 +1908-05-05 12:34:56.1179,1908-12-25 +1908-05-05 12:34:56.118,1908-12-25 +1908-05-05 12:34:56.1181,1908-12-25 +1908-05-05 12:34:56.1182,1908-12-25 +1908-05-05 12:34:56.1183,1908-12-25 +1908-05-05 12:34:56.1184,1908-12-25 +1908-05-05 12:34:56.1185,1908-12-25 +1908-05-05 12:34:56.1186,1908-12-25 +1908-05-05 12:34:56.1187,1908-12-25 +1908-05-05 12:34:56.1188,1908-12-25 +1908-05-05 12:34:56.1189,1908-12-25 +1908-05-05 12:34:56.119,1908-12-25 +1908-05-05 12:34:56.1191,1908-12-25 +1908-05-05 12:34:56.1192,1908-12-25 +1908-05-05 12:34:56.1193,1908-12-25 +1908-05-05 12:34:56.1194,1908-12-25 +1908-05-05 12:34:56.1195,1908-12-25 +1908-05-05 12:34:56.1196,1908-12-25 +1908-05-05 12:34:56.1197,1908-12-25 +1908-05-05 12:34:56.1198,1908-12-25 +1908-05-05 12:34:56.1199,1908-12-25 +1908-05-05 12:34:56.12,1908-12-25 +1908-05-05 12:34:56.1201,1908-12-25 +1908-05-05 12:34:56.1202,1908-12-25 +1908-05-05 12:34:56.1203,1908-12-25 +1908-05-05 12:34:56.1204,1908-12-25 +1908-05-05 12:34:56.1205,1908-12-25 +1908-05-05 12:34:56.1206,1908-12-25 +1908-05-05 12:34:56.1207,1908-12-25 +1908-05-05 12:34:56.1208,1908-12-25 +1908-05-05 12:34:56.1209,1908-12-25 +1908-05-05 12:34:56.121,1908-12-25 +1908-05-05 12:34:56.1211,1908-12-25 +1908-05-05 12:34:56.1212,1908-12-25 +1908-05-05 12:34:56.1213,1908-12-25 +1908-05-05 12:34:56.1214,1908-12-25 +1908-05-05 12:34:56.1215,1908-12-25 +1908-05-05 12:34:56.1216,1908-12-25 +1908-05-05 12:34:56.1217,1908-12-25 +1908-05-05 12:34:56.1218,1908-12-25 +1908-05-05 12:34:56.1219,1908-12-25 +1908-05-05 12:34:56.122,1908-12-25 +1908-05-05 12:34:56.1221,1908-12-25 +1908-05-05 12:34:56.1222,1908-12-25 +1908-05-05 12:34:56.1223,1908-12-25 +1908-05-05 12:34:56.1224,1908-12-25 +1908-05-05 12:34:56.1225,1908-12-25 +1908-05-05 12:34:56.1226,1908-12-25 +1908-05-05 12:34:56.1227,1908-12-25 +1908-05-05 12:34:56.1228,1908-12-25 +1908-05-05 12:34:56.1229,1908-12-25 +1908-05-05 12:34:56.123,1908-12-25 +1908-05-05 12:34:56.1231,1908-12-25 +1908-05-05 12:34:56.1232,1908-12-25 +1908-05-05 12:34:56.1233,1908-12-25 +1908-05-05 12:34:56.1234,1908-12-25 +1908-05-05 12:34:56.1235,1908-12-25 +1908-05-05 12:34:56.1236,1908-12-25 +1908-05-05 12:34:56.1237,1908-12-25 +1908-05-05 12:34:56.1238,1908-12-25 +1908-05-05 12:34:56.1239,1908-12-25 +1908-05-05 12:34:56.124,1908-12-25 +1908-05-05 12:34:56.1241,1908-12-25 +1908-05-05 12:34:56.1242,1908-12-25 +1908-05-05 12:34:56.1243,1908-12-25 +1908-05-05 12:34:56.1244,1908-12-25 +1908-05-05 12:34:56.1245,1908-12-25 +1908-05-05 12:34:56.1246,1908-12-25 +1908-05-05 12:34:56.1247,1908-12-25 +1908-05-05 12:34:56.1248,1908-12-25 +1908-05-05 12:34:56.1249,1908-12-25 +1908-05-05 12:34:56.125,1908-12-25 +1908-05-05 12:34:56.1251,1908-12-25 +1908-05-05 12:34:56.1252,1908-12-25 +1908-05-05 12:34:56.1253,1908-12-25 +1908-05-05 12:34:56.1254,1908-12-25 +1908-05-05 12:34:56.1255,1908-12-25 +1908-05-05 12:34:56.1256,1908-12-25 +1908-05-05 12:34:56.1257,1908-12-25 +1908-05-05 12:34:56.1258,1908-12-25 +1908-05-05 12:34:56.1259,1908-12-25 +1908-05-05 12:34:56.126,1908-12-25 +1908-05-05 12:34:56.1261,1908-12-25 +1908-05-05 12:34:56.1262,1908-12-25 +1908-05-05 12:34:56.1263,1908-12-25 +1908-05-05 12:34:56.1264,1908-12-25 +1908-05-05 12:34:56.1265,1908-12-25 +1908-05-05 12:34:56.1266,1908-12-25 +1908-05-05 12:34:56.1267,1908-12-25 +1908-05-05 12:34:56.1268,1908-12-25 +1908-05-05 12:34:56.1269,1908-12-25 +1908-05-05 12:34:56.127,1908-12-25 +1908-05-05 12:34:56.1271,1908-12-25 +1908-05-05 12:34:56.1272,1908-12-25 +1908-05-05 12:34:56.1273,1908-12-25 +1908-05-05 12:34:56.1274,1908-12-25 +1908-05-05 12:34:56.1275,1908-12-25 +1908-05-05 12:34:56.1276,1908-12-25 +1908-05-05 12:34:56.1277,1908-12-25 +1908-05-05 12:34:56.1278,1908-12-25 +1908-05-05 12:34:56.1279,1908-12-25 +1908-05-05 12:34:56.128,1908-12-25 +1908-05-05 12:34:56.1281,1908-12-25 +1908-05-05 12:34:56.1282,1908-12-25 +1908-05-05 12:34:56.1283,1908-12-25 +1908-05-05 12:34:56.1284,1908-12-25 +1908-05-05 12:34:56.1285,1908-12-25 +1908-05-05 12:34:56.1286,1908-12-25 +1908-05-05 12:34:56.1287,1908-12-25 +1908-05-05 12:34:56.1288,1908-12-25 +1908-05-05 12:34:56.1289,1908-12-25 +1908-05-05 12:34:56.129,1908-12-25 +1908-05-05 12:34:56.1291,1908-12-25 +1908-05-05 12:34:56.1292,1908-12-25 +1908-05-05 12:34:56.1293,1908-12-25 +1908-05-05 12:34:56.1294,1908-12-25 +1908-05-05 12:34:56.1295,1908-12-25 +1908-05-05 12:34:56.1296,1908-12-25 +1908-05-05 12:34:56.1297,1908-12-25 +1908-05-05 12:34:56.1298,1908-12-25 +1908-05-05 12:34:56.1299,1908-12-25 +1908-05-05 12:34:56.13,1908-12-25 +1908-05-05 12:34:56.1301,1908-12-25 +1908-05-05 12:34:56.1302,1908-12-25 +1908-05-05 12:34:56.1303,1908-12-25 +1908-05-05 12:34:56.1304,1908-12-25 +1908-05-05 12:34:56.1305,1908-12-25 +1908-05-05 12:34:56.1306,1908-12-25 +1908-05-05 12:34:56.1307,1908-12-25 +1908-05-05 12:34:56.1308,1908-12-25 +1908-05-05 12:34:56.1309,1908-12-25 +1908-05-05 12:34:56.131,1908-12-25 +1908-05-05 12:34:56.1311,1908-12-25 +1908-05-05 12:34:56.1312,1908-12-25 +1908-05-05 12:34:56.1313,1908-12-25 +1908-05-05 12:34:56.1314,1908-12-25 +1908-05-05 12:34:56.1315,1908-12-25 +1908-05-05 12:34:56.1316,1908-12-25 +1908-05-05 12:34:56.1317,1908-12-25 +1908-05-05 12:34:56.1318,1908-12-25 +1908-05-05 12:34:56.1319,1908-12-25 +1908-05-05 12:34:56.132,1908-12-25 +1908-05-05 12:34:56.1321,1908-12-25 +1908-05-05 12:34:56.1322,1908-12-25 +1908-05-05 12:34:56.1323,1908-12-25 +1908-05-05 12:34:56.1324,1908-12-25 +1908-05-05 12:34:56.1325,1908-12-25 +1908-05-05 12:34:56.1326,1908-12-25 +1908-05-05 12:34:56.1327,1908-12-25 +1908-05-05 12:34:56.1328,1908-12-25 +1908-05-05 12:34:56.1329,1908-12-25 +1908-05-05 12:34:56.133,1908-12-25 +1908-05-05 12:34:56.1331,1908-12-25 +1908-05-05 12:34:56.1332,1908-12-25 +1908-05-05 12:34:56.1333,1908-12-25 +1908-05-05 12:34:56.1334,1908-12-25 +1908-05-05 12:34:56.1335,1908-12-25 +1908-05-05 12:34:56.1336,1908-12-25 +1908-05-05 12:34:56.1337,1908-12-25 +1908-05-05 12:34:56.1338,1908-12-25 +1908-05-05 12:34:56.1339,1908-12-25 +1908-05-05 12:34:56.134,1908-12-25 +1908-05-05 12:34:56.1341,1908-12-25 +1908-05-05 12:34:56.1342,1908-12-25 +1908-05-05 12:34:56.1343,1908-12-25 +1908-05-05 12:34:56.1344,1908-12-25 +1908-05-05 12:34:56.1345,1908-12-25 +1908-05-05 12:34:56.1346,1908-12-25 +1908-05-05 12:34:56.1347,1908-12-25 +1908-05-05 12:34:56.1348,1908-12-25 +1908-05-05 12:34:56.1349,1908-12-25 +1908-05-05 12:34:56.135,1908-12-25 +1908-05-05 12:34:56.1351,1908-12-25 +1908-05-05 12:34:56.1352,1908-12-25 +1908-05-05 12:34:56.1353,1908-12-25 +1908-05-05 12:34:56.1354,1908-12-25 +1908-05-05 12:34:56.1355,1908-12-25 +1908-05-05 12:34:56.1356,1908-12-25 +1908-05-05 12:34:56.1357,1908-12-25 +1908-05-05 12:34:56.1358,1908-12-25 +1908-05-05 12:34:56.1359,1908-12-25 +1908-05-05 12:34:56.136,1908-12-25 +1908-05-05 12:34:56.1361,1908-12-25 +1908-05-05 12:34:56.1362,1908-12-25 +1908-05-05 12:34:56.1363,1908-12-25 +1908-05-05 12:34:56.1364,1908-12-25 +1908-05-05 12:34:56.1365,1908-12-25 +1908-05-05 12:34:56.1366,1908-12-25 +1908-05-05 12:34:56.1367,1908-12-25 +1908-05-05 12:34:56.1368,1908-12-25 +1908-05-05 12:34:56.1369,1908-12-25 +1908-05-05 12:34:56.137,1908-12-25 +1908-05-05 12:34:56.1371,1908-12-25 +1908-05-05 12:34:56.1372,1908-12-25 +1908-05-05 12:34:56.1373,1908-12-25 +1908-05-05 12:34:56.1374,1908-12-25 +1908-05-05 12:34:56.1375,1908-12-25 +1908-05-05 12:34:56.1376,1908-12-25 +1908-05-05 12:34:56.1377,1908-12-25 +1908-05-05 12:34:56.1378,1908-12-25 +1908-05-05 12:34:56.1379,1908-12-25 +1908-05-05 12:34:56.138,1908-12-25 +1908-05-05 12:34:56.1381,1908-12-25 +1908-05-05 12:34:56.1382,1908-12-25 +1908-05-05 12:34:56.1383,1908-12-25 +1908-05-05 12:34:56.1384,1908-12-25 +1908-05-05 12:34:56.1385,1908-12-25 +1908-05-05 12:34:56.1386,1908-12-25 +1908-05-05 12:34:56.1387,1908-12-25 +1908-05-05 12:34:56.1388,1908-12-25 +1908-05-05 12:34:56.1389,1908-12-25 +1908-05-05 12:34:56.139,1908-12-25 +1908-05-05 12:34:56.1391,1908-12-25 +1908-05-05 12:34:56.1392,1908-12-25 +1908-05-05 12:34:56.1393,1908-12-25 +1908-05-05 12:34:56.1394,1908-12-25 +1908-05-05 12:34:56.1395,1908-12-25 +1908-05-05 12:34:56.1396,1908-12-25 +1908-05-05 12:34:56.1397,1908-12-25 +1908-05-05 12:34:56.1398,1908-12-25 +1908-05-05 12:34:56.1399,1908-12-25 +1908-05-05 12:34:56.14,1908-12-25 +1908-05-05 12:34:56.1401,1908-12-25 +1908-05-05 12:34:56.1402,1908-12-25 +1908-05-05 12:34:56.1403,1908-12-25 +1908-05-05 12:34:56.1404,1908-12-25 +1908-05-05 12:34:56.1405,1908-12-25 +1908-05-05 12:34:56.1406,1908-12-25 +1908-05-05 12:34:56.1407,1908-12-25 +1908-05-05 12:34:56.1408,1908-12-25 +1908-05-05 12:34:56.1409,1908-12-25 +1908-05-05 12:34:56.141,1908-12-25 +1908-05-05 12:34:56.1411,1908-12-25 +1908-05-05 12:34:56.1412,1908-12-25 +1908-05-05 12:34:56.1413,1908-12-25 +1908-05-05 12:34:56.1414,1908-12-25 +1908-05-05 12:34:56.1415,1908-12-25 +1908-05-05 12:34:56.1416,1908-12-25 +1908-05-05 12:34:56.1417,1908-12-25 +1908-05-05 12:34:56.1418,1908-12-25 +1908-05-05 12:34:56.1419,1908-12-25 +1908-05-05 12:34:56.142,1908-12-25 +1908-05-05 12:34:56.1421,1908-12-25 +1908-05-05 12:34:56.1422,1908-12-25 +1908-05-05 12:34:56.1423,1908-12-25 +1908-05-05 12:34:56.1424,1908-12-25 +1908-05-05 12:34:56.1425,1908-12-25 +1908-05-05 12:34:56.1426,1908-12-25 +1908-05-05 12:34:56.1427,1908-12-25 +1908-05-05 12:34:56.1428,1908-12-25 +1908-05-05 12:34:56.1429,1908-12-25 +1908-05-05 12:34:56.143,1908-12-25 +1908-05-05 12:34:56.1431,1908-12-25 +1908-05-05 12:34:56.1432,1908-12-25 +1908-05-05 12:34:56.1433,1908-12-25 +1908-05-05 12:34:56.1434,1908-12-25 +1908-05-05 12:34:56.1435,1908-12-25 +1908-05-05 12:34:56.1436,1908-12-25 +1908-05-05 12:34:56.1437,1908-12-25 +1908-05-05 12:34:56.1438,1908-12-25 +1908-05-05 12:34:56.1439,1908-12-25 +1908-05-05 12:34:56.144,1908-12-25 +1908-05-05 12:34:56.1441,1908-12-25 +1908-05-05 12:34:56.1442,1908-12-25 +1908-05-05 12:34:56.1443,1908-12-25 +1908-05-05 12:34:56.1444,1908-12-25 +1908-05-05 12:34:56.1445,1908-12-25 +1908-05-05 12:34:56.1446,1908-12-25 +1908-05-05 12:34:56.1447,1908-12-25 +1908-05-05 12:34:56.1448,1908-12-25 +1908-05-05 12:34:56.1449,1908-12-25 +1908-05-05 12:34:56.145,1908-12-25 +1908-05-05 12:34:56.1451,1908-12-25 +1908-05-05 12:34:56.1452,1908-12-25 +1908-05-05 12:34:56.1453,1908-12-25 +1908-05-05 12:34:56.1454,1908-12-25 +1908-05-05 12:34:56.1455,1908-12-25 +1908-05-05 12:34:56.1456,1908-12-25 +1908-05-05 12:34:56.1457,1908-12-25 +1908-05-05 12:34:56.1458,1908-12-25 +1908-05-05 12:34:56.1459,1908-12-25 +1908-05-05 12:34:56.146,1908-12-25 +1908-05-05 12:34:56.1461,1908-12-25 +1908-05-05 12:34:56.1462,1908-12-25 +1908-05-05 12:34:56.1463,1908-12-25 +1908-05-05 12:34:56.1464,1908-12-25 +1908-05-05 12:34:56.1465,1908-12-25 +1908-05-05 12:34:56.1466,1908-12-25 +1908-05-05 12:34:56.1467,1908-12-25 +1908-05-05 12:34:56.1468,1908-12-25 +1908-05-05 12:34:56.1469,1908-12-25 +1908-05-05 12:34:56.147,1908-12-25 +1908-05-05 12:34:56.1471,1908-12-25 +1908-05-05 12:34:56.1472,1908-12-25 +1908-05-05 12:34:56.1473,1908-12-25 +1908-05-05 12:34:56.1474,1908-12-25 +1908-05-05 12:34:56.1475,1908-12-25 +1908-05-05 12:34:56.1476,1908-12-25 +1908-05-05 12:34:56.1477,1908-12-25 +1908-05-05 12:34:56.1478,1908-12-25 +1908-05-05 12:34:56.1479,1908-12-25 +1908-05-05 12:34:56.148,1908-12-25 +1908-05-05 12:34:56.1481,1908-12-25 +1908-05-05 12:34:56.1482,1908-12-25 +1908-05-05 12:34:56.1483,1908-12-25 +1908-05-05 12:34:56.1484,1908-12-25 +1908-05-05 12:34:56.1485,1908-12-25 +1908-05-05 12:34:56.1486,1908-12-25 +1908-05-05 12:34:56.1487,1908-12-25 +1908-05-05 12:34:56.1488,1908-12-25 +1908-05-05 12:34:56.1489,1908-12-25 +1908-05-05 12:34:56.149,1908-12-25 +1908-05-05 12:34:56.1491,1908-12-25 +1908-05-05 12:34:56.1492,1908-12-25 +1908-05-05 12:34:56.1493,1908-12-25 +1908-05-05 12:34:56.1494,1908-12-25 +1908-05-05 12:34:56.1495,1908-12-25 +1908-05-05 12:34:56.1496,1908-12-25 +1908-05-05 12:34:56.1497,1908-12-25 +1908-05-05 12:34:56.1498,1908-12-25 +1908-05-05 12:34:56.1499,1908-12-25 +1908-05-05 12:34:56.15,1908-12-25 +1908-05-05 12:34:56.1501,1908-12-25 +1908-05-05 12:34:56.1502,1908-12-25 +1908-05-05 12:34:56.1503,1908-12-25 +1908-05-05 12:34:56.1504,1908-12-25 +1908-05-05 12:34:56.1505,1908-12-25 +1908-05-05 12:34:56.1506,1908-12-25 +1908-05-05 12:34:56.1507,1908-12-25 +1908-05-05 12:34:56.1508,1908-12-25 +1908-05-05 12:34:56.1509,1908-12-25 +1908-05-05 12:34:56.151,1908-12-25 +1908-05-05 12:34:56.1511,1908-12-25 +1908-05-05 12:34:56.1512,1908-12-25 +1908-05-05 12:34:56.1513,1908-12-25 +1908-05-05 12:34:56.1514,1908-12-25 +1908-05-05 12:34:56.1515,1908-12-25 +1908-05-05 12:34:56.1516,1908-12-25 +1908-05-05 12:34:56.1517,1908-12-25 +1908-05-05 12:34:56.1518,1908-12-25 +1908-05-05 12:34:56.1519,1908-12-25 +1908-05-05 12:34:56.152,1908-12-25 +1908-05-05 12:34:56.1521,1908-12-25 +1908-05-05 12:34:56.1522,1908-12-25 +1908-05-05 12:34:56.1523,1908-12-25 +1908-05-05 12:34:56.1524,1908-12-25 +1908-05-05 12:34:56.1525,1908-12-25 +1908-05-05 12:34:56.1526,1908-12-25 +1908-05-05 12:34:56.1527,1908-12-25 +1908-05-05 12:34:56.1528,1908-12-25 +1908-05-05 12:34:56.1529,1908-12-25 +1908-05-05 12:34:56.153,1908-12-25 +1908-05-05 12:34:56.1531,1908-12-25 +1908-05-05 12:34:56.1532,1908-12-25 +1908-05-05 12:34:56.1533,1908-12-25 +1908-05-05 12:34:56.1534,1908-12-25 +1908-05-05 12:34:56.1535,1908-12-25 +1908-05-05 12:34:56.1536,1908-12-25 +1908-05-05 12:34:56.1537,1908-12-25 +1908-05-05 12:34:56.1538,1908-12-25 +1908-05-05 12:34:56.1539,1908-12-25 +1908-05-05 12:34:56.154,1908-12-25 +1908-05-05 12:34:56.1541,1908-12-25 +1908-05-05 12:34:56.1542,1908-12-25 +1908-05-05 12:34:56.1543,1908-12-25 +1908-05-05 12:34:56.1544,1908-12-25 +1908-05-05 12:34:56.1545,1908-12-25 +1908-05-05 12:34:56.1546,1908-12-25 +1908-05-05 12:34:56.1547,1908-12-25 +1908-05-05 12:34:56.1548,1908-12-25 +1908-05-05 12:34:56.1549,1908-12-25 +1908-05-05 12:34:56.155,1908-12-25 +1908-05-05 12:34:56.1551,1908-12-25 +1908-05-05 12:34:56.1552,1908-12-25 +1908-05-05 12:34:56.1553,1908-12-25 +1908-05-05 12:34:56.1554,1908-12-25 +1908-05-05 12:34:56.1555,1908-12-25 +1908-05-05 12:34:56.1556,1908-12-25 +1908-05-05 12:34:56.1557,1908-12-25 +1908-05-05 12:34:56.1558,1908-12-25 +1908-05-05 12:34:56.1559,1908-12-25 +1908-05-05 12:34:56.156,1908-12-25 +1908-05-05 12:34:56.1561,1908-12-25 +1908-05-05 12:34:56.1562,1908-12-25 +1908-05-05 12:34:56.1563,1908-12-25 +1908-05-05 12:34:56.1564,1908-12-25 +1908-05-05 12:34:56.1565,1908-12-25 +1908-05-05 12:34:56.1566,1908-12-25 +1908-05-05 12:34:56.1567,1908-12-25 +1908-05-05 12:34:56.1568,1908-12-25 +1908-05-05 12:34:56.1569,1908-12-25 +1908-05-05 12:34:56.157,1908-12-25 +1908-05-05 12:34:56.1571,1908-12-25 +1908-05-05 12:34:56.1572,1908-12-25 +1908-05-05 12:34:56.1573,1908-12-25 +1908-05-05 12:34:56.1574,1908-12-25 +1908-05-05 12:34:56.1575,1908-12-25 +1908-05-05 12:34:56.1576,1908-12-25 +1908-05-05 12:34:56.1577,1908-12-25 +1908-05-05 12:34:56.1578,1908-12-25 +1908-05-05 12:34:56.1579,1908-12-25 +1908-05-05 12:34:56.158,1908-12-25 +1908-05-05 12:34:56.1581,1908-12-25 +1908-05-05 12:34:56.1582,1908-12-25 +1908-05-05 12:34:56.1583,1908-12-25 +1908-05-05 12:34:56.1584,1908-12-25 +1908-05-05 12:34:56.1585,1908-12-25 +1908-05-05 12:34:56.1586,1908-12-25 +1908-05-05 12:34:56.1587,1908-12-25 +1908-05-05 12:34:56.1588,1908-12-25 +1908-05-05 12:34:56.1589,1908-12-25 +1908-05-05 12:34:56.159,1908-12-25 +1908-05-05 12:34:56.1591,1908-12-25 +1908-05-05 12:34:56.1592,1908-12-25 +1908-05-05 12:34:56.1593,1908-12-25 +1908-05-05 12:34:56.1594,1908-12-25 +1908-05-05 12:34:56.1595,1908-12-25 +1908-05-05 12:34:56.1596,1908-12-25 +1908-05-05 12:34:56.1597,1908-12-25 +1908-05-05 12:34:56.1598,1908-12-25 +1908-05-05 12:34:56.1599,1908-12-25 +1908-05-05 12:34:56.16,1908-12-25 +1908-05-05 12:34:56.1601,1908-12-25 +1908-05-05 12:34:56.1602,1908-12-25 +1908-05-05 12:34:56.1603,1908-12-25 +1908-05-05 12:34:56.1604,1908-12-25 +1908-05-05 12:34:56.1605,1908-12-25 +1908-05-05 12:34:56.1606,1908-12-25 +1908-05-05 12:34:56.1607,1908-12-25 +1908-05-05 12:34:56.1608,1908-12-25 +1908-05-05 12:34:56.1609,1908-12-25 +1908-05-05 12:34:56.161,1908-12-25 +1908-05-05 12:34:56.1611,1908-12-25 +1908-05-05 12:34:56.1612,1908-12-25 +1908-05-05 12:34:56.1613,1908-12-25 +1908-05-05 12:34:56.1614,1908-12-25 +1908-05-05 12:34:56.1615,1908-12-25 +1908-05-05 12:34:56.1616,1908-12-25 +1908-05-05 12:34:56.1617,1908-12-25 +1908-05-05 12:34:56.1618,1908-12-25 +1908-05-05 12:34:56.1619,1908-12-25 +1908-05-05 12:34:56.162,1908-12-25 +1908-05-05 12:34:56.1621,1908-12-25 +1908-05-05 12:34:56.1622,1908-12-25 +1908-05-05 12:34:56.1623,1908-12-25 +1908-05-05 12:34:56.1624,1908-12-25 +1908-05-05 12:34:56.1625,1908-12-25 +1908-05-05 12:34:56.1626,1908-12-25 +1908-05-05 12:34:56.1627,1908-12-25 +1908-05-05 12:34:56.1628,1908-12-25 +1908-05-05 12:34:56.1629,1908-12-25 +1908-05-05 12:34:56.163,1908-12-25 +1908-05-05 12:34:56.1631,1908-12-25 +1908-05-05 12:34:56.1632,1908-12-25 +1908-05-05 12:34:56.1633,1908-12-25 +1908-05-05 12:34:56.1634,1908-12-25 +1908-05-05 12:34:56.1635,1908-12-25 +1908-05-05 12:34:56.1636,1908-12-25 +1908-05-05 12:34:56.1637,1908-12-25 +1908-05-05 12:34:56.1638,1908-12-25 +1908-05-05 12:34:56.1639,1908-12-25 +1908-05-05 12:34:56.164,1908-12-25 +1908-05-05 12:34:56.1641,1908-12-25 +1908-05-05 12:34:56.1642,1908-12-25 +1908-05-05 12:34:56.1643,1908-12-25 +1908-05-05 12:34:56.1644,1908-12-25 +1908-05-05 12:34:56.1645,1908-12-25 +1908-05-05 12:34:56.1646,1908-12-25 +1908-05-05 12:34:56.1647,1908-12-25 +1908-05-05 12:34:56.1648,1908-12-25 +1908-05-05 12:34:56.1649,1908-12-25 +1908-05-05 12:34:56.165,1908-12-25 +1908-05-05 12:34:56.1651,1908-12-25 +1908-05-05 12:34:56.1652,1908-12-25 +1908-05-05 12:34:56.1653,1908-12-25 +1908-05-05 12:34:56.1654,1908-12-25 +1908-05-05 12:34:56.1655,1908-12-25 +1908-05-05 12:34:56.1656,1908-12-25 +1908-05-05 12:34:56.1657,1908-12-25 +1908-05-05 12:34:56.1658,1908-12-25 +1908-05-05 12:34:56.1659,1908-12-25 +1908-05-05 12:34:56.166,1908-12-25 +1908-05-05 12:34:56.1661,1908-12-25 +1908-05-05 12:34:56.1662,1908-12-25 +1908-05-05 12:34:56.1663,1908-12-25 +1908-05-05 12:34:56.1664,1908-12-25 +1908-05-05 12:34:56.1665,1908-12-25 +1908-05-05 12:34:56.1666,1908-12-25 +1908-05-05 12:34:56.1667,1908-12-25 +1908-05-05 12:34:56.1668,1908-12-25 +1908-05-05 12:34:56.1669,1908-12-25 +1908-05-05 12:34:56.167,1908-12-25 +1908-05-05 12:34:56.1671,1908-12-25 +1908-05-05 12:34:56.1672,1908-12-25 +1908-05-05 12:34:56.1673,1908-12-25 +1908-05-05 12:34:56.1674,1908-12-25 +1908-05-05 12:34:56.1675,1908-12-25 +1908-05-05 12:34:56.1676,1908-12-25 +1908-05-05 12:34:56.1677,1908-12-25 +1908-05-05 12:34:56.1678,1908-12-25 +1908-05-05 12:34:56.1679,1908-12-25 +1908-05-05 12:34:56.168,1908-12-25 +1908-05-05 12:34:56.1681,1908-12-25 +1908-05-05 12:34:56.1682,1908-12-25 +1908-05-05 12:34:56.1683,1908-12-25 +1908-05-05 12:34:56.1684,1908-12-25 +1908-05-05 12:34:56.1685,1908-12-25 +1908-05-05 12:34:56.1686,1908-12-25 +1908-05-05 12:34:56.1687,1908-12-25 +1908-05-05 12:34:56.1688,1908-12-25 +1908-05-05 12:34:56.1689,1908-12-25 +1908-05-05 12:34:56.169,1908-12-25 +1908-05-05 12:34:56.1691,1908-12-25 +1908-05-05 12:34:56.1692,1908-12-25 +1908-05-05 12:34:56.1693,1908-12-25 +1908-05-05 12:34:56.1694,1908-12-25 +1908-05-05 12:34:56.1695,1908-12-25 +1908-05-05 12:34:56.1696,1908-12-25 +1908-05-05 12:34:56.1697,1908-12-25 +1908-05-05 12:34:56.1698,1908-12-25 +1908-05-05 12:34:56.1699,1908-12-25 +1908-05-05 12:34:56.17,1908-12-25 +1908-05-05 12:34:56.1701,1908-12-25 +1908-05-05 12:34:56.1702,1908-12-25 +1908-05-05 12:34:56.1703,1908-12-25 +1908-05-05 12:34:56.1704,1908-12-25 +1908-05-05 12:34:56.1705,1908-12-25 +1908-05-05 12:34:56.1706,1908-12-25 +1908-05-05 12:34:56.1707,1908-12-25 +1908-05-05 12:34:56.1708,1908-12-25 +1908-05-05 12:34:56.1709,1908-12-25 +1908-05-05 12:34:56.171,1908-12-25 +1908-05-05 12:34:56.1711,1908-12-25 +1908-05-05 12:34:56.1712,1908-12-25 +1908-05-05 12:34:56.1713,1908-12-25 +1908-05-05 12:34:56.1714,1908-12-25 +1908-05-05 12:34:56.1715,1908-12-25 +1908-05-05 12:34:56.1716,1908-12-25 +1908-05-05 12:34:56.1717,1908-12-25 +1908-05-05 12:34:56.1718,1908-12-25 +1908-05-05 12:34:56.1719,1908-12-25 +1908-05-05 12:34:56.172,1908-12-25 +1908-05-05 12:34:56.1721,1908-12-25 +1908-05-05 12:34:56.1722,1908-12-25 +1908-05-05 12:34:56.1723,1908-12-25 +1908-05-05 12:34:56.1724,1908-12-25 +1908-05-05 12:34:56.1725,1908-12-25 +1908-05-05 12:34:56.1726,1908-12-25 +1908-05-05 12:34:56.1727,1908-12-25 +1908-05-05 12:34:56.1728,1908-12-25 +1908-05-05 12:34:56.1729,1908-12-25 +1908-05-05 12:34:56.173,1908-12-25 +1908-05-05 12:34:56.1731,1908-12-25 +1908-05-05 12:34:56.1732,1908-12-25 +1908-05-05 12:34:56.1733,1908-12-25 +1908-05-05 12:34:56.1734,1908-12-25 +1908-05-05 12:34:56.1735,1908-12-25 +1908-05-05 12:34:56.1736,1908-12-25 +1908-05-05 12:34:56.1737,1908-12-25 +1908-05-05 12:34:56.1738,1908-12-25 +1908-05-05 12:34:56.1739,1908-12-25 +1908-05-05 12:34:56.174,1908-12-25 +1908-05-05 12:34:56.1741,1908-12-25 +1908-05-05 12:34:56.1742,1908-12-25 +1908-05-05 12:34:56.1743,1908-12-25 +1908-05-05 12:34:56.1744,1908-12-25 +1908-05-05 12:34:56.1745,1908-12-25 +1908-05-05 12:34:56.1746,1908-12-25 +1908-05-05 12:34:56.1747,1908-12-25 +1908-05-05 12:34:56.1748,1908-12-25 +1908-05-05 12:34:56.1749,1908-12-25 +1908-05-05 12:34:56.175,1908-12-25 +1908-05-05 12:34:56.1751,1908-12-25 +1908-05-05 12:34:56.1752,1908-12-25 +1908-05-05 12:34:56.1753,1908-12-25 +1908-05-05 12:34:56.1754,1908-12-25 +1908-05-05 12:34:56.1755,1908-12-25 +1908-05-05 12:34:56.1756,1908-12-25 +1908-05-05 12:34:56.1757,1908-12-25 +1908-05-05 12:34:56.1758,1908-12-25 +1908-05-05 12:34:56.1759,1908-12-25 +1908-05-05 12:34:56.176,1908-12-25 +1908-05-05 12:34:56.1761,1908-12-25 +1908-05-05 12:34:56.1762,1908-12-25 +1908-05-05 12:34:56.1763,1908-12-25 +1908-05-05 12:34:56.1764,1908-12-25 +1908-05-05 12:34:56.1765,1908-12-25 +1908-05-05 12:34:56.1766,1908-12-25 +1908-05-05 12:34:56.1767,1908-12-25 +1908-05-05 12:34:56.1768,1908-12-25 +1908-05-05 12:34:56.1769,1908-12-25 +1908-05-05 12:34:56.177,1908-12-25 +1908-05-05 12:34:56.1771,1908-12-25 +1908-05-05 12:34:56.1772,1908-12-25 +1908-05-05 12:34:56.1773,1908-12-25 +1908-05-05 12:34:56.1774,1908-12-25 +1908-05-05 12:34:56.1775,1908-12-25 +1908-05-05 12:34:56.1776,1908-12-25 +1908-05-05 12:34:56.1777,1908-12-25 +1908-05-05 12:34:56.1778,1908-12-25 +1908-05-05 12:34:56.1779,1908-12-25 +1908-05-05 12:34:56.178,1908-12-25 +1908-05-05 12:34:56.1781,1908-12-25 +1908-05-05 12:34:56.1782,1908-12-25 +1908-05-05 12:34:56.1783,1908-12-25 +1908-05-05 12:34:56.1784,1908-12-25 +1908-05-05 12:34:56.1785,1908-12-25 +1908-05-05 12:34:56.1786,1908-12-25 +1908-05-05 12:34:56.1787,1908-12-25 +1908-05-05 12:34:56.1788,1908-12-25 +1908-05-05 12:34:56.1789,1908-12-25 +1908-05-05 12:34:56.179,1908-12-25 +1908-05-05 12:34:56.1791,1908-12-25 +1908-05-05 12:34:56.1792,1908-12-25 +1908-05-05 12:34:56.1793,1908-12-25 +1908-05-05 12:34:56.1794,1908-12-25 +1908-05-05 12:34:56.1795,1908-12-25 +1908-05-05 12:34:56.1796,1908-12-25 +1908-05-05 12:34:56.1797,1908-12-25 +1908-05-05 12:34:56.1798,1908-12-25 +1908-05-05 12:34:56.1799,1908-12-25 +1908-05-05 12:34:56.18,1908-12-25 +1908-05-05 12:34:56.1801,1908-12-25 +1908-05-05 12:34:56.1802,1908-12-25 +1908-05-05 12:34:56.1803,1908-12-25 +1908-05-05 12:34:56.1804,1908-12-25 +1908-05-05 12:34:56.1805,1908-12-25 +1908-05-05 12:34:56.1806,1908-12-25 +1908-05-05 12:34:56.1807,1908-12-25 +1908-05-05 12:34:56.1808,1908-12-25 +1908-05-05 12:34:56.1809,1908-12-25 +1908-05-05 12:34:56.181,1908-12-25 +1908-05-05 12:34:56.1811,1908-12-25 +1908-05-05 12:34:56.1812,1908-12-25 +1908-05-05 12:34:56.1813,1908-12-25 +1908-05-05 12:34:56.1814,1908-12-25 +1908-05-05 12:34:56.1815,1908-12-25 +1908-05-05 12:34:56.1816,1908-12-25 +1908-05-05 12:34:56.1817,1908-12-25 +1908-05-05 12:34:56.1818,1908-12-25 +1908-05-05 12:34:56.1819,1908-12-25 +1908-05-05 12:34:56.182,1908-12-25 +1908-05-05 12:34:56.1821,1908-12-25 +1908-05-05 12:34:56.1822,1908-12-25 +1908-05-05 12:34:56.1823,1908-12-25 +1908-05-05 12:34:56.1824,1908-12-25 +1908-05-05 12:34:56.1825,1908-12-25 +1908-05-05 12:34:56.1826,1908-12-25 +1908-05-05 12:34:56.1827,1908-12-25 +1908-05-05 12:34:56.1828,1908-12-25 +1908-05-05 12:34:56.1829,1908-12-25 +1908-05-05 12:34:56.183,1908-12-25 +1908-05-05 12:34:56.1831,1908-12-25 +1908-05-05 12:34:56.1832,1908-12-25 +1908-05-05 12:34:56.1833,1908-12-25 +1908-05-05 12:34:56.1834,1908-12-25 +1908-05-05 12:34:56.1835,1908-12-25 +1908-05-05 12:34:56.1836,1908-12-25 +1908-05-05 12:34:56.1837,1908-12-25 +1908-05-05 12:34:56.1838,1908-12-25 +1908-05-05 12:34:56.1839,1908-12-25 +1908-05-05 12:34:56.184,1908-12-25 +1908-05-05 12:34:56.1841,1908-12-25 +1908-05-05 12:34:56.1842,1908-12-25 +1908-05-05 12:34:56.1843,1908-12-25 +1908-05-05 12:34:56.1844,1908-12-25 +1908-05-05 12:34:56.1845,1908-12-25 +1908-05-05 12:34:56.1846,1908-12-25 +1908-05-05 12:34:56.1847,1908-12-25 +1908-05-05 12:34:56.1848,1908-12-25 +1908-05-05 12:34:56.1849,1908-12-25 +1908-05-05 12:34:56.185,1908-12-25 +1908-05-05 12:34:56.1851,1908-12-25 +1908-05-05 12:34:56.1852,1908-12-25 +1908-05-05 12:34:56.1853,1908-12-25 +1908-05-05 12:34:56.1854,1908-12-25 +1908-05-05 12:34:56.1855,1908-12-25 +1908-05-05 12:34:56.1856,1908-12-25 +1908-05-05 12:34:56.1857,1908-12-25 +1908-05-05 12:34:56.1858,1908-12-25 +1908-05-05 12:34:56.1859,1908-12-25 +1908-05-05 12:34:56.186,1908-12-25 +1908-05-05 12:34:56.1861,1908-12-25 +1908-05-05 12:34:56.1862,1908-12-25 +1908-05-05 12:34:56.1863,1908-12-25 +1908-05-05 12:34:56.1864,1908-12-25 +1908-05-05 12:34:56.1865,1908-12-25 +1908-05-05 12:34:56.1866,1908-12-25 +1908-05-05 12:34:56.1867,1908-12-25 +1908-05-05 12:34:56.1868,1908-12-25 +1908-05-05 12:34:56.1869,1908-12-25 +1908-05-05 12:34:56.187,1908-12-25 +1908-05-05 12:34:56.1871,1908-12-25 +1908-05-05 12:34:56.1872,1908-12-25 +1908-05-05 12:34:56.1873,1908-12-25 +1908-05-05 12:34:56.1874,1908-12-25 +1908-05-05 12:34:56.1875,1908-12-25 +1908-05-05 12:34:56.1876,1908-12-25 +1908-05-05 12:34:56.1877,1908-12-25 +1908-05-05 12:34:56.1878,1908-12-25 +1908-05-05 12:34:56.1879,1908-12-25 +1908-05-05 12:34:56.188,1908-12-25 +1908-05-05 12:34:56.1881,1908-12-25 +1908-05-05 12:34:56.1882,1908-12-25 +1908-05-05 12:34:56.1883,1908-12-25 +1908-05-05 12:34:56.1884,1908-12-25 +1908-05-05 12:34:56.1885,1908-12-25 +1908-05-05 12:34:56.1886,1908-12-25 +1908-05-05 12:34:56.1887,1908-12-25 +1908-05-05 12:34:56.1888,1908-12-25 +1908-05-05 12:34:56.1889,1908-12-25 +1908-05-05 12:34:56.189,1908-12-25 +1908-05-05 12:34:56.1891,1908-12-25 +1908-05-05 12:34:56.1892,1908-12-25 +1908-05-05 12:34:56.1893,1908-12-25 +1908-05-05 12:34:56.1894,1908-12-25 +1908-05-05 12:34:56.1895,1908-12-25 +1908-05-05 12:34:56.1896,1908-12-25 +1908-05-05 12:34:56.1897,1908-12-25 +1908-05-05 12:34:56.1898,1908-12-25 +1908-05-05 12:34:56.1899,1908-12-25 +1908-05-05 12:34:56.19,1908-12-25 +1908-05-05 12:34:56.1901,1908-12-25 +1908-05-05 12:34:56.1902,1908-12-25 +1908-05-05 12:34:56.1903,1908-12-25 +1908-05-05 12:34:56.1904,1908-12-25 +1908-05-05 12:34:56.1905,1908-12-25 +1908-05-05 12:34:56.1906,1908-12-25 +1908-05-05 12:34:56.1907,1908-12-25 +1908-05-05 12:34:56.1908,1908-12-25 +1908-05-05 12:34:56.1909,1908-12-25 +1908-05-05 12:34:56.191,1908-12-25 +1908-05-05 12:34:56.1911,1908-12-25 +1908-05-05 12:34:56.1912,1908-12-25 +1908-05-05 12:34:56.1913,1908-12-25 +1908-05-05 12:34:56.1914,1908-12-25 +1908-05-05 12:34:56.1915,1908-12-25 +1908-05-05 12:34:56.1916,1908-12-25 +1908-05-05 12:34:56.1917,1908-12-25 +1908-05-05 12:34:56.1918,1908-12-25 +1908-05-05 12:34:56.1919,1908-12-25 +1908-05-05 12:34:56.192,1908-12-25 +1908-05-05 12:34:56.1921,1908-12-25 +1908-05-05 12:34:56.1922,1908-12-25 +1908-05-05 12:34:56.1923,1908-12-25 +1908-05-05 12:34:56.1924,1908-12-25 +1908-05-05 12:34:56.1925,1908-12-25 +1908-05-05 12:34:56.1926,1908-12-25 +1908-05-05 12:34:56.1927,1908-12-25 +1908-05-05 12:34:56.1928,1908-12-25 +1908-05-05 12:34:56.1929,1908-12-25 +1908-05-05 12:34:56.193,1908-12-25 +1908-05-05 12:34:56.1931,1908-12-25 +1908-05-05 12:34:56.1932,1908-12-25 +1908-05-05 12:34:56.1933,1908-12-25 +1908-05-05 12:34:56.1934,1908-12-25 +1908-05-05 12:34:56.1935,1908-12-25 +1908-05-05 12:34:56.1936,1908-12-25 +1908-05-05 12:34:56.1937,1908-12-25 +1908-05-05 12:34:56.1938,1908-12-25 +1908-05-05 12:34:56.1939,1908-12-25 +1908-05-05 12:34:56.194,1908-12-25 +1908-05-05 12:34:56.1941,1908-12-25 +1908-05-05 12:34:56.1942,1908-12-25 +1908-05-05 12:34:56.1943,1908-12-25 +1908-05-05 12:34:56.1944,1908-12-25 +1908-05-05 12:34:56.1945,1908-12-25 +1908-05-05 12:34:56.1946,1908-12-25 +1908-05-05 12:34:56.1947,1908-12-25 +1908-05-05 12:34:56.1948,1908-12-25 +1908-05-05 12:34:56.1949,1908-12-25 +1908-05-05 12:34:56.195,1908-12-25 +1908-05-05 12:34:56.1951,1908-12-25 +1908-05-05 12:34:56.1952,1908-12-25 +1908-05-05 12:34:56.1953,1908-12-25 +1908-05-05 12:34:56.1954,1908-12-25 +1908-05-05 12:34:56.1955,1908-12-25 +1908-05-05 12:34:56.1956,1908-12-25 +1908-05-05 12:34:56.1957,1908-12-25 +1908-05-05 12:34:56.1958,1908-12-25 +1908-05-05 12:34:56.1959,1908-12-25 +1908-05-05 12:34:56.196,1908-12-25 +1908-05-05 12:34:56.1961,1908-12-25 +1908-05-05 12:34:56.1962,1908-12-25 +1908-05-05 12:34:56.1963,1908-12-25 +1908-05-05 12:34:56.1964,1908-12-25 +1908-05-05 12:34:56.1965,1908-12-25 +1908-05-05 12:34:56.1966,1908-12-25 +1908-05-05 12:34:56.1967,1908-12-25 +1908-05-05 12:34:56.1968,1908-12-25 +1908-05-05 12:34:56.1969,1908-12-25 +1908-05-05 12:34:56.197,1908-12-25 +1908-05-05 12:34:56.1971,1908-12-25 +1908-05-05 12:34:56.1972,1908-12-25 +1908-05-05 12:34:56.1973,1908-12-25 +1908-05-05 12:34:56.1974,1908-12-25 +1908-05-05 12:34:56.1975,1908-12-25 +1908-05-05 12:34:56.1976,1908-12-25 +1908-05-05 12:34:56.1977,1908-12-25 +1908-05-05 12:34:56.1978,1908-12-25 +1908-05-05 12:34:56.1979,1908-12-25 +1908-05-05 12:34:56.198,1908-12-25 +1908-05-05 12:34:56.1981,1908-12-25 +1908-05-05 12:34:56.1982,1908-12-25 +1908-05-05 12:34:56.1983,1908-12-25 +1908-05-05 12:34:56.1984,1908-12-25 +1908-05-05 12:34:56.1985,1908-12-25 +1908-05-05 12:34:56.1986,1908-12-25 +1908-05-05 12:34:56.1987,1908-12-25 +1908-05-05 12:34:56.1988,1908-12-25 +1908-05-05 12:34:56.1989,1908-12-25 +1908-05-05 12:34:56.199,1908-12-25 +1908-05-05 12:34:56.1991,1908-12-25 +1908-05-05 12:34:56.1992,1908-12-25 +1908-05-05 12:34:56.1993,1908-12-25 +1908-05-05 12:34:56.1994,1908-12-25 +1908-05-05 12:34:56.1995,1908-12-25 +1908-05-05 12:34:56.1996,1908-12-25 +1908-05-05 12:34:56.1997,1908-12-25 +1908-05-05 12:34:56.1998,1908-12-25 +1908-05-05 12:34:56.1999,1908-12-25 +1909-05-05 12:34:56.1,1909-12-25 +1909-05-05 12:34:56.1001,1909-12-25 +1909-05-05 12:34:56.1002,1909-12-25 +1909-05-05 12:34:56.1003,1909-12-25 +1909-05-05 12:34:56.1004,1909-12-25 +1909-05-05 12:34:56.1005,1909-12-25 +1909-05-05 12:34:56.1006,1909-12-25 +1909-05-05 12:34:56.1007,1909-12-25 +1909-05-05 12:34:56.1008,1909-12-25 +1909-05-05 12:34:56.1009,1909-12-25 +1909-05-05 12:34:56.101,1909-12-25 +1909-05-05 12:34:56.1011,1909-12-25 +1909-05-05 12:34:56.1012,1909-12-25 +1909-05-05 12:34:56.1013,1909-12-25 +1909-05-05 12:34:56.1014,1909-12-25 +1909-05-05 12:34:56.1015,1909-12-25 +1909-05-05 12:34:56.1016,1909-12-25 +1909-05-05 12:34:56.1017,1909-12-25 +1909-05-05 12:34:56.1018,1909-12-25 +1909-05-05 12:34:56.1019,1909-12-25 +1909-05-05 12:34:56.102,1909-12-25 +1909-05-05 12:34:56.1021,1909-12-25 +1909-05-05 12:34:56.1022,1909-12-25 +1909-05-05 12:34:56.1023,1909-12-25 +1909-05-05 12:34:56.1024,1909-12-25 +1909-05-05 12:34:56.1025,1909-12-25 +1909-05-05 12:34:56.1026,1909-12-25 +1909-05-05 12:34:56.1027,1909-12-25 +1909-05-05 12:34:56.1028,1909-12-25 +1909-05-05 12:34:56.1029,1909-12-25 +1909-05-05 12:34:56.103,1909-12-25 +1909-05-05 12:34:56.1031,1909-12-25 +1909-05-05 12:34:56.1032,1909-12-25 +1909-05-05 12:34:56.1033,1909-12-25 +1909-05-05 12:34:56.1034,1909-12-25 +1909-05-05 12:34:56.1035,1909-12-25 +1909-05-05 12:34:56.1036,1909-12-25 +1909-05-05 12:34:56.1037,1909-12-25 +1909-05-05 12:34:56.1038,1909-12-25 +1909-05-05 12:34:56.1039,1909-12-25 +1909-05-05 12:34:56.104,1909-12-25 +1909-05-05 12:34:56.1041,1909-12-25 +1909-05-05 12:34:56.1042,1909-12-25 +1909-05-05 12:34:56.1043,1909-12-25 +1909-05-05 12:34:56.1044,1909-12-25 +1909-05-05 12:34:56.1045,1909-12-25 +1909-05-05 12:34:56.1046,1909-12-25 +1909-05-05 12:34:56.1047,1909-12-25 +1909-05-05 12:34:56.1048,1909-12-25 +1909-05-05 12:34:56.1049,1909-12-25 +1909-05-05 12:34:56.105,1909-12-25 +1909-05-05 12:34:56.1051,1909-12-25 +1909-05-05 12:34:56.1052,1909-12-25 +1909-05-05 12:34:56.1053,1909-12-25 +1909-05-05 12:34:56.1054,1909-12-25 +1909-05-05 12:34:56.1055,1909-12-25 +1909-05-05 12:34:56.1056,1909-12-25 +1909-05-05 12:34:56.1057,1909-12-25 +1909-05-05 12:34:56.1058,1909-12-25 +1909-05-05 12:34:56.1059,1909-12-25 +1909-05-05 12:34:56.106,1909-12-25 +1909-05-05 12:34:56.1061,1909-12-25 +1909-05-05 12:34:56.1062,1909-12-25 +1909-05-05 12:34:56.1063,1909-12-25 +1909-05-05 12:34:56.1064,1909-12-25 +1909-05-05 12:34:56.1065,1909-12-25 +1909-05-05 12:34:56.1066,1909-12-25 +1909-05-05 12:34:56.1067,1909-12-25 +1909-05-05 12:34:56.1068,1909-12-25 +1909-05-05 12:34:56.1069,1909-12-25 +1909-05-05 12:34:56.107,1909-12-25 +1909-05-05 12:34:56.1071,1909-12-25 +1909-05-05 12:34:56.1072,1909-12-25 +1909-05-05 12:34:56.1073,1909-12-25 +1909-05-05 12:34:56.1074,1909-12-25 +1909-05-05 12:34:56.1075,1909-12-25 +1909-05-05 12:34:56.1076,1909-12-25 +1909-05-05 12:34:56.1077,1909-12-25 +1909-05-05 12:34:56.1078,1909-12-25 +1909-05-05 12:34:56.1079,1909-12-25 +1909-05-05 12:34:56.108,1909-12-25 +1909-05-05 12:34:56.1081,1909-12-25 +1909-05-05 12:34:56.1082,1909-12-25 +1909-05-05 12:34:56.1083,1909-12-25 +1909-05-05 12:34:56.1084,1909-12-25 +1909-05-05 12:34:56.1085,1909-12-25 +1909-05-05 12:34:56.1086,1909-12-25 +1909-05-05 12:34:56.1087,1909-12-25 +1909-05-05 12:34:56.1088,1909-12-25 +1909-05-05 12:34:56.1089,1909-12-25 +1909-05-05 12:34:56.109,1909-12-25 +1909-05-05 12:34:56.1091,1909-12-25 +1909-05-05 12:34:56.1092,1909-12-25 +1909-05-05 12:34:56.1093,1909-12-25 +1909-05-05 12:34:56.1094,1909-12-25 +1909-05-05 12:34:56.1095,1909-12-25 +1909-05-05 12:34:56.1096,1909-12-25 +1909-05-05 12:34:56.1097,1909-12-25 +1909-05-05 12:34:56.1098,1909-12-25 +1909-05-05 12:34:56.1099,1909-12-25 +1909-05-05 12:34:56.11,1909-12-25 +1909-05-05 12:34:56.1101,1909-12-25 +1909-05-05 12:34:56.1102,1909-12-25 +1909-05-05 12:34:56.1103,1909-12-25 +1909-05-05 12:34:56.1104,1909-12-25 +1909-05-05 12:34:56.1105,1909-12-25 +1909-05-05 12:34:56.1106,1909-12-25 +1909-05-05 12:34:56.1107,1909-12-25 +1909-05-05 12:34:56.1108,1909-12-25 +1909-05-05 12:34:56.1109,1909-12-25 +1909-05-05 12:34:56.111,1909-12-25 +1909-05-05 12:34:56.1111,1909-12-25 +1909-05-05 12:34:56.1112,1909-12-25 +1909-05-05 12:34:56.1113,1909-12-25 +1909-05-05 12:34:56.1114,1909-12-25 +1909-05-05 12:34:56.1115,1909-12-25 +1909-05-05 12:34:56.1116,1909-12-25 +1909-05-05 12:34:56.1117,1909-12-25 +1909-05-05 12:34:56.1118,1909-12-25 +1909-05-05 12:34:56.1119,1909-12-25 +1909-05-05 12:34:56.112,1909-12-25 +1909-05-05 12:34:56.1121,1909-12-25 +1909-05-05 12:34:56.1122,1909-12-25 +1909-05-05 12:34:56.1123,1909-12-25 +1909-05-05 12:34:56.1124,1909-12-25 +1909-05-05 12:34:56.1125,1909-12-25 +1909-05-05 12:34:56.1126,1909-12-25 +1909-05-05 12:34:56.1127,1909-12-25 +1909-05-05 12:34:56.1128,1909-12-25 +1909-05-05 12:34:56.1129,1909-12-25 +1909-05-05 12:34:56.113,1909-12-25 +1909-05-05 12:34:56.1131,1909-12-25 +1909-05-05 12:34:56.1132,1909-12-25 +1909-05-05 12:34:56.1133,1909-12-25 +1909-05-05 12:34:56.1134,1909-12-25 +1909-05-05 12:34:56.1135,1909-12-25 +1909-05-05 12:34:56.1136,1909-12-25 +1909-05-05 12:34:56.1137,1909-12-25 +1909-05-05 12:34:56.1138,1909-12-25 +1909-05-05 12:34:56.1139,1909-12-25 +1909-05-05 12:34:56.114,1909-12-25 +1909-05-05 12:34:56.1141,1909-12-25 +1909-05-05 12:34:56.1142,1909-12-25 +1909-05-05 12:34:56.1143,1909-12-25 +1909-05-05 12:34:56.1144,1909-12-25 +1909-05-05 12:34:56.1145,1909-12-25 +1909-05-05 12:34:56.1146,1909-12-25 +1909-05-05 12:34:56.1147,1909-12-25 +1909-05-05 12:34:56.1148,1909-12-25 +1909-05-05 12:34:56.1149,1909-12-25 +1909-05-05 12:34:56.115,1909-12-25 +1909-05-05 12:34:56.1151,1909-12-25 +1909-05-05 12:34:56.1152,1909-12-25 +1909-05-05 12:34:56.1153,1909-12-25 +1909-05-05 12:34:56.1154,1909-12-25 +1909-05-05 12:34:56.1155,1909-12-25 +1909-05-05 12:34:56.1156,1909-12-25 +1909-05-05 12:34:56.1157,1909-12-25 +1909-05-05 12:34:56.1158,1909-12-25 +1909-05-05 12:34:56.1159,1909-12-25 +1909-05-05 12:34:56.116,1909-12-25 +1909-05-05 12:34:56.1161,1909-12-25 +1909-05-05 12:34:56.1162,1909-12-25 +1909-05-05 12:34:56.1163,1909-12-25 +1909-05-05 12:34:56.1164,1909-12-25 +1909-05-05 12:34:56.1165,1909-12-25 +1909-05-05 12:34:56.1166,1909-12-25 +1909-05-05 12:34:56.1167,1909-12-25 +1909-05-05 12:34:56.1168,1909-12-25 +1909-05-05 12:34:56.1169,1909-12-25 +1909-05-05 12:34:56.117,1909-12-25 +1909-05-05 12:34:56.1171,1909-12-25 +1909-05-05 12:34:56.1172,1909-12-25 +1909-05-05 12:34:56.1173,1909-12-25 +1909-05-05 12:34:56.1174,1909-12-25 +1909-05-05 12:34:56.1175,1909-12-25 +1909-05-05 12:34:56.1176,1909-12-25 +1909-05-05 12:34:56.1177,1909-12-25 +1909-05-05 12:34:56.1178,1909-12-25 +1909-05-05 12:34:56.1179,1909-12-25 +1909-05-05 12:34:56.118,1909-12-25 +1909-05-05 12:34:56.1181,1909-12-25 +1909-05-05 12:34:56.1182,1909-12-25 +1909-05-05 12:34:56.1183,1909-12-25 +1909-05-05 12:34:56.1184,1909-12-25 +1909-05-05 12:34:56.1185,1909-12-25 +1909-05-05 12:34:56.1186,1909-12-25 +1909-05-05 12:34:56.1187,1909-12-25 +1909-05-05 12:34:56.1188,1909-12-25 +1909-05-05 12:34:56.1189,1909-12-25 +1909-05-05 12:34:56.119,1909-12-25 +1909-05-05 12:34:56.1191,1909-12-25 +1909-05-05 12:34:56.1192,1909-12-25 +1909-05-05 12:34:56.1193,1909-12-25 +1909-05-05 12:34:56.1194,1909-12-25 +1909-05-05 12:34:56.1195,1909-12-25 +1909-05-05 12:34:56.1196,1909-12-25 +1909-05-05 12:34:56.1197,1909-12-25 +1909-05-05 12:34:56.1198,1909-12-25 +1909-05-05 12:34:56.1199,1909-12-25 +1909-05-05 12:34:56.12,1909-12-25 +1909-05-05 12:34:56.1201,1909-12-25 +1909-05-05 12:34:56.1202,1909-12-25 +1909-05-05 12:34:56.1203,1909-12-25 +1909-05-05 12:34:56.1204,1909-12-25 +1909-05-05 12:34:56.1205,1909-12-25 +1909-05-05 12:34:56.1206,1909-12-25 +1909-05-05 12:34:56.1207,1909-12-25 +1909-05-05 12:34:56.1208,1909-12-25 +1909-05-05 12:34:56.1209,1909-12-25 +1909-05-05 12:34:56.121,1909-12-25 +1909-05-05 12:34:56.1211,1909-12-25 +1909-05-05 12:34:56.1212,1909-12-25 +1909-05-05 12:34:56.1213,1909-12-25 +1909-05-05 12:34:56.1214,1909-12-25 +1909-05-05 12:34:56.1215,1909-12-25 +1909-05-05 12:34:56.1216,1909-12-25 +1909-05-05 12:34:56.1217,1909-12-25 +1909-05-05 12:34:56.1218,1909-12-25 +1909-05-05 12:34:56.1219,1909-12-25 +1909-05-05 12:34:56.122,1909-12-25 +1909-05-05 12:34:56.1221,1909-12-25 +1909-05-05 12:34:56.1222,1909-12-25 +1909-05-05 12:34:56.1223,1909-12-25 +1909-05-05 12:34:56.1224,1909-12-25 +1909-05-05 12:34:56.1225,1909-12-25 +1909-05-05 12:34:56.1226,1909-12-25 +1909-05-05 12:34:56.1227,1909-12-25 +1909-05-05 12:34:56.1228,1909-12-25 +1909-05-05 12:34:56.1229,1909-12-25 +1909-05-05 12:34:56.123,1909-12-25 +1909-05-05 12:34:56.1231,1909-12-25 +1909-05-05 12:34:56.1232,1909-12-25 +1909-05-05 12:34:56.1233,1909-12-25 +1909-05-05 12:34:56.1234,1909-12-25 +1909-05-05 12:34:56.1235,1909-12-25 +1909-05-05 12:34:56.1236,1909-12-25 +1909-05-05 12:34:56.1237,1909-12-25 +1909-05-05 12:34:56.1238,1909-12-25 +1909-05-05 12:34:56.1239,1909-12-25 +1909-05-05 12:34:56.124,1909-12-25 +1909-05-05 12:34:56.1241,1909-12-25 +1909-05-05 12:34:56.1242,1909-12-25 +1909-05-05 12:34:56.1243,1909-12-25 +1909-05-05 12:34:56.1244,1909-12-25 +1909-05-05 12:34:56.1245,1909-12-25 +1909-05-05 12:34:56.1246,1909-12-25 +1909-05-05 12:34:56.1247,1909-12-25 +1909-05-05 12:34:56.1248,1909-12-25 +1909-05-05 12:34:56.1249,1909-12-25 +1909-05-05 12:34:56.125,1909-12-25 +1909-05-05 12:34:56.1251,1909-12-25 +1909-05-05 12:34:56.1252,1909-12-25 +1909-05-05 12:34:56.1253,1909-12-25 +1909-05-05 12:34:56.1254,1909-12-25 +1909-05-05 12:34:56.1255,1909-12-25 +1909-05-05 12:34:56.1256,1909-12-25 +1909-05-05 12:34:56.1257,1909-12-25 +1909-05-05 12:34:56.1258,1909-12-25 +1909-05-05 12:34:56.1259,1909-12-25 +1909-05-05 12:34:56.126,1909-12-25 +1909-05-05 12:34:56.1261,1909-12-25 +1909-05-05 12:34:56.1262,1909-12-25 +1909-05-05 12:34:56.1263,1909-12-25 +1909-05-05 12:34:56.1264,1909-12-25 +1909-05-05 12:34:56.1265,1909-12-25 +1909-05-05 12:34:56.1266,1909-12-25 +1909-05-05 12:34:56.1267,1909-12-25 +1909-05-05 12:34:56.1268,1909-12-25 +1909-05-05 12:34:56.1269,1909-12-25 +1909-05-05 12:34:56.127,1909-12-25 +1909-05-05 12:34:56.1271,1909-12-25 +1909-05-05 12:34:56.1272,1909-12-25 +1909-05-05 12:34:56.1273,1909-12-25 +1909-05-05 12:34:56.1274,1909-12-25 +1909-05-05 12:34:56.1275,1909-12-25 +1909-05-05 12:34:56.1276,1909-12-25 +1909-05-05 12:34:56.1277,1909-12-25 +1909-05-05 12:34:56.1278,1909-12-25 +1909-05-05 12:34:56.1279,1909-12-25 +1909-05-05 12:34:56.128,1909-12-25 +1909-05-05 12:34:56.1281,1909-12-25 +1909-05-05 12:34:56.1282,1909-12-25 +1909-05-05 12:34:56.1283,1909-12-25 +1909-05-05 12:34:56.1284,1909-12-25 +1909-05-05 12:34:56.1285,1909-12-25 +1909-05-05 12:34:56.1286,1909-12-25 +1909-05-05 12:34:56.1287,1909-12-25 +1909-05-05 12:34:56.1288,1909-12-25 +1909-05-05 12:34:56.1289,1909-12-25 +1909-05-05 12:34:56.129,1909-12-25 +1909-05-05 12:34:56.1291,1909-12-25 +1909-05-05 12:34:56.1292,1909-12-25 +1909-05-05 12:34:56.1293,1909-12-25 +1909-05-05 12:34:56.1294,1909-12-25 +1909-05-05 12:34:56.1295,1909-12-25 +1909-05-05 12:34:56.1296,1909-12-25 +1909-05-05 12:34:56.1297,1909-12-25 +1909-05-05 12:34:56.1298,1909-12-25 +1909-05-05 12:34:56.1299,1909-12-25 +1909-05-05 12:34:56.13,1909-12-25 +1909-05-05 12:34:56.1301,1909-12-25 +1909-05-05 12:34:56.1302,1909-12-25 +1909-05-05 12:34:56.1303,1909-12-25 +1909-05-05 12:34:56.1304,1909-12-25 +1909-05-05 12:34:56.1305,1909-12-25 +1909-05-05 12:34:56.1306,1909-12-25 +1909-05-05 12:34:56.1307,1909-12-25 +1909-05-05 12:34:56.1308,1909-12-25 +1909-05-05 12:34:56.1309,1909-12-25 +1909-05-05 12:34:56.131,1909-12-25 +1909-05-05 12:34:56.1311,1909-12-25 +1909-05-05 12:34:56.1312,1909-12-25 +1909-05-05 12:34:56.1313,1909-12-25 +1909-05-05 12:34:56.1314,1909-12-25 +1909-05-05 12:34:56.1315,1909-12-25 +1909-05-05 12:34:56.1316,1909-12-25 +1909-05-05 12:34:56.1317,1909-12-25 +1909-05-05 12:34:56.1318,1909-12-25 +1909-05-05 12:34:56.1319,1909-12-25 +1909-05-05 12:34:56.132,1909-12-25 +1909-05-05 12:34:56.1321,1909-12-25 +1909-05-05 12:34:56.1322,1909-12-25 +1909-05-05 12:34:56.1323,1909-12-25 +1909-05-05 12:34:56.1324,1909-12-25 +1909-05-05 12:34:56.1325,1909-12-25 +1909-05-05 12:34:56.1326,1909-12-25 +1909-05-05 12:34:56.1327,1909-12-25 +1909-05-05 12:34:56.1328,1909-12-25 +1909-05-05 12:34:56.1329,1909-12-25 +1909-05-05 12:34:56.133,1909-12-25 +1909-05-05 12:34:56.1331,1909-12-25 +1909-05-05 12:34:56.1332,1909-12-25 +1909-05-05 12:34:56.1333,1909-12-25 +1909-05-05 12:34:56.1334,1909-12-25 +1909-05-05 12:34:56.1335,1909-12-25 +1909-05-05 12:34:56.1336,1909-12-25 +1909-05-05 12:34:56.1337,1909-12-25 +1909-05-05 12:34:56.1338,1909-12-25 +1909-05-05 12:34:56.1339,1909-12-25 +1909-05-05 12:34:56.134,1909-12-25 +1909-05-05 12:34:56.1341,1909-12-25 +1909-05-05 12:34:56.1342,1909-12-25 +1909-05-05 12:34:56.1343,1909-12-25 +1909-05-05 12:34:56.1344,1909-12-25 +1909-05-05 12:34:56.1345,1909-12-25 +1909-05-05 12:34:56.1346,1909-12-25 +1909-05-05 12:34:56.1347,1909-12-25 +1909-05-05 12:34:56.1348,1909-12-25 +1909-05-05 12:34:56.1349,1909-12-25 +1909-05-05 12:34:56.135,1909-12-25 +1909-05-05 12:34:56.1351,1909-12-25 +1909-05-05 12:34:56.1352,1909-12-25 +1909-05-05 12:34:56.1353,1909-12-25 +1909-05-05 12:34:56.1354,1909-12-25 +1909-05-05 12:34:56.1355,1909-12-25 +1909-05-05 12:34:56.1356,1909-12-25 +1909-05-05 12:34:56.1357,1909-12-25 +1909-05-05 12:34:56.1358,1909-12-25 +1909-05-05 12:34:56.1359,1909-12-25 +1909-05-05 12:34:56.136,1909-12-25 +1909-05-05 12:34:56.1361,1909-12-25 +1909-05-05 12:34:56.1362,1909-12-25 +1909-05-05 12:34:56.1363,1909-12-25 +1909-05-05 12:34:56.1364,1909-12-25 +1909-05-05 12:34:56.1365,1909-12-25 +1909-05-05 12:34:56.1366,1909-12-25 +1909-05-05 12:34:56.1367,1909-12-25 +1909-05-05 12:34:56.1368,1909-12-25 +1909-05-05 12:34:56.1369,1909-12-25 +1909-05-05 12:34:56.137,1909-12-25 +1909-05-05 12:34:56.1371,1909-12-25 +1909-05-05 12:34:56.1372,1909-12-25 +1909-05-05 12:34:56.1373,1909-12-25 +1909-05-05 12:34:56.1374,1909-12-25 +1909-05-05 12:34:56.1375,1909-12-25 +1909-05-05 12:34:56.1376,1909-12-25 +1909-05-05 12:34:56.1377,1909-12-25 +1909-05-05 12:34:56.1378,1909-12-25 +1909-05-05 12:34:56.1379,1909-12-25 +1909-05-05 12:34:56.138,1909-12-25 +1909-05-05 12:34:56.1381,1909-12-25 +1909-05-05 12:34:56.1382,1909-12-25 +1909-05-05 12:34:56.1383,1909-12-25 +1909-05-05 12:34:56.1384,1909-12-25 +1909-05-05 12:34:56.1385,1909-12-25 +1909-05-05 12:34:56.1386,1909-12-25 +1909-05-05 12:34:56.1387,1909-12-25 +1909-05-05 12:34:56.1388,1909-12-25 +1909-05-05 12:34:56.1389,1909-12-25 +1909-05-05 12:34:56.139,1909-12-25 +1909-05-05 12:34:56.1391,1909-12-25 +1909-05-05 12:34:56.1392,1909-12-25 +1909-05-05 12:34:56.1393,1909-12-25 +1909-05-05 12:34:56.1394,1909-12-25 +1909-05-05 12:34:56.1395,1909-12-25 +1909-05-05 12:34:56.1396,1909-12-25 +1909-05-05 12:34:56.1397,1909-12-25 +1909-05-05 12:34:56.1398,1909-12-25 +1909-05-05 12:34:56.1399,1909-12-25 +1909-05-05 12:34:56.14,1909-12-25 +1909-05-05 12:34:56.1401,1909-12-25 +1909-05-05 12:34:56.1402,1909-12-25 +1909-05-05 12:34:56.1403,1909-12-25 +1909-05-05 12:34:56.1404,1909-12-25 +1909-05-05 12:34:56.1405,1909-12-25 +1909-05-05 12:34:56.1406,1909-12-25 +1909-05-05 12:34:56.1407,1909-12-25 +1909-05-05 12:34:56.1408,1909-12-25 +1909-05-05 12:34:56.1409,1909-12-25 +1909-05-05 12:34:56.141,1909-12-25 +1909-05-05 12:34:56.1411,1909-12-25 +1909-05-05 12:34:56.1412,1909-12-25 +1909-05-05 12:34:56.1413,1909-12-25 +1909-05-05 12:34:56.1414,1909-12-25 +1909-05-05 12:34:56.1415,1909-12-25 +1909-05-05 12:34:56.1416,1909-12-25 +1909-05-05 12:34:56.1417,1909-12-25 +1909-05-05 12:34:56.1418,1909-12-25 +1909-05-05 12:34:56.1419,1909-12-25 +1909-05-05 12:34:56.142,1909-12-25 +1909-05-05 12:34:56.1421,1909-12-25 +1909-05-05 12:34:56.1422,1909-12-25 +1909-05-05 12:34:56.1423,1909-12-25 +1909-05-05 12:34:56.1424,1909-12-25 +1909-05-05 12:34:56.1425,1909-12-25 +1909-05-05 12:34:56.1426,1909-12-25 +1909-05-05 12:34:56.1427,1909-12-25 +1909-05-05 12:34:56.1428,1909-12-25 +1909-05-05 12:34:56.1429,1909-12-25 +1909-05-05 12:34:56.143,1909-12-25 +1909-05-05 12:34:56.1431,1909-12-25 +1909-05-05 12:34:56.1432,1909-12-25 +1909-05-05 12:34:56.1433,1909-12-25 +1909-05-05 12:34:56.1434,1909-12-25 +1909-05-05 12:34:56.1435,1909-12-25 +1909-05-05 12:34:56.1436,1909-12-25 +1909-05-05 12:34:56.1437,1909-12-25 +1909-05-05 12:34:56.1438,1909-12-25 +1909-05-05 12:34:56.1439,1909-12-25 +1909-05-05 12:34:56.144,1909-12-25 +1909-05-05 12:34:56.1441,1909-12-25 +1909-05-05 12:34:56.1442,1909-12-25 +1909-05-05 12:34:56.1443,1909-12-25 +1909-05-05 12:34:56.1444,1909-12-25 +1909-05-05 12:34:56.1445,1909-12-25 +1909-05-05 12:34:56.1446,1909-12-25 +1909-05-05 12:34:56.1447,1909-12-25 +1909-05-05 12:34:56.1448,1909-12-25 +1909-05-05 12:34:56.1449,1909-12-25 +1909-05-05 12:34:56.145,1909-12-25 +1909-05-05 12:34:56.1451,1909-12-25 +1909-05-05 12:34:56.1452,1909-12-25 +1909-05-05 12:34:56.1453,1909-12-25 +1909-05-05 12:34:56.1454,1909-12-25 +1909-05-05 12:34:56.1455,1909-12-25 +1909-05-05 12:34:56.1456,1909-12-25 +1909-05-05 12:34:56.1457,1909-12-25 +1909-05-05 12:34:56.1458,1909-12-25 +1909-05-05 12:34:56.1459,1909-12-25 +1909-05-05 12:34:56.146,1909-12-25 +1909-05-05 12:34:56.1461,1909-12-25 +1909-05-05 12:34:56.1462,1909-12-25 +1909-05-05 12:34:56.1463,1909-12-25 +1909-05-05 12:34:56.1464,1909-12-25 +1909-05-05 12:34:56.1465,1909-12-25 +1909-05-05 12:34:56.1466,1909-12-25 +1909-05-05 12:34:56.1467,1909-12-25 +1909-05-05 12:34:56.1468,1909-12-25 +1909-05-05 12:34:56.1469,1909-12-25 +1909-05-05 12:34:56.147,1909-12-25 +1909-05-05 12:34:56.1471,1909-12-25 +1909-05-05 12:34:56.1472,1909-12-25 +1909-05-05 12:34:56.1473,1909-12-25 +1909-05-05 12:34:56.1474,1909-12-25 +1909-05-05 12:34:56.1475,1909-12-25 +1909-05-05 12:34:56.1476,1909-12-25 +1909-05-05 12:34:56.1477,1909-12-25 +1909-05-05 12:34:56.1478,1909-12-25 +1909-05-05 12:34:56.1479,1909-12-25 +1909-05-05 12:34:56.148,1909-12-25 +1909-05-05 12:34:56.1481,1909-12-25 +1909-05-05 12:34:56.1482,1909-12-25 +1909-05-05 12:34:56.1483,1909-12-25 +1909-05-05 12:34:56.1484,1909-12-25 +1909-05-05 12:34:56.1485,1909-12-25 +1909-05-05 12:34:56.1486,1909-12-25 +1909-05-05 12:34:56.1487,1909-12-25 +1909-05-05 12:34:56.1488,1909-12-25 +1909-05-05 12:34:56.1489,1909-12-25 +1909-05-05 12:34:56.149,1909-12-25 +1909-05-05 12:34:56.1491,1909-12-25 +1909-05-05 12:34:56.1492,1909-12-25 +1909-05-05 12:34:56.1493,1909-12-25 +1909-05-05 12:34:56.1494,1909-12-25 +1909-05-05 12:34:56.1495,1909-12-25 +1909-05-05 12:34:56.1496,1909-12-25 +1909-05-05 12:34:56.1497,1909-12-25 +1909-05-05 12:34:56.1498,1909-12-25 +1909-05-05 12:34:56.1499,1909-12-25 +1909-05-05 12:34:56.15,1909-12-25 +1909-05-05 12:34:56.1501,1909-12-25 +1909-05-05 12:34:56.1502,1909-12-25 +1909-05-05 12:34:56.1503,1909-12-25 +1909-05-05 12:34:56.1504,1909-12-25 +1909-05-05 12:34:56.1505,1909-12-25 +1909-05-05 12:34:56.1506,1909-12-25 +1909-05-05 12:34:56.1507,1909-12-25 +1909-05-05 12:34:56.1508,1909-12-25 +1909-05-05 12:34:56.1509,1909-12-25 +1909-05-05 12:34:56.151,1909-12-25 +1909-05-05 12:34:56.1511,1909-12-25 +1909-05-05 12:34:56.1512,1909-12-25 +1909-05-05 12:34:56.1513,1909-12-25 +1909-05-05 12:34:56.1514,1909-12-25 +1909-05-05 12:34:56.1515,1909-12-25 +1909-05-05 12:34:56.1516,1909-12-25 +1909-05-05 12:34:56.1517,1909-12-25 +1909-05-05 12:34:56.1518,1909-12-25 +1909-05-05 12:34:56.1519,1909-12-25 +1909-05-05 12:34:56.152,1909-12-25 +1909-05-05 12:34:56.1521,1909-12-25 +1909-05-05 12:34:56.1522,1909-12-25 +1909-05-05 12:34:56.1523,1909-12-25 +1909-05-05 12:34:56.1524,1909-12-25 +1909-05-05 12:34:56.1525,1909-12-25 +1909-05-05 12:34:56.1526,1909-12-25 +1909-05-05 12:34:56.1527,1909-12-25 +1909-05-05 12:34:56.1528,1909-12-25 +1909-05-05 12:34:56.1529,1909-12-25 +1909-05-05 12:34:56.153,1909-12-25 +1909-05-05 12:34:56.1531,1909-12-25 +1909-05-05 12:34:56.1532,1909-12-25 +1909-05-05 12:34:56.1533,1909-12-25 +1909-05-05 12:34:56.1534,1909-12-25 +1909-05-05 12:34:56.1535,1909-12-25 +1909-05-05 12:34:56.1536,1909-12-25 +1909-05-05 12:34:56.1537,1909-12-25 +1909-05-05 12:34:56.1538,1909-12-25 +1909-05-05 12:34:56.1539,1909-12-25 +1909-05-05 12:34:56.154,1909-12-25 +1909-05-05 12:34:56.1541,1909-12-25 +1909-05-05 12:34:56.1542,1909-12-25 +1909-05-05 12:34:56.1543,1909-12-25 +1909-05-05 12:34:56.1544,1909-12-25 +1909-05-05 12:34:56.1545,1909-12-25 +1909-05-05 12:34:56.1546,1909-12-25 +1909-05-05 12:34:56.1547,1909-12-25 +1909-05-05 12:34:56.1548,1909-12-25 +1909-05-05 12:34:56.1549,1909-12-25 +1909-05-05 12:34:56.155,1909-12-25 +1909-05-05 12:34:56.1551,1909-12-25 +1909-05-05 12:34:56.1552,1909-12-25 +1909-05-05 12:34:56.1553,1909-12-25 +1909-05-05 12:34:56.1554,1909-12-25 +1909-05-05 12:34:56.1555,1909-12-25 +1909-05-05 12:34:56.1556,1909-12-25 +1909-05-05 12:34:56.1557,1909-12-25 +1909-05-05 12:34:56.1558,1909-12-25 +1909-05-05 12:34:56.1559,1909-12-25 +1909-05-05 12:34:56.156,1909-12-25 +1909-05-05 12:34:56.1561,1909-12-25 +1909-05-05 12:34:56.1562,1909-12-25 +1909-05-05 12:34:56.1563,1909-12-25 +1909-05-05 12:34:56.1564,1909-12-25 +1909-05-05 12:34:56.1565,1909-12-25 +1909-05-05 12:34:56.1566,1909-12-25 +1909-05-05 12:34:56.1567,1909-12-25 +1909-05-05 12:34:56.1568,1909-12-25 +1909-05-05 12:34:56.1569,1909-12-25 +1909-05-05 12:34:56.157,1909-12-25 +1909-05-05 12:34:56.1571,1909-12-25 +1909-05-05 12:34:56.1572,1909-12-25 +1909-05-05 12:34:56.1573,1909-12-25 +1909-05-05 12:34:56.1574,1909-12-25 +1909-05-05 12:34:56.1575,1909-12-25 +1909-05-05 12:34:56.1576,1909-12-25 +1909-05-05 12:34:56.1577,1909-12-25 +1909-05-05 12:34:56.1578,1909-12-25 +1909-05-05 12:34:56.1579,1909-12-25 +1909-05-05 12:34:56.158,1909-12-25 +1909-05-05 12:34:56.1581,1909-12-25 +1909-05-05 12:34:56.1582,1909-12-25 +1909-05-05 12:34:56.1583,1909-12-25 +1909-05-05 12:34:56.1584,1909-12-25 +1909-05-05 12:34:56.1585,1909-12-25 +1909-05-05 12:34:56.1586,1909-12-25 +1909-05-05 12:34:56.1587,1909-12-25 +1909-05-05 12:34:56.1588,1909-12-25 +1909-05-05 12:34:56.1589,1909-12-25 +1909-05-05 12:34:56.159,1909-12-25 +1909-05-05 12:34:56.1591,1909-12-25 +1909-05-05 12:34:56.1592,1909-12-25 +1909-05-05 12:34:56.1593,1909-12-25 +1909-05-05 12:34:56.1594,1909-12-25 +1909-05-05 12:34:56.1595,1909-12-25 +1909-05-05 12:34:56.1596,1909-12-25 +1909-05-05 12:34:56.1597,1909-12-25 +1909-05-05 12:34:56.1598,1909-12-25 +1909-05-05 12:34:56.1599,1909-12-25 +1909-05-05 12:34:56.16,1909-12-25 +1909-05-05 12:34:56.1601,1909-12-25 +1909-05-05 12:34:56.1602,1909-12-25 +1909-05-05 12:34:56.1603,1909-12-25 +1909-05-05 12:34:56.1604,1909-12-25 +1909-05-05 12:34:56.1605,1909-12-25 +1909-05-05 12:34:56.1606,1909-12-25 +1909-05-05 12:34:56.1607,1909-12-25 +1909-05-05 12:34:56.1608,1909-12-25 +1909-05-05 12:34:56.1609,1909-12-25 +1909-05-05 12:34:56.161,1909-12-25 +1909-05-05 12:34:56.1611,1909-12-25 +1909-05-05 12:34:56.1612,1909-12-25 +1909-05-05 12:34:56.1613,1909-12-25 +1909-05-05 12:34:56.1614,1909-12-25 +1909-05-05 12:34:56.1615,1909-12-25 +1909-05-05 12:34:56.1616,1909-12-25 +1909-05-05 12:34:56.1617,1909-12-25 +1909-05-05 12:34:56.1618,1909-12-25 +1909-05-05 12:34:56.1619,1909-12-25 +1909-05-05 12:34:56.162,1909-12-25 +1909-05-05 12:34:56.1621,1909-12-25 +1909-05-05 12:34:56.1622,1909-12-25 +1909-05-05 12:34:56.1623,1909-12-25 +1909-05-05 12:34:56.1624,1909-12-25 +1909-05-05 12:34:56.1625,1909-12-25 +1909-05-05 12:34:56.1626,1909-12-25 +1909-05-05 12:34:56.1627,1909-12-25 +1909-05-05 12:34:56.1628,1909-12-25 +1909-05-05 12:34:56.1629,1909-12-25 +1909-05-05 12:34:56.163,1909-12-25 +1909-05-05 12:34:56.1631,1909-12-25 +1909-05-05 12:34:56.1632,1909-12-25 +1909-05-05 12:34:56.1633,1909-12-25 +1909-05-05 12:34:56.1634,1909-12-25 +1909-05-05 12:34:56.1635,1909-12-25 +1909-05-05 12:34:56.1636,1909-12-25 +1909-05-05 12:34:56.1637,1909-12-25 +1909-05-05 12:34:56.1638,1909-12-25 +1909-05-05 12:34:56.1639,1909-12-25 +1909-05-05 12:34:56.164,1909-12-25 +1909-05-05 12:34:56.1641,1909-12-25 +1909-05-05 12:34:56.1642,1909-12-25 +1909-05-05 12:34:56.1643,1909-12-25 +1909-05-05 12:34:56.1644,1909-12-25 +1909-05-05 12:34:56.1645,1909-12-25 +1909-05-05 12:34:56.1646,1909-12-25 +1909-05-05 12:34:56.1647,1909-12-25 +1909-05-05 12:34:56.1648,1909-12-25 +1909-05-05 12:34:56.1649,1909-12-25 +1909-05-05 12:34:56.165,1909-12-25 +1909-05-05 12:34:56.1651,1909-12-25 +1909-05-05 12:34:56.1652,1909-12-25 +1909-05-05 12:34:56.1653,1909-12-25 +1909-05-05 12:34:56.1654,1909-12-25 +1909-05-05 12:34:56.1655,1909-12-25 +1909-05-05 12:34:56.1656,1909-12-25 +1909-05-05 12:34:56.1657,1909-12-25 +1909-05-05 12:34:56.1658,1909-12-25 +1909-05-05 12:34:56.1659,1909-12-25 +1909-05-05 12:34:56.166,1909-12-25 +1909-05-05 12:34:56.1661,1909-12-25 +1909-05-05 12:34:56.1662,1909-12-25 +1909-05-05 12:34:56.1663,1909-12-25 +1909-05-05 12:34:56.1664,1909-12-25 +1909-05-05 12:34:56.1665,1909-12-25 +1909-05-05 12:34:56.1666,1909-12-25 +1909-05-05 12:34:56.1667,1909-12-25 +1909-05-05 12:34:56.1668,1909-12-25 +1909-05-05 12:34:56.1669,1909-12-25 +1909-05-05 12:34:56.167,1909-12-25 +1909-05-05 12:34:56.1671,1909-12-25 +1909-05-05 12:34:56.1672,1909-12-25 +1909-05-05 12:34:56.1673,1909-12-25 +1909-05-05 12:34:56.1674,1909-12-25 +1909-05-05 12:34:56.1675,1909-12-25 +1909-05-05 12:34:56.1676,1909-12-25 +1909-05-05 12:34:56.1677,1909-12-25 +1909-05-05 12:34:56.1678,1909-12-25 +1909-05-05 12:34:56.1679,1909-12-25 +1909-05-05 12:34:56.168,1909-12-25 +1909-05-05 12:34:56.1681,1909-12-25 +1909-05-05 12:34:56.1682,1909-12-25 +1909-05-05 12:34:56.1683,1909-12-25 +1909-05-05 12:34:56.1684,1909-12-25 +1909-05-05 12:34:56.1685,1909-12-25 +1909-05-05 12:34:56.1686,1909-12-25 +1909-05-05 12:34:56.1687,1909-12-25 +1909-05-05 12:34:56.1688,1909-12-25 +1909-05-05 12:34:56.1689,1909-12-25 +1909-05-05 12:34:56.169,1909-12-25 +1909-05-05 12:34:56.1691,1909-12-25 +1909-05-05 12:34:56.1692,1909-12-25 +1909-05-05 12:34:56.1693,1909-12-25 +1909-05-05 12:34:56.1694,1909-12-25 +1909-05-05 12:34:56.1695,1909-12-25 +1909-05-05 12:34:56.1696,1909-12-25 +1909-05-05 12:34:56.1697,1909-12-25 +1909-05-05 12:34:56.1698,1909-12-25 +1909-05-05 12:34:56.1699,1909-12-25 +1909-05-05 12:34:56.17,1909-12-25 +1909-05-05 12:34:56.1701,1909-12-25 +1909-05-05 12:34:56.1702,1909-12-25 +1909-05-05 12:34:56.1703,1909-12-25 +1909-05-05 12:34:56.1704,1909-12-25 +1909-05-05 12:34:56.1705,1909-12-25 +1909-05-05 12:34:56.1706,1909-12-25 +1909-05-05 12:34:56.1707,1909-12-25 +1909-05-05 12:34:56.1708,1909-12-25 +1909-05-05 12:34:56.1709,1909-12-25 +1909-05-05 12:34:56.171,1909-12-25 +1909-05-05 12:34:56.1711,1909-12-25 +1909-05-05 12:34:56.1712,1909-12-25 +1909-05-05 12:34:56.1713,1909-12-25 +1909-05-05 12:34:56.1714,1909-12-25 +1909-05-05 12:34:56.1715,1909-12-25 +1909-05-05 12:34:56.1716,1909-12-25 +1909-05-05 12:34:56.1717,1909-12-25 +1909-05-05 12:34:56.1718,1909-12-25 +1909-05-05 12:34:56.1719,1909-12-25 +1909-05-05 12:34:56.172,1909-12-25 +1909-05-05 12:34:56.1721,1909-12-25 +1909-05-05 12:34:56.1722,1909-12-25 +1909-05-05 12:34:56.1723,1909-12-25 +1909-05-05 12:34:56.1724,1909-12-25 +1909-05-05 12:34:56.1725,1909-12-25 +1909-05-05 12:34:56.1726,1909-12-25 +1909-05-05 12:34:56.1727,1909-12-25 +1909-05-05 12:34:56.1728,1909-12-25 +1909-05-05 12:34:56.1729,1909-12-25 +1909-05-05 12:34:56.173,1909-12-25 +1909-05-05 12:34:56.1731,1909-12-25 +1909-05-05 12:34:56.1732,1909-12-25 +1909-05-05 12:34:56.1733,1909-12-25 +1909-05-05 12:34:56.1734,1909-12-25 +1909-05-05 12:34:56.1735,1909-12-25 +1909-05-05 12:34:56.1736,1909-12-25 +1909-05-05 12:34:56.1737,1909-12-25 +1909-05-05 12:34:56.1738,1909-12-25 +1909-05-05 12:34:56.1739,1909-12-25 +1909-05-05 12:34:56.174,1909-12-25 +1909-05-05 12:34:56.1741,1909-12-25 +1909-05-05 12:34:56.1742,1909-12-25 +1909-05-05 12:34:56.1743,1909-12-25 +1909-05-05 12:34:56.1744,1909-12-25 +1909-05-05 12:34:56.1745,1909-12-25 +1909-05-05 12:34:56.1746,1909-12-25 +1909-05-05 12:34:56.1747,1909-12-25 +1909-05-05 12:34:56.1748,1909-12-25 +1909-05-05 12:34:56.1749,1909-12-25 +1909-05-05 12:34:56.175,1909-12-25 +1909-05-05 12:34:56.1751,1909-12-25 +1909-05-05 12:34:56.1752,1909-12-25 +1909-05-05 12:34:56.1753,1909-12-25 +1909-05-05 12:34:56.1754,1909-12-25 +1909-05-05 12:34:56.1755,1909-12-25 +1909-05-05 12:34:56.1756,1909-12-25 +1909-05-05 12:34:56.1757,1909-12-25 +1909-05-05 12:34:56.1758,1909-12-25 +1909-05-05 12:34:56.1759,1909-12-25 +1909-05-05 12:34:56.176,1909-12-25 +1909-05-05 12:34:56.1761,1909-12-25 +1909-05-05 12:34:56.1762,1909-12-25 +1909-05-05 12:34:56.1763,1909-12-25 +1909-05-05 12:34:56.1764,1909-12-25 +1909-05-05 12:34:56.1765,1909-12-25 +1909-05-05 12:34:56.1766,1909-12-25 +1909-05-05 12:34:56.1767,1909-12-25 +1909-05-05 12:34:56.1768,1909-12-25 +1909-05-05 12:34:56.1769,1909-12-25 +1909-05-05 12:34:56.177,1909-12-25 +1909-05-05 12:34:56.1771,1909-12-25 +1909-05-05 12:34:56.1772,1909-12-25 +1909-05-05 12:34:56.1773,1909-12-25 +1909-05-05 12:34:56.1774,1909-12-25 +1909-05-05 12:34:56.1775,1909-12-25 +1909-05-05 12:34:56.1776,1909-12-25 +1909-05-05 12:34:56.1777,1909-12-25 +1909-05-05 12:34:56.1778,1909-12-25 +1909-05-05 12:34:56.1779,1909-12-25 +1909-05-05 12:34:56.178,1909-12-25 +1909-05-05 12:34:56.1781,1909-12-25 +1909-05-05 12:34:56.1782,1909-12-25 +1909-05-05 12:34:56.1783,1909-12-25 +1909-05-05 12:34:56.1784,1909-12-25 +1909-05-05 12:34:56.1785,1909-12-25 +1909-05-05 12:34:56.1786,1909-12-25 +1909-05-05 12:34:56.1787,1909-12-25 +1909-05-05 12:34:56.1788,1909-12-25 +1909-05-05 12:34:56.1789,1909-12-25 +1909-05-05 12:34:56.179,1909-12-25 +1909-05-05 12:34:56.1791,1909-12-25 +1909-05-05 12:34:56.1792,1909-12-25 +1909-05-05 12:34:56.1793,1909-12-25 +1909-05-05 12:34:56.1794,1909-12-25 +1909-05-05 12:34:56.1795,1909-12-25 +1909-05-05 12:34:56.1796,1909-12-25 +1909-05-05 12:34:56.1797,1909-12-25 +1909-05-05 12:34:56.1798,1909-12-25 +1909-05-05 12:34:56.1799,1909-12-25 +1909-05-05 12:34:56.18,1909-12-25 +1909-05-05 12:34:56.1801,1909-12-25 +1909-05-05 12:34:56.1802,1909-12-25 +1909-05-05 12:34:56.1803,1909-12-25 +1909-05-05 12:34:56.1804,1909-12-25 +1909-05-05 12:34:56.1805,1909-12-25 +1909-05-05 12:34:56.1806,1909-12-25 +1909-05-05 12:34:56.1807,1909-12-25 +1909-05-05 12:34:56.1808,1909-12-25 +1909-05-05 12:34:56.1809,1909-12-25 +1909-05-05 12:34:56.181,1909-12-25 +1909-05-05 12:34:56.1811,1909-12-25 +1909-05-05 12:34:56.1812,1909-12-25 +1909-05-05 12:34:56.1813,1909-12-25 +1909-05-05 12:34:56.1814,1909-12-25 +1909-05-05 12:34:56.1815,1909-12-25 +1909-05-05 12:34:56.1816,1909-12-25 +1909-05-05 12:34:56.1817,1909-12-25 +1909-05-05 12:34:56.1818,1909-12-25 +1909-05-05 12:34:56.1819,1909-12-25 +1909-05-05 12:34:56.182,1909-12-25 +1909-05-05 12:34:56.1821,1909-12-25 +1909-05-05 12:34:56.1822,1909-12-25 +1909-05-05 12:34:56.1823,1909-12-25 +1909-05-05 12:34:56.1824,1909-12-25 +1909-05-05 12:34:56.1825,1909-12-25 +1909-05-05 12:34:56.1826,1909-12-25 +1909-05-05 12:34:56.1827,1909-12-25 +1909-05-05 12:34:56.1828,1909-12-25 +1909-05-05 12:34:56.1829,1909-12-25 +1909-05-05 12:34:56.183,1909-12-25 +1909-05-05 12:34:56.1831,1909-12-25 +1909-05-05 12:34:56.1832,1909-12-25 +1909-05-05 12:34:56.1833,1909-12-25 +1909-05-05 12:34:56.1834,1909-12-25 +1909-05-05 12:34:56.1835,1909-12-25 +1909-05-05 12:34:56.1836,1909-12-25 +1909-05-05 12:34:56.1837,1909-12-25 +1909-05-05 12:34:56.1838,1909-12-25 +1909-05-05 12:34:56.1839,1909-12-25 +1909-05-05 12:34:56.184,1909-12-25 +1909-05-05 12:34:56.1841,1909-12-25 +1909-05-05 12:34:56.1842,1909-12-25 +1909-05-05 12:34:56.1843,1909-12-25 +1909-05-05 12:34:56.1844,1909-12-25 +1909-05-05 12:34:56.1845,1909-12-25 +1909-05-05 12:34:56.1846,1909-12-25 +1909-05-05 12:34:56.1847,1909-12-25 +1909-05-05 12:34:56.1848,1909-12-25 +1909-05-05 12:34:56.1849,1909-12-25 +1909-05-05 12:34:56.185,1909-12-25 +1909-05-05 12:34:56.1851,1909-12-25 +1909-05-05 12:34:56.1852,1909-12-25 +1909-05-05 12:34:56.1853,1909-12-25 +1909-05-05 12:34:56.1854,1909-12-25 +1909-05-05 12:34:56.1855,1909-12-25 +1909-05-05 12:34:56.1856,1909-12-25 +1909-05-05 12:34:56.1857,1909-12-25 +1909-05-05 12:34:56.1858,1909-12-25 +1909-05-05 12:34:56.1859,1909-12-25 +1909-05-05 12:34:56.186,1909-12-25 +1909-05-05 12:34:56.1861,1909-12-25 +1909-05-05 12:34:56.1862,1909-12-25 +1909-05-05 12:34:56.1863,1909-12-25 +1909-05-05 12:34:56.1864,1909-12-25 +1909-05-05 12:34:56.1865,1909-12-25 +1909-05-05 12:34:56.1866,1909-12-25 +1909-05-05 12:34:56.1867,1909-12-25 +1909-05-05 12:34:56.1868,1909-12-25 +1909-05-05 12:34:56.1869,1909-12-25 +1909-05-05 12:34:56.187,1909-12-25 +1909-05-05 12:34:56.1871,1909-12-25 +1909-05-05 12:34:56.1872,1909-12-25 +1909-05-05 12:34:56.1873,1909-12-25 +1909-05-05 12:34:56.1874,1909-12-25 +1909-05-05 12:34:56.1875,1909-12-25 +1909-05-05 12:34:56.1876,1909-12-25 +1909-05-05 12:34:56.1877,1909-12-25 +1909-05-05 12:34:56.1878,1909-12-25 +1909-05-05 12:34:56.1879,1909-12-25 +1909-05-05 12:34:56.188,1909-12-25 +1909-05-05 12:34:56.1881,1909-12-25 +1909-05-05 12:34:56.1882,1909-12-25 +1909-05-05 12:34:56.1883,1909-12-25 +1909-05-05 12:34:56.1884,1909-12-25 +1909-05-05 12:34:56.1885,1909-12-25 +1909-05-05 12:34:56.1886,1909-12-25 +1909-05-05 12:34:56.1887,1909-12-25 +1909-05-05 12:34:56.1888,1909-12-25 +1909-05-05 12:34:56.1889,1909-12-25 +1909-05-05 12:34:56.189,1909-12-25 +1909-05-05 12:34:56.1891,1909-12-25 +1909-05-05 12:34:56.1892,1909-12-25 +1909-05-05 12:34:56.1893,1909-12-25 +1909-05-05 12:34:56.1894,1909-12-25 +1909-05-05 12:34:56.1895,1909-12-25 +1909-05-05 12:34:56.1896,1909-12-25 +1909-05-05 12:34:56.1897,1909-12-25 +1909-05-05 12:34:56.1898,1909-12-25 +1909-05-05 12:34:56.1899,1909-12-25 +1909-05-05 12:34:56.19,1909-12-25 +1909-05-05 12:34:56.1901,1909-12-25 +1909-05-05 12:34:56.1902,1909-12-25 +1909-05-05 12:34:56.1903,1909-12-25 +1909-05-05 12:34:56.1904,1909-12-25 +1909-05-05 12:34:56.1905,1909-12-25 +1909-05-05 12:34:56.1906,1909-12-25 +1909-05-05 12:34:56.1907,1909-12-25 +1909-05-05 12:34:56.1908,1909-12-25 +1909-05-05 12:34:56.1909,1909-12-25 +1909-05-05 12:34:56.191,1909-12-25 +1909-05-05 12:34:56.1911,1909-12-25 +1909-05-05 12:34:56.1912,1909-12-25 +1909-05-05 12:34:56.1913,1909-12-25 +1909-05-05 12:34:56.1914,1909-12-25 +1909-05-05 12:34:56.1915,1909-12-25 +1909-05-05 12:34:56.1916,1909-12-25 +1909-05-05 12:34:56.1917,1909-12-25 +1909-05-05 12:34:56.1918,1909-12-25 +1909-05-05 12:34:56.1919,1909-12-25 +1909-05-05 12:34:56.192,1909-12-25 +1909-05-05 12:34:56.1921,1909-12-25 +1909-05-05 12:34:56.1922,1909-12-25 +1909-05-05 12:34:56.1923,1909-12-25 +1909-05-05 12:34:56.1924,1909-12-25 +1909-05-05 12:34:56.1925,1909-12-25 +1909-05-05 12:34:56.1926,1909-12-25 +1909-05-05 12:34:56.1927,1909-12-25 +1909-05-05 12:34:56.1928,1909-12-25 +1909-05-05 12:34:56.1929,1909-12-25 +1909-05-05 12:34:56.193,1909-12-25 +1909-05-05 12:34:56.1931,1909-12-25 +1909-05-05 12:34:56.1932,1909-12-25 +1909-05-05 12:34:56.1933,1909-12-25 +1909-05-05 12:34:56.1934,1909-12-25 +1909-05-05 12:34:56.1935,1909-12-25 +1909-05-05 12:34:56.1936,1909-12-25 +1909-05-05 12:34:56.1937,1909-12-25 +1909-05-05 12:34:56.1938,1909-12-25 +1909-05-05 12:34:56.1939,1909-12-25 +1909-05-05 12:34:56.194,1909-12-25 +1909-05-05 12:34:56.1941,1909-12-25 +1909-05-05 12:34:56.1942,1909-12-25 +1909-05-05 12:34:56.1943,1909-12-25 +1909-05-05 12:34:56.1944,1909-12-25 +1909-05-05 12:34:56.1945,1909-12-25 +1909-05-05 12:34:56.1946,1909-12-25 +1909-05-05 12:34:56.1947,1909-12-25 +1909-05-05 12:34:56.1948,1909-12-25 +1909-05-05 12:34:56.1949,1909-12-25 +1909-05-05 12:34:56.195,1909-12-25 +1909-05-05 12:34:56.1951,1909-12-25 +1909-05-05 12:34:56.1952,1909-12-25 +1909-05-05 12:34:56.1953,1909-12-25 +1909-05-05 12:34:56.1954,1909-12-25 +1909-05-05 12:34:56.1955,1909-12-25 +1909-05-05 12:34:56.1956,1909-12-25 +1909-05-05 12:34:56.1957,1909-12-25 +1909-05-05 12:34:56.1958,1909-12-25 +1909-05-05 12:34:56.1959,1909-12-25 +1909-05-05 12:34:56.196,1909-12-25 +1909-05-05 12:34:56.1961,1909-12-25 +1909-05-05 12:34:56.1962,1909-12-25 +1909-05-05 12:34:56.1963,1909-12-25 +1909-05-05 12:34:56.1964,1909-12-25 +1909-05-05 12:34:56.1965,1909-12-25 +1909-05-05 12:34:56.1966,1909-12-25 +1909-05-05 12:34:56.1967,1909-12-25 +1909-05-05 12:34:56.1968,1909-12-25 +1909-05-05 12:34:56.1969,1909-12-25 +1909-05-05 12:34:56.197,1909-12-25 +1909-05-05 12:34:56.1971,1909-12-25 +1909-05-05 12:34:56.1972,1909-12-25 +1909-05-05 12:34:56.1973,1909-12-25 +1909-05-05 12:34:56.1974,1909-12-25 +1909-05-05 12:34:56.1975,1909-12-25 +1909-05-05 12:34:56.1976,1909-12-25 +1909-05-05 12:34:56.1977,1909-12-25 +1909-05-05 12:34:56.1978,1909-12-25 +1909-05-05 12:34:56.1979,1909-12-25 +1909-05-05 12:34:56.198,1909-12-25 +1909-05-05 12:34:56.1981,1909-12-25 +1909-05-05 12:34:56.1982,1909-12-25 +1909-05-05 12:34:56.1983,1909-12-25 +1909-05-05 12:34:56.1984,1909-12-25 +1909-05-05 12:34:56.1985,1909-12-25 +1909-05-05 12:34:56.1986,1909-12-25 +1909-05-05 12:34:56.1987,1909-12-25 +1909-05-05 12:34:56.1988,1909-12-25 +1909-05-05 12:34:56.1989,1909-12-25 +1909-05-05 12:34:56.199,1909-12-25 +1909-05-05 12:34:56.1991,1909-12-25 +1909-05-05 12:34:56.1992,1909-12-25 +1909-05-05 12:34:56.1993,1909-12-25 +1909-05-05 12:34:56.1994,1909-12-25 +1909-05-05 12:34:56.1995,1909-12-25 +1909-05-05 12:34:56.1996,1909-12-25 +1909-05-05 12:34:56.1997,1909-12-25 +1909-05-05 12:34:56.1998,1909-12-25 +1909-05-05 12:34:56.1999,1909-12-25 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc new file mode 100644 index 0000000000000000000000000000000000000000..f51ffdbd03a43fadbedce302ffa8e5967a30ad59 GIT binary patch literal 30941 zcmeI51yq!6yYFY1p=;6DUCQt4DAB&1V9K%}IV7?2VLBo!ov29;7` zK;jJWhPdA^&v(|^XK&V0x!3*F%`9hK*Zu$fuj|p&kz)n`u+RxnF=92e&IS)ywE$57 zz+C*2r4<0gpdEwhZ|Y0^$;CI|xXKhhSJ^aOw}{T`T6|~omgRWf_uxH&b-{8>HEA1~ z&-69|wFhE}mp3-YR-3*LqL2e@k#{G>0s#O3LVV;S#4#{7BS4VXqbSISQDC9v17D&n zgOG4rb^e(|N#XkWa*vfFZn=GcBF8k-doL zdUxG5y@)$fL5*g@Y-!l5WNh)8$2NB;-b9edZ|fjsv@i|6|Hacd*q)BT9gf!s#8N^o zFTR2t^E=_!A-6rF;MF-&S@bm+)o%-ojN|2L-Mvko47YE_djj%;<7!FE5*~$xnnBM# zvD4-CfD`Gb*t$Q*$iW!>iNGkBX#@~*G;>r!;#^ETPG50A)`8U;Kx930LipKgJ&8EN z4}Ix}b2i@4z^sYWO?3>KdXUR+A0nqhTO;_DEoBn?-d)^39*F#SJCstA;l=B$)66L` z-y!0z=u-gHlaEl@AS(1OCe%6E0<+B2jF|PQv7zlc-e9srY_v`W?`8(T~E+{q`PMBnLfU%6#gJ=NC}oggCw= zLp@Luj++C(jDujty{EujCFD&0Y!rNB0;3go8R!B)ReBJL{L`momkbxoS~Ll^C&T=& zNBEvAAnyWL9lKmLHwr}GkeJJhe9V43g@1Hz*>4MakB;JQYLb}R3bRU_&O9-@yvE57 z`$+1iHh9-kUK20c7%uXLiK(T2@(=rzYRz9P)v8gHj zn?X&<+%uz0y~EFqlzPb{^&9#_pKH@@zy`m@+Zc4`%6AfazFz#A#Pu=0^dpTCk7*^Y zkGauKx98W0)iD7HPSYwz9~;)4&OU0mo%0*M$VX!B7BQ|=+)K5_t%=4ph|rP-$q%mR zWeYLJU77X%o9rU4WX|(t6YZ1rspwal5K*??-Hf!~L#H>yS{5SjOMNG4nRE2-7dZ%W z*<+mMYKHq9y~95UquaYIwJC!9rnA3q(=i9L(F3use#F?Dz5E2C@`TV+mg{Snf+)u7 ztm6|R4OyOyFqwLcJXeyZ9aI+KfI(MG3Jiu|^7sxB-up%dFhV_7bjSMpy|Y}mW(Yht zz&_sLphnp3O|A?$Zr>${{cU#sOT+`S5^)cx_{i!k8ImlS*V^}Icq|!d+x4zI%=9Mo z+2q8Fq-)o1e3;-(WQ(a(UHv$vLjw7~>(Zb?PeJK*rGo2RQ>r!9$&@SfHG|rp^Am=7 z{2!yOT#6plP0LRgd@KEwOzKS(jt!GlYB!Zwa^0Ou#Er7~Oyjpc4RQE)F4HYIZ$?hM zwY?k5^Nr1HLAm*jKn}6t4UQ-jl6>H$&9Bj%3dt~hCj-I*qD6b?| z@w1WHTrcYZLv?PF6ji^xPC@case=QUn^s~}qorD-5ztDd1ei;aa3XT znfkbTx<}5?zK$)dOMI|WV=FrAi&4{i)h3ke<(lRXREIfr``YVU2~$!B>U3L?(9imQ zgD(3~fZSHw2QmM zR__dD;jdJNiQIH{?rr1j?U_ieX%L~>7`@e-HW_{6m>fIO$sS_7h zXD}(s^o{wIv2lZjni{N9xti zpsBI$PT@)sPiL3aHttod#yUqI7m=M-=heK)G1gb;QsW_hm5M(cGZ@P5Q;@yv z?%=*H;;i8ZCv{%YTk9*GZ2DA>URMBhm)M=(SsSC7h?ID#@0AO^V~g)yUAIM!f3N#B z?USoKyS1h>iKYvyjpLtuaBaLxy|%>H=eMWEs2U?hbecu(inw$FhP)#vaQJfh)i%(U zyd$xVXR+2@)tlOIp>R^V+xIo=yKJv$m2W;M_ihup@&i_QSkQmvU`P?V6VbBi7Vw44 ze?@8AP01Iy4fPs0Xfp|(4nBUB)h=-SC zNWKP(e?(Vz#TbW^yAPAe&}D=$)Pt>Dai79T-wh`-&~VGKgfd{=R%%mExn>q7ZxF6q zkNw4!PI!jlR&n`j?kiz_-gx&n2xDf1GKxfIsDmEFBoPp`V#teL!asia&5nvhn?iSh z`&pjfu$D`nu(*PW`@{WZYTN;4%3_y1_LuT*IF{n3ooYB%SN+J4-`=H7VeVP}vRM=3 zYtvooz7IYj$h|h%*XG==Xi2qO`9>EaNc%CBmaS8HhhKr2MH$|(yh(7!=TRhash<8s|ToLA?J<8Nu;VvEW z8_&OAt^UYf@i2a8MT}-pRs87(cNLeF10{kqPE0v^88DHoFjK%mAaOnu$yi+4S7kZ8 ze7}QO_xY>$?y!IwV&jb8Y#|6z1!4!r_^tR;p5WNdYrp$;E1l#$;^rn(om&RTk;dP&WF?<4|WHJ8S*v zw0cYWQ5WA~L9UeK$=F}y@R8NNs{FAFn&`C8ep4axpY@u;mf@}Bheun!>npoi{_86( z(^*G4LuMTNL*M;;T3pt5R}bqxO$9jp&8b7pmgALOsl$Ndp#YbsE!VdSmwsgX9}gXy z9d8{2)=om6MAesJ3N*OnT!Bf1vXwaBsLe{VyV*;g2L8-twZeqS2iL`m)i`8qbRw>^ zBr4g652R|iKPAdsaTm`%8C1PC`}hu)p?tvFyyP>z%V(6ZeKWJX?u(^KS;@AuiPV8j zVtH+erEb}?5og`QUWQF<4 z!p-w7>8&?zi=w6(Ufly1uogR&^(V1effr)r4QaZN>uc>E*jr*J$f-g5XY(D%Y z24bF%f!@6WSLO*G-EDJ0WK4hScx|@#aC%dyWr^wtR(BA$PMW5eQTb*SfN;(JWiCuDtX^v0pu^;ko* zj(;x-8hvbI{0hh1*}y^f4w{>l5N)N(kf$G<)v>gRa9_D$Iux_&S5ysltv-3q}Gcj32C{eK|*Xl4N>eP5MjtrVQ zAGZ=(KoW#vPQWEGQ9F7O_tXQsePYqM+ze>g10Txx?+UV7VD-{6?Z7Y4;8z z-Mk*OXL44*A-Vq>z1h5Wb5DYAf~_=r^^5D|s2obISlYrWcvIJ~3a)e56b|>8+CL5% zgdUizJWd$AY-^*%-BVJYUK&mKK_%avjRI}*0&NP3JmDP$E9xjxUmN$%H_OpgeB@+S zf@PiHX5pJeDb)>v6hvv=?JU6Ql!gW&>?*;_ZnmR~Q@3DsHyvqv`>v!m-4#avZW)80 zot|)4z!uw}x~7^O+YrOog}~8jVRV$^`QWqXnYtl#&pX8pNA(L{p`kk0Y6vHWTu3%eXAs_<3b~Bv1PP-;K#OQ%m79a@x?o?ceP-TJkXh!Gm_&%%ZH1-JYJI(_ zbVn%?JT4E0;2|rI?r9_IogS&d(*2E37=lsdb#qfEt^h8?86RM zQM_S!O$A-v@8xKBFqrQ()NNyxZ*jORjZ95Bvb}a}%KLDawS2d43U1OE_pRy8tGfbd zDV?hC%pVXWS_F%Tb{QGN3H812JARg%fPE;o=5<4OL46n#1aU^ za+MBNk468KlLN(lW==NT3bhNCdym$YlLy7#hg0LUGLwfm@}kz6KW)ZXai!pAz|npn zN9f#^vCq%h#{ay{#Nhsq3QT+pRX-BX8TPNQ-9$r^U2)21uKLmR5jyrRlZ3)Ju3#p%;Y2r-)EP>)KGe% z{LMO@SUL6jaC*MRj&cHZys#LH$sLt@pPl%blC9Os-aMX&7ei%NKg{4~SJQrYE#7(X z%H-3I*8zhZlLa0xHPg3ZYghPIi_C_gl7({jl)JXlG*-AW>DXlxm@P1v9+=B^htEG+ zj|+aBe@Q|oLDIfEsQS@*LQIB|W}h`Z8GrocR9Y!|H#B~6qnm1I0*_~+t~|VvcW3eVLHV68jeog9ita$q2tx#3CqpknkKHY=-m^G5J#;L!yLF86?H@)y z){oIj%sQ#p#ZRMNyh1*IriQI}W}kbm05UGJiVsDM&G6j1Y2L?lL^S=qY5m8t?KRK0 zi$VrNUl!7CycW9mZq&Nz#?qJ7mj=mS9GZ+i;0N4TC|P}GkXYhSV>D87`0NIGiKmCL zN6FALvrPThpI=at9XS|I>Iq;Qq?x?H!M;)5ZP4cYF=O%ts?iIQS5{v-e%yLZ`!PxK z1&xoT&(1r%Nc@j+O)rRitZeaI1?%QhI_^`?@+#KjCc7Dx!Bxr(ecMMSvK;F{AMPlY z&1o37AVR6?u|@9aJI^{#yaQ#K`u3cUgk8QJ>xi`J@3;YQqoZ!GNPXY_?7g6t4aR^4 z)}zk;NFK1E0ag@pYi}NeLB(L^%T6rIe!WejD6Hu{2Ko~dUuO3<<>jy{UJNn|;>(t(0pPRo=Kp-k~&CT4}Rp06eb7zSykZAypVx z4UHd0VaQi|GZNlrT%6V8Y}a|-o@pR;pnI!5d5=P9(X zN;ydsg0OTdWFJSzhj=}fWlvNYW3PFkPq~m&jURALd?A0U=LW^&NPZ4|8LBi|}>nM(3Q-MuHt z_fVmOj(+7L`!X)uc~dK_O#z5k&}A~e;qv|b=`9N2)ey~dk+5IuTqGnpzaO+XzaJd@ zWk1M&em`hNRfVZGZk!f{scvnV5a{x5Vj?ZFuZ0gO|eQq^QKEyZ);=9 z-0P+D`#~2@OQ+`(I(h{K?B965|>Yr58 zfl?pO?FJ9*KI}|6e+`)&SA12a>51?E0{w^Oqn(kguj!H!uqUnkcyrR|Vjmg$I^SfL zUJEqK=ou5Msq&&+f>-wqtiZ}O)-y6&8$|QmJXn?%-{Bv-+Iy|=bGyeynjpU_8TY!l zCI(93^c zO+BCP*3|7%_G}ogFMCYA`O;J+;!Wm~@9Xl-c`6F{bdykk-14eT3Q`5xBqghFCz+Z_ zyGJL22sH9-r-9DUW^YRS1uFV>5H-(?p+i@B{1#`jq2DSs52CYw7`1Guw&X&;IR>nY z9JRU}=1uRhwiH4)(T`^z9gSokq)VaG=M-!H5(lx*$3aun7L&mLWF##AB@&7|R?lwX4ZIbobw0&;pS-FYMuKGyL z+jQA;LHkhJImas14%vaD}IpZF_j1rx>JioY_(BCo?jvTdcq`XgT%PdS}Bfm@7O za{JnUG!n+1kA(5(BH=XSzY+;Io%biVrqxb^+t1POw|!s#ii?XzK=E0`I*oH#L(@|k zzi;y-3#Z-on|zMHzuc7?|7|ZgwLbadFQ)u7j}LwJKixdpMXsWYp2j}^>7ju2tfSgg zWF(b3+C84jIx@Qp0HAS6W=Q5sK9{VPY?N%5d?z_9Ie{i5`BidD@&F2k;y{U^)KDfU zCz>Sm3iK*e4SF4F47G;dg1SR}Ws%zLFjD*m5X);1Fg&O3=#kN>e+bEd^xe2iLxlDh55A1LBaJCo^Ey6$(HpLwPcCNy;^q5Cx+i zE3_|rW*t;0#t;}-BE7uVhfl343jpc|Y3=pkQ*)dRN9NIYgicQ=((^a}KyZ59OGSTk z2?j94l?0=7eJ`jt3WD$o;u!{02l-8S?SvD7g#O_-cs| zX;?|1-Q+5W5xEc};6QE12Jjw|k66x`6Pd?C7ElJF2(!%7l-r^rGeZcv16on6D^Msd zSHmu;5fY__rihXUl3x{2z8!}41)Yh-_NfgE2tf1`bdf`j>9Ej{h~$KlkRU0E3LZvX z{a{;_HDt4PVR!AC7J%=+V3dMt6v%cUl9jNX=;}?l!RAOXL5@<~93wOl_EEJfXHSJMrTo7%_Z_I(05;Nd%Zu@9nS)d>} z7dr-L3x;#T7LZc^fKt70aGlR9zu={76z6e?>mIn}!)Aql@Q8qf_#cF&jYd@zr0^4v z5WlM2ONKKjq=f;-490X-Cy58E>qIXDm|`EZLx#;?%5VuvSzsRVc?G8LYB-zTe8oc6 zg7}W8r;l-P`YhyUoWjCrA(HWEorSw~`wqeR12ffN zx_)u)){zy;KMo+s2>pSB0Kg~ax8wqMKd|vpq9KM_TbJBgGJG!eXzgY5&01D`J(bdS)8;)rM=zr3!WvuiZ+(Z_Jf3N9aOiJkVHucqCSlUf68eynO#9IH}n*cj(v&m%M_ z<4}+{tjt<$)|mJUa({asxy%28+)Ms0a$jfv1-U={1-W1U&ms4?Q9Ja~kr>0m%xY5! zM-ak*s-{HPUc_lMNaKrYCNO(hsY`r7RBo-;Ae??;X?+F`@kb=zd#LzLeHd1i;H^dbJq_m>UnMu!>|AMSNkWIE70(pKF z0>1p}dP}u?@Z)m#&~7@psRd`(hekL{>}Sf!4&|eVPcn&4rID%r16VQBsWjq;ltx~j zsU$)l4q#6hPiH-n4`5}ir^9WhKV%fAn7tqeg;TNS-q-J-FQ{jCzP5m_rz#7dM)ju! z036wz$iu9|V697LR4L>D?>`g}_{@m{!huvkWYbO+5VJQpCklvw^@#%Vi9GB?0ZG9O zEvT0b*#;3N2n_Kj0Dl)iPRyp#KbuYQJb0int2ACmg6NpOjiZh0FZBVS^ZLiBS2X$pN!xplWXT%#~<#X3_Y~mDupj6G@}G zovw1k_2dugq~;SKof>gHXNanBwK22ylvORVn9bZo3uibYZaJP?*_+WU@vSi-DY>It z>REEH#!Lv?YNRZt#M#4eyX3orfr;K+o0)paCx@Fr)hg8_3x-Hi3e}Ie=~YIx=Bl-p zUbmXOJA}UmIgK%Xs${LT?f+VVh@~03EP*sGqsA)7JL{q&CR>=gSY;rxABI!=$lQg9vg0!JH5a#`#E|Fj=-Rm5-P; z6=ak%^wp8&p&Fu?O|GP9Gnb;;bW=#pum!-e;i}!z7zFLHoXtUE6KyC-6~Ph)uO)Ah zPPdWLTA*uI+_x2lVN;V$w_;AgIZ@(BlP@vK5^yz>Mbs0x-M&m}$*wo6@tp4HlJKm| zt&Y5p5v-MD{j(L`j4`vi8Au&lRXVqzpq+C{MX8N2PFRQAsX&&az+qG4SwYjV5_)lB zj1FsNzW-*Ppc0w=g)w$2!H2Gj%Te6lmkA{MgVh zxTc{}o{=ciz?OGMmt`Y}zeKuVq@brrOP)gy6Eb2v`7F%vS>nrUjvnR0FfF6)BG1pt zt0Pt7MPwe}X7OSXkI_H{N7d+ooN(!<1WtJi<9e5?bxNu;^+NOw^JuE|Bpfn52k)&u z$A!x|4VR7QPgdVR7cb@UK=aLKr1^4ndby!_1iNYY@t8|n#k^*8;Pdl@5hGhifXnlNmxc)* zRX3?VRyS2sY>b|FWMrOqWFQ7;zeY-q>-)TL+4%~%k0vD+1)`ZZzb{I~>Q$=&^!Mbj zT^jD2vd;(^5?3q)wTL#kd7>@NM)i%PW~2*99ZVKX7QC&0FVF26@#?nQXURHO)w(-g z&Z}#aos#uj+s&UESnIs{*WjiT15Gc9wu2;m{zHBY66XzI9pZt6Z<{0(iDDp+5D!^> zUsvJ#j+^bjdoATRZ*}60+O6}SN%oib`}+6&7};oQ01z=e5S&hrUn?WyM|!=LyC}r( zJ@|eOZj6j#6W)I`Mf#OO{a^Yq)UR}Y@P6pBnGqPOk%TGNq<)@4aTh1}4Km%&SC3In z&WO+2=c9;MXM9Blz)Cf0b0?r)(YUNdHxq^U&n#q`|{%2sO{3N zZtgL}KXNCW8YWIi)vJ{N>NAtY$=GY?SoFj;cziSP`>7|fsqom;=MS&Oi6?OvX|Pyb z;SoBsGS=<>*zup93V5}8_(Aw@P91NZxe$@g#N88K6|moR>Qp=~57-X?up{%OzW`5- zze62z)R}xwO4c5{V2ofQQQ4=ip>>fP^wEil@=R_AT}~S$`up0LKyWTr_x9W3EfXYb z(nhsznggYuP5O7Dk48YG>7NDq)@}ybuz(>CIOxs1G^Bn_V|xBfV_MoVPSTi3Uug=3 z9JO2s5jqGW5L7{|Z&%?jZF%RW;Z6R3g5wnJw|!NDY$m~aibVg4<9wI`LW)a%&trb8 zc}W2t-lu26kQK*Oh3phr0Z9Ro3OOk`xli#zKyBoQU5!BJHOafHsm(9Ne<^G^Sba2| z#k`uTH$?hpT5?+1asxcyle$=DN1-Nm2y}5En)Gt`)Omhn?hF989YTe?QmCF1zWf zs&!MQ{_iuM4?<89YrqA(xu_Jms1&)V6uGDrxu_KRUt1}10dFEN;LYEPf*0@xSugpo z`C~8O%|BNuashAt8&-;3z?*Z~Lok9Nt|%o@a4?U$M`>X8n6TQjq`kCHP{5$kUD(!fU zj|lN`0|5sg0?__~-t-m#A#*WE)#K+*ALM`_yI>1!PDb;wFp_85%-$?TB`~Dyp+u;a z8AR9#r*lKEzq>mxq?o~_%Z==p@jHwmj|!ceYVp5DDWrJNU2D7yqxr+tN{-pD>1ula z3z6EFGGb(xir<@q;2@iW95=S~BM*${{&YYjX#?LM01&9Ona{N3SAn&`;oqBhxEl$~ ziw^p`@0z~#d`ylTGT_dehN}=6jN~@IBOHQjgeXSA`a2m!=70-;a{+KJ0L}%#xd1r- zY(eb>!1ovcFhmq_@Kr}49C3VH+fn&=#dc$z$Nvkwb>!!)wwDfj-~ZsBwb^AmqgiNO#2? zQ~~Y3q%%KLoSvU4PTIdoaR$da3%DSgu=vtxWl?L5#JXdqvqCm{lSj<#jDx%uM0x`O`T7{N^+46Geq7`V@VWDc>p=v47 zn4kRX#BaT=P|Mzye2B*z2*fc5fbR0~8lw~*x``X3tbfTPq(M%R!vJXCEX@&k9_edd zn2l|h9^xIP2k~f~pq)z05NYyo3sxnTdIo;vZ)V~Q!Y-+HHjl%7E)OOU;WfcXQK`R_ z8x|Zaf9Hb@YC|1@6yg0)zeMPL6#mH*4=C*3H8qhe(eL-^GoMW%pXg6vr>5&%d^w1{ zFKiQtcDghe52zL#ep_mc5HX+13q`K!WEx=uOM-nLX&~fPW&7#!3g*}=4*DpE3tV46 z(j5R~0RSIM@BW=PY_!P4OOQ7WO<@RTM@i3sOYPh2)xR0$dr?_)(Z(zGZ^=>@ZM-hp zcwMydx@hC|KfaCE1)RBnGk-G*Uci|PIP+5*uZzl>|KV-CF5t{5oH^NzpR_GGg){A@ z5U>nH2I7ttX>OI5h_sH5nAw08l@fu@GZ@9xa1xDJ4(Nvs+CDV1+s`cYk-b2SQHcwp z6h|tY&c?lSyFu!TCq*(j%WDlm9-3!EmxiE`P-Kq~UFprnMYJTJlg1^$jxX_41{p|} zgSVJ*v93Y&G%WrM_1w4$l{t;r5--S@uVo|+0%G7e*@1v zJT0y{?gal?TqE&s;hBV|CPysH8=I|D?1iBqasU@n&LBRMGjLBDv7=!GX+e_3v;Zu> zJ3z_9w25P!2WWVY1((tG(2C@(T9as0f@Sq%W6^>batxj{mbFUEa=W^h4%-U^ zqLG^i2ED*Bo;s{^DUna3Eo6Czj>8KBfN&n$&?mz_Ht98h*RpL=@R8>sPiA{7%^ftk zDV_`~g@*ol`8=WM`l|+Tc(3?xS0CHM(b)iis*0wRTYL{ILy`SDe5AJAzw=)}my4>I zi>jH6s+o(bnSZ*A=>>H8p9)=?PoPVAt0%GIABowPLabP$7UYHNZx{kE+3H? z#gTX=cekJrCv6Uba7 z=Wf*+Ei=dON1&G$eAFGm+6>f|&=|J`K9Q>Q;BB7ngG%ww0BEzed6m%VB z{dTr0yfMAa_awLB-N;7sSNRjJ%nYO5ca04Im~>;Ju-8<&g1Lx^3GpsnjeNYq$!yZ& zSca2wyw3?)<7!5QX3f^{Cng;t;783)_+D4$U)|061EA1zf(%hQs`m-&+avzqC!sdd zs^+xi&EW8N7;~T#07MRxHbAQe8(;fDxUA81?nJn*-zBafPvx^)1|17-pmGBMMt&cM zJhzc#1;jsyByK*e+f#Fv7sW0YJmqgHNf$ijf~Q>Ylnb76!BhVA*P>i>D!b@Z_OHJj z?4ncIf4A47Ty!e?-}YLR3!ZYyQ%*WUobVJa;;@;w1jdvH+#dX$Y%%hiT z8SS*)`}-g9BCg)ZQWj)$c<54N(2nwHOS1j#yP#Ld?j0w$@VE==l|7v`9B&MYJ|#1< z4n0l&y=3Fbf8EU^_*9l?Y5j?tVE;8Y!9fAc4K7(Oq-ayZV9Z$gjR?`&X~fZyZ4<1^ zwr;mi6@>L26jmxK3iag2KA3(3CI;B2z|lu}Tk-QPx(ejduYsY?KKq9zyaunqQaRxT zMk&a*MZpB!i$a8ue+Atq_J*jLHZ~2vH-9lT$EEo!jawBUfOX5a?LluiNef=nCIlh6 z3M(cd#h@yiwjwrYNFzwc{kfXPFBI1{nIG%Q8p!^=T;0rIua&9?DwH-QVC@NxRtml~ zNJ7z;H4nC`MQ=tn8LT|S9gskXK$x#TzSbsfVH%5ii#i&JGQ<^ROyfTHv&tep`aLkp zcHyHFCyWXP09*$G=FCZ#eY)5>-nMe`hG|mAFCObdkIVLhl;dqzWm5CBy}lXNwVTQ?|S!os`gV)y=$*m zUVjn;Cqq&3Q<&Q^h>QO7AzzQGy5`K)8077Do<4p0)IDUxooCOUJ$2?7_$S_f*XcV? zoj!Z^ABVpk{^O$GF8U9wmjI;_AWs5pl>_tTKzBJ%Ta2A6!j^Hc<7{kU0d|0i&0%0W z>DV0_cDD>z$-&l8u(MgP^Z*>&2Mf|*e+oQIf*m_xiwI~+fJYydSl$Ir7{Q_raIhUL;{n||a90bqw1BJ4V67UQ zYXr+0!0~#pPzjE6ft*^fvj*I$2AiwE6)9L#0nV0#rDAYQ1Qr$n{Q~f?1nl5}TU@ZI z2wW-ztJ&Z*3luQHwnAV~{o}s$$EKnm=MH};j{ndX{9(uSLtXC=Q!PJ?Br)B*A2z5z zRBwGhzVLmX=lgc+_ba;Z%WJ#Q`Tgm=*Eb~q7`ZsN}-z*GZ%1qx3slPdpe$!m^&HUll zCGoHOgJ18uzOL{6T42Bw%D(R5eZ5Kjx@POEiG{E7JzsTLzgpFORZ;uZfD)5K|Ee|l z<(%ltV&<27BFxUp7j^zGrW{`sbbis@@Wn>?7uD=9#xtMi zt$*IY#H@^bUT*q)ME&`p^z)XY&le6a2;vt8f*1B&7iRa+vn>~jWEXmQ7q+MuYPUX{ zT=iwTgJ3cAw{G_Mhlg;u^YS^DlWIo<kmB@$X zlOK)@e0bRYVTi9Ul~Re$d9wHMQ?f2;R-7zuS>~XEpLp#pFAq1MlP*-Wi)kEsEYLWxg}G_xAqE+fDwr z=NxYrcfQ@%@b*sm+hspaW!@@Sf2(`;t&Nels!eZ=tKZ72e5<|at(C(!+f1mD;G2i8 zH(Pq&TxfYyAbWFw_vRk;&4#TvW*6RAQlomUZ*1w_sI7ftQt$?o{zhl=_4UZ>m6NZJ z4ZL1fiE2~6z9f3RjQRS|-fIUduQmH$n|Hic()n6{!)rrDsCxEm)0tNb*I(_KeRXr> z)f&^Q6Y5vdiCHCSrLC_H26x+*ek8QuPnB_QYw38koU?y^_8ZrbIoyNvFBW$ z_1uo`TwU$CDZ#k{`nm4p%NvoGt0!Nc4<%|8j$o0b)D&sGV1z*S=d!eoO zg{77k%49DL@m@HfzA$ute&P8N&-4A(=XZ6_*VjHjEqK0={(MjJxy{IP^)lqdz;pSA z=Q@A~>^h(Rq<+7(ocu)6iAzHScT3C2W;CX7m`qZB8 zsfOC8W(7|b(Vyx~KDoJosGWRra^Oj(;mJ*1nUnC%s#X?@=ygGF{^&4 zsPduSqKCE)AFPc(I2nA9Ird;@?}O_t4~{0`W4s4*b02KmdSGecfilknL)HflbPqJw zJ}@tMAZH2QpFF!8Ia@z@c6#7!q2X+g^6aMQYz_15#NPd_lkg7z{i}}qD?0BVZMZ*2 za(^rP{>99FrR(<%&fd2;0BbX@xbnWfqWg9Z@2!j9I~BaQVC>%R-g`?8@M_t; z5x@TqKo^sDUBi4Hkb@#N?-ZQ9#3+VR@B=6pf+}$vF_w2ykMTWb3m3ME6 z?yhCty)S|@qjz=s?^<`V!{tb&NQ3O%&X6oRG#TC zIENA(V|O+`y4iAPjqJ_|-ktfmcXn)@UR^j{;W<5OJ)P5bx;P436r3)l zpB_x!u^+jkY4VP_fjf!~cl0Um*b&{)=Z8)0ohpc)>h_=7aGa{{JT=~MDo=8%oqcL0 zbNh}1HZptr;mGYRrrQ_Pw+kw7A1Jzg@9<UYl-33e&wwN5?DR+*6F=l z3Zu95_;1;C+)~qd%S6L1`I1{Y*tc}EVHN8qMrTjtI8U^iPAsZVlvbV?EIP4&cym+y z=8X)rczCBmmsMUrRCN8p;qi78bUt{zWbAl<@A2K1$C6j%E?vTWa6_uBlT$@9=yfuDwF?h6e?C4 zURKa{S$FMa8-mNK>6eWsFU?!Jv_S@08Mw6EaOsHh(nHauEzC<7_AU`bFB$M(vgf#D zmIs+_xTHvONiX}7t<1%>>laVXUd(h}+-bUaU43y;E@Z6e;#|tbZSf;Z!6Rj3M}~Tj z9JCy1mK~Yr9qHWy`nSlt3*>qadD==YY$Ny7k~an98ajC*N!nTfIwnc01EdNAX;ewd zDI>KqNsD{L(kOAzPptI-O`XKK24b;<*vBUBWC(TZgsE9Vfs@c}B1~F=>Po_R5h0I4 zXpiGpg81?={75hUum#^D!!z3eK`wq^3%9p`Yw+M^t+=8#TyHIIOMt7TjL1w0XBDn^9^8$1ng&nyBSRVI%awnQ|QE0(t%Ai zrlt}zQH05-U^?RH)gZcJ3_aS5&e5XBeqZFFOLNhKTd4g7RFelaXGImaq55i3I|5Yh zS5rx3!4k5261g#etTrIWmB_p@WIGeNvWIAUXT*;PEl3ha>G2Eb3qGyqaN#2ki(C~*HPiVmcG0u|jvvpJwzg>~6*i4dND zNYIy)GOIv({D>rev59?YPYd!+XiBXEZ*eR^1UCZ@fMJ9@Wv^`4L!o~*fFWIk)0 zd0-~>V3z-IRm&rmkw?QTkJ0H*&}I=1(^G5yr#ZXNXiA=QH$T5U^n!5tMFZ_+f7Lm~ z9#T5>s(I(NQ1Kfy^_vRETlUD?QR=(w%JF}qLoC~Ik z&x0LbP`zJ9TQD}xH_NJTSp(lSE`0Ai{2^Zc6VLFo&hr;<{1+!55ZC|K)&F~7{trs_ zZ!+=!vFiUN;{I30#=ofw{-KUyV|_UQH%JAvnOwjl$^()*Iv^U$2aM|sV1~&8vb6<( zi;lMh3W0E%1JFykfU>n1aE$VRwbc@U!w>)(xe#!hOM!%pAlxeh41#jN-y#7rBNc!& zDh15+Dj+1Q253DpKrvkd@Oy~S5(U84)B);YCE!}F2NJXffM4AR=)0Q$-;^3i?=%DY zX_85;1%i%NfV!ju`Xbatdr+yqr_Im({ z<`QGp3(SapKvq8mRE_%q%f>ViE|>-A$^f9W&H;{K5HJm2$|+j_G;LwPJ+=sJuSWnO zYZ)*oRsg>x3SiUW$ZO~B6B0;2M5fZe+TsArRaYi}1w2=)MK z-sO7dKH!UHfHXY|NNNrNQx63aoX&wzQ&fnYPlMPrc@TN^G1f{xq%n^PaaOY+@$Lc$ z&(DVFl7$d2uLzP-b0OlPVu-E#ia;(OLa7o!WL-jt)mI9MY>Oa_VlhP3R1O&)lt5yO z6_DkrE49)ph{sqBNqS`v(N+z_SX2v{X;47219gz9uo7aSs35H5RrHQVh;pI{;@DI} z)(SNcj;aNsv1uXhP%C6RtAhy3^^iv1HU4n}B(u>0k+M4>W@Q&7WbKB~=1maAK@Y@U z)(eTYSs+gJwd$A^;$qn#3B>?}-)D#D=LR9ZeFr36It-EMMj$4a6B2h^M`eyd
y zt#=%Ava}44c+JGdN;t&xn0Wnr>LA;eWW_>%5 zs+}anQk;T>oAw}dM;fAx>_Z&X1IU^*3*mGgLQTm ztrSBg{pC>8yaXCNsDM(%l~B393Tks#LziPRD62pY4RCLD_9>w8xjHB>qlD^2DyUc2 z08P0Xq2l!>Xcw~?8mQGkDNVP@0$Qjw-3pBe+n|hAJybQ?4jo=KKx6qvC|BMI)tbAY zvcZ$dlnE*ln4!j&Ug*q-1)5#ygI3Y|p%$498a53;>C<+ob@6sb$q;m{c^JwW9)W6> zolrN;1>LS1g9^LdP{Y(X)W73_Msn_uswbglhYuQxOhIY6(@;g_4AkB^3yn?&pzQ5A zsG2(u)vdr1_5~<^F$~q`M4-NkC1|>187lFvKuw7#G{{+lQXAHxD&w8Dg$?NPVI0bm zB%qCkEvVD84UNZlpuECesIEQ*_4e;UhrMUS*?nl2_y8KvXQ7nwL#S+n0<#uSVUfCA z7{f|~se*a1SmLgjhymlaF=5&<7R<9=0826pVIl*hI@t1DwrW!4fE4$U>TVlChe((nWq)7 z&~6=!#;=DdG%A>9;J)Z`BaEG=hN-KYVXkfsEHTvr<0rK+Jx>SosoP*_haQ$(xZjl9 z0Si_dVbrcpnB3O|vu$_7mbpDJR+AalIM@qwE?QtB$^$%UKTKz|!Mxr9SSn$MiHjVt zu7)94U|<+V36H>Jlu?*b@?gX;24hUPVXDn>*l^(lELQJ@acz?@ZO8}nWT#+B@ifev zhcJ%M!e%xCuxvpPR;8SWS*#&gIJf|#AB17bvIxx4wgk(19_GYWU>epc%&l01ZClo1 z!nqjCu)hKGmu|u`tqGXawFRr%d?dt3!f3U-Fhy?)W}n@IMfcJ$wlD)zYY$*9XBL)- z9>OezkLhcua9>X@JUvZ=OHz4o6F(mw)G*-G5hh%|!h+lK*zj=u<17;g-smraJCj^^ zoX3OnnoHoiAwJx@B!H)KOX1=w5u9#&BH$ClDLdtGS#bs2+9ZWX1}ou=NEKW~mBELl zHSm~G4p)Yr)NU!@o+2eY*-#G`*;R03xB)&xX@X}ZYIs#gGu$$vfjhEK(K)SfrAh~P z*xKN0Aw8UP*bdi}cfj54M)>x4CtMisg0G36@hf}a8LJsCo$rO44=nIdnH5gc_rn!# z8{8fnfJa$_a8CQP>b@bkYi<~x*dKxOMWb-N&IR|m#^C8SH(bK>z)iIi@KiOD8kmI3 z(>}OO=!Y+Br{S#88F=IBEZmtt2aneT;XLy^+`aLS zQMgRD3b&fp;1T~ie7oR9RY@E^+`I{o4JF{*}gUASl}1vl>O!G+3~ zvP~IymE!1;SZiuEbtNbLjfXMUW#BFLg!LP4C==-IJ zOy-sJK@~zGmLW{~8br`7M^HCv5%PjMgso1ASnjJwu!1UtwCpvfs0k6*sS&)fW`u5C zgYYu7h*WJWLTu3?x&m#8Kw6J5x4kZF?Lb&Zjflu>UV3~E2Z@0dm?SZ_;n0|;~F93s>iM9_Tm2*q{?VdsVs(Z)pt zdoY4fFD@bMigywfQ3T((iqL!45Wd7ZBF))ANE+e@)4(PoxR5|lDcgvs~BB-^h;s*`O<7q1e$%ypjb|KT1-AMh~ z$0lD7GPrFmdK;X`)W9fG z9CjhQC~jmxGLEElc#yJ*31nLSS)|a1WT>W)Dw`iU9GXVPvNK3-c>t+xpF?`agUIB@ zJW|qo!Ke%)XRM3JY%qeXI#@zl%2tr!wkVSBUPUTnYe)wxhBVE7&gqLIHFKLt_kIGo zUAm1F>UNL@R}$%8+eKy=fBNy-G%~pNMQApIq@@p#3gIEruBD)&&Kwkbm5Ng5(@?IO zJXE5Gj-m>`(x(_GAD@LvYYI@35jM)SQiuxX6``myE=q1HM%ny4lwA8Yt3-fmY!;%N zL#3$rk_g44iBY<$a+J4Af=c-+P~sga%I5qg&{U10IAkbUqy}ZB)}kU(1&YyGhf+-{ zQNvsHs8~@0YB~C?*4~8jgw?1dMS~Jmw4jU~TGY%$D=NFGLsfC~D2u8c#mfJlzR-bE z9vV@O@-Ea`dpC;XF`+c^9+aEii`rINP{RH`R3M4*AM~R#WdkUw-i|W62T`Gz14Szs zMk(q>Q1(72Dmpica`ybF7P(O_-8d@Y@}T(Z6DU1%66LG)q0+rmC`rJNGNq?c@#&wa ztpSvLbPi=(4WgFwLnxMf0o7;@qntB~sCX)Z;t7^fywuO$ktiy)vWgPZ*HK-v7%E`e zKvAaSDB12N%3898iZpMdbo^gb%SqHQErp6z?V-5cX_R(qALZG}ppwN|lt_JuGCC+| zuja2=YA(8}l7_Z)=ApxrbToZCAFbpv(T+wIdd*&d<}9+&so`JU6-DUn4lY{gEk+v> zJhY#~M`u(5v~)m-HZPQ-Lx&=?c;z>Rp&V`ZNYK%E1)5!0iB{KFp9qkPm(5Z9>T3p(R?rQBq2S&Tm=5j#BFr%&VUUbB4K{ICi(2%32o_hahIv@eG zfF6htG^8}ydNJox-(|AP*N*A0q(B|l@UIITSKg4gvFm2~Ev{R|Cmk^L9jVinGwi#) z_f*{%y^_vp-HiYMYv8lR>;>wHf1Lg>W+GI`GYD(iL68`8Iy-`2jHdC&HN z674@P|0MRQ;R5xG;V*?>2ftB$m%x~QWc=*@i{!77Up2p{|FHg@^DkXk*}t#-L!Sds zasdYo;M0LXKA>a*2^K)d(>Z{v2oM(oVIH9711SMuDFxVNfL9Ed2rLP9rU9$QT^ zZa-EY#U}T#b|y|-hBK)ElL5!I<3f`--4c$zi}TU(Dj_~oi+8r+xke!2!CS+4;WmCb zmtZU*uxkhbEkR=;P~C*kAfVVJq$orOmnf+s#+r%d9wKj)7!DBiF`{k?@E4LAD@fT! zlBh+OOHtFj`&i5#dd`6 zJrY?wV%Ryt%DZ@mf3aGAF-3Q=yYFHZ1Ck6~Y)@P)re3mEe2J<05^l>SA@e0V*Cq7e zC7Du)>hO{b=TfKi(kk_(iSA3SBbN$iFI`@{)R?){SqBM}U#4lejM{mb$8nj$e_3kf zGRNL!66VpE=qUFZp5bVC;HZA`C}ZiUKY6r~etA}KxvTbaS=;3a8^q?hTok@Mx^;P1 z?lDfuv7qdjR(p(QIp!TZX7xhS@ngf3E2P{j;+0oeny=vZToG|zVF+Boid`X$K-7g- zQYx-=H(n|4x-vO*rG5HJ@#>Xp=_^gFtCn{lq57+I9aqr@uk!h>QY~MV*}ck{e|1&q z)d|Jb#yqIdcJ=ba)yBoE**jMUXxC`?*HGoxcv`Pf^j*UiKpmlLB#CQcIoFztujN%= z8`fN_H($$eUF)B_HXw&)v)8&f*U6;UZ8u$K>%LAja$R)hx~{eBIGO7-I;gh%IIZEh z*LYm%IG&z5KD=^Vx_3OzxZWbVo@#|g+OIbZT+f=kekO9gI(a>XeuG|J~$y#}nu=%EC(@jR_ zP3*u;f%ThG2`IJjW{>1%MdQt>&YK-WH%q2(jzw=ar*G!5PB^GAefczb5>cIVXXRV%kAQnyv^XcveW*S(~Z5Sv#!&+M`ZES+lQxZMR$rS z?~JPN>@wZSao!o6y;Hk>r~eS&ZHa840oQ94xNen&sd^o_~|nd=3R|Na8~_Y zGlsj=_PZ#)yWC56$#?Hc((kql?-ncW&Pw5?{=2yocZb4v>$dNv)9&$=+@q?wC!@W` zX}QOxh9~Clv2NZY%(-`&d#|zTUbg1mK+nCJ(R-!ewE_!H9`jAQe5ZCljXyhT?>_hbRhv?^2?1wYu4?7zkuIhX^;dt2Ue^?lO zczN$(Bl8iC1QAd^qA@%|wLjvSd_=MINGkb=gZ`*Q_-L&5(O?6D*Z*kP^Qb=jC}aCk zf9_+AC68rgkGZsu$t;g)I}x_|$3&ZtMJbPWaUbVYJsxa+T-)P?hdEZA`BhL>fpO?~Khznk@)V^Rb zAS2cn44xNQ;TLAMUQp-0NGW;IEqhU}eKFblBHxY_&%d}9f6+vFiOYQ{RQZyw`6YVK zOFrjIs=!N;Nu=}OrK-Z06Oxy$jV}wkUS1x0**N_&d-df&`enlsk}5vusXwRaIG3`Y zbNJ3lme0j@&zbXI;g!C^+C}R1uP|({_$OXzTznmifsHpqZ zuF$KT#H+!a*R;j2(W+nbYF<;CUrUd^rWT^48?VK)uUR;+^QEsxnqD__zs?$YeP-r$ z_1fzc1Ib9sEw<;az`|Rat+%MTZ+m!eD`an{THbc_zMb@-V!^k~@wa)D zcfv*Q=qul0G{56Fz0>G?Cp-I&J&ckaytB=I*Cu&a)c9_+^WCnYcRADV2BYt4)9;G6 zQQorml=bhW4et%x-;?^@i!Z%r*?o_n|9(XH{#q`YrGI~>|9$nu`;^7^-P`ZWX&)p@ zKCst(Aa4D@RDw3SKj4Nw2yK3#%lVLA{GqSvLzU*kOwWhT(GR&b=tS&8YxYAS=cDC{ zkBm(pvAaJC41c7V`G~sqQAmqc6r4|q&pTA-CC2ly!Sm*+^SqVw;naCO<71r#?XUZ| zvHjz$?PHhsV_D?m?VXQpd7p>`pG4)K(A{W`^^@TEC)$NiXj`9nsh=u&pQfun9d7wl z+WTqdUoF8;`SDL9ho2gXK4VpWHlzMbZTgJj{LDT3nQ9ZA%zS2NUl2iJ4G`c8zV@zuZSBFNS>FtczmclGiFbTs8T^Jn^-W~? z8$;?F7UNstC`MiPEv5Zix9wZG_uJ&+xAvWH#d+VY@xL?4zgrGqLVe%q#=oO4eCJDi zr=otJDgNGB{e4x-_X+d&#u!Ez{C;`kd*k8v?4ln6(jPSHAE>4uJR?6SW`A(90Y~Ns z3H!%b`H$v?A91P{(N8YrPcp+#+XFw@CVvtw{S;09)J6Z9 zBm7xY0czWRruF~q_57?1|D4|Xc{ukk(vrW#Wq+|~|3Ym7BV&It%>RY8`Ii~WFKX^D zl&W9c&A-Tden~oi@pOUW*e`1bznBXD%B}cosPV75uD{ZU|LU9mt7`SHieb>n`n5{@ zYoh*FYsas`!C#kszZ#c+Wv6}()@ese7TCfWHdM*H2&|D7lQ z-LVSl`+jGP|Lza{-kA73oAZaO_zzk2AKRKgZ00{C`(SkLkFJeBIEQ}(Ie*hi|3*{) z&D;Gq<;dUCGk=S*u+q%m;st-Vl>eRI@b`%E?*_-;S^mGzto&WQ_jis2aIaX=+j%y$ zP&R-znCxi3e-NEXI?&RRVYIn<1RWZ3{^`vBML`=w+pFB@XxBKJJ>@~GcP7xT;z@L( z$%p1UrqKF`|4;M&uW9KlTGAOnny=$ zo&z|T9L!p65r$*oVl;tbj62Q4Yzz4qVXFXR7!_jttEHHTnF>hdVvN~bjtR|3Ftoi2 zj6zU}vA0xVq9fH9c2tH@)8!b(Y%Y*6DKPx$I*fi-iSd=FFzMz7jAXbGV_I&)1Zm9} zYPANV+M@xsDJ^DsrxnBEwP707dW_T2j)_MM7+$UsqpR%1cssi=>NcEs+l1-jnlXXK zUJPZ>f{`uuVXQg*m`H^U!!QnDR9-tK){+mzI71k2!!SlWFoN+cI5Ek?QH)43hA|r4 zm>JJFCL8x)xK0LOsh`Az`+XRCXbPju`Y{gi3}#I~i{Xq1Fq(}yjJseSqm42Fp>+Xc z2!=8K%pxWuTEa-%mNDkB6-;P7ilMRAFbc&w#zQXvqJa$zdq0j*mnJZ-)-6n8bQ{B8 z+rj7=yBME5g-M(DFi9C3FzxMQf`S7KRhz}gorf4(lmaf(sUWK+7i{dIfzIhXP}EZh z@c0Z+r(uHLVHTKLE&#=Og%zN{bgwb{V**Z?Sr*+CP_0R|OAAhmB8l+TTTwtXiU zUgiTV-5A*Da)ZvbaWKx90C}}uP}e&NdS`uLYHtb@3#UPPo)8E)XFniL2J!C z80iUtjF|;cl?sEy{0JD+EP=}EQb4=10($aR!DRIsC^D^sMt=;PNp65y-X>VpoB%CD zTcD#`1khb{Z0<7dv zh&7dquo|8OpnAkudAuBJV^?68l~OFLzY^OxUxjrZRAb|1HCUcrj&-Xm0B=lzO|g_% zaa}#Ot51au%r#&snMSNkq{dox&De-bgH5axRLoZFaIFp->utkw1A44B-H!DLJFrQu z5i1()#2Qz-u)?KEAY0RetumXjmYH5`IAy`o1y-!Gr621Uv0>L%2Cy9ZAl8su4Y*B1 z*lqtXR=7KYHI$5E{mm|HW@rp6U3O#5G!HgZHG%b4{&`S4HHo$F_^{DpKbEbY#;P4N zSXX2go1o5N`ISMezH=U%>8t_LTMJkTcM)r9j9`QIB`kGu87t3;Vr>lM~0rRhwAeKmw~<*ur`bx3Q`6Bvx$L#ddj8*g$*_Yu>H}Wc3-Wwf_Jcna^Sw z*+Z;KoP!(IQ*kkOE{?lF!)XiXxDdAvNcJ&sq97Az%&>4XA~r6oE5ucead4LPB3zhR zjH4@fI9ihua0K|cwX^`oDJ{imT17bbXc=yMRg4odBshb-0_QhNaf-ouK)P3jGYe$6 zP)iMt=9J?U(OR6HUWbdylsI-zJx)EX!tq(8L`f5luTkUl!_7F~vIdu?X>pS3R-CC@ zhYL=%;iySHE}GK_*wh`kWrq>RTI$3#=62(pl_p%gvj@lXnQ^-9UYwWPhhs~d0P$cy zu4~bT3*^{w6zL#NW^~}J-XUBhF^pprIdQ6nQJmVS24V|iI4)%zrWAK>G9%3 zo0B+W;S_GB-jB=nPvczPX26o2#f8OlIC^^!ryQThIW|JLwSq8?qg=#ktPz|$xP(h2 zG=Q)yiZisW;{0Q4xXk)GPRiQAnH6zd$g+u}%_VS({Vg27NDD+;lQ_0(7pGoJ;arR~ zE+OB?@q05k{p;vcw0|CetDXKXQi0rY)FJpFSG%X+;TjlN`hB) zRp5txQhaQ?63;EJ#%r5oc+X%BKDj8z>lI|9v<^RGRN}MVdVJNE3U4WD#D^Q2@N~Nx zuM9Wi9h4TlNn!vv9j$oHgbwfCY{PFCw&R5=1Kwck!23f+d?wq8mzH=e+AEIiQ-iwtN7uSHGC{DhF`8X0a{ZW@9}TqlgR{LRI-gX zHt*nPhLZT~(k{M=wuiS=rSU9V4?v&F;FUWEct`Oeeyxc@;5c#!nh2HPrqT%8(maB& zlTK(1nF0S6gODj=5u}X;1hbt@2!#s?w45S>qJm4XcN7z%6Fh=5+Y6{U0)k5=BqVI5 z1pb1Epg$}l_{z%(>2?W0;;A5*;!=Xoa)er6O_2A?2)6ke!t#Nfz!ED6jrux*)2$@L zWAy}HK?8x;ZUwx3O@!2(njp?J6S_n#gn&*|JMASmg> z1cz*dux4@+IMbtqRDm6Em$(Vr&Eo{&u!mq+o*?*XlY~r_k09-yBABQAgwW13L9G0< zk*E$3?2b7?G!i7Rb3+7m)9uy;u(UtFiYsFKO_YD zDMZS_Fd)lPiB>U<7}4hu8RK-KY9pUGT)-s8>R3dswScG%vWc=XCy*2s5k+lWqH(O4 zIJ3?pW|@3ql|n$YScHG}J4=c5eGxGoxY*HJPFx$65IJiVL=B^o=$2Ozx6Rc=;jD~k z*sCG>1+~P;mxky4k{F_^h_sppqN1mfXrFE(Mt9XjHeW+jYg&km*cgylZYA>b z+KBpUJ<-?QPE1c3h>~On(ZuT{2Gw0e>QFaP#To}}xn|;WWiOG{Wg#~D`iRbLD>2Ts z5qV7mMBSjB=v^En4l6u>SUODXGL8@fUMG>VHA<8fjS;O4ZenC$oX7}!h$@Pg7_&?O zF@ulDotPqOH~mCU;S4cZKT8zZ0z_kIjyRJI60_nEkvr!FEaPEfcw>=BXD<Ogrie7%9#P>+ z6FsFfS@5YnDPv?B$U7!d#MGOC$N5d8BldPD-}=0aFc=6zpMk4SNkXE~ zq$F8YCCRup14Mk)B*u=6q$-w^hMQ_hu|Wli8>u5{sr4j}R7FY}8%Q&Z0ASo|BFz*v zld=sOQkA`hWC?3YVTz7KuV^DFJM<*SL^~-fp944?BT1v`B)M%}r0q~QNqA@?8OqHh ze|s+}Gj1VC<9(#6-XIWC+DNqi0g__gPO={ilA>ipB({E-q;`*xTrnpp!E%u-v-5zy z&rR~pjg!**9+E`lC7E=Sq@c@3qOMPoa{bZ@LmXD1p=hT^c=}4oF~P#ArfzN zfuvgvlf3y6QmST&BsMRT=)y1%NJU8$!5T@{vQDyL2q4HARCNm9uYq+!z*DdyiM zDYc7$wq%#&X-<)nLwh9Aa++kMWk@qs2c&FQmQ*!$NV4ou$PQ-&psRDqN(YVXh~$yi zsQF}0C4;Q#WRl&JEb{hN0a?f`B%4Yv^V^HanJ||u&Eb*F6(!_Q2cJyy3do9tkZk9O z$Wc`pnUlW)s29q~u0shqQ7$F(4V7fQr;6;0SCiB18nUEbPB!(|k~K9^fSRo%%fx>;2;ZA>p*s7 zm|PVdAzSF9&AL5aqHKZgAi?masCZ`YnV5kGYti^7rt;28Yl3qO5-`ZxlG rKY=+7KOH~yz<;dzz(2F>eJV1Q6-janc;sX=Ou8cL8_qSP9oK!94K)*7IcDgh$a z{hm|oQ%Y#lOWM@>of|(VoFq;1%{kvY^UVKqe*eSYtJXZb(Na3!dcV~mgnb(B+N~CY zt$eZ4v?V#&Iz!(*6}umvWR;$12X z7y3-reEsps+6&DVVH1kTY*(>lD3+Y~&t)o>%FM-*+fpofti_UdO0iTqwOI1milu6M zvE-joEY&+?@B0WPxr|@?2qn2a0*po;gggj&5V8r` zgls}KA)Am*$R=bHvI*IQY(h36n~+U-{kQ*##vr7Hi~eEBQdyW}ktYkIEQBm1Sr}wt z=1vB}gMlXko8foeiZ>=to+itaSB|K(p7p#R;||6$pRe^>PMt@Y^Y%76N&qGyW! z^{Rh-cG;>I{-NmKtz7Xm?%Z0Wu8K}s6)n?LQ9n}^RdZBPI$sstkEr7GW2$I)TopT? zP(}G-Ra{@9iuR{e(Nv_0+GkaiGI_{_hb`}hC&e=Yy-S3ak2Z~cfve5z4; zsBp+ACfJ)=%@#8R3|Zew%1nK0P#(~?##>vHGGG69o-#|{nw7cwR#v9#Tg$EOBy44i zWRsNbl77da)M~FZ*bIshTO+mx{Q=`|4;XJfU{vz-2TaPj$@(iyiba2gNm2C&@{~OW z?2v3`<$3)HX1qfClV;^XeZQ=%*0+{h+eyl5yTKy#|4ewD!LBh+r`*5S)c+hj_cw*+ zHnGLMSQlHGu7CZvgLIjDmS{>ZrI%=0&R(1~tVCniC7LOkL?c6@F(~Z|Gqq^ki=nOj zYiKin4{he(K%4m=q0RhH&}RN;Xfyu{w3+`E+RVR&HuJwhn|T`A%>NE;=HEk``9Gk| z{1dd970_n>PiQl%pw0Z>&}RMvw3+9j&HO*mX8vDjGyfm7nGy-!j05Oh12|p*oNEEs zI-qO=;NA#$UIo0HfXdeZ-xi>Hn^Y+Iw@X~Fzyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m_W`Gpj7kQ0);+{h$}X@c8zP-xOR>Oj{!tKXYR62hQFLd;t~o3B zS$q|etu`SkO=ph%SJ^ge?lzvLBpyp)Wf;)kb9|-RO8f$<^EzleQqVEB%bwI2hXbS@IkARL4kZ1t9!a&a_ zK(Y}?H37X5z;O(4egU|?1j>#B?vsG$6yQArRGtNV=Yi_40RPuO%{M^cJD~1+I5)u` z=-h~$uczP#U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R}{5Hg=Z-x(76>(Kj z&Q}%jAml;FgOH1SxyYAI$R=bHvI*IQY(h36n~+V&CQRepOth8eoIk?(BfsnUBmD3j zrmeIt7{>V;YTHf%a$rpxVuB zU8BidySe`^+RfoyXgAuN8_AZ1c{aCcH-Us?Yp6FUO~DWTA=}zb=Y{kGZL0RH8G4{i zdHEpi#)P>(Rm~Y|H~3OeSS|v=E+EtmgnNL-pMXdbXub|aQ$XuYAl3`Cl^CGdIDn2d zK*9-hl>$93Ah{k$l>xowfMXNjd<}4I0m`-k?(Kl*b-?=uQ28d{dkd(32k`FzYIXsE z-9X)5Ah-_*y$gi*1C0j^g;L~w16MRK0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XB zkn;qn-B7!sb|XZea@2P$aCIA3w{dkFSGRGUFPo4}$R=bHvI*IQY(h36n~+V&Cd_#g zrqOOV>ID6p3q3m{L#ESi@TH)z1cAs$KywI)HUO<*AodB+)(FI# zfQ|@|_#EhJ26|e6WE4o90D4;i#~HwR7I2*h%Dw{JUjv?R0PlA|<@bQ^2cY@_;J*aa zTm}MHfVv-n;8h@W4G7-=8h-{NzW~j@0?`+Z_6S4`}glxhz+KqwZVK^R!6IM83B`XtFXfMHh7$?V47PXsvs3fX5{g^5m z9#_TACsa|sSQXcosG|KTRWuc;qW0N|s?n$;{qC}+CH0Fp1MgrS%@76-9=b;f2Lo#| zux|c2E={3k&RFi@M(bwWSvLWLZq=9-ZK%ztS~s(-1|jUzEv57AQ!P=a(bj$f*3HfG zm!xcKHw|~0cGG=xyJCGv{}>8B#>3OCrYBdfeA+HAS+RV@%I8)n4ci83Hx`>!X+M>* zc7tyPg{2gTxq!CyK)ej-C$?p8~B%f!Jq2+c6;i z1<>&&w420nYBz#wI+y?xU;<2l2`~XBzyz286JP>NfC(@GCcp%kK+Y4Ob~8{n5Q1wu zxVnw2+qk-otJ~OwY(h36n~+V&CS((`3E6~fLN+0rkWHBLCQPH*W$ z7HCiYgK9SgFj~wu!>u7V<7?Q>zTvbRO~;Wi^rdg;%_L>D-Cz;=e7IZ z`k#a6TJ!8iNG{gfw43+|qpjO#Qku@+^k!YVslS+KovGV3`I_W2#nLI;Fp*0sS$A^w z$g~@)ZHm%RVa#;e4Zal=mR2Bf8t94vJ?DUA8<1)TdgFlOJHYup;Q9e5y8yT^0iMf% z_X<$?BjCFVR9^%9H-MU-fxs_7-LF9KMH3X8mw@ogK;tVwWG&FV4v205S~mi*SAn)o zK>RhJV+)Yj26Sx)dR_;TZNfC(@GCcp%k z025#WOn}-AwHr>m;j|l0yD^yto(EiF$t9LtV#y_zIbUMQgOCRy4?_OGWB$KmHX)mk zO~@u>6Q;F5F7yArFY~9p#KBQKi^tDdoJH-XfD>*;G~EZ@&7fQUm}oc|%(@wY zFO!K`vs80xrVMdgraemOX57uXsU1$b>62};E#36^Otz9MkZwYqfvih6-HG(lO~bMM zGl!IJN)8U1bt53%w68T;bU9|Qh_K+Pc_a2TjN0t7z=LPvq{XF%gIAo2y!{3Q@Q z4z!*GVyA$%GeG<-&~Y9}dm3xQ$K7CS((` z3E6~fLN+0rkWI)YWD~LpbKZn$tQ!*-%8#Fg^0MI1VdT%jc{e-}WO1)2LKdwXRdmX# zXql#p`kAVznxl%+`KsuCL=~qWQ$@q$Dyehw@7O_jrv;;`f7zj=)-MdltWZknc!o4RGH8JU;{8Ux3PA0pE*xP;6cT{4WDFuKSZvlySfUX@t&n_Uj8%XU1diMd20N{KNaMb~2^?*AF zcs|N2l)RxlUPqJ(FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^xoF_o-hT09a8xFhS zup17$VH2_m*@SFDHX)mkO~@u>6S4`}gls}K;W%r;G}?{ej1tZ$p?XEvhpx{AbA7U? z-OPhdqKc*>Rn$JKimj_vQSuk4IBO)hIO~D>%|KNJKs~1rYSKU*V^KOQWS}RKEfYu0 zNWB?56=k*l#nn5En1SDYls0 z;#rq&?l$bEBpyZ5@FGoa=e z5cmS9`w|Eq2SO)-@F}423=laBG@l2eUjeOO1F>&_w(o%W_dv%FK;i<>bqVOX3?#1r zsULyftAOJ=;7kFon?PAF;4U%a^Em+T8lch%_)39lm$^{#uQ%s>IRQTa6JP>NfC(@G zCcp%k025#WOn?b60Vco%m;e)C0+epJtcc5sxU6WLEi2+BrFlu|oL^F!2O$qa9)z6m zzzGj*LN+0rkWI)YWD~LpbKZn$q#N@DTWU^&h6W7{8X7dLwJb_EGH2XmJl`iDQqmB{ zN5QB$I7@{wgLyZDhr*1K_R{}nb^j$NZ%XOLHpHZx97#964Q5;ISx7fu?@nc1y6L=- ze$q|Vo;5>Hy6N_rtkbOqA?){AO6S`xm?mUWN<30p6DMDP;$-cKW{a>1MZjR$pcyP1 z0spH&%_bo58c??d2yO#H+kx=wK;s)g0%1M$5;$37tOF3`0f z=s5r+-v?430KFdqju7B%09;|9>=VG<2zZ(RZv?3P9Pl**)h&QO3e=nc0NfC(@GCcp%k025#WOd#h8(7K^@L+ggt4Xqnq)|^eq zCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMk)$a6sRc3f+vE+6Ja8q2wAjl7I3^x z#-n`}i0(&|7XJE z2D`>QopS$PQ~z`DTx*`)2tma^8*04)fn!;P$2I~y{p{aQ2;Yn8M z3A?mNDPKHj??St@RNFgGf9@RZxwEa4AD#Qcs<{h`o>^5?WS5t$SiWNAb1Rg)xi<~T zqeHtlEfs^@8xw+Pk7e%O;A_ESIS&NC0zzK{;ctM(?|{hnK=Thk^a9X&35Z<=+O7cc zAAydmK;jzEbpz=68A$#Dq<#f@UzFk5tO1-(z*P#Axd8Wiz*7cz%YjNa;M)vTdjS7d zpvDUXDuB95AXo*2d_eeZps`vmlp;IjoUbb22VeqBfC(@GCcp%k025#WOn?b60Vco% zm;e)C0!)DJ4ObU&brDw=jkDE7ys$JcES>WUOYYi7~C$%jv(iqns&qTz8>?0iBM<%?BueTgdC zpHf9rkt%ARRq5yS`8l#^Suge5Hv{irCe5gtI0Zt=K)p$$+bq@881pFImT7;P-p#o4 zZUP3~r!nh(O*-$U&$r38cN5+v+gh%{yLsy+f7ZR5hP%wX>AtyLu|A}GJBsG*Ot+ez zT)Fb;zY06$<%7H%v(2J3ykW?A<_*3U9G2ZcWG~RX4~V`CwC)FD2Y|Nsf%peN$A>`T zW1#CG&~pe#9tKiJfZk65M+9(w4!D|uvKGJ{1w1DJZ!1uF8t}z{>T`g<4X9}c0&$@3 zTOim0ggSw60%*JlM7n_HZXntNwEhI|CYGdk!tO?V2 zH+h`)Aq!6XFjKqWiNF(KB1wo@^ll0@l|&Wwn}Mndpi@>w%QWnni5+uPQ956xnxkt; z^PnXa3q+J&z3JC*mSx+)S(K4mA&b*uGHQCmF z#G*8rAn9$~F|1QnK#k?2&ml7MoS6`Zja#244#f%MBp* zGtl-65dRhEc+mpK<|Ux(WuWI3Ah{Mutpj>D0FH9N=>}Yzfie%^-U@iUfVTputOR^j zK(!C>zYWw>1A(1DogWD90YWuExE5#(0Fn2A<~kr+53~k>*hfHH2#7ZT9brqMl=#HL zONlZ8Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaaiz^91PKaAgr!7I9_KI9pl7%S!XI z(mB7ZG!H@^gggj2<$+Tk*o16CHX)mkO~@u>6Xv`L(|9*jPN@E6gKqg_;^Aa4?Pg>~&1%h=vEIu~)m~2PX53jfwZmCAqciO$e#Bzy z-ZV*R`Yv`R>()(oBE5CfaBTm~A+4K+ZG)!WSZ!03l3m7(r`_Od!C?6mNE`*aJ_CA= z0m(0b)R#c-alp|EI8Otv7*KW&aJK=TcEB44D!&DM9YA#_;7$%4`ar3(Y43!VtF;E6DCPJ}F4Hym%1*=U~v(f-sR z-Dc1~myM2nlP?Sz7&rH>&rB&6Be<>Bo=oFr+!;4}a%0@|Y?@>%iNm;QY`l_n(ZIX`JB5Xntuva!|_R4EOY75Z24RBNd z&Pu>l1(f*!_uGJ{8u0D}D*b?O4^UkL_-lch01$W&sH+2l^*|^Hgg*irLqMbfXbuC> zPk`1&Al3x5MS%F{Ku0r>XaTySK+g#v*$Sji1HCc8aUO7f1>?r`HH{m=NfC(@GCcp%k025#WOd#h8(7556BCaXonxb*Griho5=H;YwemQ9#gggj& z5OTT$r#r9-*@SFDHX)mkO~@w9c@w5FZn(yN;;gZ!F(C{7nsn%lJP~jrWYM^p2a`k< zO+~7xeO47)SF57rFR*adNUCUA57ci4sw#j^8M~%o&rIx?ql(h`s_1@16{jClMZ@Fj zA7EZiA-?-S&t#G2oD3d&GjLC~Bpfv(>t+PP%&>M%zGl>jVQ=%a=VVj7fnQWCOvB#h zY0nw{<7M|bg+f@(aBPGk>sv`F(6DqY>{k|a#+&u7?fJ=l?IzZF=A`P)}TLN{Otkbtp|)sp8kMIIX78*^Hl}>08D@hFaajO1egF5U;<2l2`~XBzyz286JP>NfC6S4`}gls}KA)Am*$R=bHvI*IQr1p&cp}Jx zC&I)z5whss$bVp$W~3fbKHQ`sRFi^HQ*o9Gk-@~9(Q;9i-9@ljnwcjvtVlDWClvlZ zt(!jE#&XNP$+2~_`j&Omx6PJy{;rfXX_+R}WMN0slupO$Z1y0Ciy?_z4hd1j0=~V+4qN4m39d z(H5XJ3dBwTZLL83G|&+P66b)fHlU{+NXCKGw?JUI1K|fU?Vg`wHOs5%69G zDz5>)8$k8Xux|Xn(7NHo8z#U6m;e)C0!)AjFaajO1egF5U;<2l2`~XBkoyE^-EdtI z*A;PH(KuUI#0yLF!qPdvurv=s9)vsyIp=|M9@vCzLN+0rkWI)YWE1AR3Da0NTxUOV z*4fjVkOhBDdUSa_5pW`8(Yjf{DK{BU_*sAnKm9{gFnVbS0|yT+%c_Kfx644h$>#8x zA->&$0lp24#t~^VS$Q(uo6INP{8=A~W5$deZHSIpc~Ez5rM(~vwmw0qNXz?9yz$#~=qz86H6lF1Nl96-$)Am9Y*N`asY2(1UgWk6#& z5OD*|n}MhYXx$3Lyg*w85U&I}s(^$K=z1IIsRoiefs`NU-2*uG0nT>;*M6Yv0N{Qf z@O%JxKLjd227CvB>O+A4Fi>*@2z(0E9i3b#1wWh2tBEoJCcp%k025#WOn?b60Vco% zm;e)C0!)AjFaaiz^8_g0P`;sjL-~gC4KHubCS((`3E6~fLN+0rkWI)YWD~Lp*@SGu zan^)sthxQbp~vDg~WBK}QxX>!p7EX5bwRr5Qq2xpx((Ko~P{Z^l}K(H)qayy<9-_Zj$` z4!mC|1e_JKO(?Rym6QT~Yfv80x5isrlQLibcb+m!-}G6+-UJMIQwgHY z^Vk}&9W36A&d{5{=aX#>-YH5`Y_T3@gn3byUrSvvVzW$dc<1g9G7J=VeKw|k^lUSO8U<(k60^t)t zV=E9j4K&Aq=sBRZ4T!Y^ZE+y}Ezr>cBszhv1kiI4NOl3KZlJdZa9jnP*8tZIpzLSB z{R`mv74W_|1(MB6fbV6X`W3*x7N}VV1U3M58-d`fKxh*Xehp~cGNn+8Y@3qvB?bHd zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2~fP@(jqP`;?kmVwzP;Bm*&N#bAEAY z9)vsyc@T2e17|(33E6~fLN+0rkWI)Y%y|>05pTHEe&Q^(r#OMJ4>X8q5b;F7iI7F{ zrciT9R8hYfsHy-uWmU9H!=9PgF-H}p^HsVzx|cK$UQ)3zs=_!>$++|WvTRE@i!!rr zRBir@dCYFBwfCZR0~=?2SvUJ~W8H+kQ*14rux@sL-I;akrv74{b*8SwrT!dnRR2fS(K7JhRm%Sd@mR*6+omCXs!aHKA`n&AXW{u?F8a}pkoh^r~$fa zft~=6d=E&~0loEr<3qstG2l7~lpO-xhXKzK!22msc@*${22>vd{9gbyUjl*SK;20o zcnS!e0m5g2#`8erE1>ynAo>l^`W>vB*!Q$<1ebI$0Vco%m;e)C0!)AjFaajO1egF5 zU;<2l2{3`2CqV0l)(x#2S~s+AcxiJsA)Am*$R=bHvI*IQY(h36n~+V&CS((ivnEVq z-AJC;!?i69G}2*wF>B4p9JDd32k5smkOb~EUfKPDDV2E%R!3pe?i7GoLJ z+dS>>)4UmX=1uKz=8ZN7M}mOTH}__evf6I22>m}3_8aUP^K{Dndrke%!E>#7b|VBB z>nz=cnXb7oQ!UZXDYo{y`;?~j$4X4r8Tvz0@zBGQtkM&9X^~RCc+lR3c4?`$cb@*- zIofk)TPHs{_k~q+7ZyFUs;Fqd*qTIo^QPh0{+UCXH|6Dn=G|CqR;9gr1m+FC7c7

O}i`k9SFV-gx&zcZvu^P0g-or<{dzE7tp#Ji0uX1_5tyCfsXz6 z6-tQ%_vL&|0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WG;g@Jh--_uwrHHK zE#hURd0FY4Usjq2ArC?xgq-rgDGzKyHX)mkO~@u>6S4_&-h^q)8?LpVIBV@`PRN44 zCS5u+PXwF@Su}4r;3l)-J_VxvsX@xkpnonK9Rnv{NEui+LkTv^G(ETCV`HAAz>3 zK>QleaRW&F40Qbh^!y4WUz}PfrCyrK)eKC42`~XBzyz286JP>NfC(@GCcp%k025#W zOn?dGJOL^|-(>BN(7YLU=1q=<-Sn)PYAZPo^QQUj+N_&5-FF#wQxXp;Lz*|Wo)skz_9^vZUkJf0%e;3 z_iKP>3*g-bRBi`+uLIR@0RA_Dnzw+!J3!qIAh-(%?FPbofyRA63T&E*2xc6>2XT@khPKm*AUXR*V@M zhOBQTr9j^rln3;!@z&O)%-8>&r_9o~W@WCvm6iMTt>xBsk}};YVj5Li*Jv*_Dd#5ZuP`YV{S_uf)gQ=HhShFX;{}o}+XHVL z$EVuL-<+m2?JIA}x^~lfA^pIcsy%Cl9(dER&1ALfGR+iCrja4j7?kn{Z?|sDmc3@ zfif@Pt^hoh(+VYT)if?>U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WOd#h8(7K^@ zL+ggt4Xqnq+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMpt>D_s*dLJn3V^0;YLwr{Xp;l5PBa7e*iRo2t+;xnhyfeLqO|cAa(?3`xJ;D1v)+h632k9FMysef#h)@ zbrR@31vp}W^Bmx61IpR~cO3A13wS$#%1*$S0IDxSyzzHYyy3VTCcp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaaiz`vfT7aA^^j7IA6OI9po8t4i~#(mB7XG!H@^gggj26Xv`L(}*`*YCmz7+Ebj61%FMdbP`VloCsMIZ}K6NsN(cvs%Usz z6+53$MfqY?TwkJ!_NP?QRHTa9XI08MeR7U0TGmTxRTK=Q87SS{dH)_YoB|;-@NP1a zWzv5BA$AsMU+u0_Z3?u9sNO)S8Gh!?SgJPxgRa<^mFMrG-e~)?Kl8?Sd77>EW1G@+ z<@;+{S8p2bGV`YU=61#UkUrB!!AzTnr&~==u3Y)FU0$+c`HGd#tx!r1nr_!@OmL7| z-jQzKIn>Y_d@pz`R{;NyK+RPka1E%t0R(>rLcajvUxCIKZSZVf0-9e2qOSn0Yk}B0 zplt&X-w1TP3M4iGU9SN>TY%&?AhjLneI0OA0Zt#_dK)OK2HZOVk00>v0V-<%UoB7_ z0Q~O(HFZFs9;gf23Z>vjwwzzilOKQyFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zx;J!h=-$x1p?kv%oU;kpgls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)=)PZnIP&&B#? zS}QyeWWf_*;+zOsbZ-hZokSJ&n}Mndpi@>w%QWnni5+uPQ956xo};Tt^Pnab3!x4- z5@uPpJsb?7$w0ihs~$6iIWwY%mfx<~EE;p=29nS3bl^?jpqsH7c(ZRf>t=KY-UL3j z*&5d6D^1bk8}qC)bQ5N(X2MLeN>A9OMN0YNL3P7gDV9#zhPF~l)}5R^GVA6y!DhPt*Y4B)ng!yNS*cA( z)_i^EWNoL}B5XntFjzj;43>jH@DLC>41|vWjh_OMqd@a#K=c^U`UMdC5@31*E%IAUxCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO z1ah7LtsAZ@;<_TPD;j6(ig;mZURXNk7nbHh$b*mvA?G}B&I6l}O~@u>6S4`}glxi` zH(?s%C`LG*&FDNVKcJy6A+33;m?7_W+2i6G)IBx381wV+D+^=cn(6||hc!1`uK-3GgRsgX|psfms z`+$zO?S)dJ+RoJsOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3FJHhDmPqI#6?A1 zR5Z>O74d@7yr6W>FDT7}kOv_TLe6*KdZ^AUn4Hwx@oJICj zCS<{1lMN%Bl?dEQiZc5@IWk~I&)-!0*jny_qDG3`hpLB!o z1%+i7kk}1$?FD-F0m*lP)PA7%0N|(xoI$|#5l|Ka+zo&y40t~QDjNY`6Hpxi{GS6g z%|M_9sEY!@6F{gH2%iQTV?g8_(A)+@+kw_N5c?Kr>j2`NKt}>dTm-thfSzt3*#qq+ z^%J!lPP$MfZ7eU8)`SyZm8Yxs^)A$ zHX)mkO~@u>6S4`}gls}KA)Am*$R-?TO_)ZzG0TGM^tn!7qOigfK^8m_CeDeFMeRoZ z18X!RRfzJTCG|Tucit%&H4A5{kTRHdb63x%K=Wg!jK-}%`?EA}et+i8^H#G(vRRbX z`gW4C+HSB2{XY}-8|)hMbjtmEP5sZobFF!Hqos7db(Ym2gnc^h+NWB2uG(!SpG;Sp zS{;|OZr*fWNI&qVYR{UX2i}yI4>E5A%$xR|(qQw(qSRIxtoiyEnyh^xvqjj1B4Dvx z)hw24KTVRZDkO~659|4Yofb-Dw zLdkV_dd@H9$q&E;m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28%^R*Q;@TpvEgENQ zi+EjWUROHj*Olf$$b*mvA;&y$%mbT{O~@u>6S4`}glxi`H(?s{hHLF7&RTn#6SCm1 zNtZ56Wasl|{!I&ofTUuEi6QG-Nh#2`2IT>LYrM5JDf9Jz z=P9%Fty!6?Z)N3veQUY3ouo{+%FtUR<*=mRF(|d#D-AY-V#L;ntwDdl_}c@HRLJ**ZgaSf;{Zd3cgldcrO(Qpy((+Ply$E!FnU)1Ny>d+u!OTSZstux^61d!P1Poo)vo{jDVk3s!>2JQE&t+n@x}rTsi8i-c!RG6 ziKP*6H34N2!2LPkX$HJ4KxGv0odBv^0sm>BCI$q~0d;LaupJ1+f$+CLV+RoF1ey~- z^divO1;o06wjLn<6VQm^b0tBcQBM@ zpmcNR{d?4K3WOm8@8+LlqrfOB7KVGcWkIDGk&jcT$vgj?Z+b7Ux_9qp29y^w?2bPF zW^}!qfI;_b%!)SI#sJ@DY`hz{dxov59p25>SY_6|n})kgyy?EVU9mo-do_yY?M%0t zo?N-|>Awm)4ci8JHwK$YseLwc?*?BB4$Ef1>j5ga0zNNLT>w7?~4rr?f;z6L}BOnn1x*C9dvVs;J)#R8;_-vMO4pVb4tL zn4^l)`6|^MT}zq=EeU>#s#z-O?VG;+%d+j@EXqi_xfgfl)@x<$tC=#tEzq7!?*=~2 z@bhlQ(!1F=oOh%7I1(hF)%tdlvf6I22>m}3_8aUP^K{Dndrke%!E>#7b|Wko>nvS( znXV0znQHO1&#=|j+^;mCUUEMq8wZeB19Uloo>Cy`0#fUN-ZH@PD&X7%xLyOwwgB#JfM+}4eI2NL1Ms~G zRKErI-vMfN0D)aV-R}DfrQqKCxu$^$FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zoF_o>hT;vy8;Um+Z+LNYHX)mkO~@u>6S4`}gls}KA)Am*$R=bHj8{^b`yKIIqTX@cOt!Z({OD6%ptX#l7pt( z4I6FfPNfC(@GCcp%8p8&NR zE-d20A}%Z%XA6sXS!rHYI_H;_=0V7VkOv{BJaEbbn~+V&CS((`3E6~f!kjl@8tsM) z?I+GcdukK1;IB!M&dd`5Cqfpr8xFY1Y`9N>Xn*QX~5Q107{Rq8#XQ13jC8qz6cC1$w=J z<8{FK2H<)VD0>TVzXN!70N!0dY7pUF`_}>L;_5*NfC(@GCcp$} z-O#$Bbwlfh)(tOg&L(6NvI*IQY(h36n~+V&CS((`3E6~f!g1DwX{;L|3og^=GW|UI zDm)Q5^lBmvy~?6>GY>Y2Dw>K^QTwbawysu1$zNdNtdUgFvL2}43{+JBoicV!!=9Pg zF-H}p^HtIPh$>D$rizBg)jz<{oI-r}fx^ck&HpeAam$v7qh@5@jNROs)tdWaRfUzq zZpNW@lcQlb(Z~a~_N_COri!(_S=VmvHteP(9#V$XZmRYS8g^sVv>T@}bL|G-3ku8U zK&%;PYXRa>pyLFPXa%}X13fVyc@9Xm0ln>j;~T*F9pL&NDEk3$UjRIp0Pkg>@(SSl z5vaZj_^$yqH-Ny;K;17u@K+%8;!G$uF9D4&1CdvN=Cwd{9niV~h;0PgUIpTtfR5K@ z7D|aNGr6LH2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6ft)8m?S>1BxUh%|i^kc) zB3@IP*OborHKlnF@*w0v$Po`5@xUfz6S4`}gls}KA)7GgO_)Zzp>aavgzA-n#>qr7 zPWWr$*UX}JBmaRlnvp6*`OuOEYBz)K7mS*Pvs4%}7HyXp4bq1{YT8k$F--QascVetZq3ZSbJ z=&1sdJ|Oir&|3{Sb^*@afNL*MwhwT>3wZVe-UC48`+)BQp!!3=|1nT=5C|Ls>J9_J zBS7d=Abb>P{0xX31Dd}8qF(~7$AQ>MpzRb8KLd1}1rq0huCIWeuYu$@&~8%SQM(a2 zUsAyjzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%sNGPzp>{*;~=hy;q_3juN z^w~B-k@c;l6zE&S_?B<}j1OtBnUy?jjo3bjtz?sx=k={cS*>p;DXZ-Ui_rfw;c% z^^ZA4`@edNF{*Yi-6-Y;JgC3 zegw*{0`6;o=LX>Y8L0dP@cjx@zo@{oc?qa_83?=r)U5@A>wwS(AiNQ1d=-dn0-9d~ zqFaF0Z9r^0(DphIe*@@v6G*%TbiD)g>;RIxfYffFcQ4?m1)Kq;P;$Mea9INrU;<2l z2`~XBzyz286JP>NfC(@GCcp%k025#WIZuG@4ObU&brDw=jkDE7ys|W}ES>W!OY(273)K~E~7w~ zd3d_j^yJEwPycmVPI>tq-i<*i|IcstZmc$|(&99x^=|OB;IO<8xIO^NJ_Ot!1D=C` z_YhEd81Nkdsy_w%M}eBpfWR@J?h7FJB@j9egiivEr+~;Ap!qBiJrA^g1;oAv+P(qe z-vJ%p1BoAit_wiVB_MejNL>MXe*_#!zNfC(@GCcp%k025#WOn?b60Vco%m;k*SdN=fL=-tq};q}eggls}KA)Am*$R=bH zvI*IQY(h36n~+U7&YCcdcaw+f?c12C6E6 zPFWQ#)39eIcFa*l>3o%Hj;D>(H-Rv9AyBVE%H}02a*{Z&TcT-h=KI`61{lz@%OkJ$MFOb*=biE7o><5wufYkdy?+1Y6BfuE~ zTn#{37;t|Acp3q36Hplee4hi=&G2sgE%a^#7j-ZJCcp%k025#WOn?b60Vco%m;e)C z0!)AjFoB#WK<|bti@36gD~rb2$|7D?nwOQ%`DLYf5b_}8LC7f&obtdXWD~Lp*@SFD zHX)lZ=S`T#yWvXviL=t4-h?dpYtp0>JP~jrWYN1R;EbCQ&G&(LGw7B-CLT@(({4s) z)U4K=8PmPoRPE)oZZO5hs62qdGG_Q3+RPfWEl-)HZ_UbFeJd+tIqjx)IO}F~rrr3y zoMo$hW46-t;@elVZryYz(pxtT$M(-0(zXFcF811ie_pBt#&4EQ}j%~l}b1?nnh7fQj( z**RZOzz@I#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28wHslW3RHP-Eh85 zX0v??MEg^Ne49c4TsAuT4MxgjpxoTMKVvQyBe<>Bo=oK?Bjx6PUA?g=ecMUObbVTl zq#Ty?I|ijzlXVO>gJQ(ih^;|?!1&t(##;{#h-wXQ6n?Ud_AoLCp-T^f30wTMC=Dk34 zAJFx24OGVf|2d$h4G6RYb#Wm0EfDGe!ks{40?JL~B9$9Xx?uuLfC(@GCcp%k025#W zOn?b60Vco%m;e)C0=Z9s$_*D4aZwQ$6^*k+MZBOiFDRY!3rh1K88K|lNI%VvdhCMT}V~#3H=c}Uo5mlUiOcf1}tABurIfeM{#oBi1?vWs%^bNh4 zq^!0ZEJFX!g#8A)#yp*J|6Wu7bMRbip4|vf#yU$^W~Sq=eX1pVX|}E9!vdx0@Xv2& z-MqQmu$z*2NEzC^2@D!`W3gG4@}?1(H~3z#SS|yRD?sy)K=dlmdJTx(0NQ>A;=ceL zzXFLD3t-v21oXTNBwqniYk}T%fTIj>mIE#~P_`LxdjQW?!0QDnD*#_5P+bN1eL&6I zK%g3^+X)2yKxhvTt^pcrfk*&oeh-M&0j>1~g;Fe7!0U)I0Vco%m;e)C0!)AjFaajO z1egF5U;<2l2`~XBkn;p+-q5_Ec|-Gt<_#}w&L(6NvI*IQY(h36n~+V&CS((`3E6~f z!g1DwY0Mk5EGS-3ypSkf@I;UWPlSncB4p9Lk^jIZ%}6bxe3(i7+RdGJ3Pw%DSt>*Z z18;^GWrj6r(mr9?UG$ozWA5C*@+s8bRE+VhPzZfsjZkELD=7u~)}TC~Z;iLMCPf=t zW47ffv-GW5nX7N*TZ3-KX5h`R+D)HslWpxL`eA{seeHuv)7#&Fly&W<^FsQ8H&uJq z3_b9s%2$CV|Lx zpg9FZZvw5oK&<3Jd_D&dUjuYFA1ss-r4QzOK>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e)C0@QA}u!swbxUguPEiB@7rFmWHoL^U(2O$qa9)uk8z%dVOLN+0rkWI)Y zWD~LpbKZn$v>Pt8pEwKcsZGd&za~YxJe~+R5wfV=EZ~rvj0gNIz<{5A;RBPG?lf-( z?pc;q3kPqPfq64Vvu74)?#qGn3V^ym29#yIx}zLYag_ApMrez^6?X6Cf_t9j}9&0v>X^D-%Npg z(|%$E@(sQhM3!|xVgt~%5$JgpNNxgBuK~SV0EZWFRsgO_psWgT`vA||fVUc`+zI&n zK=mHLUjx+C0)YTf_Z|?e148vcI0!U;1VlnWa{~|!1FfF`u|}Y+35Z94j?aNaGtkun z^hAN=3CK68R?0VmD?69~6JP>NfC(@GCcp%k025#WOn?b60Vco%m_W`GpnOC5hVl*N z8_GAlzB!waO~@u>6S4`}gls}KA)Am*$R=bHvI)mo6Q+@G46@*AeXiCQ)K+*R$bu)r z#5obNDBt8mC{e}f$5heqxGHu&p^Eaws<^&H741){qNzv~wa=;)bovAxS+uN|`t_TE zcQBP^pm=lV{d-h#3WPBO_vV%l(=W!1_MJkCH{){ZO~9ZFHfCM0G1%@DZ$@Y8P0y(Z zZ6(!zQkptK7qc$jG~8wCP4~_1iuEC@K?u8IDV_iDbgSvfl`EgN%S%=)U$OGJ6-u|y zWVP#$Pr>7M*&;OY#y)uH%@n2OOQSVkf8u2AiDo=eC;|@4Da~Oy1EkIZz2^Z(JK&51 zu5W>|4#3?BcoKm3B2d`{__~4W9>D(-P?H1#*MYhe5WES5dVz4spWxUyfXEu4*$G5T zfmRm~TMx9A0r7I6!wn=h16>}VXDg8O0;vk1w-Ru?2{_;SQ=#N~=TA9bQNRzt1egF5 zU;<2l2`~XBzyz286JP>NfC(@GCcp%k0KFToEaJ){t}Gg7D~ot-X+6>%eI5FC^P9s)hrn3 zKVx)Tt-X;|9Wei9v^5|6{ncap9s9y86apTL85V}DZzZKb-x@NOZf5AuwPqiV<5Qwg&wH<8KcbZ#`hd?3!Vw-R#SabmOT0ldb$GNH?GQ z*G#g`(EXLEng}z=Dm`JB7AfV62kl*GmzHXK=jqR#qdj-Fb@HQgUsyGFVbL?Iii-Z~ z&eUJbv(D6?mai!>Q!Jgb4Q-{AtUEb-q|%K+Y5BX`rJLU-;G~mo@U;znXpllD| zt^qu?fHweCz6bc~fa-d{9|USX0s=NL<40x_Uy7B%<>4w8@m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzyz2;?h~MNL+OUn4W%1OH@vVpn~+V&CS((`3E6~f zLN+0rkWI)YWD~Lp$5|7mk!~beaNdoHtMsX>@I;UWPXyzHIT5lb-4t-Z&4`Bkz`7Z9 z%O4X7CxcNpgJqli;pCq?NAiqx&pcrm1Jh>ETj(Qp27lSnc8z{t;!W*v;?47LdnB7h z>6>~pNm*?-ScLwc3HuFpjd?od{=KID=is^4Ji8J8i*=T+!c5mxn5h=`Pk*vig&$Iy zc6utZF5Yw}(u+3@$M(-0QoJcIH{CAT7;Gk`)+yb-b13r$-wPH?67XIJDpP>(CQ#iA z_)8vwW#a$>Yk)c@5G(~kE+D)fXed_d3J zK(ZQ0?F4%LfMYM<+y}Vc1M}fu@K%^CDJ`F@;K6S4`}gls}KA)Am*$R=bHvI)mo6Q)sageVmp;q7d2_d6Hzo0qGNgG^>lrlc z#*8UH-5U*=54*wlg2mznf}4Sm2MBKk8ofZI0%)!TqE$ev4~V@Dv{eJ~oj`{lNbCW+ zYJi?vAQ=Es?*YAafa3$e`61x?7$`dkxDNrI!+`e)Q28m~I|@{P2KbKwHD3UMFM+z_ zK=33GIt7H!0F7s1-bBvRyb)a1!33B96JP>NfC(@GCcp%k025#WOn?b60Vco%a-IOq z8?G(l+9Iwk8fR;Zcui?uQ#$9@l;%OmgOCRyM?7%E1DlXd$R=bHvI*IQY{Hy3VH)#> zYwah_T6>xkvf!^tm(IWw0VhHh%^Ud-Y|@O>BFcxE)UVy#d8c61M4Y94@b8Ibp<|<8}o%tr~+D+$$^aF3I_N*Ct;7!A} zLE4SQW>s2Fj6l1=_kzOG4n*QW^S3~>18D68VhNz_A`tHaI=X>G576}!(31p`*MU?D z=)DOzeg&K_&V^$05>WOs;C==0tOdO5fXWSkZzE9sD&XG))Vu}+wg7e8fZ%o@^g0lJ z1895`h`a?fzXL>f0Ij>`7D}<*b926+fFFPfFaajO1egF5U;<2l2`~XBzyz286JP>N zfC(@GYB$twsNGPzp?1TIo3jbogls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)kmC=0IB z=Q@28g%zF%vfzm@aZZFRYBviwrV^C_fR~l>v#fYsDTZ8_9@wW$zw;mXYdGlxePnzNCXn)eIJczAilad3$z7*_I6nnkM}e}>0QWJ#^9A7j5~w^5_)Y@VrvU#Mpyn(PI1ki) z1q8naLf-)4?|{bdfyfU)^93M!3240x#I68sKLYWqK*u$hH;EfGZ#eLV2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?dGJ^`9HTwBDoMO<4n&ej(3y3)L^bk46U&4Z8!ArC^1 zdEl4_HX)mkO~@u>6S4`}ggI}*H0BN0+E1Ld_B1DC!C#Xuox~FXCqfp@n|#b7I^88)g8*RU2v)poUCMm1!28+=D zGhx5Mt}#!i+`re<{~SEmnrAm!O6OZ=Sq(zir{k`DswJM9YwK?M8>MN(yPK@m8Tvz0 z@zBGQtkM&9X^~RCc+lR3c4?`$cb@*-Iofk)TPHs{_k~q+7ZyFUs;I~=FIlmC#meVa zC=GX+deeP#yJCGvf4!o;e!A842M4(~0`5&i#US_Qx1DzVFHONfC(@GCcp%k025#WOn?b60Vco%a-IO? z8_GA7Zz$hTzTqX#*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djeL_Q3$E7ZYJD@c z6`lyP;E6DCPJ}GVH-(x{qKf*>Kvf0MDXXGo8urY@jyb9*ov+f*(cPqZaFdFKP=y$% z-Q0P9S++%-MH!hlqdt9Rx#rp6yWhIuQ;osMn{;pb*U!-ZVX}K?`RzaPBB7ZqW}8rC zeJd#i`qpr3)Xm68-pm-qkvIE>({4s*b&pK0gV)8X7 zW{Ra#wxO+*l65C%53Ake>o1(Fz0eGqMkvCXX4h@XjW&F!d@=NuCQV<70IAP`-e$n@ zCEz>`xK09PrvUdEz;hPxo(C$w0(@Ts)!zXA?|_=`fxr(y-31_c2?$*V!dHOCAA!hK zp!pgQy#ciT48(o`+I|J%FFp*t<|QEUGSKx3(6bgut^-mVfZmOO!won$KU^reJP+r5 zF#$gS6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e)C0#t6esECV-xTt8HEh^$=rFmKD zoL^R&2O$qa9)z6oz$p)GLN+0rkWI)YWD~LpbKZn$lp8LxpE!%`sZ7X%za}L*!4m-| zLKc;q0?xP@(R`mf-H!a*{ftpka59*7GwLSFpzmJzn|w29|ET^`e^bmB8q-_hJv%qk zbw|gd^lc|8qv_n#=Ek{kY$*-MQ&bq<3x_j_sd0q;u2lGg;HvG#N`b z_*M{DwgIl~K-uen`whVJCg6PwsC)NfC(@GCcp%k025#WOn?b60Vco%a-IOC8%j5nZYbSQ zy5Tj=*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djdWv@1y|{FmA-+x3Qq)C@I;t6 zCqfpb8xFR~Y^YCxXn$&Gn`GJO$T#`In1OIJrVa{}7icI$r!g9j{!WFP8Dl8i?8%LA zpD9QHDIg(V~r3YV671&L<}`Ztx-b^SWA#vB2)|*Yt&c+)EYHn z#QJ@n19l;Vl1kciyU&^UIN>B|l4s6&`p^4uUa1Fx^g}@AVPN19AiD|3JqqNv07F{= zTMJ-s1sq<$xgBu%0QU~S;|E%H0p0-MYy0PVoj>@`6<pJY2g&lJg(KJsHgEuSU_^pbFEKvRd#^u!GyRQ+3 z;oD;M{z$`R$|d0_qqJ@ot3HfztZ#RyZ!yx>AB#u-POY2k$I-f}(3l(Vo`1Hq-TNuK=Na zK=@T4x*zC#4T!xCbRPubhk)M0K;lgxc?3wk1*DGwnYV#~cYy2(Aonhie-9XXAFzE0 z*wcWcA8=*>*J;2#0C)z0mMq}?0`TPk|5+fA2ZHB-_90j|p~ipVf(9nQ1egF5U;<2l z2`~XBzyz286JP>NfC(@GCcp$Lo&c>IS~s+AXx-4d;Wf?Kgls}KA)Am*$R=bHvI*IQ zY(h36n~+U7!J4p$byF<~F4N~SeI0!jo(PiQi7`a+0?3fO!ZjYYSlkW@y1kbF~~P)VvSLrr){g`+1j>Ro~vyoc|wUd zf5C^+SxoY3Z97F?ZPlBE_MZ{^^;VTt8l~aA#^LASx#lXX0mEa=v$Z)gGt@aU(@nv9 z|HTq{6XH$wkYlQOruNWuJap3(vu=@9w^(joQm}WSRkuvtTcVPDwIH#q1w!`$ z;roH;13>3`Aod{8{SXj;80dWjNNfUPJKzb{Xc?=kM9LPQaWVT!1prX4+DV+5R3xt2Y^r~5bgq^F`)B}f2r5Sx+&g> ziZ80*2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0u*n!w1`WKxU^`3EiK|zrFm89 zieFWl2O$qa9)uk7z#$K8LN+0rkWI)YWD~LpE8c`f#2YTPpFB(LDNaa&za~|>DxL^9 z5y~ju%;$)klE?eZ$9SJLLU;a#pW$-%=Hfle%j)6a?MiTOO0sM2P;D7gS#FEfzeDk6 z;)*wag;=Aaa3+O%u#NP&b-vJUQfaJSC>OCO+K9KnU82AXtehlP30rIDS zp)-JO5U^(f#}|Mz2e{4x?mXZ*2eb?U-o`peHa5V&4hYzRU=z^p074spuoH+j1D!4) zwh`!d1M$s3ucxkFm)KUvH4RLF2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgJOPR~ z6mKZrP`sgd!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g@kTERuGHsBeL-b~ zCxRq+B21nWp^V~94P+8U9KTf&kp+s_u}BfkOB8W-sUrGrQ$$yTBHHd$DCeA)b0pET z;evNFEa05KgPAlJOE;I_zeWwGR_IIcZhnYOQ?I%+=4-goyP0s_O;E4-HPx~@*k=5^ zoA_gOmcCQ)Zk~TKRCezsa)p66gXgx&<{xVVb7Xa3&J45hPb*jc`EQeQB3lZ)8?$Aa zJouEMeK`R9P4J%Ftpuy+8C5a4_n zaD@SP1n@+GmIHvd6YzBb{umH=0|<5l?L9y!4up>a(O#hQI1oz!-6w%~66oy%5-A}0 zA&^P~>3(=OnGC%f4!mIkOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p-fZh#P7I9?} zR~AjMl|{U+G_NaN@#{+SAml;FgOFn$IOc&($R=bHvI*IQY(h3+#hb8*cf*zTlV_zp zy$MP1*Q7~j* z48Hgixoguy@v?h29jB|zvoyV?26~NknyFv1ps?h|^~Y`;op&SP-9(P)%~tKNO;i7x zWD*v=Jb3-3`}Ek?yaW=j1YM@H7D}4#2ws@Hqj0GZ1hA!Hqz>8whO%!X6;H4d`qEVy!^87l>~MdVN4*2axmw zsa-%i0A$*Lfgq559>}%-q+XZr_({bt<;f4g1egF5U;<2l2`~XBzyz286JP>NfC(@G zCcp%k0L2@MHxzFu-cY>Z#m(7-Y(h36n~+V&CS((`3E6~fLN+0rkWDzjny`p?qmu+z z>T{*OfyxR`1WE8jm^>#!8O57g4!IfCfFGDQ1-F`U5phZwcQXoOrWChk+10F?!FkuJbb>jfc?IC^*YD>^SGda2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6 zfr=+U>xS!!xUPuniYC~)B3@ORSCy{#Ri$|l@*w0v$RQ6L^1vo!6S4`}gls}KA)Bz` zO<2Ub;X3=tv(BE@ge3TD(xa>9iGUNKjMfe3+mtrjr&jdcHnPW|`5?N2(Ki?W^M{SW zsS%*Dl?hmu<#Jy4iI#>1J$Z-PoGu zS(tL0xdwW6=?SYq3uA}2Sj%Ooqiy;3+N62@iw402qc~d zlI=jM14xH}%*(()7|2F|TolM30ERjN+d;s72yh$*oNofIBY^uYNH?Bilx{@D7gX>A zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GN;i~lDBV!Hp>)H`nzISngls}KA)Am* z$R=bHvI*IQY(h36n{a|PVG-#@NP?^Mxk|r^x(ZJON$^CNJSRdKrJFj)B#P*2P(<6E zirBnb5smji#960PM9&7GVpJY2g&lJg(KJsHgEuSU_^pbFEKvRdM&{Jx zyRW%uzRTibF3QDrN4Y#4r6loY{B)Gns`_G7gz@4pJ3jYm-AvR`H?H1!me%b*mAg9P zTgqam(kM%B96rYok{6x#=drB_v0jsu=YZ4@kZ$}b6dM~b zunx%Dfm{=icK|~h0NX=={b9iI2;kfVxE=-ETL8~ipye^Z`#9ix;-~dG|C2xEiUuaY z1egF5U;<2l2`~XBzyz286JP>NfC(@GCcp$Lo&dEQE-d20A}%bNU<-?QNoihEy5g6V z=0V7VkOv_rJaEDTn~+V&CS((`3E6~f!iqOx5$%Qx?I+JddukJs;IB!MPR|noCqfyu z8|fcdqZzG2R0A#PV(q5je(ji9ILm~ggn2iRV8-=UPwhs&!CY;swiqsKjq*J0-&OK# zZCfqR)wYs+y|x{nVK;488+NlAA4X>}$>+CI4%m(X_O}7YJAm^9;CdHuzXy2U2UX953iAn+Uz z+yk`l1wtNfC(@GCcp%k z025#WOn~wYmltt)5tkQDu;oR(v@|a*UGYmx^C09w$b*oR9ysZNO~@u>6S4`}gls}K zVa1!Uh>~3D5ph4i-;!zPJ}YbH#HDS6mk4kMMM@TV#gvyG%r!a*`Y4T@;HQ=y=9UeJ+5&j#JFesl31Or{%PgP zKU<}xEACja@~#zfb8~@sW3rg#o-?BmZ}7b!u|$E`0ie4Rh<5?KF(B~NfC(@GCcp$Lo&d!giZ>K*DBe)K;T6u=gls}KA)Am*$R=bHvI*IQY(h36 zn~+U7!J4p$cvFQd<#VMzSL#z);fWv#9Z!Twb0U;cys1}R5=C@u1X{NN{gNVjuEU;L z*fB>DP4g7GIhvPL2QO)jKt#pen_&-UdATK=#U)ucSGQ~GR8PiS##@d0*C^ghIPqrp z)x?{znRpX-{@l`c65`Fv_kLY=@uuT+m3fxt*VMqTu}(AfOBNKC+_?VOjiVE9%$8|# zThLH!x=pctkXW3m#L^5TTtIRoka7d*%|ONj3~U3kEkLdn$a{gI?SSoRz`hf3JOenN z1zfuU_j7<}574p~@V)@}UIhFv0fAS5;69-JRUot<2)_nIUk5r50NP(_&Orqs!wTL%5Vkh>l`)BI27nc73s)fqHX%(_KZ-D0_UNx|NQR^2jnZ=Lqs zIqGw7G*7){?%k{AE^N4cRYSwy#?}mGifcEK!+U0psNF=i6wJF3&~763%jVq}<;edj zuHE2!L19S(nLc131!O-2a%mvn4-91h+Zn+ADd6}FaDEQBz69J~0iLgcmTv&>w}9_E zz<=*ApxCShg7*RK_XD8^fbe=C`XJEx5DNfC(@GCcp%k025#WOrYWkP`jaa zL+ysz4YeCy+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35#eqMoDm;KG*5%DXj2B z;M}XpH14O2+6@QXls4R_R`lIg;M)}ZbNSflH#I_Ef^u{1{)|{7MsZuMKKYuI8%2G& zaa6a}>hrJZO@F}4>xF8uR9l2BXOPanXfeu|n^Ro5!S{l`vJ1!ufT1?Pwg<591spE`&KCjKOMv?oz_Sl%c@^;P z2Yjyq{?~!PK_GYtXg>^u-UPx&faqI5=P@AmHqiYJ5I+I*z6&JY1Cs9psSkkkM?mId zVBixVdkV;%0rH;$L!SY*F93TE%8lbJl^agFVFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wl}~`m4Hp%0Q4tpvO|V5pyr48MC|&UjO7kG(LCAxU^Bp+fflbIJWD~Lp*@SFD zHetn^u!wR)+l00WwJQT{lgVV8@YlqzSw`ii4l0Qvx*8PGcBdjXuU16kJmuOY`ghDtEniAX0Yo=1RkE8dG6; zWb-CyEEH;ta^G#EFmLd^V6l7yIKBm(-vO?B{}qzoDskk1>6VzwO;4x{MU+K#*-g_2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?b60h%{7Z)o1oyrFr+3!AeE*@SFDHX)mkO~@u>6S4`}gls}KA)9c5HDM9+ zrdkpdFDPEYHkOWVJ$#Wu<(Y%phlR&MRqll(?iWt0E5yx*;L}Y;?b}Ujv^Abgz zU8>N|(afY8m`N9#HwE`=$4taoCPWG5%~*|rS$+k*X4xob)fCleM!S>R%~c29TvfYS zJto?X>-B%Nv~K#P+!bj%UUu!K|5WjTH?6zYjXdzCai6hJv8k3LUR~kN5hmW?i@{+z z2zU+wEr$W`n}F{K;C~AU90P)H1MTksp%Xy(T_E}%(D^NfC(@GCQ$JN=-qH-5my#*Wzhs% zS;PxV^TN^%csB+A)O3}y&BMXlm7v}jN2{^?u;r8 z6wvuJ5ZeiKKLf;{1$uV_iRXai9w4Nfc6dT8`?LtZ)o4}3g>J>HX)mkO~@u>6S4`}gls}KA)Am* z$R?a%O<2Uf(PLpO=Y4RwKD`y52$G<;A|}m=P)7TvhLdhepYkJ#o(=G6pxqSSFqfu4 zzA5ob}q8c{E%$u>9dlUTNmzKzLH_KhNrl#rUnP$BZ*6F6Ec{fcl>lRsci{<7e1$!4- zb<5Pfb=q_1sL#F8JoT2jcdweeu;KPq4GmUl>54m6th{T59J#{Wo56EiW%G}<*UReb zXPAwDTDkJie|zDe_cHUwB)9F=k7Di(z8EZ)kATp}K=>0NdJ5<~1H?WBx<3Qrp98&L z0*SAH6V7UKeY-nTr~j025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5sCWW2Z@9LIYm2zHXo9UR;^n1zdFhH@UYZ9X4?-S$Ywc-HVDQ7FF(>$I^4FBa4>u#rXx`MTGKnHOHUh2N zfPP64J=bB+EbN%0h^Bc8Zgy8jyNNz`v!&-dXg42x{&?B7n~u{}=2@CuQvd{ z`NP1_n}972*pC8^Uch-Aa3uiuNx+i?TKWKQ3h;dh_|rh39|&fE_R~OU00?6amoAzG zI==v7IiUM25YGd>=b+srhN#_e;0+UC0!)AjFaajO1egF5U;<2l2`~XBzyz286R3Ow z)NZKVP`jaaL+yqaH)j*F3E6~fLN+0rkWI)YWD~Lp*@SFDHsJ(o!Xny@?pjuMQ*q#w z6xt!QL)ak`aAsH;wVPVFB#LNWqKLCg714K_BDxwB(RQaIHm_Dh<2~?j)cyDH^j4?-eX2JTPQ7WnntG#7!qHhw^7+X(Q{>fFy-8^Q z8L?k)RavD`8s2Lheh!{%uCf|nz?f%iGRzEBhM8`Pe|NK`@9=!N>%e<2lwG|U%oJB| zB8T_P8d1GzZZ4R3W426_2g8O@OuWGtgT`|2d}uanf#iKa>V6>o0FYS^3_J*A9|Cd@ z1NldQp-q5oGhp`sj%|Rm1#qN zfC(@GCcp%k025#WRByPrh>MH3xM+ecF5-oyd12{_Us#$4ArC?xgq-uhIS*_?HX)mk zO~@u>6S4^_-h@Te8za>TsuNTvCZIaOk0c2L&&?lvZkAEK;fR~k#{1MlODb?~3jVo# zY!sXtp({bV8Og9&uKF?4%6nU;{y{o76VAEWbv5VaylYc#=O%S{zGZOhujH<$4!v1+ z=cegQap$Jz#Oc|ib8ZBjo5)MLLg%JhZa$@gtsHjsY@7&-yil7PJraHIg|hkz>$xcdQ52531Acn1LAAmGmefiHky4ro6M zgz`Z691tA>Ivallzs3f1uLI(CptlJ~IDq5^Ams$o%|ONl3~U6lZXmb$SM|EQ=T}_L zzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wr7pmRg#hRzM08#*_ z6S4`}gls}KA)Am*$R=bHvI!?x6Bco91V`3zWDUJ5x*b2*?O=2hGrBT5H+39wQ}TG9 zdeJd!jnEC>7OVFQr{7$>r(7D2Qj&Kw3aw_os?3OMzo8&TrQXa}J)P=nzv2JuYlMEB zZ}mbITBwLDkbO7ivE)^uSzMV@JvFjht< z@7HN}^m3c}O4au;U~9luuRUP+{sF^<2MltR_JC17IaPawQ8sC>Fv^PdK$Tqb#GCVz zZ^||CCcE`lmc|V18~Y0%m)*X((!`s_R9GI-zG-tCFLi1JvW?VTD&JIFs^zxK733Ry zFNiFU0r|&)p(g;_cEIif96JD~A8_pg+yTJT2DAhL@AH7K9q@Mmfe;XU8E6j!p$HI; z0?`9NXD1Nr0=i>B{0*SD8%Xp3$vBWY3Z#31%yD2K0c1}Cxg?P91BOz7?IXbcG2|P^ zCzNkQ#TQoa126$5zyz286JP>NfC(@GCcp%k025#WOn?b60m?UAUc}`^TwXN6mKX85 z(!8#8#jh*PgOCRy4?>Q4;Ft$CA)Am*$R=bHvI*IQ6>q{K@=Z18YH+Rw1uP0!6tE_l zMN~%lMuJeHh}JoZXqu;p!J8Fv{8mLo7ARuJB1JSWQN-D$3I!ca(5VrdR}Y&vm*1%! zGZ$x>(3ddu=2~Q$A0o0qM!9y4X0(5y{_j(}nF%$f+A=}3o3R;sW6S)?(%kWHa@WV+ z@5-*-^q(p|^rm&!x{-(8M79)YH+qXvZvB5pq21tnL18%!I0gXcAmGXZ?k@mO4rnAm#+Rn}N6s=-mh;+(2?Okn#ZOZ9t|4 z7-$8uULdy}$oqhy9f0i_!2T@Y*bO+J16+Fm_uhZ2*Lhy}H?C-40!)AjFaajO1egF5 zU;<2l2`~XBzyz286JP>NpyCNoyP z6S4`}gcGa@i)c57Nw(OW4hSZig}ZstQJQAFcCis)abQ$)`OpkpJ@x((=; zuT*7Moq48NFNAfv zscGI#Q_Q+WR^4K`c}cjOqZvf$LAld_T#(~&Tpt~1{9|w99K;k5jOaiGsAe{m-9|8kuAlnb*GC=+`Ff;(z zJ_GEZ1CB2N=U0I1Yry>t;Q1D4`3~^jdka3_+FR;%{`+pJ_@V-S04Bf$m;e)C0!)Aj zFaajO1egF5U;<2l2`~XBzyzq?aB&eA7jbdX1Y2Ch3rq9D(iOk3G!H@^gggj2=Yew` z*o16CHX)mkO~@u>6IQ$li>NnxdJ6Ou=qX^)I!^?i2$ShVD5H8)!znkVPxz5kHEGyT zx%dud(p;?FTz>x=MVwk8N-%G(?#h(n(-b_Xc0|3VR-HgI-nYL~`R4j@9D5VgYm!a1 z{1IOPD12GTKy$y)B0KKh1!V4s~11TSn z-T`F%z`!mb8vt@`Kt2czJrCGk0PHUUj+X%ED}ZYs;C>bG><3z21H7*TzJq}O5D+*F z1m6VOk3hZ&y+!#(aAgM*U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0+eqk-%!4x zd_(z$*EnYrvI*IQY(h36n~+V&CS((`3E6~fLN?(9Yr-P(O%(^la8L}r6M83;$vfe% ziC?pf@=ZNQ-IO}!r%voxv_=R;h>NwG%kM8Qw}`X2B=cs>4VXJr$;LE%Tddw6^EWMc z3u-r4oq4nSYTC`%%)AL6y~Pso{#x#Gq&v&5-E^F;GSAZNni^H|nP%#jEEvosH?BW+ zpNg&n-bfxvR?qX9FRW?4CMjaw}AaSz;W-dq1dbiT=xO)`vK1bK+Af-`yk+Z2=G4)1Reo` zn}GI5fzTEpycLK(26R6D>v~=6iCNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0@QA}u!swbxUgt~EiB@trFm)TieFlq2O$qa9)z6qz)25mLN+0rkWI)YWD~Lp zE8c`fv>P3#eMrK{b^5d}cp~sbm`o008MT{QxFm{bUZRMzOBK;~nPWKe-pyd9xOWpdyl2*k-p!!5VBC$_VwQXC zhEaGo_+oHawgWLA(7gkQ`+?qFKq3Gn+kjLMNIwr`+JS)%AR7X5F9Z27Fcbl7`vLoF zfa7(*c@S_N0^ElI&znHY5y1Nv;5!ER-v$Ej0KpSL`@2BsJs|u(5d8q?{0NAB40L}2 z#7_aeXW-o=KBafVaW_nW2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!(7T~`L+^&( z4ZRy)+MG?uCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35$3)R89<3PNYs9$T>eaX7tQ%9cWnwuuyDH<{#0P(E=?nk6-1WvQxw1PqO=pTbH$5j#&mOIF zGfi$gG#cjyUkv_A7D#*nBy&LOERfCvnRCFv5Rh&Bclb3nAioY6vIDjU0Q-8t@gU%Q z2yi_NxE}#Ln}C)_0q+*Tw-xX|1_T}lf=>YLPXeK*fbi2mbSKdH3=n%3=-v&)p96aL z0ExXo@&zFEB9MOR-|KamSN@%s5oH2QfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Od z3DCLWsv@o`;;NzvwyKDil;$O+D}G679)vsyc@T2K11CJN3E6~fLN+0rkWI)YtauX^ zac+bpIPRvJi}a~n@I;UVPlT#Tb0U<{xvArPo04by)Wb_!BMig0#p?aS(Ki?GDVK($ zjMBSVtQs-X#`<=L`WEzVCY*Oup;)VHb<>y% z%OiR>ZEmC4s{Q?G`1@ALBrMb9$p0xEbz_qIp3-0XM7>3i4+n)MtST%KAQJ@!4glFs zAlC)tW5CcGfbC7degts51vrlZuD1dAJAmf|(DE+eeGl-x5BNU-0v`dvkAe12fY2!* zd6S4^>SQ8e}Zn)5X@+`EcHX#ZAniT2uJP~jrlu^5pIO3+{ z@jf+T^Xg&e=JGqWV`kwj6S@+{-CWVLsa5@$k@TP+{AjKEXRm19)T$4e#^P41{#pGv z-|B@rN2b~$WJ%lVL9#MS{ z1GWZi_1Xi5?;kK+c)%c6X%86XlT)=<7-f_83ZtxO4^+ulm2dun7w9Y|`TTZ@yxOWa z3GF{4_Uo-Gt29c(dyT`-!E?=3Rs&QP^K8v(nW0)O(@oh8zp*r)fPCZJ>qZ`W)7*TSe502eZ@E;yF(Hgd#}(uod@qPBP9Wb547mW?CcyqE;Mf8 zAh{n%y#}OT2QmkNfkQy{Fpzr_$R7cQ-U4jBfc-e+8%Kijjo``-Ccp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaah|@dPN}aCs4z7jb#f1Y2IjOH1?8(iOk7G!H@^gggj2>4B3T z*o16CHX)mkO~@u>6IQ$li^w+yNpQ73SL++8t?)#U1W$y?b0U;czL^i9L=lbmD58Iz zP7ysDfR2qo>o%ZY!mjJEXBKvpEU74y^l#1=gEy}cy7M>8(YoBfxp>d=vbs2UyAu4H zk_?+WRAZ(*=E`FA?~I9FGs2&(r+HJddGl9I&8dc~qslzh@{cs*Mv_-wFmKLp%Vpl! zPW;Bwyz^GM>(j>YxS2PR+`*B}o7P7P%p1MMDEIw;qcCsqyT^ zfahbNL?K=qn)nH4yy<==>IleFt>kdn+uPwLtHEK;nKN z`2dhw52POiG7kX*4+GgpfZQe^|0poD1+Z-c>@9$!6>xe1*LJ||yR}~D*>P*d*A(yr zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GCP4Fs<_*monm06Wc!_g1A)Am*$R=bH zvI*IQY(h36n~+V&CS(&%uqG^G-f*q`cF5Nle z%o}_$SS-%~o@arU-GKKwz_$nR?*#%c0Kpf5_LqRrD?oT35PcQs+z-TF1G--a;s=4= zLqOs%kbDzJ9RbpB0hwdKz}rCf9UylC$iE8=y$9G*fc-NfC(@GCcp%k025#WOn?b60Vco%DxUz&8?G(l+9Iwk znqX^-cxh=~TDszwmgYgogOCRyCp~b|1DlXd$R=bHvI*IQY{H5+VG;ADN)lYE&!zg+ zbXIsGU}OfJl}U3Vl+nDY=a`#PNBq=5Ov11jMO9Ub`#0zJFE6)+gUK=_XgAkl&s=yd zPLMUCy7b$8^~p4EN-}S*hXhk?G0Eq*Q{@Z&V{khYwTUkvHD2Znd<%_8)TB zbH~q=-Mr~IU1gr72{ttlY^>8v{gMT9x#Y(6$8H>*d87Uct&RFom^b)huvoqZ{NDh9 zZ-L-NM_0n27B5WWwH-Vby>0L0b<-46oshk)LPfy5&~aubkx6i9CYGFyRx$AIkP zK<)`3|0FQ<6kyu{*!_TG7vKy4t~S6O1U%0JE$x7}1Mr0a|I0ui30AW)#XyDRRxSt7$dM$6dOC=2Ne}Y2i;W)_X7)2&zi8MaYu2)ycKm zRxjV6ZLdD@W<0%{wySwJ=Y5-UdpE(?{=*V^%e_A=9!u#GhLlRGsUc1WYsN} zo0k;qU1-%UQ}@_iZ5Y4v;(nq}~P6?*W_x)lOY_3g6~C}F4?-SD1q!_!&C973eYcIQcPt+p|E5NW5}cc{9XnH}$}Z-DTaCJBY~T0tzfbMvs^e~U zT}`_gn{hYMM}BMRc^}%%=er*(yLQubrnq*~bK>;u(P=kk%QU&~0mCT9-QbHsVc7)4 z9tFC$0P(Fr?_)sXaUl5wka`kGKLuo-1_pKl*=K;B83J4{ z1MV>3i2yB8zQ@on}l|g=%aQc zxTb>%FaajO1egF5U;<2l2`~XBzyz286JP>NfC*GQ0ctnYZm8W*yP6S4`}gls}KA)9c5HDM9$hO63t;Hq{C0g_;JwAKtf5hmt|P)6;h4myb< zx*8PGcBdjXuU16kJy3Dh>EPmQ06I1Tt=jo^`_8^ zDbQrf_|Mg{%o$+dQy}{pkoz3Se+dkI1=#X{ z{T$#J0-TNYaBOUVdmZ4h11(K}*8%u80DdPBXa<5VpnW3{as%PbK-2?tZUbU1KzAz; z_X54!frJl8?f_DLAib-;UY7~fSA0bQKL8V80!)AjFaajO1egF5U;<2l2`~XBzyz28 z6JP@LZn(0DD~q_YXo9UQ;#H-2Rq2XfRhkDO4?-S<9P+>+4{SmLK*q{%ZApA91Yi0CqB#yZ$dBjf*yrg02=JGqWW2WIO6NVB- z-oSzx*IPZk8~FxvwW-=-xUewLDkbO7ivE)^uSzMV@JvOgf8B-mlZ{ z=;b!`m3oU_HehSORjahDMG+g>b^~Y#l0)=I_s<1o_9);u0604VR~O)p0iHL2mTth?1Nh>A|0odX1%k(c_5=_*351hC zv=8V^0kIE(?lchZ2YNF=;xv#P08)cMItygJ00weE_AHRgL%YeJqjtk_H%x#DFaajO z1egF5U;<2l2`~XBzyz286JP>Npz;Y&yP6S4`}gcGa@i)c4APH3D^y{e*mB?+Dglj=k$qjoc&BW_9_?=xQv-n>Q_&ficg zmzy^i?^#|}3kPpkx_Kk1Ld@UB@{H;I`RW^)&fg^U&NxlonXkUlNVA5<;?a7c*ZdhF zOWIZ^*J@jR$^Oku?fK*E-~1JCs<%|b*in_9YWYXl>MWA{7j0|0u$>~Ww(3nn`_G8S z^;VTt8l~aA#^LASx#lXX0XB!Q|@wq{^7X!H<8@Ik^P&V zZI}5sdO5P-Qvb$mG0Q#nEBH6~UT|5y1M>I&CtRDgfXxorn*fIcaBcuxPQcv^cw9is zM!@R^e47Ek2MBBff-OLMD-iMm;q5@w2XyWLVt$}|7Z48sy=_1u2qd2eQtd#x1IUDc zftP`77|2C{d=wZu0N7p!><9m|UgtRUpS+?d6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e)C0u@hy{tZ_cafK0A7)`JhM!dQ-uP$Bjt4s4B0qrH7KI(PKA!nc}GVQJsWhx2F}HIFqmd!%jFtvoLZqPLBA=Y+$_7A1#~T%%5qhJ z87JRn_>Jl*-e?nXjPl8;npb0#O`2C@lojoPDmkcHLOO^xs$f%0yjiV1Zj#S$%O&2} z-uO>Tv*&km*H_7NWfyNER~URVcy6m~{;}r&$nbw|nqfBnY30g4TcxEd?pU$%t`%}~ zv++{NMsG37t!s6c?i|s)!54$Y(hWFz0B0O<9R=LIfaf^Sk^sCX0bdgE_W^+v5d08m zPXnQTAe;fBr-9A^AT|heXMy+^KyMC6oCT73AaxE%4*{9R-@&r60oipx&JN_8fFTE9 zdl0Zc1UMcBoR0vmO@RB+-_`3pTYgvZH3j?tOn?b60Vco%m;e)C0!)AjFaajO1egF5 zU;<2l3DCTuc|-Gt<_*moUgexk$R=bHvI*IQY(h36n~+V&CS((`3E6}btO<*lH#%1QcFa*k(>#T8jwUA6K}=dB z5K(dYW>~~oUTzC#aY@?E)u+JJsiKUrjJF!~uhG0QRa*=fM&B6ad5~JFEmiVtZCfqR z)wYs+y|x{nkvF@qX5M^1@}|7zjmz^pOKU&Oo8z&KR`X2FhMBI~FjLIBMONKnxp_&! z-i21(GIejA_S`w@b8j?Hy=Cs*tL83jxP4Va!`}p&j?-1vp_G5tKZNT{s;5q@g z-vvDH0WI$X-VXrZM=)>vAJe=MT-Lz^m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyvCu z0L>e&E#lfDt}U8iYm0bkXlXO zxo?Ah6ccao#o(}H0RL$qFaQJxf%YsA`T_{&faqDEGY`bh0o_ADyz#%_*w}!?Iv{BW zQcXbG0c17+15O~@4CGuuej_mC25egZ`&Pj57~p&ya6JLIp9DNl0WD7h-kpH&8NmN6 z5ZDa_p99+W{8zm$wD-R%zM_C1fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)AjFadfu z^ls?g(7T~`!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g?}jVwC(lZIdJ~f1 zuSt`xnkNEIgfee{2z?38&9w_OrZr*|x7F&C>D){> z=Vn)BoSR_#e_0~y{#)+ycn_4_xoJ96+_~vFaeDUXoEwwHEJu!v#<{^4gTK-NghD|0 zWgr>`IwL?V3UnU;;+;Tm7m$bn$v1#hH<0cDGI3zwD3I+1a>s#u0vI|8*xm!|?*onx z0Ov=5>tn$E3E(*ew44FFp8~$m0RQJe;7cI*70~`Q5c&oPe+xvv13K^h?|NNq?SFGQ z0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfX)q96>(J&R~1dLRYkm{G%qP# z@k>hcAml;FgOC#*IN^a!$R=bHvI*IQY(h3+#hb8*b0Z|dMfzN%UqxGmCxRq+B21nW zp^VN=9b6JcbTufV?M_8(Uag46d*I=$(hayz2>tMFv3kF7^v%V4%BA5bC3!caP-d>`*VL5e)YOf;cLUR> zUVYQQ{R-oAzh0;|OSMJFlD5^!wc1uM-=J*`7q&)up7!r5dA7E#mgj0)Nxoj&nl5ao z$TQ87NoUc?`*qqKz1*h0Qg6}A25b%3>a_<9-#=iu@PI+C(jG9%C#Pz!Fv=$F6-HUn z9;lM9I_~B#c!3U6U(S!cnIf;Y>P89wq|F-nRq27G+;K{P9H&+^W)0hg&Bda&Tf^j!y%QU&wZ5YM48+NfC(@GCcp%k025#WOn~YQ)f=ieRBx!>@Y3dNLN+0r zkWI)YWD~Lp*@SFDHX)mkO~@vkU`<#=z2Rc}$+Os=>VzctYf`4u^F+XjP)7Ae`Uh5N zMynCkKusD!^r#&(6K9zaC5*fotLZb#Rn=wM<+o+(&aryaEB$49H`ngnocC?Y?cK!V z|841OT_AUz{QC2(F-Jx1E~a%J_%%!z(606O#!(NfqWVm>IZD60Q(uh@hRZ^3~+r8 zxW5EEUjZ#&1Kw`{-?xDOJ0NiH0ys8nf%f}=(EUL80U){_=zI`}Jp^<=48$J+dN%=y zM}g!PAhi`pKenJ=mw9{vmozW|Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaah|@dW7I zaAgr!7I9_K1Y23eD@*gr(iOk5G!H@^gggj2=z)VC*o16CHX)mkO~@u>6IQ$li+DE% zNpPV)7wQ{ntnfsT1W$y?b0U<{yP3~1HzkkwnU4`aYlJv|L#K@3;NYRjKN*00f*@K3YG+JX2FyrbA`9X^L65$f{c`H!msJyU?mzrtYoNo;yc< z?v3WDx6HkJ)!c;*x36kwuu4l;+_7TiT`S}(2{@75!I1@=p1nqMjrRAas#;C8Nnm`D zxOD8zG&nhZN3WpY;A=r;@dBCcz<>|P?f`OrAioP33IMj}0Q(-mu@`W@0JvTR+%Ey1 zSAdp%fcI6vw;%Ao1_WLQf(L>2LqO;-5PlPg9sxSv0%FI2?ze&XJ3#LVAn`7cd=E&y z52QZ;G9Lj09|PG>fZQqQH~BNvZ#edb2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bg zJ^|`C)NiQYP`{ym!%LjA3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g{e}zfC(nX= z>JyUSuSua!#}ff3LK*d&8Ym@-IDV@lA`29;W04}7mnh=wQbqLLriiWvMYP?iP|-QB z=t!bxgKk(nx%dtS(~K;GT%(LrD-0#rHy4bUi$Bg~)h8~?O& z<)8m1>>{1$M12e|M3A4oQ9 zftLFK@BM)90l>c=2s{V`9|GDR211Vj;Y~pFQJ`}R5ZelLKL*4f2YR0X5>Eojr-0Pc zKzb*Tc?KAG7Rc@fa?b(zJ;2ajz}5lSL;q8+bG-aN6~CesSaM!8evd`VPups?%%9Ipb- z{ebH=!2LSlIS8~I0=$O--vbN- z@42FZ2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fr=+U?S|S7wHs6Hc%uETY|Tq5b4pXisfI68tqO(g~gjI1$RI-PCf% z&8X)4z`H59)r^aVQ^K?xSTL8|YSe!k$MjaG{(Y)9C8;;(2i=U%z?-(KsW<8%937_b zoF9BMMP6;yn}qhC5&QL4l~o$0;l0M;=is^KDysn^i+Q%@w9HVQmgy$f`rlhx-+_A5 zx8tp{t2cw0;_6N0@Sa&CsyEHe#!D?5BPIYvj*dpX!54$Z@*v=O2xxg2@IC_gHUa)e zfxs3ZxD{xB3VZ@9RKi;K9pXo4*+;)SJoVd;urSegeR4?-S< zob$jr4{SmR@OC1^Jz88*vRKc>38w`J-dq;oUjoSR)&b8gPNHsyA1yzl(p z()RQp+9nrpbLr4Q8t*5lvGiA_;?W z^m6kO_$%+I{>ll!|1J=C4+y>ww0{7EJ_5oY1JO@_&Qn0_4AA{45dRG5{TxVq2_(M) zQeOkNfC(@GCcp%k025#WOn?b60Vco%m;e)~cmi~8 z=-kk`p>sp$hL<*H6S4`}gls}KA)Am*$R=bHvI*IQY(h5S1Z%<~&J9=DPo7owbS5Oh zUy~M{fhPh^gfcoeb#O@((bb@cwmTKEd9@-M?}3N2PN#^T4M4|6pmiJ2FJaep*fR?| z<|v|Ro+1WsR>bjJ6%kpW`~!^5smFI;BXq;J#p?aS={FbeDVK($l;quvLaUjtDl?{Q zze(yJp?Wjn)SC*8y9qx12TSBjs5jomwKL5#HHl?9B$k_|n01S+y2a2~3id9v>XxZ{ z>$K<2QJ;IGdFm~5?_M=`VZ-gK8XBz9(iL~CSb5h9`AXw%8dG6;MD?c4T`=y(WHHN) z$Npz;Y&z2V{_E-vEY zq6xORh*y>7Ri!I_RcRiCJP3IZa>xURJg^Dbgls}KA)Am*$R@0K6Bbc#swKg-`dq88 zqqM>kK@vO>CeMjbM)gMe2UclDs}a>eO&a!YF27SdW+u)up)X6eDl& z#o(}f1;oAvy1xP9-vYhg0f~G62*+kEkh%{@-w$LS00!0r*$08#LqPsvVCWIR<_7GW z0fz^0ZUbB`fV&m&c!8GffY%53b^v}q5ZDC-13-Hl5DEg}=YeQD(AfdRLO}P+Ks*fe zM*di@OGN)z@r!x#126$5zyz286JP>NfC(@GCcp%k025#WOn?b60VY82hTaXm8+teN zZg_cfHX)mkO~@u>6S4`}gls}KA)Am*$R=bHPOv5{;@xni{p49`Pj5mJ{55IPRq;f? ziBLxGWE}Hy7_&URDhUZ&!kPGirCHsBW{kWTEDcarbUu z`qZm$df`_Xw>$MhZ?1-7BVf~B&tCw%kwuTE^qdZUhca=O_+g8hSwXGyyuWd~i zwo~MpX33%+H$WzSU)B0h-$eZtXTD9Rj(=dF;D&f~K%7gz8G?x9U#_}4FcpXR{1X729^kE?LCNOXW z$i4;Sjsf|%fuVN*+eyHl1RQ;UGX=Ol1l(!B(+{*{0PktQHvsqtfj|}regU-SfY4bW zoCl)kfX*Qx)~GtXQK(8H0GyzEmklFyGol3nf)2wh+0~25ZOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;-6Sfa(nw7jbbB7Z**i#YMcbG_NdO@heO7Aml;FgOGzBIOu^* z$R=bHvI*IQY(h3+#hb8*dZU*F*XnbvzM!ejKl<>Y|;@{M&Kc(?Xw_0I@z4G^>-~?4)NC)NSFWBm_EmUyS{W^hZW1cqQ^Ce=9fZo(L z#ayGkW~%zpt4#uOmAG^sjuHM%>oy%^mWNfDyzz|iA>%?sGK z0}dbH+yS`!fO{9<2>>l^fHw&Eo(KHxK%fH%hJf~$flwF-M}TM)=sW<#I)UykARYsH z-vAQbK(Yr&#ewuuAkzyB90#%qAa@c1PCiKiM^t=q1wQ~2U;<2l2`~XBzyz286JP>N zfC(@GCcp%k0282qLji{Z4h0+vIK0j|n~+V&CS((`3E6~fLN+0rkWI)YWD~LpCs-2} z5pcNVe)252r$8YI{+d+kj64x=B9u|UsaKs6MRaTgu*zH!{gNVjuEU;L*fB>DP4g5w zI+~wU2R{iOilQnfBdH(D%Pr$9F3G;Rq9b!9cV@Kv@3{1fFIKNfC(@G zCcp%k025#WOn?bgJON5KTvo(oMO;=i!Il;A;?lghbj2?&&4Z8!ArC^%df==FHX)mk zO~@u>6S4`}gcWbXBGQdc5?rOvRr&_%Dm)P+!4qNfoCsxiyt3M3Hxv*uD4 z<#HjXED}x$GjB!`fiC}4*S@Eo){T6FX3-cfY>o0f?cY`MY;9XD&(*e)Jf7A~TV<>p zTVSE3`5dg9`?u!HZru!Kid#34!+U0pXx$8YjYVhJl&;<2dqH7o0~|rX`8?oi2izTi zCj_*-40ywUF9P_ZK;Qrn>;&4ofKUtwzX3$MfzBQv76-bI0`Xp;_c)M90Lha;DhZ_f zfJ_P)_z=jZfm}b3&j3TG0o$j5{WHMvIpF*faD4^1zlL_>`G(pJXWcLXCcp%k025#W zOn?b60Vco%m;e)C0!)AjFoDV^K<$Rw4YeC;H`H!;U2`@en~+V&CS((`3E6~fLN+0r zkWI)YWD`!XCM=@eaH0L=S!hpfLK6HnDbiK*M8JtqM(u{fZAu&LQ!DyzyIi@sc+c{& z(Qj&mp#`&=;>u0WiPN)3tK3w}jqmG9S8njVpsx%8p2q)$USk8i>j0k}@HYVg2N2uNfC(@GCcp%k z025#WOn?b60Vco%sN8T-5f>G4QPBijRKyEP^McY9zo0Y^LLP)X2sz(@^BvfPY(h36 zn~+V&CS(&gog#WR0392F)@?w)gk9HR&n)biqll(?iWt0E5yx*;L}Y>T4=^#O9^ZY9Fbv-o ztM|nSw{lrH$|$v)#R3XW^;qBTP~T#tuRj)#QoE@#OD3JgD4(1Pl}C>|CTtDZD%t~8 za)l<{cw_%-Y1_O=?mC#=S$6H_N|SCHQ(<{T?WWCLFzKe+Vv_rw(wDB?;Cn$~c?0lw z1A!hO7zf&q0-;_Yd>n`-fXj`yr4>1Id0Ml>ySHfy@9fFbHI`K<*15 zp96-@0=91e`?rAOJHUDGA}BU%0r!1?=YF8&0l>Q+@I46l9|8go1Hngt_DzfGb)iQW zaYX|YU;<2l2`~XBzyz286JP>NfC(@GCcp%k025#W6;FWL4YeC;H`H#Z-SDdBY(h36 zn~+V&CS((`3E6~fLN+0rkWI)YoM25@M7!Za`^mG=p4x;Y_-j(6)AK~YiBLxEM*0WV zXhy3L)j&%cc5W`eQ#)oB&N87ZVcN|VJ)2t9k1>tKtycZBG;d7R7K40)Hqgc>&(pS5 z@@#EeEzi}ql02RRZ?0CZ3Q~LKx{kE?E~UFfL=e4*aaj5K&lN$2Z7A2rQ;S zv0cr_aYk*pU1dLF=&vU?vA+#x^NpznxhsQ}zlAh%0 z>C=DSkMlYN_>KVnqd?#|5c~uPodDWS!n_H8M)O7#eN_cN025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5pn1c!MO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFD zHX)mkO~@uJdJ|?bZ*)e%rTSc|ucx!ZKLn%TAHt944HEuxxC=z%s!0G+`4#DThrW zi^C7dqV+*pY@I2Ky4kY0I7b%U56hyxN*2wJ$+UB>+BrtivGSUDGicykeSoPn*IPF? zKE6#4r$T6R5N~cpr>RurnQ^zVQM@TR@y4fB1sj8;jJ7G9coTj7AxpRIpQQHFZ=B7$ zc++~5kvIL9)=B1{sgrjkOy0S7f?5C2!iA4mjdK>vU$F4;1ybwU4Dm)N;>}vE*{Z&2 zoHDM*Xc88!)G*mRK|S^p<(SE&x9FvY^%__#uPYXdABeR8T>&8eHqa9U60JaA2uQvQ zq}qUVJ1`IiZ2JKFM}T8LQ2R09JP5cB0q!F}{ZYVk9BBLm@SXsgP6EEq0RQJe;0qvl z3J9G6+P(zBXMxB$AbK9?ya2?$2D-ih;+KJ*D==>oOaH0pYYO-Qm;e)C0!)AjFaajO z1egF5U;<2l2`~XBzyz286QFs+wMAT8#I;2wwzi13mgcRci+*cqUWB{|c@c8b11CMO z3E6~fLN+0rkWI)YEP4}WF>lI@f=l(eRNp{ng?|VbnL%geNArh}NAspqkx69HvKqiD zb6NBnWzlg5j!eXX$+D=KCR5H)#iXeala>grEVLQgO~#M&@@?VF%1OJq9ed{5dyR^M zGXj%l`bb_p#Bat-G6b9ZHeGqKzQ9|h(8FUfScGI$_Zq1}-D{=0)xGZ8UN22ke=d_I zse6MoMco^vyVSi&x>Mbc!OupM#-fq-Xw(NwwI-qd)8n|-s<2F@aqy^q z@HKd?xy-7Ap<X0 zI>V}&s~nxGzIL+m+I!4n@1OF-qA4?~9$i#b^_@f0a=OesQGHpta<*}%UZVw_B~`CD zc+ar%%?+MS|6SjAZmH`Pn&{qQ_9YAU&kgf#=>Hym+!2Ueocmb$g z3pm#St``CK2B3Z;;MoK;ZU($tfTouL-z$LsRUoh(2)+h{b^vW}0O2=*$Xh^kC(!v0 z5ZeWG?FQm|fS&h&#QQ+sULg4aklOc8m74TNbZ-O~b}#`Zzyz286JP>NfC(@GCcp%k z025#WOn?b6fubis_lE8b-5a_$bZ>Zvb2cHHkWI)YWD~Lp*@SFDHX)mkO~@u>6P8#L zW^r%0+Me18{VPH3#3=ZO@T2-e$fJ8x!AUnGn(}j_AW~is7bl0AHzSZ{hVWnppPD zkzKoK-M3@nklIc4E`6qAqnA4Vka=tf?FQcq3QH779{>hA0oxJ4eiU#V2Wme7oF@R+ zNx=OXQ2#mL`2uJ>1$fT@OL3xLfH*w_EOQsb!qdC?aX@B=UbCcp%k025#W zOn?b60Vco%m;e)C0!)AjFaah&?S>1BxUh%|i%M)^5$`O`J4+Y+&eFUHc@gp=3E6~fLN+0rkWEwb=YeGqHjq2t^S7ZDP&^{}68N4@nvC%`*Z+8Hqj1WF$U(Uo!MvEEzeUz(r)G0q%9WurqkG;Pe?P68 zVTavp8_v4z2G*H#KLnTQ?m?Pfr@wy2%(~W00CRXhxvj z;EO?F*$6l`0kxX}=N7>AGT?p%sDBmkYzG=&1H3zcrZ)iJn}Gifa6o3wg+(b z0NfC(@GCcp%Wp8&NRYB$twsNGPz z;hoLdgls}KA)Am*$R=bHvI*IQY(h36n~+UdVojJuyWy&K?X9h9rx0KijE>ftj(-TH z`G=54?Pe-;5?QoY$)fo&S*%$si|W5a#aX72MaN2@Wi`;S9_TgV&>c835eFvAqGp;b z`tOs);Rj^V`k?#|Fg2$V-+gcx3?9tNX&y`-OdL5U^#&G9fwslUi43wZ+~~!O_K_c` zcXJm8%or>`>S;I5Ge5U9?EGh`z5mmWyn8n{n|4zj4@yINHx1kL=I?no1`E8K#7Mjw zd@(pIUjpv4K>azua~^2C0C>L!n!W*imjVA3Ah7hG;n+M0gq{N0o(96pfyfFV`V7$d zED(DR=voEDp9gwg01|6~zI8zIMIf~SNN)rNHUYLqz`hl5c!An&fU^m3H3M!RQ2#pM z@dJ%5|6Hl@2L73M5@iBRfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Oj3DCRY$|9~T z;>w~DTUo@rO7pJLMZc>wFG5~~ya+kufkPhHgls}KA)Am*$R=bH7QG3xcsB;lhcODe z1Deuw2dG4G;tjo?Tl?G0qjzKc2li-2>JXK~OS;~>$#`5bY8uX5q03?94J?>~Zk6-Mi^Mm3`h#!?tBZ z&%5dO+~D1qqz0dM1oLk2#o(~K19*1D(|5O^O5?gc_00B!q#@JB#oKM?&G z=sXC-4gp<9fcR0M=Qxn~1n4^fBu@gV&w%vjz`z%P?KEKT104N8Z4z*P1-Mdx`yx=E z20WL5#sR=v{R@0P8{k_8`0c-_)C6jNQS=oB`~XaV2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?c{yP6S4`}geBI5S-cx6 zCsa=8UX{_kG7A17{HXpA^61@6=Zu@2=le{@e4iykcXdOpWOz3jKhDdmhJ(}PuHL|D zTp~<`*`_Ne(iV8*)SEK1(WJ5HrDJ1Z_Gt0IguM=XS$(2RYS!X_2E$$croO-+m8mZ< zNIzAN8>PRhdy}+S-H(wLTeT*k{?p@qTC2h`mBzuN`oY)WwdOLb&QvqaJjtvT!a4yD zt>aC;<-f4B9)f-2d%Jp~`A+qz@p$UqF=ow7t7eu|H#_6#46A0Ya&)Tt+R4gm?=g?P zf65b!rp%~%bWv56)i`It`~?diUm)GgzG+SE9ooL>Sf63vm@HNfC(@GCcp%k023&B0<>?qzKH9KxW1^w))(>S(!9BJ z(Qhuzi;x!~FG9|G;H(EWA)Am*$R=bHvI*IQMQ_3^_KlWrGw9Qj~GtC>5!BSdtZ!py5s(GUrHd+`sndZ$^ z<0jwcP3X`sEFGJFDYdWKx+d@DP3ukO-t=EuCz*eynmdwW?o2T2A6mHZk?)k9y1ESW z#%viUb??-TVD1gR7%Y|}K=>#SISxcW0Xk0rv6Dd8XF&XOpyvx9aSG@=10=r$QfGnm zIbh&CV7my|(}3d=P&)uPtA7d0#s;{T0rhskQv)NfC(@G zCP4Fs<_*monm06Wc#m^7A)Am*$R=bHvI*IQY(h36n~+V&CS((qSQBP3Z#eNo&>kpR zdw^0Dr6@{KMxnojKZHD*HmQJ8)zo4osFs%`};Ejw&Wig_yKN zAfoK@&7g=gFW(lvRScMwg9o0fv#78_^UwAb|CQ@(6<9fz5%4(1k!H-13Lj*5U{rbju24$ zF5qkfTRuyNsC%t+x4JK_i8sx|c{igo@g{!mmzMtB zv!wQeSKiONchjH9?%lNR+c9xS@1}LF{`-oJuxO>`d$TGwSr5WtIj1-*=YhlppzmuS z`3;b|45Y6B150PYu~`P#?SP{OsC59&m4K@jaMuC#PQbGoXmkPIH9(Ua@T~{@^+2Ej z2zr3fOF&yA5Z($zyg+mt(AfmUnt?7K5Pu!$@dJq#pf3O<-v&~_S(Tb}>nz?(lnF2a zCcp%k025#WOn?b60Vco%m;e)C0!)AjFoB{cK<|d$4ZRzBH}r0JcXKu&n~+V&CS((` z3E6~fLN+0rkWI)YWD}NH6K3&lZe^AN-2u7-g>eU{L>YySLvQE?-ohV39=#imxXEq2 zPlf1yI77I}_;KFo_&4Q3n}c)n!xLz3!GMAOatrUe&9~gXb91Na=9r|r)cqLjjV6sn zBkj?s51@fe!!ug=AUf=I*lX1%bl*OqyY_@mDpQ}(OUK5l@6byo^&NUiR-Y)7wiU;@ zN$#FysjizXwfj$fl6U8(=1g|yrsL@8Nh5P^v>0Ji_h?qVChI}?E4vkcWe<>k4;XkK zu!RBphkzpj)OG;QDBwB(xI2OR!+<9SG#&%IT|iSe;EMzPPk}%W5bOm)383vX5bgsa z{XjGcbbbZIQb5;5Af5(#E&+)Fps#v1KA#OpEd$baV4w!DtpM!L%&ycpo}FFv)dc(i zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCLWsv@o`;;Nz&TUEqcO7oV|MZcvq zFG5~~ya+krffF9sgls}KA)Am*$R=bH7QG3xI5&a<4FwtsH1t~NwcLW&Qp%T29-W)1 za7kp*UL}j>$7Hc)u`H_p4i9IUMiw0_ftJ-k!+N0Ch(mYa$V42NEQ^|Hvgp507Ka~@ zMeBp|KfuhKN__Y0r@PEjCbwjczPbKLzBC*;C+}utV`r}aA~!yU>644AGPOt+OfH`- zzpwdWsW-!pyZI|z9gW2#T^)NfMp|sunuPjKkKXC9 z9G$AZcCzx?d(30+pYp__DKn}bT~t+NHO^Tuf5F1X7f8LQvd_F}*tTrwnK#wDGQ=AL zjH8bJ?2gS%#2b7wNG#oeI}X%;3V3>e#$Ldi0Gdt%zCOU;4+N4x@GBsc0@^MD;WQAr z1Vjgb&gx%5vatbO%Ye8Y=&1n`4xn!(kgNq#bwJt)46FuhF97znfMXp{`y$}n0Jt^+ z?oB}bX27!rXngrsl^X9Wzbg8Y0)7A{zyz286JP>NfC(@GCcp%k025#WOn?b60Vco% zDBf^s5tkNmX;FzSE#keUd2i{W-&>j&AumE+gdFw2Q4efFHX)mkO~@u>6S4`5-h^4i z8(m>mfg1%+q~TlDq5P0mApreny@5+SZ`_@C>ba+CGJyu5li zI9(3zO%^@oMq%dmk4#sT8`IUzs63#4V=`ED(%l#(W3cF@Y3jaAnxyUx(iC-Xl!~I? z{0%49S`1Pd_B!l;ioM2SltyRnjc41hEX`@?H%FSb6imNqP3;|8zv*}*L%%UWziDmJ zW;JcH9)!xWO;K5z0BDf#d-o)d{2z0|PO@b{w#O0ys_pwI>1RXMpQ-!2JbKe+uxN0UEyqyk~)? zbAazW;J*OmORU@R9FE=Jb1RP;bmq>yx^y zmQB`!&{)!n#&QV=3;@CEe}QIW1KO4WVLK410iq6|b0rX~1-j~hxD)7E4J2GZ-x?t4 z22$&RbUiT80N6GG_RWA}3sCzq;Cuyey$ZOu1NE-~o*h8r8-Vvspy@5Zw-fNc0|a&f z!QDV;5774Bzf@|%@Ba&LD#`?y025#WOn?b60Vco%m;e)C0!)AjFaajO1eie46QFv- z#YJ3P#KlD=wz!BlnC1uYUiU?ya*tS?X7f0Td+{l~l#qn-J?f+uwaQ#|pUuQp* zckia+WTl#A_HctY9r-0zoKxjG8wgL!014Nz$qR#=HtA1Uni9P@8qOU06 z2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0!)D34ZRzBH}r1k-SGD2Y(h36n~+V& zCS((`3E6~fLN+0rkWI)YEU_lc;@xNpvk07S3*8nfCsa<@HU%4fl}GQUf@5w*G~x&1 zO~zJU5DzDZfj61TO}Qe*j2qQ$s`C5k-jtks(>$De^H+5yj!C*Y`(})^*s3)N^`G9P zv1qLd%TyW%kLm|sgV&nNtU4$#=1HmzGeNOo#+#z9Ut79Ag?n@9Y+v5poBl+0_oj8< zj)_CMH+6LxGjD|A-aM(v9B*TgS`XZCP%pvnfyd%fJeD;;%nfv{2jcZWPXmzf0DUh3 z$wnZx6-ax5fo*_oJ79keaO?nT-vFF%0q92UB`j=CqT~$xHpNDbZ-O~b}#`Zzyz286JP>N zfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5pONcTT2)H*3!HPc@gp=3E6~fLN+0rkWEo%{)br5&7NbD!*_j>1LkttRcVKT;&%Ib<)vzwo>TfsTeFm zGOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlOjnZA}-gIq0274pS7LBw=qdw3|&B{Bq z7OkYiUWdI_eM0x`6S`|p=%h0B3B7b|tojbUWK!Rumt^&cGHKgz(#=)bCg0Lc{L^1s z`hEW@wI4rjpJcvM^>N0-$GLZmSu@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMk zGpZh4R8?g)&RH;j!NSKENHu4&OE(=yPfr?Fy0NOif1Kh58BGG&_TTq?=f+?$NX?0x zIXC!X@K<_(L@&^n0FtMHR3DJ;2L_UW?L1(=064w|YQF)TmjTxmz`gWe;nzF~c%A|p zp9Z|kfu z1xUUMq_+QSr6&E_zZQKp0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R} z(7B;=L+6I~HfIyE3E6~fLN+0rkWI)YWD~Lp*@SFDHerc1VHW2`6lMW9y%u^c)J>?H zux$!9_bQLh%~ZG~vS_c8Me}2_ShH9b)qjVFvrHq4j+H>mY5*f2=;n0XQ7Kv`E)kl+ zZI<#lbNtQqNAjiN$T@j8BhYH5E6R-a_BZ^WFTYw%F2AkvhpFD&2|dPODU5nkqclG9G^KPo+L1{?!rrD)8e^0&9TJ%!K{~d{XgD(b+<#izK2L@UI z+fKm#4&c}Y)b0kHdjQvafct%*elOtp0BGC?cs~M~_5;3;0slcDa0mz<0YXQCw&Otf z6CiQ|h@J#GKLcW)16^ML@l!z086fc`(03L{o&!?nf%FAn;A_Bk39t`9y>V3k8&@?j z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ1k?--f(dd7Z-7HQHd=s;$5YASLveP zRhkzeFG5~~9P+>+4{Sm6DG zjS)Ht+?vU!2)qacHvplHK-(rDycvjW0irJhov#3~SAnkWK>RhJX9tjY1L%7b zNWKN6b^_^lfPr0rEdNfC(@G zCcp%k025#WOn?b60Vco%DBe)Kp?E{_hT;wHZ_XxU6S4`}gls}KA)Am*$R=bHvI*IQ zY{C+2!YtyAt}v^>DYsB=p>sm#gl$u>*;jcKZ>Dq3P0llZrbA2`oD-vtd--nu&5a(; zyu5liI9(3z&8_$|OB8?Rd)us>jhgw&_?bd)KT!APu42r*`5R8Eg=$iUy$<`IVz03n zrO|P3oFDz0rQyZjNbS`dBL#DBT2p(6c5kX1Gu#_x@=)F75xF<`X7E_{1MZK3`h$Sy z5YTu8@E!%4jsw0=0RIUfa1sc9282Eb+P(n7r+~;AAo?ZHc@~JB1G>%w@e4rD*FfSM zpzkt}yaJ?_{sx}Slfb}JfUO3wI{?Q@ptcrp)&VXj;9d>Xy8zD`pwa!CN{x5@Z@8?1 z2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5$`R{drKGn z-qO4Xc@gp=3E6~fLN+0rkWEyh@cTsFe8E|;*_nzh=_S~ zZ^}97CifvfM$xenx=ohpG8jv9qkVJzk=rzJDul>Ey%`k?rb3ZxOgGyql&41JyhHq2 z+Bd^azVT^Q%f=x6^(OX>a$Exm<*J185Wd51@F(iD9dncIn4=r5y zh}Ae}!TbdaA73C<@5-=mOct}$zgzqLaX3RyzQH$x$+7|PZUmY(0lv+Ee+v+J83?`t zgkA;OwgchUfXEIY`UcSXCJ=iI=-LUy-vN4d0g2r}-yR_O9*}w;Nbdy(J^*YHz}^8k zqCo8dz}X474g>BOP=5^YbODXsfHw{_eG2$`0Dmv+n?Qp04JY3)0Vco%m;e)C0!)Aj zFaajO1egF5U;<2l2{3`;CqVm#_6_YD+BdXsc$afFA)Am*$R=bHvI*IQY(h36n~+V& zCS((qSQBQkZ^{a@6r6?&4Ht?h6i?VT1)G4CNBgEy5lUpyvKnYu5A+&k(QyZkOvHi7 zvZ$FRQ_xWbovC8$%q2p|LSz+i1`p25w}~?=C-vs0g3QgtnUNl!uZTA0!EKiEnBI-a zV9`l;LuD~o^wKnSUnWgb_XcT-x;ILNoO-i;IPYe3rr!8Y{l?Pz)*Pulbn$ZDy_=TP zW#);hUQ@1EKI2TiMhn_Xs$OyMo{@PsW_ULpcHKz48+0vv0AS~uWa z54h?9cLPxG0X#1Ojg5eJE70TxeA@tj6A)+yf<7ShI?(2yQ>h8J%qjZ)Joy2b025#W zOn?b60Vco%m;e)C0!)AjFaajO1egF5pm)QSMO<0Ll|?1CvWPdA=FO#xesgJFguDoO z5pvc8XFae9*@SFDHX)mkO~@uJdJ|^xZZxzZXhYD3D3}ew6LHK|VI;&ndN&o2Nn}ws zTNW4R$fEmUS+rNlqWLjdtXV9J>c2z8Sq2-2a?Uj|3F1x0R$dSfCx@9g-zhPJZp^4Y zsHAvvO}Hs7@us;r;!WtSIhKy6e=D_b`t>bto${fX@2P3yiL6NeOU`aKx~Z^l{1N!)oyMX9!pmPrpdk^S(ABgV-dOiRW`+&ZWfaHE4^)Zk>2n-woY%#!o z3~+P-wcUU-4!AxA+&w^jFW^Z4ji&)`AJEhf_>zGCD6X{ zY(VF--&Sg3_TO?%0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;;%?fZ`3s8;Um+ zZz$gI=H_fdHX)mkO~@u>6S4`}gls}KA)Am*$R;eYCd?w<=nJz7oN^207CI+%PS`dD z8+?^V@rDy_a+~f`0VyfNxXJi&-slK86S4`}ghg+{EY6Lf z2|*KrCPcwZ2%d=3wF)C3=Fz#C$^kby5BI4QEfbdr-QYG$d7L@>=K3S~(s1MvdN;Ea zBW9=%mX~MeE2sF*yP2gtHjVZ+Upa-oz+0uzn+@=5gk)6r8mU6vYjYlUbEo=xlXRE5 zAA`Noq_JqEJsS0aR%)JxXS5crq{Ci^y;glf_w5t9YftE;GW7|)bZo5p4!vYj-=UXe z^@%d6NW*U82Y+kn-!xZh|7_=bdG~H^HteQ49+ZakZkka5Z6&sVqB(=8O z%)7xCgTrzNNE`wBjsnT!K#i zQVnw}HEGXWE@@x_On?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm z8{XHPO~@u>6S4`}gls}KA)Am*$R=bHvI*IQCDw#lyc3Rg!1+hno2A0sWcYE44@ zr^j)vRbiP*WG*L%7~cprsSr%mg*$bo3<^-^RC|Xp2|M* zreWK%p-0}-)#<;l*=RAzrmkA^{ewfhH~41oST+OcEx^FbfNdLKZvq_6K&=mOz7Dwj zfV%~#4*;IGfyN- zrwd4Q1ATEI`6-a<0n)v|KmxFx0qkGGy>XnSd&8MGOn?b60Vco%m;e)C0!)AjFaajO z1egF5U;<2__zBRx;p!r;F5>E<5?fuwTTAoS(nY_uG%rG4guDnj>4B3T*o16CHX)mk zO~@u>6BfM*v$!`p8WA)iXhamuh~SAhWUDYDVjkU_>F`NpQT=yW^e)rLqGKh{vKnYu z5A+&w=nfpAp942(I^3isLUVP)wSJ?1bN!Kdc~x<6x*YVIoD`b*iZGL(u`#oh-$VBX z<8XA>#^2~AWq6IjQYKAO_XcT-x;I`McT?c8H-E#)wH5;glr1mYV&KJzRX-(}N+P!I5mEqp#5k&vHBXV!>&ETX4An65C+kkWvFwhLxb^!J_0LPm^?OT9zC*XSLca<9VuHO~?j-LDgOn?b6 z0Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCWvdqekz?hV}=-s7B2$R=bHvI*IQY(h36 zn~+V&CS((`3E6}t)`VHy8*O10g41uI-$Lz#+6mjHVDqo?=-!lb)=lnnevFEnG-%&k z{{WL|uJ>+ke0-ZG4hGERpx)eyP*bVsGltvPsNR&EdgIfoij6^1#@iH5y>Yhw&eBl( zd#SzV%E}7!o$5rK@tBBn?-;XYrd2abs+*m0bcR(kS2;RWeeGoBwfC6E-aqAuMN?)} zJ-Vo>%4(dmVE%%Ik1vo~Z!-3#|I#|i{4@3alJfouX8l787d~>`8`oy2HwMi6=~%8E z$=DlwGiWR!!2K>z-v)TvfyOZ4{Sau10KN{u9|ZyjfM6#OIt;YMfbcOO(gj4jfzCJ( z`xNNv0ph(tPXb7s2KxGdWIvEf0_m@SffQi-8nAx@I4%RVR{-bI-{bQ=3AmpE>YoNY z%YnufzpvDIpZPskH824tzyz286JP>NfC(@GCcp%k025#WOn?b60VYuN1gPF{aS<06 zadAz+n$;LN+0rkWI)YWD~Lpi{6A;)SEI25fmaQ zL=;Sj;E6bAt1up79@U#lPP)l;%Fk4>b>=v-g=H#@gGcp)ufc20WmX+T7xN_5d6|HR*6}7!?e8tk$DrPP zk$5TZ>P^e(GV?_BY~{+?#+iDJ7L4YSs#hGmXJqP)$zqn8BO_98@XesH)B#>6(6k!x zxd8tfAm9dq>w!=`(AEHiJwW6oAle9YZUtgqplcftZvuLnfrJm}dmTvnfm92S4gdph z1GZg&eK+9P1Ju3;INt|cdjaa?z^AXUvAMkz*G#vzdhXDT(s5gP5RBuGlmsRis zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@Gsy9?`sNPV$p?bqRoU;kpgls}KA)Am* z$R=bHvI*IQY(h36o3O;1FpGMlDa<->>Mhh;Xr0hHVcQgJ`c)p)n+o_OvZ$Lai;Ht) z(fzP2+N)&I{Fp4(ES5#}-(}IeOe53J`PR>Ydy}!17evL$VdhPq$Ir}F6c_Ul+dSpj zJfC#fUr6z$%xr`hqnD12g*T(c0~7W->}BQ?DSGV=!C3=&Hh5a@!1XNP zeh#Q#1$dqZ8eagsYk{V9fbT`XzX1qr1cIA@&}N`*%O5H=;g|owH4RLF2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?a#Jpqb0Tw27XMO<1`VoQs7YiZtEy6Cr-=0(VhkQX5* zJ#f+kn~+V&CS((`3E6~f!lE}}7V$<;9fCRpb%=uL5IhlwY!${r%%gY%vBn6m<_=l3 zOq4~#WLeZqlSTi1vN-&JELtCwDdwnBQU#=>4C5x_$9bb8;FJrIgLE^LATxwJljBoU z6~$&;##XKzp?8BpIHfi4X4`Py&FBog33>lu>9`2*=9SQayn8n_XR>=Y9Y;@38ku(^ z;N4V*Zt!l*Qpe7WDL8t#Oc<&IhsCQnEZcx^6A)|sE0O)xeNCbht zRv;MyQttxkHejF~uzdj7_W_QNfZF|l^JBnu5O5y?>W=`Pqd?NfC&^o0eUy|Zs^_6yP6S4`}gls}KA)BzonlOuZ zBMP$!oNf!<7AhxHPS`dD8+?^V?`A58+~hpqrxI?`5+Me+S<2(g`8U@e$(M>F=j7gG ziZL0UOdfw+tSC1EpQ@1J&0=)`kE|#_`T}p2LXV*`28)o4>RuyNsC%t+x4JJa@uo-v zZ=x6fVCioAcd314)9Jj6H#Zx2QymXVLy9*I+w|sgvsMV}SX0e3tHC7jUr~I2+>O}+ z?Z%$r;tXmx_+C(0(m?DI&@}+WtN$H}jSWaF1N!VhvIa;wfb>dWpcb$_2iR8uj^}~e z7Xas4z_ku=zX;TC06ZIk#!Y~CGtjgJ@VyN9UjYKI0>SM-=ry2i2M~S(h`b3z-vT;! z0NfC(@GCcp%k025#W zOn}-A7Z!105f>Jf*uo;-RhoB|F8W=ic@gp=TT-XE~5q0rWisB%cLR&;7AdlV0^lE@@x_ zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm8{XQSO~@u>6S4`} zgls}KA)Am*$R=bHvI*IQCDw#lyc=C%7J<`kq1!^`gvtrqreI^Q^61@6hf5-h>c7jP zcbP^O9V>yB)j-2~px1~)ci_lG9LU*Ek$w*%hSfTkUQ?+w8JCJ=ZF2<`+z?*MJPfbec0vImI12Xwv<#P$MR z9{}-vK+i`&Vn5LLF_1h6qz(b;Bf!8>z}5xWyP@AW;?!?M(U(^6126$5zyz286JP>N zfC(@GCcp%k025#WOn?b60qQqgV8jJRTwqjU3ygSoY2IDB=y#XqMaYYg7a@l|aM%Nz zkWI)YWD~Lp*@SGuqBmg{{YFbEf>H#fh=NHGJQ3$?6~;x(qkdBkr9>8oACN`sgR_-MSGPjnjibI>NN65zq?VmOh+&8W&a6+)APeFG6DbDOIi z(T~zLPx;MMZ*o#^t`58@jC$kKs!EMPRcy3SZZg%ItIAEj)f?N%KU(Tudsu4Ues)LR z)tlCv48G~Vv`#YrOf`TcWpK^}v;Lul3m^GzVoqIMhI*q%AkF(nq~740L1Xz0aC{Eb zegQa70j@KE`%9qyEZ{i@G@b{%7l5X(0pB-(|1uD`0tA;n49(_ApzSFj{4@|*4n$V~ zozDQVXMwKgfcPq)=XoIU0?@Y>NUj4?F9PWez`#bp_7Y%k1RPs|S})+-2Dq9YuGF}j zALgnCCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO1d5&j)f=ieRBx!>P`%+@&e?=) zLN+0rkWI)YWD~Lp*@SFDHX)mkO;}=0m_@xQE6h4@>Mhh;Xr0hHVcQgJ0#+W?n@SG5 z$#u}rRG3Li1m*!{w{HeboO$`yaAxJ?-3;&7OjS%7Lmszs<+oA2DLM6K`*7;b=uEwF zzV@)C;R@87y5k?@UA<{JU1pxB8aCz1jGb|&UZVxmxuoh92k#k~dZWx9s&i>aGW7=E z3>wQ0!2JeL|0dvh3uxR4c;5kzJZYfu z64V>-0M#2#yw{&>J1kcad8nB7nRuJ zBHmn@H5J4fLU_t~> z#4%fi@euQ<-c)eX&4{M_z`e=X$_t|6nylGvVG4sY`F-!fi5s5eWW{_Bx{s$zR zCxND?0N>Mqe>o6X0R*1`LeBzi&jI08K;(HK`U23o7Kp6_x?Tk08-SjTKw=Znw;4!o z0a7mm=~sY(R{>iyVD|xz*MV9;;A{b00l@t>P#*+5tw3W4@V*N)wE@0%z#smPN=@Ly z|0w$1Joy2b025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5pm;;^hT;vy8;Uo)zd4(b zO~@u>6S4`}gls}KA)Am*$R=bHvI$G93A2bd`ogRNr`$rhh0Y0`6ShskW?$t|ypct( zQ5GF{$f9MUEE*=uqGp;b`tOs);Rj^V`k+iPM-_7_MEAoP#!beL^F~L&DHqxtq?Z#v+`#dvR8BhlXrKJPrc&t52FNu+GOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlO zjnZA}-gIq0279APW6?-^H0lGb)U3QyYtc$N>~+{{)hBe{KB2qzgib0`pU_Lk#;Who zOD6RldP!EFD3i7oN4oLt`wvU&x<5(n(bvDpyL3}?CcAXgarE@0kx4gZ%Q&fHgKi|# zZt%?@uzUmr_5;C>fzUyq?GO+?0z{4i(c?hpCqV23&~*}se+Kk?4kW$+`c47KGeGJ~ zAbl1XI0x8Lfc+xiNCUN(0OtVUs{RuM8yir+4Di^2#u~uu0Gd_;zFNRv2Lzlza5WHe z0c~skRH+HO|HSnSOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2^2j6N;h0q#AQWX zR#alkig;6L-c-8iHO~@u>6S4`}glxj1H(?g(Mo@*I3PBa3 zU@8Pp#PM2%F%a`8-Asi{B8&DaSu{T;i#3a7QT=y_ILkCJaaIB?tAU2~0Og!Yh)GL? zc5s`eJkA_`bN!Kgc{p-T;!QpW&y*{E%eY~;smg2edEQ)p8{L~>-J4+t-uxA^j>ck= zt`5E#BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignN?@1nP#43)(T;rfQQ!crqH@SSvtOe zd$Xfq)&FC@Q+;YYp1OC8Su@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMkGpZh4 zR8?g)&RH;j!NSKENH-gJQymXVL%TP=jDa@-?oIWD8wTEtlNwwZLXO^|mr#E2Sk@^X z%ZosG0}$B=L^lDQn}OICpzCEI{tD3ZDv;O?^t}cocL1q3fb^Tdz*~UrZNMG`9IZfY z2yng&xY_`BJ5V16JRbs$5y0C4G(`d50l?o01P%ki7!W!Jv~>aDZXgl|qMrhtJ#cSg zy>xFx(brY*126$5zyz286JP>NfC(@GCcp%k025#WOn?b60lGJIZ|L68y`g)6S4`}gls}KA)BzonlOudV<^l*aQZFuTd18-J7L=tZ17bc-5VnZ z-Q+yvryOq5pm=lRgNjj8apnq?LXnzECU>Nptc5ZIsn&7z+DT}*8v_U(6}1#x`3uNfX@y1*8_oiAlLwe zJV4t^K)4ZzYz3lTpmQ4#YXZ8Ofw&LodHp{tH3|QJa!CUdU;<2l2`~XBzyz286JP>N zfC(@GCcp%k025#WMNfd<4ObR%Wf4~vmDtK6-ddWsmM;3OrFjwZBIHHLNe`U#z$Ro9 zvI*IQY(h36o3Q9jn8mx%(T1Q6K^vlAHUv+^AzOu!5cBBWOy``NoM-$@7yb7Q>L>W0 z>y7Ho^+)FARl~vQa!_xw*fE12IFApeU-#Ydzu2s{&Zr)&{9)QRcVg0v0h4C#Qukx9 zH)6JoM%trMA84iKX?R8pGen2I4tuTogznoXbl0BHNx6@``J4JH4S0w0D-F_5vDa9P z(&&u6iNE=umj0*aN$uw^gbHTgw5IkBZQpdfkzwB`8DrGW1l7p-iDKlKOhel@_+l_w z-U1RkfxdTu^C~sz zW%G)@u7Dqa2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b60opgTZ)o4pzM*}?JDjr# z*@SFDHX)mkO~@u>6S4`}gls}KA)BzonlOufqb@t={y!Ek*{_t<`-0;Lz#~z8N%@r-AfxU|;odN15rQF*#g7@K-b$q zJP7o(0*MgN_b!lZ15)ikIt&ba2-rRX?E9hKI6kI&Be<@E2`~XBzyz286JP>NfC(@G zCcp%k025#WOn?a#JprmWTwKJ(MO<7|VvCD-e`(%dy6E?p=0(VhkQX7xJ#gFun~+V& zCS((`3E6~f!lE}}7WJl#LIi~f3K0bpB6uQB+A55Pm`C-dQn5*7(Xtw7SP%3XWzlg5 zj!eXX$+D=KCezMQ&7`R?la>gbEOc4zo56$gt{V%JRcsllhBGTS??zTc7=6LE#mc$H z6?prBPQ1A~>83CfZ?+f5yRjXZXQ}i2S!#dtqh#K_o0ijM=839aQ?3lD8E5J>TF_Qf z^@@Y{jLf^yLqzI7J|gc1-wX~*C*U{?)W!hkF~HRYxVwS+INNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0`zX^-O#(CcSG-nw>M`KvI*IQY(h36n~+V&CS((`3E6~fLN;NEHDMO-MpKwY z;B;H)woo~ta>BML*yyV~dN&mub2Fk5KM-#+w(^2_I5`Zw$y9F26*0yzs@qiM_tU+} zY35BK-J9m&+#6*Uj>ck=uFk$0BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignNE(|UYSEP#;loX)y$IWW@j9oVb#o4j!soyJ6U<{ zJ?63ZPkCa|lo?fzE~=`!Zgfo|yL;2RZ^y(T-J80)jF~qEeDMyCb|f=z@Xg?{yac!# zf%>h0#|t!W1H4T@Q#0W60shy4fFB690HFZT_BId>0+CiA8Ui}s1!8SLS33|713e!C zi3rfw0VJb9>Hv`L1O^TRwxfXkINNfC(@GCcp%k025#WOn?b60Vco%ik|@88?G+m>LRW#DzVi?ytOoMEnW0m zOY<4Z8+)Xs%(>Q z>BiIlXG?SNzohoB8$EfKZfeeCmu@BeL+OU<8;NV>r{gTRsmyk7xLDZqCT z@TY;mB_KEegsT4wf{hIbF9RZWAX)=-I)KCf zfNdjS-vl@|1GQTK=gWZW6~O%}P`@4UyaqJx0K9JiO>Y9ew*ddn|Ekmk-uW-yOOy#P z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`8CqU_j(ha2>N;i~lcyn_$A)Am*$R=bH zvI*IQY(h36n~+V&CS((qSQBQEZbV^LfKzOt*h1fgz6sl=V1uvnDBVnjOd^Z+Dp@o? zCW|$TWl{Zih&anMvglX|w5$di)&spp9J&KXCgQ+kS=3CEMgM)WIQ)PtS|61E0S4z( z;=3;qn!#AOVi8mwgYNjjBjQRFAqw-7W-jtksQ>1}6zTkgZT30?I zwa1)+yt_C3HyL}135 zJ)&~BNrU3ejSnhDO~sijv^fmDxgD40R{WXDf?GH6eAJWbmxzLHl|pYeK(G;#QQd2# z3U#lQ?pF7@YkR#kP5rq{nxyUx(iC-Xla`Q6FffX62n)i&oNM zuftxeKB4>e3Ej0PbW)l6gkCx}R(*$FGO6#-OS1YznKZ0;Gde?WLMtD!bi^Ru>>Svi zck!n8RQ91a4cnFtJ@lq^ZH9OwAl_7CF3<$CRtW1)Of}Q2CR5gfdW%77ZqYzuS*b`Y zwLrKIh&X}hYM|2v#MS^^ZXmuM=&1)14M3j zj!?W2T-Ct@m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyyk(0L2@MHxzFu-cY>Z4bItw zY(h36n~+V&CS((`3E6~fLN+0rkWE-(O_)Wz(G_MDIOP_~Ep$%koUm;QHvB4&;>~mp zy2*LS&vXp=St7JoH~h!G(Y?9;$h^FII5=Gn?#-?EGdE7HnWY#tBA0Ex^0K0wG(Y0a zEJffk=8@%)+(ueed3FPxi_t; zy+gY<)s1?yRn=_9DVmKDnvGWKwwlY;V`G(L1{@QT_`Z!}P{f-A#2b7qNG!*ISQpUM z4aDO>&!<452k7esk_jMn8c6p61O0&Q9AG~WI4%ISUjxo>0M})}eFdmrS_R4GNucp5 z!22}Nv>foQ0Q}DYfoFl>b3kYn(DpnKegTNA1)}SK&KH5$2B2#r5Z?s!Y_6)*B(_u) zeMtd7025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-3xxU`5%i@3C?#FiHE#?rj8 zbkT1t&5Mv1AumGCdElG}HX)mkO~@u>6S4`}ghg+{EaHupIs|nH>JSCfA$TH=*eZ;L zm`CxZ95RV44nH7^)(2&=b*3!pX3OH@99eWfEQ|IkSu{T;Q_i_6=NLuDO3k2pbNvI% zq`6+Yx$*IBYB&`_ocj82n(??o`QoE;-XVT%rE$r?$0SWq_oF-Z#-~*k8-w&$?6puxGSwU9xW;0-rr?Z`7F)F@q5jk3xYnw$Or>$~ zsDAJ@c&)k2s)Mv*o}~IK6BK`CyeYo5%F>^LdUN^U(#hsK)u+bese8wmH8ZW6SyJ8X zjH5HGnz_o+sp@MdE3duBJof%6Pb``;qw3K`RaI8woCWh2EPQ-{)OwS#H~p8^N#>ua z@0XPKPcZ8rTDb6$@7`FwD?`1}TJ%!u+}zb0d@X1!ULdgz=xYL!%|OZrq+bUH{DAE( zz`hf3yaUwk0-U=6*B-$A9#H>2;Mof_egJs)0ZktPzWspzV<2!42p$4LM}W4YK=?Qi z`2>ib06I?svCn|6&w=4aR%u75=fo}QsNfC(@G zCcp%k025#WOn?b60VYuV1gPFny`g$T^@i#V?{Cf~WD~Lp*@SFDHX)mkO~@u>6S4`} zglxhRYr-t*O<7^qfm3gx-a_ky)(P9DV6(6CsNPg6Hi;}+Rs#*|fnK95I_|)ci8wG> z7B$mk+BvG3G!ynJgovvTrouFc}fP-F7=<9y|NeK%`Hofkvt26J$9 z(%q^|qnDgx;IEu)V)zEq;#`=IO%3|=G`PyRhH`a{#$DA+PyaK(oM_hGV?^$ zttnR&pK+#MqXlgxRj)XB&#=NfC(@GCcp%k025#WOn?b60Vco%ik<+i8?Gzjx+1PCDzSA%ystFxD_!*a zO7kM*MaYYgV;(r>flbIJWD~Lp*@SFDHeu15FpG7gp$kD5f-XeCTnL_s6SfMYAm-7! zso;>C5e@jcaSTg&K`fjc#@&1;#9S}OjNao)MZuwVlaqFHm#W_sX4Xw}akLxTd;e{z zTlJ{azU$T3@~++VC$eidt^0ON98$aK_vp>v({2nH{?nWofp&xM1%>5(z_Ayo{Qz+8 z16&^g?)^ah$AIS`(0B;&9s!z;0>0yb{}Ui^0tlW2LZ1O`p9A49fXFEzdIsqH5{R7z zy3PUd^FYr9An`TO_YIJ|45Y3A>7|cCv3U}(*#UbE;BWx7D*NfC(@GCcp%k025#WOn?bcyP6S4`}geBI5S+pB{Vb*|CZK2vie2tQaK%!o03y*wiQRYajtsQ(r^OGO~cx~c~@>~&SY0^ zI*y*6G*acpB-LRUP`Nt(W~?&)#$XZ_A&IPN4)m2(ioWtZ;C=z9UkiBF0gW#L-VH$0 zM!>fT@NWhJTY%upK_W-H) zfb{#oz+S-iAz+UHjt-zU3OEk{u1>&x7^sf{o?}2`7nB=sHNfC(@GCcp%k025#WOrYoqP`TlvA}%W8qM{O8RKy!f^M=wzzo9fQLSBTt2sz(@ z^BvfPY(h36n~+V&CS(&9y$Q1@H-aJrMF@%z1rs57B97K7jDMI%c835eFvAqGp;b`tOs);Rj^V`k?#|FfpeR z-+hVD4Q{iP$3uB1`Lb~25o$NH6d}em+S`2P6hnRe(Rj8}Ib%WXBAgC_kvgeHNJe$9 zkt)=^R=Qi=>#ptf(lqtwGHH^!H%L>|y;1rHm~`Vg@u;PF`(LE?rB9v6yLNN4NjKH; zpfsd*)9lKabYrrZrT%jx&~EU(ps<_-yq^J0p98)x0RJf|0xk8h}w40lH zHWi8=V;+sILix2cZ%S_9P4jRAZ~h7$NMkWci`D%YX|Yvn66!xaj%%$7%TyW%kLm|s zgV&nNtU64JF;7y5#!OI##*8=lw*SS_dKuqWgx&<&-U7lqfyg^RbQjRM8;I=zy50lg?*l!1fy4(u z-##Gu5s=ysq(24*4g$8rfIS8{jsdk@fU_HL#R2!HKz$G3=>-}SfcG@e)Cc(b0e=z* zd<6tkKNfC(@GCcp%k025#WOn?b60Vco%m_YFppn1c! zMO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFDHX)mkO~@uJdJ|?bZ*=q_ z=tIzlD3}ky6LG>;VKl@%nm5xqbdVyk^|w+q?-~{)?sKga47*_Z+L6 zV!l(Aa>hf-xp$0NGt;V>CDqN&I6A|snX4R~s=juz^4fdMWAC5x#G)xPsvccbRb@5K zSulUW!p9d#H#2WqQ+tOtZyHu*m^T9EP4x+l*{Ys&oMH_bO~Rs;I{sht1ohZYlw&6R zXEjL82@N!s%ZkQw1qd(wA80mD0@0^{&ZmLca-eGk5Pt^fc@{`K2lTB1lFtLF7l8Cy zU|=0!s|V~2fWrgSz63ZM0oPW*?FH($0iGtHu^I6CfTq_0pC9nI0D%Ayd>aS_fwop4 z90DTm0?{_0v;BW6HL>vj@P499fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj6g>f| zH&kz^-cY@vdczx>vkBRRY(h36n~+V&CS((`3E6~fLN+0ru*8}$i+ZCi%sO!DE!10R zozOaA+Z1g0RUXxwa_A(oIQ)PtS|60f)|s-Xn=Olrb7ax|uq@iEWYPSXOg-nSo?{dp zE3cV1g9^^o2N+9py>@fs zdyQ0~?zPh0>Rxwkua~B&KbJ|9)V)EPqVA2-UFzN>-Kp-!;AbNS%V?xM8ufuzYF6H< zg#e<%UWdI_eM0x`6S`|p=-|;PZ`Di3#;WhoOD6RldP!EFD3g4{&Ahof>n7jkP4t8R zv2?HfU#b12_N{q0Z(46M^QQmOI?4PqRhW?!b7z8C|IosPk9?=>w64|PDA_bj{{Ar~ zcNjp~#T$GtNG$t+*hfIuejxra&~p$-90K}|0Li03>Nt@81Q<90*ggg9J%FPZs7(OQ z(}1fFaQ6fCNx<_J(3k?e7lEcU;JXC)2Y^8J|3b2{0ik6;n;i((01*ceT?us70D<+_1^aT%bzA8a%)=uu(12y@OTan&vTxc&5SRc(>K0ZjQhnIe03aNIdhoV9cLbH zVBhSUTixDnYQRte)>1G+z*3`BtXOK0TBC+iw3aBfMyMFD)Tp%vs5NT90QFqo1-lzc zXj55jy78JFKP#-HP1e=5uKnisS>Gb+5Y!>4Lrj_u!4Kk)t;w+vO%!j+#3j*H%O;?1 z8_@66Ro5-pGXpzj>8f(BPB+K!lIFlmf-9nnE|OcnxwwCk*%HozysR4&M$h2-9hVF< z@7J3%zgoGmmdS@ss&#{pyMFQJqHr@g;?3@{#G6N8>R3Exi#N$F|JyV84aA!#UvD*C zylFXA73+@NIX>~m3TLU}Pva18a9ogBwgRc=fPw8m`gtJp z0+8JS47~(+8vx%fz~2Z|H32nYAn-a6YzAsufKUXee;a6s0*$RexD9B27l^b2(GH+3 z2DE<#e_E+8HUx(@@1ZlLEVkW2u5$AMH2FwhI6lR)Mxh&S0jiZ_a@I+y?xU;<2l z2`~XBzyz286JP>NfC(@GCcp%kK*1BBcti1q;tjS<`8czld}q(atq}aIwy2a*fx_k`N~A`rj%1|#x>#RYB8mFQbe3Q z=H2{ym*wh%!k&e4T=r2Oi{w?4KXlArNbP3Kc{fdCX*Uxy?e&rk+yT_sR?zD`Se@?GO;ESPaL;3pA_-8ht>x5@_}Vk@Y~d3TUea+G~K=CZIC_#5V)oK_IaW z=&1#gbwFPTNWBOQ)C1|AK&Ao6?gEAy0q<*oZ_nZ~i+}Ipg0Cpx7hnQRfC(@GCcp%k z025#WOn?b60Vco%m;e)C0!)D34ObR%Wf4~vO|g|lystFxD_!vWO7kS-Nyw9sV;(r> zflbIJWD~Lp*@SFDHetb=Fo$hGiHiq549VY!(-F#zyKKs=BUgywnf@ZW9!gn z8(XJ#yRn^|X*avZ(rzYZ+Ksn)v8Q_VziA!suNyR7yQw^#Tf6Bxc53GMv>OM;{Is4P zk9LCtgTm4R_#;5o+dxee2($vhHlX%hAk+@jcK{7Bpz%W>+zB*y0g*TmJq)yU1MNqF zSOVxg4#aza?p`2~1bV&#l6^p5KaffR1E+xW0FW63vT0!GN5Fdq@cj(<&jD2zfSO-` zz!U#gW(ltPH?C-40!)AjFaajO1egF5U;<2l2`~XBzyz286JP>Npx_BmyP6S4`}gj1{ub7(hea@K%TZK2vic4@pIb}HZl}Zgi7Rd9_#1pej96c}fW~3K$wJBrPo)u!;xOC+j{qQH^ z)qm1)H^!sPzqxO7Ta_xg*;P6Bu4!(|eI=F!TJ`-S_RcS{ zER?--jQ7rx_ulEg>F(K&E}uQW{GsLL-c8-E5#w%L z;@z}Z$1&~(2L^}52LvmDT0an457buy4b?zn4G`W0GzWmlW*`~_+O`4hwLq*6=nMhz z7lH12Ah8qZX#kSDfWAf`)dUQLf%NM@rWwe#07DVL`wrmS2l)2`RR@5Y_kh6rK=2?? z`vDL-1k`^7@225ndN-=zYby8!m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28y&JA9 z;>sefESh2~i+ERQ-c`Ebca`Qz$diyKA%{G0$OD^@O~@u>6S4`}glxisH(?I%#z7l` zHUw>mNwXpNL7cBOITE6Y-i`A&?9q(ZAu5KKG%Vd*{iJllG@OOXmdD5&STK|ND5H0C zY0k~mns-wm?3n#{D*+5kARwwfxss~@CZ=*DG>S$s6PfY zd=5080K#7Y&0hkMuYu?{K-;%K`$^b0vD36~IQ51JFaajO1egF5U;<2l2`~XBzyz28 z6JP>NfC&^n0opfQU&QrATwgTB))(>S(!9BJ!EY|jlaMDNPeRUm;H(EWA)Am*$R=bH zvI*IQ1#iL}_KlTJ1f2*v5tHUb@PjyJYjRXX6YZN~7$v$odXKJJ@72}L`*c-(zpltEMG74V{aIj#G85w+#C?mp{Q=n#;|btDj$|i&LsBdB`_8gqwvT$4sA~ z$0GU7G;drE&(xZG6Sf*gjl(c(te&gQn~TPc+2&2#z!Fc_+e@{M4)hzR=53+%a;AA#QDI|#Y>kxyi}{M9%0@nm^T#xi~CmNhyF#ZA(tUA z)vX$_(}C!kQWj_|1ER4E0A~d4e?LhtWK*I|_;|?JF643lI5P1cNz6!MM z2HIZ(Vtatjy+Hg8p!?0GWtPNSOACHWPksR=zyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%sNPV$p?X90hUyJ(a?U1X6S4`}gls}KA)Am*$R=bHvI*IQY{DtlggMlkqRCkY zPQ8VC3#}7cCv2NZ8-Qh^dQ&DgiLP2U0d?Deey6UwZo!@z*fC32m2-94Ifj`u2WHYr zg@|(7H^U~*BC|D|1$lWlWBWC8#FSAcK8nYh?Q>DgnIn3R-NZ++{5HoVKgyH?LW{$r zG^eq(Xr;#1s@-91^S5tqH{L%@yWN;kW6=&+j3-vDX)a!|daRlaTN}1k;|1H_U$9+z z!G*iVnF&sAkztCyMUoM;5`EPJ_Y=r0aeF00g>;4=np{KPeA(_AoerRc@BtQ0J?tx5>G6{_ge)dp9K2WEGx65)-L0^1}4A+ zm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyu1O0OcDlFXHkdE-#v5%Zqq>Y2IGC;J26N zNyw9sCn2XjaM}Z#kWI)YWD~Lp*@SGuf;V9f`Nl#ef=UFHh)Gi-_(2@BH901tiSkV; zXWfiz&JXOH5s%_Yk#X{vdNV$)W<`GD%qVkhbhDkBeDs{Me@^e_Hp91Z!M72AXJWmZ zrm?)6iJ5wn^e^)a9))-FRLw7@dpCor+}=&=p*=H3^=?|X*xgrnH&!?{t$)5!u^HXF z!GXbHSr4SDfPrctT?1q`0oedBv>EVj1$@r|{_Q~3^FYlDKwt+Dd;@WN1HyZN=Dk4V4Iug^(DoM4{tgh^2XyWS;s=24_khIvK+i!S`2o;(2uOVd415fv zKLIjF;N4_DrFWwWzNUg-fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj(7T~`L+^&( z4ZR!Q;G9j!CS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933GTi_Q_cUPPc__3zZWpCv2NZ z8-8V?cf%<+`AzsKRlN_65N<~NbJ4{3H^s`DhjW9eGQa-uD+HPqrVg>mN4UXz=-iAs z>te$H{Ct^(dDz}mp`<;yu8Huz|zG_mp!~xt2~|Cx#>D~YUY^E&8^1o{}-8c<8s+O z4y|tQh`BbSOE)+&2rLO8dmI?*0lZ%TzApj)*Fe=bK+U&6;3N<{4b*-IguVyre*hYO z0vgW%;h%x#b3o(*5d8&cd*aU!Y*qoWCxOm2KzuFGy$(n`1@t@(BsTzk8-dg_z`(OW zdJB-*3S^%HhPDIV5a4_9&t(>W{hzs>feA1HCcp%k025#WOn?b60Vco%m;e)C0!)Aj zFoA+6K` zW&CHyert)a$e?q0uYLGEc(1#t#0HbaJ=2g{ZWXEJW|#M+KYOZwf_wAPw~eN|H`khX zQ<02nqq{d@`<1h8960cb?6}+;92q>8mjVAPK-H^2&2Av@8W7wA)b0gBZvgdg0u65g zjqd>AeL(YmAaVeRz6Z3u540ZyVjlpVhk*D;K=;Q$;uE0f2$1{~==%&v9RmhF2ht~i z%ojlROJL}0z}pY_Qh@&yP&EM53<7~P+?(K!bZocz`zSYdIyks3CO+-47~z) z8v$Pv;12^;uLCvBK%fN(Mu6J4flw5vZ~gZ&OGDeg7yNdf`~pmX2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bcyy4O!E-m8HqA9kti1(J}y`>9&Z)u){JPCOca?}GyJ+KMc zgls}KA)Am*$R;d!6Xp`NtsaL~gslzR-(hR$vOvvwATR_3 zE0#mG@dBaMK)nxWs014QKzKdST(!K+5~*I!l?_aQ2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?a#JOS!A)NiQYP`{ym!+V^w3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEiZx*l z{l+>uYr(0wP;sGoLi2=eGimd$Ow@0Rp_J(A=smh>y;oN|@6%QF{kl5)fUbHU)Ky2h zu9}wURCF#XI!@KK-ZCuUT>b=;X-3#LBmQ}vGES+oXe=9n$TL9nS)gqT(7qLjJqL7d2jb5I-7f%%9YD`Z zK=Ngv?-d~RDlo7cNWTVT_5j(vz|b3jHv;(H2K-T=suied0|M^?!FHgw0|>={`VWDI zPN1<12*-iu!$71Ph#m#n56S4`} zglxisH(?I-rielWg$N1}lO{y)gE(qyay)tfT0Np#h+38>o!^gDIcbqn^)z>ZnE zs+_CS&N0lSIWUt}Dx0osmrFNSKVM|FhO;0)??x99X8NQ(R>)^@ zkrQtm+TiDwD|b3DB&aE6fx>b^6qYZ5*q1=(*FgLmp!-`OaT4e`4J5w<`o0HJKL7(i z0qHY9=4T*#4j8%scq{$_#l{QxR|8c(pr#TC_<`VhptcGKRRi@kK*J`WF#v=&1IN zfC(@GCcp#=o&dEQYB$twsNGPz;oZ&Igls}KA)Am*$R=bHvI*IQY(h36n~+U7#hNgO zc4L{GHQ-cRsJ75Jp>e{tnY77QCTcgOoN_a+2|w^|Mm&lqMZ?Ks-pxqkrdYff4C0x< z#~k_PBh{N?+2fed$D9fN{W-sUnR2)tuxym(G`1G4)Yw|{o_cee@qU+fyRn^ytNfC(@GCcp%k025#WOn?b60Vco%m;e(fd;+v@xW0(%i@3gMimfl=y`_0?>4M)| znkOMoLY{;i^}taNY(h36n~+V&CS((`2@BqYIqVxdod`M+bRs6riQoru%GTtlh$h-M zoN|-jgr8E?```%eX2d@iO^k(8tSot$H=`LgV-De2sjTCDER-LaSi|P(-w*xe>YWq4 z=by{D*)^7Pv*MC-b8%~ybCW#x7ti3q|Ij*~8GOfd=ce*>Zs(@!*r}Q0b8f62yH@dd zPQ51QPWUV5#9z4pq<#Sgp7;;=HLHNklR$P2Ftir%t_OTofWI23ssU;?0f7Jz+ziwP zfzURfz7}Yx0~$j>_(h<(9*FD&q76XXE}*>;h&2J7VIclG(A^9qT7aGikbE2Hivp=u zV4w|1zYApA|D(*3?f8#^-^-I8f(Bt_J_0tE2bms`Xy|H!wM;49C7wS%x1A9Pk|m{M|s+QJ^LP1dap29-y`t2ql5~ zuYiU=ps^nar-0^DKx6=j4gzgyp#4W6mH|4?0`V-+eI7^*0X-Ef;MsVAzSTg=2MkmK zX+Mxz4`i!=p=!Xp5%4{;qRirdb_JI;FaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zCQ$GM=-zO35my&+bL;ZWrs80_Odd0D zuE(YMHG^ad<-lBR<4p9P>z{dZJI2g7JTC3xb{e)$m&Iez4p@vQR;_6+-eL7vH5;}z zY^}x%w!gn%yYzxhyMFQJqHyEV%oT6En^$6S4`}gls}K;S_7a9O8{_a#n#;ZlT;l z=Y-A)+h)>+UzsT0%;TV&yodbE!;qhqGA>3Q_+ww~-dw(Ck*OXIJ}wXU=GXW$S6eF! z6n>8@pT}Z(n>inv|L-Z3@ME|$N^=@pi&koEt=b*N)^=%Y*TkT4c#5={#@3`Ntr+K0gslzR-(hRY&+p1K_&92J1 zcTIC!?klk@(5mkrv3GunWufeyW4w2ky!TG`O?S_JbouQ0z`Gsr zJrDR_0IGHXH7@~ymx16bK<%qQXg5&*+JBZ=8ut7rmo+c}Ccp%k025#WOn?b60Vco% zm;e)C0!)AjFaah|@C4}IaCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3>Vcyk*o16C zHX)mkO~@u>6BfJ)bGSEH9?ZEnMO1^R2Ju3m8gwJ6K_e59+F;Tvts?box0L{T!$2T5lP)Z!Uj=u{4)^H&;KuP7|k8sXWx1ynLB_ zoSMtuQ!0Lq`&y4ud5QK7Y@4ZN--NA(W#iBuxrTis`z;s%WA5ae(Ch!{Y4Sd-b!^%G zoay#W>oq3d44&VrxqoliJDS)#x4P|rT(;~_|Il_S_KmP_+@9%LMYU}_lW%ZjFj<;` zh8CbP0)*cNnxjCZ6^OP0ZSMl@?Le#p=!^mJ4}tDZAkhW%#DU~tpsyQ99R&swK>9e4 z=>f96z)%wKehv7(0sP+rRVRU((?H-mAox8{`vVaA38+5NfC(@GCcp%k025#WOn?b60Vco%m;e)?eM9?(_6_YD+Bdw*Ih&A8 z$R=bHvI*IQY(h36n~+V&CS((`38z>S=CE%lo+yeZ6i+Cg+(_aH|C;z~nrPpYiBO`e zmQ6t2HlW|BtFBwHX9jl6(pBYLoq~=bC(VJJv{ETusmr~atDi42+r(LrmwIzeLFQWG z%y{=N7SU$<@MD2Ir*|{uyqn!)c{dk*8?(Keu=in4YZttmtbecR-c8G?BKHhKuPKIJ zQ!?Gv@AROpw2E~{?i`@1 z1=PO-H0%Q!_XFVrK=XS*Npx_D6yWz?rt}NopqA9krh&Pw!&7})|b7`K0JPCOca@GT9J+KMc zgls}KA)Am*$R;d!6Xx)4EKbG2H+C-6r*^>$!Krv5*lw5$!9?$-l#_18HRT85&4@?w zqo0K2HrFkM!ae3df3xd`-s+YEc2o1;>}))cgzt&H=#- zKP`jaaL+ysz4R3AECS((` z3E6~fLN+0rkWI)YWD~Lp*@SGuDb|EJv>Pt8zi}4YQ=4!q{xvDmId~!9LNHOg;e4C? zX8V+?-UmncHY5JIXkzr6Vr9!ixf$sKjreY+|FcXE)-}nGGWo;;4A?Q$8l^dntwk#} zw${9rn_G?dyR_Sl?KEtiE{n&a9k3WrtXk7tykhlOH5;}zY^}x%w!gn%yYzxhD>7cN zYsYUge!{M~jGwS;y75AhwyQA8O}zFIPwy!xH*Y7uGF`c;Je^y)={k05=9tP&iJ{y~ z7v;tY<;JS@&c9Nx8Qr&i=S3h{5A^K>QVqbsE+E|qWSW3%7#MmT@V)`~ z-UR$_0afn+HT!_Tejs=NsC^Fzy${qM1R6d78V>>CkAUWnfygI7^a#-QDbW5I5IY8R zeh$P>0Nq~zi7$bkuYu$@K;O4;Zc-=d+;G$l6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e(fd;)ZCxT=V&inyw1imfW*9i@3k>4M);nkOMoLY{;i@W25NY(h36n~+V&CS((` z2@BqYIh-5iR9vLbMfydwRd^vd6)%Jv=Rz>ixtRl(L{}Z?@UR__08qk7m1= zrQzt~^llc6rT?r{*6V*PmR~~eCa+O9x515ZV4})ocsB(ab(1{xh-WbJsMhgZ=Xuk; zn`@1_sYph(QN5d{fc;9x#_F+at!ph;?i^jb!I42?830m)z(5*E{|ICE8qPjSMj_}8RLXXS-}3&BM3#`zl-X~ruN6+=uKwr;L| zQaWKC&O&9$W8lp-U7AucWb701D3xDJ_a-m*=Av^mMckW5AOcw+0^MR#{mo9sF zsn&lo_spBRU8_f*c~f0I!o9J399muHxZE2Y89bJ^fov2QY6ZLp0N;Cn|9zn9AW-uG z5I6(`KLTn$211_z^+$k)Pl3kIfbcP(`Eww00*HPAw0#M*e+|UG0Xn}0;wORb(?H@o zpyzua`2*1R6OcLs4EzkF&jFbWK=v14=!w7L`>h6izQ2}P{FQ$#_^JYa0Vco%m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzy#>taCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3 z>Vcyk*o16CHX)mkO~@u>6BfJ)bGSD)r{ZFLF4nixTH%G@RJ;&woD0E3_hudk-Q+#w zXC8+9tW=hZ4-C?}TEDq`&mvP*9DH0J`b}Pf&0-N|9Hu-L$nT+hGj-jY$Kcjjp_&w7 zYs2<;hHK-zZ`+?x|Sc1@al)0#Orx_eW%afEv#(}yZHTE{c@21f>u zWgXyu3aEM-sM!DnHUhzCfZAt)&=#P6E70&9(6}84KMyp&07P~G(U*X>mx1ZeL#9YkU0Qk-vfr;2fQ)B_aWf#1gg4#nm7tL^DdI2HezwCOCo5O5)w z=-w2=C(+f>dvw)$uda69r>pAwb#?XuUG+YwtB!JAH7(KU=Unu2oT_X6CG}?5!MXSZ zlW8vZZmxcQohA-O*W{tz{2HOAtPuLl^$0sZ#lDnAXv{;a$_! zo7QWLy%{{eRdfH|7@MQX*qmG4_CGFL_NRZ$%W2&*LcMW$+*8wWVJ-v{)tfT0Np#h+38>o!^gDIcbqn^)z>ZnEs+_CS&N0lSIWUt}Dyy!nBfOgt z|6F9YhO;0q@5Y4bGfuHu-1$G|$(v31+;RO9sy9V$C&U`NcKjy8tg&k@Y;D+~8!r@T zyT?**M9HyW_{$1oJ55_rVs$CwKRfnYON2!Rox^+W!|%a+-9;rfh%WA#hVybOo|fF~ z3hj8z)AR$>o98baGhMxDIaTDIVSHM#eA;wZzte-l(kj*+xpRE#joUL_Yu#oW&(s?n z88nucfQFZW##eywt3dN^Ao3az-2=4k1=`;LVs8SSZvpXlfbM-jVn5Jx07$+E^t}(H z4gv!o0O>ZkOCS{0pS6l zc@XMNBu({(Q*W366JP>NfC(@GCcp%k025#WOn?b60Vco%m_XqZpn5~~hUyK~8>%6S4`}gls}KA)Am*$R=bHvI(bH6XsBFxY(Z73H2*Q>%^&eA>2?Gf{E%) zDJR{GYswGYn-P!VNl|g~n0Ye}XC@!JW}$dB(?@wMl2<9-U>45Qnt9VSmUwefxG`J2 z3IFhzr*;2-X&v5ge=uFV8BFCCZ(0xSnK7z(Q?YNv%o_#qrg!aC;*DFYcyGiY90$e* zHKi<&Sbh+RNfC(@GCcp#=o&d!g zE-m8HA}%eOVoQs7YiZtEy5P5#=1IttkS8H0J#f+kn~+V&CS((`3E6~f!h$zp4)MnB zR5#Re^BZPl1yvBLAiOr6icXaE*IfuEiZ>i`liz@!Qq}w52;*kNKNn4mfK#kg9@5Qd zg3KuHOr9^z5yeK0c;Jgow0AQn*9@Lw5rQ0({3ufnjE8Y}l;$+H7Om9STD3cj?bK3l zc8#Up{C(!l#jRO0Z`$_%m#6FL$F+_x53l?0?%Rw3H8*2G&0W*nmitO93$*I{N9>(n zVp%AA=NRvuCGWk{ebe2uA6-6se)&Vo%galg4=i20blJm8waU}E)tjzkr)G|+-jo=> zf4U6Iak`Yps#T9LZ`>ZY*5w6S4`}gls}KA)Am* zIK`SUhk3(=?Z0wiJDmWhVssSOY`hSr=7nIQc{2w_iLN@zb=9;)SDRPps^W3jIIAtX z>RJ!9Yy#@G0sT(wx&?b?V8<+7RnFDb;2(5#^d4Qc-mCuxM(32_*e};l7KnQ?astlf zd(4t?^t|kwk!sB4U)9{_D~toz<=9Q0voq!|Ec@s5Zf-OD9G7;xv7Lsk(`E5kv;!97 ziB)Twi&w00LTuRDu(cX5*#7>4?a~W2t;l%6t{uP0_zAn_GJe9Y>Bb91T7gF1#GiiL z)B8ERn|E7ZFx|Vk*2tTRWK|)!GXbH*#IOq z0zJj;c0NP_f>_ecl6Nq;K-Ekms80hH+l1G8Q1iYKnae6n3 zi#nJ96JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wf3pm)QSMO<0Ll|@r*Wf5;I%^OP> z{KnEe33(FoB;=e2&Us)HvI*IQY(h36n~+Ud@FvXR-8eWR#;NEISf-{s@atxM&;;RM zleK1|cjNpGdo<&9h>GDQjUup=PMC(XP}%aBdNY~sQMA8$pC?~Krjo`o(DohKz+sEAlZ0<#??UB2Q*g# z5kC-J542SQ?bSf62I$-b!~;P0W*`v+dbR<{TA;5ENQHob7lCv=klFdSGE27MZv|gc zz%RfAm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28#T$w@6mKZrP`u&I&Dn%(LN+0r zkWI)YWD~Lp*@SFDHX)mkO*qAxFo$?U=Y-A)<*OpfS5CzX;fA^pOcZbCamY>H1AgXV zz|Tr$yZAt@j3VLSrA4NCIQY2y-5bbxE0uk69`odbIQGWn@YpVCICf2j*El>y+Dv2X z&}JK3=cREslRWn3F?_hy;euu5ajP5#z(5K}p8_%iKz0xq zN(0`X0N)wtH~ycg-*D~?6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e(fd;-*OxWI@D zjJUvPiY+kW?WK8p>4M*0nkOMoLY{=2_P}WmY(h36n~+V&CS((`2@BqYIrJMVM~gTW z?F&Wwf)@fWgd1sMl8O3FF_aQr9lb|at@r9`=Y6`WzF${oAJA3rgSzS{*HzOJor=yy zMaQYS)?0=JoXekJGR=g_IHk&xhkcX7xLGK6jAMcxi{v*`y_s_AP1tHEH4a0u$)(<0 zRBp^xZ@k&Rd8#{BY8~IKI%&Fk(|V1`H-qQ5YVO|~29O2==&oDc_CGFL_NNl(14|b# zUH0%&t-5-IdgJi8wASs`@l3wKkwIfQ5BP_Gs*07+Y`j2VH4yXxwUt1~57e&*8mfTC zY9L$#G;abT0U){=XbS@E+kjXt&{+q>LqPY7K%yS#*$E^YfWBQosu37y0@7h1^E!}i z28LPy?^}TH9l*a2sM-(I8~_6Ett_(y-(Sg_iZTHvzyz286JP>NfC(@GCcp%k025#W zOn?b60VYuJ1gPFny`g$T^@i#V?{dy2WD~Lp*@SFDHX)mkO~@u>6S4`}glxho)`U6K z8(Jr{PN-kmX`S3i)(QWb_-mS|-js<=qN|ooK;1T=->IvvTd-#acFfXMYA)?&&&9I5H$ZQQ~L0;a?*nZ6%F=bpPK8oeHQN5XR>do%4)Em>L-rQ+8FSp6m zn_FBp9Vey0bGrB$pua_9Km8<)qe)qOQC_XdXs zk0k~KKLlz!flwDv9|sx^1C8B4_$bhv03yeMXb;fV3$!PJ*jGSj9}w>cx>G>n6wosO zBnN@MG?4lc7{~zWvp^;bWX}UbLxA^*3JXM=RYL!h!m2gGnzh2fI$`iBVeQkx&<0`s z#tKWBrQw+hOTiZy@Cz^jCcp%k025#WOn?b60Vco%m;e)C0!)AjFaah&4TlSjxX_3T zji%T_Bi?A5H<~W^jiz}L@+9O*$hi-k`@kk-6S4`}gls}KA)Bz^O_)Q&u{aei3|bgA zS{S?#oQfC1jdLNGsNs}y>dm+&{=mZ-@hFx(n16H4hbdZ49`kR;C)ljGZiPmlbOQ&@ zPnmq^*v65$?rcv2|#(jjdC=-PpP= zZKq-DbXhzWt=8HLRH1#ftH`LJ)8~O<#$H^YXEv=@!mlDz}BxdT7s#Q7xR-Eh8r2 zxINRgikEHUnSdk529ss8upuaH+$Idy3Y+VMk&rO@qOh%A*uGO3YY=ws62=>a-A%$o zSlIKrFxf2ZYZ0a*!hyGi>8LQ%D$KSChu#%>-xvB03jH4ls}2ckJ`x5#76v~N)*cat zJ{8t~CTut+Z2VjpJ|S%WLJXbAmo#)NipxKk025#WOn?b60Vco%m;e)C0!)AjFaajO z1eid<6QH3(Lx+YA4ILUfyeT@HkWI)YWD~Lp*@SFDHX)mkO~@u>6S4`XSQF+jbhsw} z##xh3!@{Zf*Q9gj;Dvw-!9+uclX3E!3RJ3k9~`0PjQHmwH6f->v9ji2=Umsc8bh_2 zzEX|ju|nQ_T_5@@ete49IsXbH#sP=sdhMLl1lT#d3S;MlA?A2m!%x7@@xLE7-Oi~z zo!idoI(BO2`0N}7JEu2#)r_3!TG#dw%8tY1(3(aQwj4*Ll*pn3|&kOs8gsF-rzyYr?y;_*@3A2^LA-~Z3l+gFI(7!=gwNY5} zj4<%5Ft|lnyHyx^PFTNP*zmlt@daUchp_o2VdQ0D^c7*-tHSo(Pr%NJz4ip>=g`h! z0!)AjFaajO1egF5U;<2l2`~XBzyz286JP>`Pk?p~*B^2H5!WA0vGqs1O*L;*UGUpf z^CaX+$diy$BRDmJO~@u>6S4`}gls}KVZoa)hn=IGip%-AoWF=(3oitx;)QVITnHxG zIdfp0=&GY!S4~TFwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq z@6lE3z4~up4o?}5eWkJvKNiUIk>hkO-(!}rqvvJlj6<#&)5Iwj`$moVm?Q5oQ_?Ax z-!}Gv%#rsLeu~ai_H@RaqVotOBa6qSUEEI7R+Ly>%J|QY{nip;kwNG1Ui8kL;(>WQAneORaYl==qGOCU4>4fd>5@X!YbQ$;KbSd~%UAJ60 zL}$9VJBP2O>Byl$XL(&1YZi942;&i9_uIlmRM^uhOtuO8-W8_Wg##VJbWE7}P?+r$ z4s{8=p9pcwVQPXk$hP9jVDo4f8n1&sltDlrkn6sJkj|Kb&WcNd&c&@+ zf=(Q6j;FU%1f36Slcozg{U>t|(y80Edh|g$75hdAI#!QeYx;l3Bk0JXL1gg?6RU+i zK4G#_*yk6f)(Z!!gz0Kwrbd|EBpeC|y<3F7twR5E!m91Un&*Xq7lgqb!rGUFp_hgA zuLv7n6*le`hF=pl?-54!3Zrib+ujtmza@;lBkbHKjPDnA9}p(q6ZX6>Odb^Wec*+l zlRD((9G$}7BelY!3j05P0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3_zCqO}mOOUt( ziA#{C*b*e(u9~;2F8J-Lc@pv@)bg{4$Xe?;gl!TbX*RPO}oR;Z|vG!V_T%nG`0?Hwy||;1<`aK!-rcv4y_1V8@9j0 z*5YwqnxEs+%++*~FmybF+g3r-dDWktG)<>9b8vJ`r)#hMN+rh*O$YPAuG~5LARRe2 z_$*z*R9rZ4SeWh>W{wK83E|Ljq4$K)_l3~^rLgL2Va+$fz_-HSNn!12Vdy(y{rAF# zAB2rR3BzZE%|8nx=Y-J^MR#{mo9sFsn&XpQ96U?w`%U+8$Ykf&)@2{|8d!} zKmFqe2SZmmIu5Pwn{hcha%^x}YK7T4;ZR8E-68b7B=o;5ta?RQ^QthgTNr#zSi46U z+AFMoL)h@9u<3BYIx zUG+P4)pZN@%)pLWx~iP3Q{yodra4fURw|pWY?s?PS3h55wy(1wKS4(qW5zLQj}`J+ z9HNug5S_^sbaod;(DA~}@l?OL8iLMuU%YI(pwn`y$UVdGaEjsKluUQ^J3Z(ttzzAg zJI5#JI6N+`VwZJ1Lv-ZWAhMhm`o9xaeJ`x}K^XW+7(64a{aF|~C#=69Z1_dk_{3^3 zyh_;oq%g8Z7+ouDTPJLPN*H@u*ttO%-ze;UMwob3*t12L+$!vQPMF#*9C%)senFVo zAzp(y*u;D#nVn^_nkOMoLY{=29l_ZVY(h36n~+V&CS((` z2@BqYIV2ssQ*l*4SM^(|x9~!6DqaXT&V^v2q{Fc}`Hc)JRlN_6uyaQIbJ4`eI>pM8 zhoCc>i?c{f8n?NRh4Kq&=S(>}2h(93@M|2JSU6T}i?Fp}`#WR5^OBWgww)7#nB!?W zB6iMeZ+>RFol|)_x1G~NfC(@GCcp%k025#WOn?b60Vco%3ZDS&9NIawb7<$#&f)FS*@SFDHX)mkO~@u> z6S4`}gls}KA)Am*IK`SUhn>Uq_&3gaeA*RG#lI$fI~y+qTnHxGIdfp0=&GY!S4~TF zwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq@6lE3z4~up6i*qB zeWkJtKNiUIkrQ<;-(!}rqvvJlj6=YgXOdjwoODA6PEeVA=*W3N6TPQQIpDWAJW6vK zTZ{Iu#@4FcVQg)ewsvi<@$VvSrm=NsvyH7&yWQBjE^Vh_>x4&R(NY%UiB)TopR{_c znhjeUwpQZ>+uvWXU3$R=Bb91T7gFBgyH9ST3_%%(y7_; zz3Gz9wMOVvB%|7>l1@`##0VV)NvHP*%axK&&YcdAOB;OJ3Yi7R2ASnkVdOJm^q8>i zb7A`lVeAWG=a<6x*TU{^go$s3Jtu|9)55;*gsJa^13w7UKM6BugxQ~kL+6CvA)&9r z2l~Cjs@1|8pD<7<4Elw&>xH2zVSTl*p+?xaNf-_Yn>Pz1L1A>8u&q|uUgv|P6ASq` zLZ|QxFbaNuR(=5{zyz286JP>NfC(@GCcp%k025#WOn?b6fr2MMNr%gjxD1KQkfzu& zB;Ki-cd9PbCfg?bAw z1gGMKaN}GECQ3Ta->`HuUg4-164S7ubM=$b33GN9Dr+8NbgpNjjasS3{V0*moomdW52aTSY*&Syw^Vb9=z9G zRAO^g&UKf%txA>Lj;AFaR~zIUPuKTi==@N<_J6u>GhVzIFWxoHZMm<+vOuf8f5hJT zC6&PdyG!quGOQD(P`ac zcb6Eyf4Y28r%S~mnB($tmCjtdF3wm{F31B1r0Ll}EW*!i+B{)({s zRbgVcu;(>la*wcYuQ2t7aNtd0`YmDR9btB#aA?2K`>xQ}F7$T@t75{M4~2nFVX#YB z8yAKS3+uau4M&BI31RrSu(?MV=@mwk!nUu3?R~;nzpyhUjGq#A4~U+V7^I%V89GdW z2`~XBzyz286JP>NfC(@GCcp%k025#WOkfHmKs|?g4)q-BIn;A_zjQVsn~+V&CS((` z3E6~fLN+0rkWI)YWD`!YCd{Gda6$f!vml>(g;VjbN#U-D7XmH>6ZM>VoST#P%%FLg z8MIPaFFqhWUTx@HzGsoCwhlfn4?}0%icL-{XMuP(mkx9ECyV86=6q=WzsJyma!52e z%^I_IUcaGpn_>94v|EksB+bxy3?F9oI52fb#_~9{-(hR2xoX!eH++1$wT>bnyeVtOJ z@(^_LvTAaE|50|9%Awh>^(d8>=;&a&j!nD6aBu9|Tw`0L%`~YLn^cgsu2c z3!I!su+8P@$bR#u=p>=%cm|)Ugrl>!zr}P%r}Y|BbOz6F)!e@~CgNx^h38hc{g2C* z{pla$cPjSTuN;qKM>JhKEcr`1a%7NM_6t)7gahvh)9(v22Zh-WghPjf-Y%gpF7zK3 zR&@(&jtT<_Veq)HwnrH171k$(4POZx`-I_sVRK3tIVFq^2-^mQ?P+1`M`3417(XlQ z&I%Lfg*`*UWJM*|=M|<_3kQ6{bfqxkuY{zNU0+E_r|=6fDlDq7|Kk^60!)AjFaajO z1egF5U;<2l2`~XBzyz286DW8BlyoTRP|~5KLrI6XOlK3a3E6~fLN+0rkWI)YWD~Lp z*@SFDHsKU&!W@zgm*w9$%kn8%I2Hez)b8xO5O5)wDCv}mccQD7O+ei?px>#hu3NBY z26oKSRpnfr9*^NL&4I%NAw?f+$jls|5Z4=hi3Ilb*U`SZ|qA*l1 ztlueYXb?8;5{4Uv%}v5cSQvd>*w!p;ZxO~K!p^sa@u;x7RhVcK_Pi@hwhQ|@gsGTt z;6q`$Q<&)zX5+%4!$R+;Lf>bi=lG9N&#_R?VFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wg-?Kb4i_MC0TLG=O|bto)N@KXH)mWkgRVAGiYG z&`ju)GWt1}#^#JYK4&uhoTkF~IbNtap6Z6R@N>@W>o?uc8BFE&b6OAWnK7!LGZ-2% zKF8&8X{{~R{Kx0Wk-=l>5&Cy~3gfGV-9BNWQrP1cCf5u5s)VU(;XsWry-An}2(z1oLqVbU zIiYX6(Eq%!>IGrV4q@P>weWL-FR$hJoWifYDEPfu`30B&6JP>NfC(@GCcp%k025#W zOn?b60Vco%3Z4M{9Qrx*bLi*L&*81o*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IK`SU zho8e0`8UpreEJnm#lI$vI|nZWTnHxmIh>c1->jfg)%)PpZqDU<7EO$&Q><)xxH+zs zYEmC%@^PG?(|#Qv)r>Q+HO}oTIN){9o?dj6pXyW6;fA)7+N(N-PVs>ib9R zonK;ED0}A^@0}&@z0-Zu-LoHEK6`%oL(9v{OPmiZUA%PJ!%MZw)4AQ8u4AWWj@Qj` zYjqdK;pWJZ!C~1Y3^odDn}ngTu>N&nL$k24MHr3NfC(@GCcp%k025#WOn?b60Vco%ra%I8 zbGZ75tB<()Xo{^q;w`Fqi|T^kqM9cmPePuAoEX805o|&>A)Am*$R=bHvIz^`ggM+C z<+^HGqN~j-bXD;<%Pn=W@2_FX&$?brLqco?nwP^opY^|J~qg2W5&}vFNt`O85 zPgA=ee$JcEZZ_S|xz_BQieyw9)z4`PjF_F{_Dt6*+HCpH&XFU7$C4E`oEJ6@3BwhB zu-Pk&tQJOn!nR6byI&YvFYK%m#;b+hHNwOuVNXDq+$`)13RByJ1GU0*oiGy;W?vKz z)eF5Z3w^H${jUnEb_;7>69)DOgL{RwZwNzg3hUnzHoPNj+$Rk07d9X8!_SGl=jZGk z&dy;1On?b60Vco%m;e)C0!)AjFaajO1egF5U;>3tfPN1B9Qrx*bLi*rKIv>iHX)mk zO~@u>6S4`}gls}KA)Am*$R?a(O_;;a;fnklXGK2!3a8>6a5_LZ`ijP zuWM8ce`#3Gx%x@zgy}j9l_ih)IoA|&N=2lZJ`s;n`L$GZrrZdfrm;roJOaPS;&Ev! zjO{dSMTym=jQ{M|Z!HlP8FUWswGY1s?{ycI*x=>3OO44lmlT~a>>N+)rgczsf~%TL zS9JPM<{qI_w`=w2BXp{(M<_Z9icarNi@U`5_R~c$a=Mhqs#VlC(k31e|#=V4*ITiAV6m`DhFjti4L!oFT%Dk&WJN|^2w zX8MKMlyK;j(EFXx_r1{ngRtr+Va*v~;Adg*oUrzSF!YPC{)u&9!zy9plfv*CVe?vH zWSubjl(6k-Vf%)4aCBlD*U`}_{Q8Ruiz@8@_yw2%6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%3Z4KR9j-#+DkQE#nqsSvc&lpOs=DB}s^&?^laMDNCr5B{1e=ge$R=bHvI*IQ zY{G&!VGc*f=2TqN&qe)q+AX{goQfC1jdLNG=;+Mj+?>2;2F=6Fpq0vU@qxKqS8F<# z?^$H3vV)JyL(|Di#93^fShFB+L1(eN(`w3Nf&89--Z4728RK`RX}7zb5MeA@%3?gR zYE5(TiWQ=W4O<(wR^tWR-(Rp@dcmd@886ti<2M;UVb^5DkIiG(bmN60ZP?Ly%y@$X zKOt{$XureO;&EPbbIf*h+Mwrnx{~7PoEvz5(j1-E%)!weow|+ot96{N|8eC$Nxu62 zYbZK$WKdZG!q{eEXHXd5ChV>iChCMeAz|`GVPCy4wNp6IAWZKPW*UXrCgD(6=-n&y zy&?3!DXe-+So4lBuumA=FRVQv4813;e_z;eP}ul^FnmbZ{E;y7u`v3Hu`Pk@RJ z6&)%%RCK85@RsRpLN+0rkWI)YWD~Lp*@SFDHX)mkO~@vkVojJs(cz-}8)s2I6$_{0 zUz5_Eg%<)Y1QQjVVrVD2I(m<;TJP1>&iiy#eZQ{GKA@}K2X)m^uB)acIyIh)8jn+T zt-oaG468a9pJ33<<$BK5&#zP1DOJ`y44q#y(5B2hcg7R}XUtzx_RnePTr_iB+GH3y z;jyOZT%4X`wxJV;oa5rsxcw->SKPZw$ZDF#P7OTiy0QE?f4e z66XU;7cX7*@KUXH%Lqfq3Y)2E$v6xhIWm|mUkMX^!k&I%G9~OgB}@$n2L^@dv@r9d zFq;t$ofUe|34IrY{$GSuPoPmOHLHYyCxyW^!rHaM&^lrLQ^JO)g^e48;f=!PXM~Yw zh0!g-wynbU=Y+BC!p`S~@fU>MJA{dsggq|{ldlN-UR@7EC$)P$hv*c3^+mz&(8@2s z1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$GMXy|Ya64xMc4bl`_gT#AP^Ip{jzgIO+ zLY{;?2{}4~qa)aaY(h36n~+V&CS(&9ya{s{Iz>*!CH-8|@1Wbl3js4Y=(gN27lMg~ zPMJt2x@y@3)NKR$ox19}1$$;-$1Gh{&ebXL7y{EA2u$!tbTLG7dpZ~QFEZQK!PJ;M z^c)i=)Sx0(DtrEqdGclxK6hNdgoe(TGjw*3W#~L&%+fL2&`Cnh@eB@$q4Q4r8Pg4& zmQzLU8HR{c3=yYfx~t#mL1k$b>yF$xrlE7I@q_;&Kj?DVF;1vDY#E25BZmf+B`i$6 zE*xkUrdx!Wh%o!Ma40JD?iczF2>tH~tKJvZ925pV5C#tkYd;c(J{H!0B5XJ!Z2VLh z{!G|>Oc?oG7(F3u`$E|Mr7-rju=5*X{99r7Nnzr&u;)8r@_S+555m+>!htiw^v}Y~ zIZ<@77pUlPhz=8A0!)AjFaajO1egF5U;<2l2`~XBzyz286PN-CP|=~HLq&&*4iz2V zFr7`vCS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933DhqT$E2cg_@S4o#Iry5N@ao!9+!; zl!J4|H8kjIo1}PBl$|^ropHD|`B*p$#lLZk@>nFVQqXZZJT~nPI4=&2W|?bji?o@> z)}hTdwodJKV>>w`behHzbS7qmP8x2Gr{d645Oiw8Pya9XZH5$cvy8Qw=C<5dVp*V7 z-#=pS{1VGT**nL0?<{%mo$j0Np8e?Z+4IXET3%lMk4ZR#soa82>!Cd}Miq1__SxN6 z2s#drOB>9LN6?W&gUC|x6qxl2hgJ){YlObFLjO8p)l?K8sAv%>l< z!iKHF#^;3L?ZW2gg^?G8(H+9JmxS#v3uCVcJ6{#XcMH2;6DIZud-e*GZwUL|6sF!1 z4!k2w?-OSB3$q7=L+=T_9YSC1DF{0L51--;9nR2U0!)AjFaajO1egF5U;<2l2`~XB zzyz286JP>`Pk@3BmmqNo5|6S4`} zgls}KVZoa)hoED3svBxq`VBJzgUSe%5nh{4MR&^j>n;Qn1sx8}$!}m#sp@@jbRA{U z#E3e@O64Kvj3&~I;@IT*(i~B7Tq7REvWI@ol=E|TjpgS|%m5uP)ErOs)++cpzdZMn z>3&Y->D+!!*RfMG$LHs`JZ`P)E$cW2=*XeLWBEwv|5#Y{iLmB~Fz~4`_?fWwm@xFY zu>ORw;R|8om%{MZ!sc&;k#B|3lft&s!uIckvG0YQKM3PL3A@h-6F&=k&IywjgnhpV zQ%_WZ1FMATCxw|c!t7e%&^n>_|7Y+0W1Fh~_}_PbY^M}H+EtOG`}l#F5dOJW?)BcU z$#N5un`8X^g~|QH@Rxs(d=`_h5pGDhYCbW{iggw&vp|IcWfrVhp^TzqRxP#agd%fR zm{@Smiena-vueeP^S+-4oG{APLAUm^&oh+ADW@&ndG4I&U+>3xxj|^`OTK zdN;bIpX2kn>E~2_?M0J8RQ3;l04Bf$m;e)C0!)AjFaajO1egF5U;<2l2{3_*CqO@k zeh&Q{`Z@G-c(ZgiA)Am*$R=bHvI*IQY(h36n~+V&CS(&`O-Vl#KA^Qv^}PYjVm@d- zBX`WzMDqeo4Bf4Xr?9{WlmMjngsd@MtyT*{7ClA|+Ppc&=Ul<}V{ zpypULVIBMMC*#!y1U_{~dhxp&ghd~KBUDA-8q_=Wt?|;}91hU2+bkaG=Ga=lgP_y* z{DHCyI@cPY)0By+V+uO0yG9Mru|m*snkFFV;LxPVvK91g1N|?9f$d=M6)^NF7=8_m zybiYS2BUkx*j_NP5A1vsOzsD}-U3r^gFWwp=>uTzdtl~$u>V6a`w=*J2+SP@^GCqK zC*bg>pyMd$>;YZJL3bMToCLkSpsx?~XTZQIFxU@4Cp17o$3Q`c2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bgJ^>0kT!O?UNL+$cWlNBFpK9Kxy5je#=1IttkS8I>MsREd zn~+V&CS((`3E6~f!iqOx5kaSx6K51b$-q#ZlEL+j4xuu_-zICVjDn8xUo6~AR5q%U z0@H|{bM=$@N%M6U3sVV0bfhjbrH=**Iuvw-u+O$MFOY&xK)N}$w*70Rp!43(J!Kbk z2F?_ppwqf*)z}ksh5}a!IVEY?urJxgSU&{$Pm0x^O@!PZV2VeqBfC(@G zCcp%k025#WOn?b60Vco%m;e)~cmfo3DCkhop`b%Shj&Y76S4`}gls}KA)Am*$R=bH zvI*IQY(h3+l{H}zL5FS%-4se%HI%dz!3$wpT?l0qbQW@IPRSF47Rrf1D~0LeL)OaG zj?U$KmXy`kkzZH3qoY8bnKdNfC(@GCcp%k z025#WOn?b60Vco%DxUx~9WF%TLL@Fks)JgS36UXn>MEpKY?0i5I zEe~qq{6m`Pdsq_(8#U4Pm`0W7qROKPxrl#6)VchLoOE-!qI3208L%RnfNw^-g_jEP0a7t@`^d>TG>GLvG>O z7}Tsme_~YI7RXnOHlu2iTa(-x^%qQkzF@laf=LQD<0v|jKNTIQI7X@Jj8=3msyXFW zbQ03ev2}k2Mdz6#&i}=FtFFM@Bo&yuW>^gm*bR%+mIp`ez29zFjJ@;q_s+$8@379i zXWo;`=iT4<$nwTUyYkS|N0%;pe5o40#w4Ag3!7EzKlIP5`1~za^FNm@``2F|Y-zbl z(J`tm|Kmy<$1D}6)-BgkbZ}@=WqA%vJ`Z-S22(G9J!`@Ai(v0MFtY*de+kTP0tYvP zxvgM+8(4T59NrE(f}nFJ=n8@EU7#lndfPx>1oXcF2HL@32N;Ti;dj7D3~Y~s(F7R# z08Dg(od>~W672dIOm%@h-B5JWDJnXK$}hjD_-$JG126$5zyz286JP>NfC(@GCcp%k z025#WOn?bgJOL^?RCK85P|=~H!@H)l3E6~fLN+0rkWI)YWD~Lp*@SFDHX)m^%9^l< zqC-1{b_z8uGwqb=WT)`ADSw+~RCF3(ooJ$Cy@Xt$u89Ff6Wuqj;E&~X-Y4uY;B(47N4KY`vn z=sOSk3t->^7#s#eO)pA@9bjY?*zN?Q&0x$0Cf0zRZZO#bc6q?mda%a}rZ>7J$diyKA!kQ$b_AP{O~@u>6S4`}glxi!H(?P)$Djxr8ZtJCwIJ^gRL_p^opsO8pcYvNK=zRzD#Xx@?3?#td z2VkfZ3?Bp|NwEE6Fxmyiy1_&W>^ur4d%&*aU@8svoCMRoU~eCo$$WkN^!G8agy|Xz0+; z;SJN-gls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb(BYc=>9Z!EhJ_;d+oW?>%L_qX z2xT;MI5nrVi9z+E@8MBu&ZvJbnH*C`4yq}^&M}W6wT%0cuG-YD6cc!?z?*OAORwYO z|DkryY-_Eh)@D&JZfD4?VlmhZYSy4XF{*6~yNnpsLIwK@ix`GO?Aa@Q_Yi*Cm~NlPL1Hy2sR;`kWI)YWD~Lp*@P8u!XkE# zPz0Crb2)zvy%t^wir|GXeJ+GD+Bx&3b)t!bjhbkCOcNVdXrk#UY3!^rXrgj?$T-uCN;|LjP-)`;B(6 zMyE2e*F5qb`Ce;{-DGKAV6C?rg|N?-Pwh5~L;5+kmNo07r1RW`Z_6&}Tx*0*QzoX4 zE$Ku?jnJ{!tZK`_2}wFQHYu~jL01BFe*k(qLGMA(mjwMEgMltE*bRnKVE8B)=>gl1 zgV8h?I|(Lw!OlJ~nE|^_fvJA5X8=rR!QQiAW)SQj0<$@A@Fy^r2lMB_LIE7U06P8v zolmcmbUg#Qp9MY7f!^oWNlC}IdL1PlN;*t{2`~XBzyz286JP>NfC(@GCcp%k025#W zOrY`!P|~5KLrI5{4kaDlE}c!tCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrgheDBF3X=j z%kn8%D1yIDYIjCn2=YQGqokuq@kClUb2ZVtKodiEYvTC5nuy=0iJcE+V2u$q)d%|G44eX78nK!`x zb}-uk4o1P;J77Kr7UJM=0(5)`IzIwkhd}pX&~pUzeggVF1^u6afn#9sa~L|IFKFl( zD!=-o;`eCf55NSN025#WOn?b60Vco%m;e)C0!)AjFaah|@dRk-a19dIAaM;+m90VI z-Ku%F>Wbg3nkOMoLY{;i9>L)eY(h36n~+V&CS((`2`k=&MGPI2BDkcVOZv@pTX-QT zf)~Q{xe&@|=q%*uoRY@|EtCe+N+B*jWT%WWbVmJiNm+#*`E?~oIz>F1tIe7l-?I>I z&a8_M1yAVdRIjJ=Z#gl?XscCg&$!RJ-K||{f&<-Z)~(H4=sIk z>9WU{s@L*#;`u{kdpg~3j`DP7Nl&Nm=!85S9Gmo6PJ*FcFx&@5GGO~DFxn5s2EarX z>^uu52f?l(FqH#)ege~Zu=hNeDS-VK!0a$M*tA|U=K%Aoz=9JTZU!BzLFWsgYc1%0 z5%jDBy&FK^OQ3%f7}yL3w}PQOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;<2_@(Iw>p{GMnhn@~S9o{vaO~@u>6S4`}gls}KA)Am*$R=bH zvI*IQRn~+>JRPpgpFS(|=~*a(zfGEV23`pALMWrBQ^)B!rB4o0ME4rQh^ce=6FKW< zl%_N4pEqdj)C*Gyn$9J4X2hYH>?aL$bky6V_EKvzUD}$}1^T~h)Z6uKtvXNND(Y-~ zYq_+YA-9UfU^A#$gZ{**w&9aTn^85%tx0Z;`U|E%Uoc&I!6c0veAKL-oT-1ptXlL> zm{m=Gp+=1u*!d$Ao#f9{ zbT~?f2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgK>}2CxCn`hkhln`$`&E6S4`}gcWbXB8pCpBDkiXYx-*`x9~zx1TTc? zb0L&b(P@Bnq6xX8TobKZBnA{sbl)uZ%#l0hYNC08MvF(cc;<_p53Cd>O_(m1bFO~A zq};mBqSE{v4Z_T&K|fRQWCgyjn}uG!Q#$buhde zjO+p1_kz)VVC+pWu^;Sw3rxNZcD)Ox4uC!HZIFIW`uzNfC(@GCcp%k028Qq0`zm}=g`lgpF=-~w@YUevI*IQY(h36n~+V& zCS((`3E6~fLN;NQHDM7yhb!`@&x(Bd6^h_*lg6Fkg&;44GWt37QaaH@%Y&LY|Bxp7 z9@fOcMoqLmriqO!G|}{w6n0igYljlgC4or_I-?$SQ{wBCFhFOtno~EPCp2ox&vpEB zzA#Rinlti64RmzIo1xP-o}+`=ItDo(=He`!8R`nV(IWJ}X1U*J2WxaHBYVvw?~(7d z*4Rx_W3krjM$2p%EjCL^>N&PPk5@W6XT4vR-O(A!7I$>wNA}Gb)6r>ZF<+_P807$> zHs>fGXKYCa$0lW#PB48C>`j81kHP*fFxw3droh}$Fy8|fj)TK#(D5bc{0ek^4Z6Pp zJ>P=f)1dD=(0>LDoCAa3gP|Y5@Q+~RXR!SjF#0PP`wdL|4tD+lCZG07c0B{8o&|fJ z1Jlogy{o~@3t<0RuatDMFM28IP|{%nOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p- zfRYZEA#oWJmmyWzG9=!sn)j-%_`Rxm67nSENyyO=938rIkHZjFq9zZNLS{X zk0p3Xj=Cx5WAW6^&bcV){D;~(v!`%^&aUz7oQqaYx$T^c6mx7teXw(mIDRg>ozr}- zxSi8|;_U4cvvbUHT29O13E4R~Hfgb}2eV#qa3h%Wf%z?9!4D3%f{yK=^A*tbD(HR< z^t=vwcZ0q?pnoqI*arsR1Vj75@LOQyZLs}aFnR!ty$2@V2RlCmlOKUyhrrZfu;&Pv z{sip(6wG`E_8$YYpM!&6fVmT3{!7?7g|BGm7%IQ=qT=^wRU__Qk&!QUo*I}a|AbG-YDyn37JLchm@-S~)#vXsz+exjJKy(7~}unI!`jPJzSypyNBxc?NWy1Kr<) zo*zK(kD%{o(Eke<_!SKP28MnI!+(I0r(cq6e+G;`3&x%U6VHR4tHI<8VAonO^&;4_ z4oq(VdtU-Go523fV0J4wxDCv`4Cc3kg;&7gS3ySzbnbddN;O~@u>6S4`}glxi!H(?P;r&bYM)z4M^2I?)m5EQ`+VftJMWt4OjDV}Je zb*?6w7iePWZcQA&R}=C3G_msmO|(3yiSrL>6nS)!r%r5KF=FUk{iJ@_SsG+Fk7u6>~UP0+I+^u7i9-Uj{ef`J2I@I5f}J{bNGjC=&P z9|EI?!PpTn@d?=ZDVY2W>^cUfJ_mcg0MjSH-Y>z-S786wVD=ku@LMo<8q9wO7S4df z=Rn60=*)qxpFnpW^qdF11sFQM3p8{%Mu!P70Vco%m;e)C0!)AjFaajO1egF5U;<2l z2~G<0a_(9ofwLqmtROlK3a3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%=zYx1Yh zntU1-ir{aP&Rq>J1bHEp(a>2atrJZ&J*A0(RR&FTuaW3jFVVV1VnC6*ZkBs!@koPd zp){CQ3gg9x^d7I4bS~esq^!b@{JIh(oe5huMa7&&P;acIJRZf{u1B=Fkxx<5DP7X} zx3$({wAHFLa%+;?zss$`rl^x6>3F1{V{1LUQA#>PAHFkXl1@B-XlzNRDL6{fvD#*- zZJSIJ8KZ+^lQPR6pzrC8lKyAFz_VcRIWY7*7+wuVUI5$Ig3%Yj*g7z=0qlGUOl|_Z zHiN0HV9z!%{W93Q9n8D}_P+{dUjqkU2Xni@{2s8d7aZOPI@&>J2k44|?sq^>4D`l9 zUjp=h00uh2;K7Yj(g`Iua*R&pZ;{$$5S9JIAAkuk0Vco%m;e)C0!)AjFaajO1egF5 zU;<2_;t5dF;W8vHL*g=|DqDucyH)dU)fK;6HBUmGgggm3Jc7d`*o16CHX)mkO~@u> z6IQ$li%2>~MQ~L=SM>|(ExZsE!3$yfTnJ^9bm}-dr}VKw3M8fxP3Q6_a?s7?g3i^? zZ&2B(7or45rzEka6#3@z_tZngnRTs4Jzk=wW2v>7)Z3&1Q)@G;3-oP`db_@@Rp;qj zMV+m0Etj@4%40Uoidog6Yx=CMn(Eqh|HwO#KsP z)uMmGtZMoTHEP5tcNnCW^Mw9}t2G_$H`pwfbe$RM3cJxF^uK1g-)IMGbSfiz%_HxT z@3q$0O_t^b)_SW^2>WdL)NZo`q@iPL+q6k)I-l-6T6Rq*evMf=Ll-uy)_>?fh$?@O zyKb?X|G8}0zwF9GOCMdj?D3^)(}7W%j*yy8e9Kjujz#VJ(Qu`^W04`nJB(6ik;f)= zmLp*36EOTK82JorKL$oW2V-A=i4$PwmtgWMur!0h+n z;16K#M=<|0Soj4T{uOjw0G-32t7(&@+W~r3fnF!*YX5Y(h36n~+V&CS((`3E6~fLN+0rkWE-+O;|+J;lljsvoN2Ug(CRdq={qcnVdN~5$WdG z;-?|#M7EwSyP(r?w#GU~_i*YU1kJJxC^k9nOKnIT_^2nsdvKfqS1!LR5#LHmkb};!0*!3!ydJXJ(9Zc^Ad-s5uyE#`D!=ri;Wbg2nkOMoLY{=29l_ZVY(h36n~+V&CS((`2`k=&Mf4nl zBDkKP>-kL-TX-QTf)~Q{xe&^z=hRE*L=!C!YU2Dun&^9269*eL(e{`oHm=Y_(^Hxl zSY^=Y@cilUjMyn7PwJ*b*C}C!&R^!t$j>&ZPa1IG6;Nd=@})QE=iH)yerg8jv{lB> zNk}!v*8QeW`Z?>KU$xMBtM0_ygrPPwtcC~dhDB=2gQND|Z#OK)-ue1_=iWb8v>T#r>T4k$rQ<^mB#+W~*KQ|7XF^QS>oI@;h(Q_x%m~ES5{7 za*FPhB8xmUDYASACeMIf=fKqWV9yU=`bV($XE5^%*#9e-{S6%a9nAd!=AZUS7M=lz zp9LMwpwk7q)_`s|=xG7H9?-WQ^n1a;Mlk3DLtDVGAB?nu?Ex^l1B?a1#7?j?1SWTZ zU12cQ2KGdJQqW1i;p6-q3OY=H2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!P|%^E zLqUgv4h0?FE1gZqCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrghd1$F3F!hOY$jLD1yID zDtEQK5afkWMnOjkIEwUdZq`J{98I*&)kO0GO$^ZQCiw(hcI zazvdvVJbn+F+hFEhu>Sy(prZ?Pc7v`I=spU1dO+`S(3b}NC&55380-T>88CbbjP!%;17I`@ z#?FF?L9lZOOy^Tpn3t;aBFf$DHH*J=7PS&xRc24D2UR3NfC(@G zCcp%k025#WOn?b60Vco%DxUxi9U3|`bZF?%(BZw(*@SFDHX)mkO~@u>6S4`}gls}K zA)Am*SY=IE#L(fI{OPkMpN54Z_}ip&XXJ$-FN880Itu6Jlsq%2PHbE;qUT)wq<+$b zoyEdX!VH~jDmnE~X=*3qQIG$YlFomgr1OL{h72}~xQE7NP$&%l$?>Sff)J z*=rtok9@DS#%{7SFR<2IjY8OG%cpjmB`5tHTT|;6De1I-e7NkA&cK=CV{}?~tr~lb zPD{%uNvBpGdfOJ`mFA8`?u>U7-zjAld2CW<`3x)^1BX8c9cj>c5_I)~?mp0y0llX{ zUq9#{00UVtcoqx|g5e=Bk^|d+0;72_b{6IQ$li%2>qMQ~L=SM{5zx9~zx1TTc?b0L&b(pf0Q6HPQdrHO%6 z22FIYk?2@2A(z_AeTv+5v)nUB?kHJi(WdGDxljzH_`WHR|p9wpN{| zZxyv7lXU(qzuYLt#?;8INpAlxw+5SXNzN&^q~nmKVQU$Lr1Mh8H&Z6*#Pf&7mULRz zjgoXMHmln5b#Wi3sJSC$7I|z^W(k0<9iTf1dUk@|5a`LX59gK8< z?NKoL4j7Api8$Dq0FxhpU7cX+AlQ=x(;tJqU0|jg>`#H&qu^i7FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zDoB8m4kaB*I+S!M>F}=UY(h36n~+V&CS((`3E6~fLN+0rkWI)YtgcC8(n#^t`yh(xSlV)k?}ipbZV@M zG;Pf4$(g!>V^%G?f@4-S{e>DeV$@xnS``Cxrq0pvNIl2a`p#DA=$zmFTiG3*_%#OU z3|-i)TK}O>^ikzRpSy0cn*X_M*}v?{LrWiBy6o|#YJAfuN5^WLr4GGfDyriY-6>rb zd2G^U83KJd(Ek${$b-T2V5k6wFMyF@u)S%kWYhu1R)Gm8*x3vwU0~N5Fy#h&TEMgi z>|GCLykP%EFzW*cw}3f6m~RCO0dRN+=y(-$z6QEp2i?0t&mPdb7xe7|{cnPS{b2B| ztNfC(@GCcp%k025#WOn?bgJ^?y9T!q9{NL+M{s-un~+V&CS((`3E6~f!iqOx5l5#+5nR;IMg6t3 zTX-SJIUKZGrp<*=Mn|WCvvW$F8#G@kOe=*^6UNK!oU5NNDYvh)s3bw>x+R;xKB&Od znN|8@A>K?!$5Lynt~ok;#&dMw?ig(4o}&|xdXBBl<(H1m7i-u3Z`NCNZ{{X=Gc&A) z2keGLYRiM8_TFzdEXLmX`g`Z%y?0n=-ZSsX<@4@ud}MiJ<6o*x$JrX|9Q|!|@@@86 zmI1{kr`xDatB>6=F-J#8M<@R2RdaN#YFpP;J1uIw!zf)Ad2G^UiGiUw7*2qZ55V?L zFnSP-CBekUU}qPY>;}71VCpE?(*vfDgS}}ma}w0t!Uo!j*7xB}~zofLv3GhO-y~ju_*y1h1Ca&{>TC@GoiT>gP-FYMH-u-2YIH?vYYU ztxc$kzBQRY3Fo4&0sL#J&#LuZBjq04XRM5LT!iyw!f6a6hzc0*?f`EnwOY_O^nV0NB3+%m%^1onS5m=68XGFgV->I`)CiH$m5a(ES$Zc^mY; z3;GU#{`bJZ`(W@xF!T`^J_JS%gY8Ga=qF(8Q!w!v*m(?sPV#dKItB_lOn?b60Vco% zm;e)C0!)AjFaajO1egF5U;<2_@(EDT;SwY+LE;jmDqDiYdsXvZ)fK;2HBUmGgggm3 zI)bAk*o16CHX)mkO~@u>6IQ$liwHVqMNF$@>Ckj^C?^_T_-_!7)>F3<4dps6(w!WPqw~AC`3~JV(KQXFp3*;+C zX^xoW)+Dz^{RPvXFPJX9U{Y)J7tHF(nffQpD%>HH&8%wr3pMJl@%)_0nV^%9YL2Zt zxJ~*wn@=ao?&mb0EAHoXpE!H_#QYqi%&PVOnTVf*Lz5m$8cd!9yL!P?AJ~%t)2G1R zelRlt_GiKDS#WR=%ngD099Z}X9L|G|UqR<@pzC+g{RilIdYh#88PNAE=zk6jJP!s} zgP|9|@LDkPBG|qTjBWs9FM)|oVCQBqxfSf%2BuyHd$w`O-Vl#KA^Q zv^}PYjVm9Pwt*BYSHl!>Wh3OcR3 z%vZ`dW(Yc43|HHQ067nSENyxbooEyO=WD~Lp*@SFDHX)m^ z;!Rk@(5dCf8AZ@CFjS{yaDB5w=#22U$yzI;p`&nePRUb)>ZHMRxuG-adHtjbJBx*> zgef}GmzmN>0}UPZHaWhg)@Hi2HLDneQ){bHZ`ZfA>O6g`TpFD-H6wH?Wawn1oMRhG zv`Rzg%UG`LhR(p5;v;lgcdZ(Ggw9alDnqAMZ8}s(7z50Yyg8VfuT)ccrzH;3bt6S4`}gls}KVU;yu5krS^3gr}9S~aw^6u}E&T3rZbG;|hnZcfQFgBFURyH^U+ z#fPkwQHIW_e=aGjup_^&bV)}#Y%4|W)I1jAi>y=p_z#_=Gh6p{rr6W@xBOD0tyZm( zTa(=WU2Y9FMSViwTGSQ#c80pbZnOyfuUWosw1YJ|m65&Xk@v{=T5IekOY;J2z11j$ zeYSjRw^?#h(XlnX__FkLVj7z@RJ-$@EmZuZX9~#@!>E1HR)2X#t)S;cmiA>VL zu}Poh1F+Bu4j%*^he78N(De!E{uK0l26~TyzRyAb7hvE782l0peFcWU1|#2q?cajY z(_rj7FmVR#JO?Jf2fKa%Q$K<|KZEICz}{cM%x_@-?_l;1aPaAuC3DY!`DekxbKvmv zpkoc_biXV;9aqcC^mHn}2BXO!D*J~&025#WOn?b60Vco%m;e)C0!)AjFaajO1eid@ z6QHNVl}KEP#Fa=@wi1aqtmX}?D}KXjo`gIJc@lDd1m{Pv3E6~fLN+0rkWI)YtauX^ z@pOzFR-y=M8iJYzF9co)(`g1%89kjk>7Hof_`RBl-=~S44``z0K~0>0NE3YzYvN#| zCfXj;=<-~2c@)vT#xP>)T>eDPx*4VEjQZydT08Z^P=cmYM9oF8A5 zES-o^cW-KS7pIt`bJ5Kyx1-~bdXBB-D>ypayn(ViI`M1F(iyt2S+)K{cYIVhKDSuS z|6I20Uw_FxEiI!Q9gEGXcAuM&ql06UF3UR5wE=X$1bQ}s-p!zIE9l<_23`h(+riK) zVE9!q@*3FwIvCvz#`b`TyF5~f=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fyyU9M~99M9UVG4baZ&%bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh z7IAc_r%+F!r)8#|GM)4k{x;=rvy6^T1GE!ObgY+X-6AodXrlXOxo3{tF;^4K3p8px zy23PHDoiVdjHtM#GotJ)DYvh)s3bvWd=Y1Uxs{r_DHn9462v(=HC9E=xiPCJXG#^v zD4$s5)+D!@{z8qqXFNw|a^~oGq@H7I-5rpQPLm^Ac1Nejn}uHw{TZqF>sfMS!C zr8ccTcE`jV9jk4Y+7>ZQWR4DwO}Z>!gT8M-|F>Y^G#LC244nbP=fKGKVEYeX^hYrE zGnn`V?EDo>{swma4yOJ9d!7zRrk??Op9M3|f&I^e+123S3t(<7n12y0tOJKPfR2rz z(+9e?fNnqNX$8Fj(6NfC(@GCcp%k028Qq0(5k^3W=+bxC*JtRw40r)x2GG#cx;5laMDNPeM+Q;PePK zA)Am*$R=bHvI*IQ6>q{Kj*dYQR5Yk)n5by*LQn)Rgz0l3l+n?t=j@yb%?*;0&ZtM- zlsG#jOwpOxK${itYi8ZZL&L`>_N6!eei}OC8ai#`89I|QMJFKT99!FOFm%2?_i5P; zouO=TLnnS@-<&ZGo%p6vQ*?whbmGpdrs&L4n_e5G?O1FUHQr&ACW|~aX|n7ALwmvS zJ}~ko*uEc(z6HkK1{3dsod>|=dtlf5VCqA#=OZwE2<$x!W{!aUpMcpF&=*ocZQ=q3G^bUZ&Ea*QA1_r_45E#mV;h(@r9&A4kLnm6Gp~ERU zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_3KF29Lqms#4hr&7&=^&KYiBZ)38tkf17mfYIz~Z3!#jL4#(z{HZrJQ z^gTRE%^CI2C6i<7)Cpq=cFv8fHe#iiz+(m8d_#7QHgQ8}1(cxihXs}QJq-dgYHDpl zRrIYvy;I*B)!X!~>C)D$F3|s7qu#D>Yt?!BR#9i`Tg#>G47tS+9D|xQ=ueDl8$M~Y z8C8?qn&j4~zhL_F1=FP$OzOCH&aTSXIT0!5*y4w_OFJj=#dl@5bDGZ;w{yBroV|Tq zJIAgM-kF8LJBmffbenF!a(vD#Y3%eJy_TGVLz5EA?_l%~F!uCz$;2~Y=d)n)Ik4+_ zFtr-&c>zqX1$$ovGwZqaBvft+YIKnf`x71@XMfM2j~ofuAQJe1bTLX-Z1EE z1N{*&@CF!c2SXiTI0{DI0o!9>G!DiRVB!O?vva$YbCL(Qb9@fR=P&^#zyz286JP>N zfC(@GCcp%k025#WOn?b6fyyU9Ifu)SxcrFAkE(3>5${sXyHr>FF4a5UZ9DgyESqAUQNXB)BX#_ z@HEI{Unz_uk41PsdYaDVd&=eOXe9|c6L4$B6>;hy-~4r?*Lpr5@3|o{=sG^$@So}E z+&YDh&bSkFo{&F`K@NwxI7w%Qy25U>2>q{F?l;=O8lB3>Uh~L%)(io`|D^ zLz6DcVK8|F?D_;ueG2w`2BwdJy`O`bFTnm2VD?LJ@GCI)HJJYfEPM+Np9URS(0LYg z4TA0=(31nbKY_kH=synz3SjU87#aq{O*>vIBOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfQ}9w z9XdL6bm-{tX6bA~HX)mkO~@u>6S4`}gls}KA)Am*$R@0^CM@FUa8>^FS(Q)6LJ|CJ z(z-M9LXa0i866$vzu38%sBcs!9j0s9Dsq4Qq$xX#g(zW&&iG2r7(e_NFD=1wnXi5< z#!d=4a=6a;Lv*T2(77n)lv~hANjJyV_bmjSReLv;UC#kK}57B8lV7^ku zG23d@c))Pw&atQH;MkG1932z07D;u;Z87e5NuC^(T~Ac7ntY zX)t{f?Ck|JePDkEhEDbr4IKjw9VWm8m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ27LC z=x_}Z*C25XQkAVi;?1ggv+9c9tePhwPePuAoE^d05o|&>A)Am*$R=bHvI#5RghdP; zlOnjJpG*49bX#~KD1sNl^tlkqXy`2DNfC(@GCcp%kK*bZFr$bMNo(?@7dOEyoI-8J9$R=bH zvI*IQY(h36n~+V&CS((`39GCLi+DO*nLmA2=F_uK1b>?}?+m;UV>HU zP3Mw2Gvd%p_LBxWI_hn@dsAJGPQ<9YIJLTqQ_RtsoLM?Kspr_596{;mbT%I|b`}p{0*5UH14=btrI^qhnFq-ZV~RmJW_h zx-4&kh5g|0TcG0|&=~_=anPLrJs*JHPSAG{^e4f<$6&Av40VIy6c{-Qw)cS1<6tZe zCQgE#yvLovtZ^d*gptnhrq!cnEMIL=fT2xaJT?Eeg~a@z|nC% z9i*c}M~4Y80Vco%m;e)C0!)AjFaajO1egF5U;<2l2~<7-Iyzj1#8pUKg;Zs$ka){# z-m<#lx2)z#$diyKAty+1f&`n8O~@u>6S4`}glxi!H(?P+r$!N6)XzozwX|D!At-_u z!t}Wi%IN4cKs(Vy$9jp@EfNEYCc1Byd*;X;b2ZVtK%>T^D@^mH!n9JDG-0~h(Ybuj zl5+bxi%Ju8G#E3MDSNEIcg>pOM+3fb>W=Z^cQpu$F5(DP(YFTmPJL^vNN-Vj(wJ8K(WcbQkzyEyJKR4j>TqGTM82rbZ~4^ zWLX8eoS?fI^teFp8qntk{Via?0|wWFAukx-2u6Hh`xY?j2V<>ZA^>*o0FyzmYbTfr zfjzsxbQtVy12Yk@{|zwP4i0vJxhR-_2Q0+E;W+4cA9Q{Qx;_Hkhd|F^(0e2(1s&ff z6m$%gUwl#VJGAl#U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0u*#8=upt1phH22 zH%w;}vI*IQY(h36n~+V&CS((`3E6~fLN;NQHDM7!hfDIO&ysu!7K-3+lggdog&;44 zG738N9Go+up+VBn8TF`}5@Dx=5jvy2l{%O#)=7QL$Nx`B$5Lynt|>Zg<4HPC=u>tq z>cwd~Gt?D!qebX{%@%{rXa{R_DkFQ%Bkz&#wbs~8(v-2*>(b0@NHaEzNBTLoR{yI~ z(rNBHUv^1nC|g|8i67ZFXG}?_rDfC<9jk4YIutWaWQq=sP0B1O(03H{_ke-pU@#4a zPJ-cHFwzIMXTazwFxC$y2EfiNm^=%14T7m5uqOwme*$~+VCFp7UjVZgz`p-fRYZEA#oWJmmyWzG9=!tnm4Pi_|2+$67nSE zNyym|oE^a?WD~Lp*@SFDHX)m^;!RjY(lIN7tNOXB-$=cM7lI;qAxxhOp^TCa$L5qa zGN@kkJv_?J8THR4lOyZY2}22j&R8zal1VUW7EigFBV{0t(5bO1(w;G^Cud5@#wed) zypKt4HT{Jeb=P=y&P6My+;&btiaEBnv#@i%JKXYrSZ~$Uotvb(bJq;3;Q_m0k=pX$ zsJ-{w4U4gNzW(01c<&w7nfJ_la{0Xb8y{KT*l1TCTKeeHWsfgao6i-ubGlEQy?tVK zj*xav{B6UPc23cq7`D@P+9)j+d2G^RX$3<8FuVhd1i|*5U^E2Ac7cg7*x3dqBVgAX zV5%ML=>XGFu=gD>69fC>U^W2`egNh=!Tdq6kOYT61|6S(&QC$tXQ2BS==mJNfC(@G zCcp%kKoulFJBM}-?Ht-Uv~zg7bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh7O``< z9)J3*$ERJP2>v$d+nIPF$P1y2cFufhooM1DVMOL zm1O8lV3f^5$Ti}|AM$FP&kpko8*dNfC(@GCcp%kK*bZFr^A&KxTck7-2Gx%x@{q-i^gg|UQ5Iyd6o7*~o3KN|3n8~M`f`D_Eeajcm^ zlksZ94eL3#>Kczly+z;7P-p8?ZwzYIpg%FHZ42ZpMk$V%@d)4FTb*pqbPn?~t5m_F3{KTFqha9~nn zc?V3!z^*u$N`O5dfay-K_aK-_g8d(Z*)DLf8_cD^{86yb0}dYt9VbBNm!Rt_(ET;& z`3CfU3;Irj{_nuR88CPb41EuVe*hytg6%(p(OGVMQsa5|jw+5S{uF$uZOWPUhin0vQNl7`!*4G0=XU*E< zQ)cMI^M}SZbee+ZD<|WaWfV<&45ceNI5MfSxWM!pu-6S{TEKn}m|YJJdcoXAFz*8k zTfkvI=y(})ZUA)Am*$R=bHvI#5Rghdn`qawJbpKJOBZi zPZK*I&_v6FnmGTECi))M#KA^Qv^}O##_46APcIt&F z!O$s5t0_gjx%@r#&~R$6^{B^7lyoe$Hj{dr^j>P^coq!9skPOpx9i(lb)LRe}Tfu8R{?+>8wN6`N>82AMY{tAYE1H-?Akw3upr$ds_XTaFA zVB$Hj^La428ti%jOsxfbUIf$Yz}^jD<|VLy6PVo$4sHc=+ra$GAt~t;wud-JhmsBx zU;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qu0+e(p=}^+4q(e!E_e^IKvI*IQY(h36 zn~+V&CS((`3E6~fLN;NQHDM7+hs*M(&$4_<7K-3+liHn`7lOPH$|&hHz&p`I$9jp@ zEfNEYCc1Byd*;X;b2ZVtK%>W_J52MX!z6_i4Nl5f18bI)Ti97tlA&`=kLFr_%|!P< z3LnQZ@>qoD)N`s%&)GAco-;W!baK+nu{He&J*TVxx&PC8tFFx41Z8H1)$oAbut;rr zaMa%W?S{qJJ70hAT)g)V>&$!RJ-K||{f&<-Z*2U_vgtTmW1XYFtxmqpKFczo*koU+ zO{`@wPH;E`I(CE3J)mnZ=-vl<-UPk-LEl@T|7|ev zE*LxjhTa3i?}L#K!S;{9=pit67)%@iJ3j%FpMqVVfvIC)&*xzJ3$XVDnE4Xy{|d~0 z4Gw+-=Dr2el6u}E&`dkQQ)N|@NH)ldKgRVAG>ZU~3DPe-nU*^n+ zLNlpP8tCU-8k`84ayPq?ZE$-*UkL;T>rk^tu7&Sh} zYO|`{PE+ZA4vtKEEI)&;UqJV-pyxNx`#b3S1N1-rx@6!PF!(GOdJYUf4@Opl?Jt1Q zwP5T;FtHBo+yEwD0=qVWsm)-|RxrH{?0p%`YzO;a0kf}ygRg?b zehydUPoEX}^eYs>-zJT_T3!h9LMWr3!+ANS%?hd)eGgyl=3KsK$>exCb;4AFn`2!m zru5N(ujA~Ts+*m&t1@nmM`}5?*0sB&o6{28Uv@XA`CM@~r~Aa&+b8Pg%u?G9Pr%K= zkx7T;ebDzI=>G@|90G%f!O#&f{0SKO6m0(tj2;7HpM!}nz|Iq3@=LJmD=_sn*z*mT z{ub;#4Q9Rr`_F*cbKv0jVD1Mn|07uV865rvbQD161<*ANx|?=MdK{p473gz<{$?=X z0)uOINjE3t-bFWuZVnS*0!)AjFaajO1egF5U;<2l2`~XBzyz286R3OwbaS}+h^vpd z`l!lQAMqB|yhU}zZ&A&YkS8HeLQag}#0WMan~+V&CS((`3E6}dZ^9yOj!*;_^K&tO z4XqYl2#Vl^FnunBGP*hQrE{W*gN>SKdrT7>S7@T?De3F1GH9ZEjYP+KiPkL=1B%>r zv)nUB?wG5I<^`GqoDV+ktd(si^6`Z`S|2vyOy2K7#TYvk-4A?&j)%?s={OF*hQwzeeA{bf+hBttbm%#Q-V01GW+X^POft@dd$?ag*D`4tXu;(=}{W{pY z8_eth`}cy`ec<4mU~WH{e+w+U4GzBxI^v)+0lGc_-JPK4Am~kkzK=nF7Z~UUgDEg{ z6b$!(k>g-{8h%dnB>fyi<=0+R{Kl;O0hj<2U;<2l2`~XBzyz286JP>NfC(@GCcp$L zo&fzE`Z@G-=;zSS;eFECgls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb&*6&v>9ZoA zeuX0V+oW-4L*RtSu6}C%+I-|kW)WtB2E36 zevbaCaiu8#qaOd~f1agtt8VpJ)Qj60>TIiGG1v@h)}TL;ipT=_4x`Pen&j3bw?_R1 z)1NPxF1=tdBew(+LuJRv_|uvyfL+ZpN#yU`-_zh=4L zXa{R_DkFQ%Bkz&#wbs~8(qysL>r%@lOD7@)9b5eM-O|!YZ8%+aOK0Fr@hLj3yH<@o zMW?0ZDoe+xHr;cjrGw;K9=wL7gCmnR%U59ZYcTc=nD`d#JPjtl1G~C9Q+N;{SM~;01Hp=mK=TtbT~n0Gw5=G?lqvt4SHKZp9l1>2LoO( zxDgEb!0;9@;s@JX!Ds-C?En)&uyf~bY3U?GyE#RNmJSnO0!)AjFaajO1egF5U;<2l z2`~XBzyz286R3Owv~;)*iR+NK4ynr4A@OF_yjgX{Z&uBdkS8HeLe7rh>5EQ`+VftJMWwdk_a&k_|Q-c=DsX;4+;o?J1 z=DJ$cxqQ!(vPwJh>q-!HN}6f&DAbzrm@|vY^#4rg=V_p#Q?jD-Z{6Xkl~NC^pIY_r zx|*Y?D=sNI7q{h7bQ03ev336nMQ7jBe@vO86VD$SThVDRCK85P|=~HLq&%-O=lCb z3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%`#i}I(>qI@bAir{aP(w%`9g1ivQsOZ#5 z?L-sD@6|;7K27X=KoczwYU2Dun&^9269*eL(e{`|jpw4qqloS`mkgZ|Rp;UpIqK$e zJ?HA@Hz@4L$uT7uI@jaf=)#M6rMT|L^?d1sl$=p-rlB+L5S_?)Lv$v`&`C)-$JX~z zSQ$##Z0MW^z2NNqBR%KqUGzyz286JP>NfC(@GCcp%k025#WOn?b60VYuK z1gPk65fT?6aS>9LEkfeWs(G{Oir=i7Cm~Nlo`jqo!PyaPLN+0rkWI)YWD~LpE8c`f z6rCDHa7{ng^w&~u;e{ZFa8PcUHWxw}6`clHCz|M3FVVV1VnESE_sw$89JynzCYl#$ zw0LxbX}&a=q%NXih)Z!%=i>e)<<@oNteO)1oa=UM{`#Q2Kvs&{(jN=)W-2EJ&v=T?6Y>KFo8{6lopKG)$w)iL zHnd@nRCInm9xl70({Z-OI!8Bg>ZFNdpJf?PY_hV{rq##pn3$qdD-XT3$v6Q;2ZttA zmP26nFgSPw%zXmpKLrb)fy2i@$8pe^23;pXcQ5Ga1HBp0cMA0PgMk4sm<2;;!SEm$ z83NmLVDu+2mIo8(!OjAhya09$gQ=!Hl06PEy$bAgf|+Kp-vwsZfP-!@*8=7}d!(XM zSigsg4iz0Hzyz286JP>NfC(@GCcp%k025#WOn?b60VYuS1gPjx(V?P4MTd$G@0iXe zWD~Lp*@SFDHX)mkO~@u>6S4`}glxhpYr-Om4j1LqPNAkHXs0NG7s9l<5Xz|N)N^vq zgr){vt)$dViLz6IqcZ`wrW6ZjG5j0L7>^}*m4Z&y2|8`#2|ANALnkNQ99z>V2s%9< zbe3Jv8OjzHbmB+$%^6eBX*w`!hK|K%RhvGXh@gW*lOoGYU||zDycu-(L1!!I3V`k% zpeG1=cY?kU=-&kf!eFos3`M~38(^dzZ0`W0Q84xnn23R$aWI(xyFLI@onX&FFr5T@ zKL#^hV1GB5O@V_)!CViRKMoes;P6S%@fGO&8iJ1N8wxsx$}hgC_zhb5126$5zyz28 z6JP>NfC(@GCcp%k025#WOn?bgJOK(iT!O?UNL+$cWlNBFt7_h=y5hI0=1IttkS8H0 zM{sfkn~+V&CS((`3E6~f!iqOx5kbeSh-tMfecFt`pfW;bgx97b=uR20??Nb}pu?d# zr406=>(I_V3!L_tpR)7V7dkD^?;f6 zV80j4ZUhH?U~UVT_k)F2a5w-uUICr2g09y<_v@f%H|X8dCjA`W-Zl=egTKoN|NdYg$CLUmpUW%P6AOYcMz2OBlf_LwF% zuFyo&Q_|U4WzaZrEecFG)Af5(! z>??&Bc`U;7u?&@RDLYz8j?QR-W|U7;#(%DWnlnoiv&KIB$#}H^flu9$Ui_{GVYSvu z5l5(szBQRY3Fo4z$&+M3k``oC+`+x2a&I#1s!>TG>$QE%0^GvvP&i@|14vj+W% zQEkH~jW(ldl3SD98ub@Uf4*S4^nyvP(O)pDCui!PFsl~*6J}M@U#L+lG(g8A-5gtM z(_Sg)Y}}GByP$Ke0Xj{Ym^!AQ)4FTa0G(M7bdF9$(7~Zek>w50*ADtSz(5oXz5|A0 zU^os&5@7oWV6+pA9Rw3eu=8Uu*#&lWgQ*nQa}-SXfW61TOd9Mz31)l2!9FmT0rRK8 zLO(b>06NZq&U2vad(iy@==l-!{tWtl0sX&%f#1O3?+|oCe^Aij039a41egF5U;<2l z2`~XBzyz286JP>NfC(@GCQt*M;bx+; zQJoZ+M(mucpVUv9ud`T~N*JOeb(txBG*Hl?pd*BRwxxN2-DU|$H^R_ueqh zFx<(^zVQ5de&=}3ndP`I&fv_sV3;{GUNEyeUO2*xXBc(%?4d@C5TKNR5dzj4r9#9~ z1B4nNlmGz&gc6`&fKsE@8laY-5hB#{J_~j?l+dP^0ftS_68 zO~@u>6S4`}gk!7;^9VY0Q|P8p(ki2*r3hXKdb#8{fy}InRgwglm{0I^XNQg;)DKpqjYYM(6QVZsWVEW zbpEN&+9{XAcQBo&T>Z1$8f=RChQ77j*-lUw*o_vU|2NCmjdrk1r_#UI-2Wc=UTc}% zBn=m9r7pYNF?9k`(y_I!4@grdd2Pd}nL4f6!-JbTZR_tdb&P83bN5O)W@+lkfnoRV z>^F6AXwqnD149WgdaUhR2SIQ4W`pz&t)*v17>@{ zTn6m>33U7dI&XlkU%}d2p!+w_^HxC8`!-nr4(NLqY+M5Rmx4{p!N3YIxDpJl0>kft zk@vyq+JH25V(S7lb!h4^0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqPq&YmvAX ziEEL@*jgmsv6^?RF8Lj+c@pv@M z=21l3GDE+sbN3TD>}GHu<{`bEN?|BK)X5{~%(+XXSqPD)n2))mXy)|$KUGw8tcp}< z%ExLhYRyF;FGBse-6`XQi!7)lTXP~09;Etkmds{3;CnD_}TU#26&d!Uw zi>~OjK46$m@6C0p^{=`sq+%G)V^;Iu=gu&W77H-kL^FtZKJ2Ekkl*cSpFyFuq3(6tw=-3Pi4 zfS!Y(_b^z01oRyP8;^tj6JXOxFmM_So&iH&f#GvtQ|r025#WOn?b60Vco%m;e)C0!)AjFaajO1eid{6QH9*M~99M9UVG4 zymLC6kWI)YWD~Lp*@SFDHX)mkO~@u>6S4`%SQF-Pbf~9LPobw}rk*mM^c4OzW6PH9DtX(2-C6ppMRDsl}ArMrVx9mZ2P-e~-~AuA`HXdXBBLWvg^_ z)*m@tbVsM3AD-z5}}61#6dp?xmn-Ip|#h)~^J8tH8$h zK>zz-(^@dF4h(((hCT$tAAymN!RQ7s_9+Z-Yx`E|=%jaS zNp!5mQ(cvm2u0rA}q%pP%iMOlf z?W#+DyK0_#=+Gcr?jQc}*b^*XmnL+4iTO3@9S-gJILr}fy@DT5k1t!oEN(Sa$|xy*R)Xq~~Q z=-}9-$r1w7yTG0>m}v#G5iqwO?2Ceq!=UpB=sE`09tYhgK+j3gdm5}i1Ny!K8_$9M z3t-blFz_`P{00nt2Zp}~BR_!AOJM8@82=GWTm{>&fytl2j$gpk4Y2E1FntT``3=mx zwM{bn_BLtg1;waA)Am*$R=bHvI*IQY(h36n~+V&CLCi;n8(oJn*8yzCZC3d zBKX&&b63s_L0$+&G;}yNr?8Phm7?>d0cy^GFXxVosZ$}01=u+cui97^ieWq!;LWsi zH0S4RE{&a&kz$Um<}~b_4&OWfi}i7R*v%ty z*v-=utcDlthFNOeiv#vPZ#T@r-s$>#r{cX&SttH&+MBOUd%pVR*Q%@S%ItZs%$xuE zJazGv{B};;xyw%u&(5*ftZM6tVc9u2Hfgae26Ha3ZyD%V0XkQLu2o>|d!YM$(6biw zt^?~o0DT{VjUR#jkHMx5VBk|QxCsn>28KTeBU`}e7hr5F7~c*ic7W|)g2|m=$8IpS z2khDlruTt82f)lhFnbux9Rd4}fsQuNnSh<+Iz>CjKs$#CFaajO1egF5U;<2l2`~XB zzyz286JP>NfC-d70oplSf5i1iTz@pi)*taU)x1r0$!}B5laMDNPeM+O;M52cdg)n|Dgd*BG)1`Hyi9^+zXns`_YZhpt z=3mm-S!B>e+cJrk)e;TsCAt;4>ruI9irg_(6N_hPqW2k1oPAakt6|irtLq4YCJ(P6DoTM}42%R^iIb@J?WNwer znV>GP8!bZrZ@+ny;f`3hFcSc?a@_9JtfwG)a?19-A~- zHh|tw!TL?0?=!IRbI`v9Z2AHWYz2ec!O#vc{3RIK2}XB=u{~gXFPPW|wjThK2f>cR zVCo3ibqq`&2YXI{nUi4lG?+UB_I(99&V$Yl(A5dnra!`F*;0u2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKtqRXkhlhkYmmm+8YJGW zns=)%`Q55{67nSENyy<593H_YWD~Lp*@SFDHX)m^{ioP|df6=!_^$C4zy0bN_GxV>^)F<_A zxjIeXD(Yl?Yq_(XAh(K~SYuGr2K|XqZN_&RZAR53wMxl7{etPv3nsNpf5EJt zpQwL_S+(fjVOBN$g)+5&n9e`-Hh3f|17r#o1*@En9l8Ov4-jRq@`nP-V>Cn z&RbnaM@`jf%^n_H)oI&qwpQpyER(=TA)1{nMm4BZ05zk!jrf|Aj9|W6Pz(5EL z?g~m#Cln5HlnzJfFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPshoTNe9f~>> zb$Hu!HX)mkO~@u>6S4`}gls}KA)Am*$R=bHjO11hs6DCrV;VY-Njaw6Hadn*z^I!z z<+_Pul(*lV{2W$LmE1%6Ws0Gsb+bm5X3f)&S$9NmI@p*4{Wrm;J}^+TLo(<9LyN$$6O1edqplrN(TOeFK}DzZ%P(pS zqO||`3orpDzyz286JP>NfC(@GCcp%k025#WOn?cLJOL^?T!h3$NL++8#ug#*messv zb;)m8&6AKPAx}b1kl+LfHX)mkO~@u>6S4`}ge7mnJc>@4BDkiXYx>J6x9~zx1TTc~ zb0HK_(W!!UqKTH(5)JDmx)n{dJu3H1kvpbpV(|=(7LRT)O_v7KLSfQ`X>jjlZn1Tp zS%vvI+EAZ+zc}6h`Z@PZ#xdH=YUhdp<8lTUbZ~4^WLW{mR)XJ_OSrfju9CnGIm}Q!uv)?E4IKG=a`$&=mk{w}I{;=xG7HA+UZI z=nI35t)M>wHth!kQ80K248_3kQ7{q*qitX;0me^(iFUC4ECiiol7fzbf({d40!)Aj zFaajO1egF5U;<2l2`~XBzyz286DWNG6m%%)P|%^ELqUi4OJ@_Z3E6~fLN+0rkWI)Y zWD~Lp*@SFDHsKg+!aRZwm*kJ1CHWLA6v4kHl{>);L0$+&6m%*%H)mKggQTG|;88Iu z!cGAbbfhnH&!Ylgnn(089Y3FvPC=5+_&^w|C=oa zo6!!I=~Vjnn)}})-)k+io1`gYt< zpyMadnFU?f!P*??z6pBzKyOWpWW59QEdm>zpnoyg%sPVFxdch_`uXBU{_;{lyuS?TR26h^b0Ubev4NA0!)AjFaajO1egF5U;<2l z2`~XBzyz286JP=*Pk@pRmmzT(5|<&3v1LfSSv7A~UGke%^CaX+$diz>BRD&PO~@u> z6S4`}gls}KVab~?kECN(1XuNQRlkvX3oisk@In|r7eWyw9gfW@Y-CWS=zM8_oipIe zxg#U%R0u->g3e$r&fF0&Y37W&nj>YPDtxGQlpj^Xs@pX}RrIYv{foXes!!y8`c`{Rdxm+ zp@Un3z+!=%x(p9+rhpapd$o2cY&@jSlbG^BcNwL=#7H)hd^HpY&;73 z<6u)87)XG@Q(&ka44(xfNicdIjCFwVPB4)I+rI^qU0_Eym`a0Pm%(%o*wYJUGGO*6 z*g3f@?HrEKVFFBm2`~XBzyz286JP>NfC(@GCcp%k025#WV;}+AIkaO|(9z{TGblsglROP#F3jv+#W2M4h|$6ie9A3NmztG0J8pJQvHuKNS3sVSH{; z-b_iS%&N!{H)i$xL@5Xv6!;*AxY*J>q0p@-M`)+}bnlB}t4$!p-taXC!#h}LpdY6Iq zwVy zP3OTt2N>)GLn$!)Eg0znqupRE4aP5ni5{@M7ffcrj-S9(7VNqXrgLD=O)%33W@|!{ zIS1Ic2z0y)I+uX1rC{xH(7gimtPDv*$Ga*-L#OnsFKP^;wEy@EFaajO1egF5U;<2l z2`~XBzyz286JP>NfC-d50UA0qbZF?%(4nEjTc)!K*@SFDHX)mkO~@u>6S4`}gls}K zA)9cFHDMk@himf3&zgK17K-3slg?ckF9dlZ6w%O`$D^zeluL2+`)A$1Hp~sgTDjcpJy)SjufC^$9u3rd*D)nW1mX)F<_A zxjIeXD(Yl?Yq_(XAh(!>V^Gru{fSX+#&;TRM%5&@Cb>20FPQ%Qg6Yl+Cbe)$=b!qI zESKMbA6c&cS#AwBMI9MQ$1VLFTSE$x&c?IXMorRb%^n_H(y3`2An6Dx=`?(M-xwW> zT6fEEueoED19#e-CMmPXW0NwA2lTE1>%E|FJ=j@^Ce9j zs@6pFs~Sa~+aix5+LjsmHJ!Vk$U!%E3p)2de@JDgQiuW^or1)gLgbsf-%|+@M?B~u zf3XqLbSh#0Ja9BnC0@EmEgFR{<=`MGxs=<4s_0vT`WJm`ykqE$EmJ38)Qy~S^$oc- zN<(L$se}Cnn?--!V4I*Wup2Ex|8JK2jdrk1r_#UI-2Wc=UTc}%WLZ4JT4^;3VV^9Y z+HDq}lyq#(%{!&3vt<9$D(mC=i;u_`pPpbfykIxXQtMtEu=jbpVGj0A*WWu8?|sTT z@o&@Kd~MqE)i1wRU2Rup&wFLw{MYBHtq&Nd(|dEBYW=JJ{VKlyF{}CS^XLELKi^oh z$85FhuTR1+s#t{dpqeM#>*QEu9IY(_?45p52ZttwmTy6S7ueJd2GU^gG8pOs!@Xc6 z14e%WV_7hM9ZclF_M2d`5A3MfDVcJBU5mi96YNgP-q|qE2YbPEOM){UVH#->Q|r025#W zOn?b60Vco%m;e)C0!)AjFaajO1eid{6QHO=QHP=qMIDMdymLC6kWI)YWD~Lp*@SFD zHX)mkO~@u>6S4`%SQF+Eb+|Nt{4C9=xo3*pF;x?bXJ~YJbdPDe^q8ctqCri$Gg2q-lexu~c4if1={(S-d5~i>-2JcI zd*HKhm=6?bM(CxgKc=7axbE|e&Lo{JL-{!)Gf5{P)f`)EAN-v3mXC|>=d@fdvrf@9 zn+j+)lPukeP4<;qv-I>+!}D`6uShyUkLf?`PxxUL%iw+v4o!M20Wh=;38h? zd;@yF1M9yBeLsMWmq7m&u<1uIa1{(*14BQ9;a|YW4KVsE{G8Y=`Z*k=!vvTB6JP>N zfC(@GCcp%k025#WOn?b60Vco%#y|q}bGQPDE0DMXX^gEv;{B?5zv`0TubL+zPePuA z93R2)5o|&>A)Am*$R=bHvI$Gxgn9fNgCe+~p9}g;G+THfD1sNl__+{@=;u^&c+Ri} z2i-5VRNSkr+^_5u#n&ldjLu-r(EY#F!|$o0pi}SwozWC@noA?-M5LQzYdgA23OYyD z?JT;W)0@sO=(HZ&I%QBnr`Kn`SH>~h%&Js-?%g@Kp@UKCTMs%u1f3s&u8+am4WRo|(6b5jeg@Wm4*Ir$ zjbDKNtzgr3Ft7s*ehG$lg5lj@WDgkK3&!?=@dIGuAlQC*mo#*eM|N?74kzd^0Vco% zm;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oh7JuK8agy|c(-&mA)Am*$R=bHvI*IQ zY(h36n~+V&CS((iu_nx8=x|N`_*s)r!$J}KYtp$Z=Y=3Igd!R`9GO$txS&eW`O^J% z&fRIp&38R3BCNIF8S;l-o>qEFH5tLm!hG{L!?JU5Y|>(hgUL3qBLSvP zfnDuj`YhO!1T*KsYzLU@1p88;<2%s#J?Q!Yti1%fuYjH(LGM+t{u=1}8EpIo^xpuR zegy-!z~FCS=&jw7;kUuaJ7Dx(Ft!AYF9j3J!S)qkawXWY3QWBRcD=t_+Bxa9yJ_c? ze&t1tL6r6%e*q@I1egF5U;<2l2`~XBzyz286JP>NfC(^xk|#hrhwG2H{)p?3#@PBJ z-lm$jsV@0#s(BLfB;-lRsS%tS!6sxAvI*IQY(h36o3P|fn8(f$ir{j7F6S?!*TM@y z5xfw_&xKG#J7>DIPBd|-S`*E$YGTa-P1O8L8as;&nrK@l(Xv`XPMe^`)AneUXqmE5 znED^H@LUY?D3-9J4b#w>1$U-=q#v)~BWUQ1IYXyJ^K(*C&aw4&!_ZlNEnRd&=Rxyx zYEof!P(!ELGhlvBxy_<>ZZHmKeh!XJnk?(VbUoP90A_q(_7gDI2=;9R9iM^D&q3D~ zu=Weky%qFq2faJM`Y%D>POxz|=-&f2?F9q-z~BKebPx<51|vtn=rJ&M9E_g;6DPs; z(_r!p*zpyZItO-L0Mi%2p0B~oH(>TV7&^J{Y3LYe=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fzl^HLx+YA4ILUfG<0~cbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa z%wyVCsM(gs)@xjG|~HvCeA*qiPq;dvGD~> z)V-*Q>$5dlJi5VDAq}Q}J?H)>l_MtX%n^nHX6QUn$*CMMm8Npk4ISwORpCReqx`56 zR(%YPP!)Y^Q2(NDjd#Z9jIA*`LrOYt$Y0(dO_c@uc7nRVZnOyfzgg}#+QBlNO8;JS z|9j+ntz~wTW$_GarPU~eeX@LNw^=gM&#~3)2un$)r^E5TS|8V6d_=zZ^aQKn1-oIE zTKD3Bz0cbXbFg>1{@$s0?^D)^f1CE^Ytx>ue)+ZPYP&Lf-YfIwzdldxzLbBAPQ&I! zgOAavs~aHcSZr3cH9agz2gfF5mNb~V4EFVaj;o;a8tD2Nto;Ra-vB+og5FzT{coV} zt*~U{+o1m)u<2beumlV)1w+fh@Cq=p5{#|_WAB0S_rb(kuzekv`~d9u5KMgpc6|({ zH-J5#f|*TV_A@Z|IoP)abOb=>wy>0RT){BM=#+i|M#=Bd%3pv9FaajO1egF5U;<2l z2`~XBzyz286JP>NpyUZq(%~{BE<@rnq%pP(iFd2!-KtA|w`!h*JPCOca(D!XN3aRm zgls}KA)Am*$R;d#6Xub0Op4&Dey-{_Q*Ysgpa@5y5G~ed(YgWIy>^?3h;Ca8fx=O@g$d7kYx(`&?3H#l1`abkrItrJwH)8Hb(ix zBDW^F)$|w2)P70lpZXih)iV7JNfC(@GCcp%k025#W zOn?b60Vco%N}m8F9ZEWsbSUXi(&1gx*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL4YV zkEFw8`Qv94kbX??&To!*=4RO?^$K|Cr3@jPZV|9$@afBa_xPwUzNj*ft%v-ZA0I##vw%z&L% zIdZ4XX_79BJT~dFybF4lfb~m3-*T{V1?XQ1Hmw2!?}5Sh!O&VTybg?f07gFqV;_O> zkHN$Su>Dgoxe4s}3`~6vc5MOEUw}PZ!OV6ry93O93HI#-9bwSf3c4a-?S9Z51wDsA zZw#zI3i{$;V;dYDe}ayVq4eu7N`9AC{sK&Z2`~XBzyz286JP>NfC(@GCcp%k025#W zB~O5k4p$*@6%tn=jj>foyk9l%S6%Y^Rr4g|Nyw9s<0Cjef=$RKWD~Lp*@SFDHetz| zFpr~CrU)+T=c4{{+AX{g7&T!WAm|MEa&EDGommA5IuGO7-1%MwvW_VHF%xetf}^7hbF|M) zctxZAkTR1rUdnAkRrIYv{foXes!!iMIM^8Sx$rgGhowK zVBj1Wya0wSg5j^h$Twj0J23V=82Cn>QUDMfwY(h36 zn~+V&CS((`3E6~fLN+0rkWDzonlO*0!*%%-RA_1m3Mz`=g)puzgd$oxl~O&?MBR&; zxISAGoiAzPP_-tSU)98#1)8Y&m(+F^Nq2`T&mE0PYB~cR6{BM96fj3;7_rfbaQO2 zKSI#SEKe0((CJO*7j#;WZJjcxpi{HQY`sU&G1|;(+y9%_#mT!~k=_1x$Pawr>TK+rf?^lfLVxaRV z=!%22ZJ;{=dQO4fcCh{|=u3i)=Rto5*whIIQef~~Fw_NxyTM2rj9!MI6YHU%!!bHc zfC(@GCcp%k025#WOn?b60Vco%m;e)C0!&~GBtSujOOUt(iA#{i*b*e(t(teMF8ST6 zc@pv@LsA zcOeu}(Ba&i!e$0lN_pvSJLUf8y02mwFjD94FMr>7oeCiekaGqTY3~13hr4GwR2=hw zM+NrK&yj<5h900Zntsmaq5PbG56~&DpA(U4j;(F?Ug_tYym6!Ge$L`6`Td->bC;hS z($9HJAE5Im4A8My%(im1`9niq7bovdDYD38lOoGiFm?@${|qL60o!kY$zQ>aTVU!p zuo_D~^yI^(+m|F_=Ee9QSpwkVyR)e)3(7guqctP)au)ZGjHGqvi(Ekb8 z)CdMPf6JP>N zfC(@GCcp%k025#WOn?b60Vco%m_W%BprAuRhk_0T9SS6S4`}gls}K zA)Am*$R=bHvI)mn6Xp?gxc>cju79UGpa@1szs%_lJd-Jtv&sV?v zT6MKunLY27dGlYNr#@(cPE9JT4r=H$Y&PHP-IT-7k)w<5(RQphtJ>x?<=rVw7I|#a zWZ45I_ktb!z|;Y->mZmu4E7uWGsnQ}aWHoR>^liM+Ck@8(3J#h&x7s`(9;QeQ(*nK zpsx#T><0a5u<0@w=mCShU?>BIe*z;}FnS$~<-qt&FwqCL*F+?f4zObpm~w($izCv| zNxLF6bZF=>0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3> zshW4HF8Q6Rc@pv@L#cwRYT!p;0f_GuYS!`kTR~02tT?27_Rz1q_G4$SyD%24k&YJOU>6 zgY8i;c?j%?fvKZlR~$^YfjtQ@a|+D1!_di{rJ-Xe{pyR7-=LMh025#WOn?b60Vco% zm;e)C0!)AjFaajO1eid{6QH3(Lx+YA4ILUfyk9z-kWI)YWD~Lp*@SFDHX)mkO~@u> z6S4`%SQF+kbSS4#PNAh$MoUW(yb#9Kg-}F8XQs4HG*R;}O>{3ZXrgVIM9XT4hV>HN zirn?6+(U~;8cZ{#!L(4AZa-wL3@~&Cd^xwM!jAm7!X+JPKQ0uO(Rs|o2U$n?Q8ntH zC=XD6sP5wkRnfNw^)LF?IJP~Vf68CdD96T>$*oCl|17r#o1(s86ol5^+bN_qfd#z=5lazC;mHGgjJDyHPDmu2Bjr*jhlijs<)I6Qm?BT&Z zowoG@JROT1np5-3usj_coAg=Efw>D{-$l@o0-fK2t}d{)8+4~Z&t=ft1J?I~z6{v- z6X?%^P1nId4h-G|Lw#VlW}jrl0Y(>rF((*b3?^J)`!Xu$tk+W_FXgUMFd`N4j zQWy%*bn>VcQ+jzj7>wz@1FoiiK1 zD!QZ7`hZzFy*JmX*1zhqkSe{QrysML|2}{IKkUlvd9Tcy|N1<&u5N&%W3^3EJ9nD$ zj?KxtQ@Skj*rdzi2VI-N+9uH540-~fcN8z|ej$90enX zz-SDN9R=fYFwq9KC&1(>u%jJJodvs+VER1R(*b5W!E6f5eGB$=fsP+Q=Oxf}1+4uM zbYBHM*Wl=Qf2O0uSvpLB2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKu3p;4jmmj zI&^e+-*h%1n~+V&CS((`3E6~fLN+0rkWI)YWD}0DCd}jLP*0(rLQl&~J!L%UDg0~7 zzh)5~ohpvcDRgwubg}V;g+fjk%CG74D?4+G?d!}cNYEL2p3QV9HIs_?sK8Gf`g4o> z9?sDzvntZSF{|e%N)^W_pIGG9B)6LWLYcZ{C`V^x=IFSko?~m+zh62!Evv5<-O*{e zTxOl3t2h->#j#JabSpMF@k^~)dits1IXXf*It_8dKpkh2+8oF)?nswK9-DMou7lnj zSbr1r^?{8w`z8GjuxSw(aDu_bV8{iAmw}O5Fj@!3++ch)nDBt@Yrv!z>{t(`>cOrC zFzo|-J^?e0V0I&z^MiexLB|%*`330O3f681-8(?fm!NkiSic+e?ExG2?w5{^f8T!2 z(JB2sQfmyNwEy@EFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d50XjNdg~U}zT!l2o zRw40r)x2GG$!}N9laMDNPeM+Q;PePKA)Am*$R=bHvI*IQC2zt!j*dYQR5Yk)n5by* zLQn)RgzR!~u_1T)}d`T0Bsx{I4swUPf&_vC@G||0CYAJGlj3x?| zS+sHdGo_?6;88Iu&Q1YSbcSc&EP!7l9_FFp+lOc9EO=;3NGp*RBrTY7n@|;fYf%59 zZ;k2``qp%3YgT9IUze#*>f3U4n!Z)k$@E1-lwb+|2FN-*QPyR{qk$o)&CiW)0@t3=(HZ&I%QBpr**B_ zYS(}MB&a=#MaXoUpT1YnnS{wYryeBe;LxPV5(WK-z@``&I0^>iV5kiYC&0)lFxn2r z&VunIm^cr%cYw)GupjnEVpyOxI`3vZ}0oMKsx^ID= z-$3tM2PEs?27T{NfC(^x(kDPchk_0T9SS-Wba=ybHX)mkO~@u>6S4`}gls}KA)Am*$R=bH zj z`>ZBfpBvAz&2WyrQp!sM%$xyV&K((1r$QJDkaHg1v@tFe!+0#fn}=rvJ^cHt{+M#k zjC&0x9->}duw0WiA_%mu-|7SOR1bnXUS zd%)VgpnD(aIRJVOg7t?%-x09!80bF^Hk|+iC&A!pFmwhCe+5R)fzb<)b7B`M=NL-A z@S^0mX5}xy1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$MODCcnb5tkov`Oz3#e#Dzp z^Cs0LzezPuLY{;?2{|)@Gb7l9Y(h36n~+V&CS((qyb1HjIYJR!&Ck{RWz<@DAt-_u z!uYumiYVty=fs?Xrv+6>f$46AWflyTfx~p}-cu}JM=MCs8HQVP_fh11E|s8#!aU?- zI^IJ^M-J8*dVtPoIyyrR(0M}&Lk62gy*)^0g1W$NvNI$a<%s} z!@WBPpP++dlQK&RjC~8nyTC*@*q#QHm%)x6Fx3loWx(`LU{4mzTnDo`Fn1H|>jNEc zMJ1hYgRXbL+IK6S4`}gls}KA)Am*$R=bHvI*IQW2_1DNIG1WKYo_wQ?gJ5 z|C-e9jJy!!g-}FEN8#k0f~N*mh&2oHDlNBnR*smnGe?L5rsxdq)C?)zn>1OP!DImJ*aoJ8U{?#64uL(pz)Tp-wt~3`*tZ{a90Z+*LDvzm z_890s4th?2-jiVcY0!5DZ2SuJp97mNfPsr(@M|#i4H*6ojC>D9e*j~b!1xs~@gvxN z6--_OJAMXJzkppgVCbZOrJ=(yI!u5GFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zLx*dSxCV)9kjB^=B;Ku>cdIV>-Ku#K@+9O*$l(zj9>FGL6S4`}gls}KA)Bz|O_;~f zF)4ye`njavOt*y>f+Bb!jGqgkh=$HgX`N`I=3koVUS!Zj+cJrk)e;TsCAt;4>ruI9 ziri7K$)b(ZpE*F4+T%? z>5N@Z=buszGRmAhgjJ}`3t%pN=_J)PX)gPf#O`n#mo7({9R@fTnMOn?b60Vco%m;e)C0!)Aj zFaajO1egF5D0u?(bm-~O)1jwBPltC+XA`ms*@SFDHX)mkO~@u>6S4`}gls}K;TUVe zJf03$=8vD1`SdIl!M`TWI|DBSc_9?h)2Wc|i6+iItBKa7G?`h-3s$8=|F zR%hs6m#I(c+j4c9zE#xGoTU>m$`3V2CkHN0K1XL{X6a<4o@1+7dq_Guecqb?-TJuI zD1`kH%i+{ss2h7sxy}3@c{#E~e72p4u)%^GQ^Z)UmZ|wEm=jd3~*0)W=addEO(q%ac z=Hg&q8|XL*I!}YHGhppkp!*!?xd3`Eg7sg6zHh+B??C_eVABs^;1U?T0)~DB!&kw` zH8A=!82bf`-vAT8g6+4!NfC(@GCcp%k025#WOn?b6fzl^HM~ADBxC)7@kjB_5 zB;K-`x2!JtEvtDF@+9O*$O#ghAi*YN6S4`}gls}KA)Bz|O_;~gDN_U&^>a~wIqeo+ z2#Vl^Fn%tCB04%%9G_F@=%DFRVOl6mnlRmM=iL8%Zn1rxS%nEY8jKn1s67_ovp7U& z%ni}mQW`ABX-S-f5E;`o~UC?Q{TxOl3dpH%+!?91YbSpMF?n|v%dits12|8BW zB(>qtuml|(n-p2rfvyk0+7ChZN1*3p(7OSw{}l9X0vkUA{hxzPTfo2)x4DCkhop`b%Shc`@T6S4`}gls}KA)Am*$R=bHvI*IQY(h5S7;C~jf)1DDkDn#^ z6f6|Mzb2JC!3#lN2t^chDmgf3SVM!Pp)=r7F)G4N0V8wHx(u~dGmVS<{Vf$ez>9l6H6kXEkP3M<%T90j=GN`0e zS2tjaj)0`|w&C6>I(c_O=jjcbq|73ZP0B1kfZj`B{T0ynBiMKq^j`y;eg*@-fWaGJ z=vOd&3yk~*M&CLt8G9RyzXK-T1>2W^$)#Y&axk?5>{{>9l4($5? zbToiYAL#l7tZf9{8$pjB^lk?0n?PSP*cdo0B_02^!O|(9zQR2}ho=VaA(f~VWz?XAJM%Jkih5`hg!KT^Fh14R7`G3gIv=WHIz&be*n6x+`6NioOP{B}&*IUjW#EV`Yu_)2~|r|sP3 zCx^6i?7BuY2^x`N5jLaR`;4_h-!~Ea%H=+xihF&XNm9+}JSEi}9GFyCc7Xmb!KR&H zU^f`t1BUj3;eBA_02n<8#twt=BVghf*nS*Lo&Y;ef~nJB*BLPV71(nQ%v=Do7s1@u zVBa^OqYHF)gRV4Kdl_{1fSz8^n*r;80)1Jq@jB?wflW8TKpz;aiAgml#F_<~sQH)Fbru<5#RZ?GCC=C6NS$IBhoX*{Qie>9) z1?f4%uy1Cx2TWl+X?Dqt0ITg z7}T^ue_~XdXUJEK(j76$tx0Z;`U|FizhJuaf=MmYUoflZC+go}RxSE>m{m=Gp-e5& z{G5Q4b8M|AVCXnv7mIG_JZOGSO)9JoYUnh3%+`Ah9iz>xw*7yF8#*{JX|lWnhTa9k zOTfrdFuEL!tpMXI!Ne-C{XH=GKG?AqOsxaEJ^<4nf;}IBnUBHj1~B(2*tZFEYzCc8 zpsN|I4S?=#peG1=Tfq7d=-UN0hCzQT*c1T+`@vuo3>^Z)F)(rzjK*Q;#M)@+7-;A) z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3>qnh`qF8MvG zc@pv@l_MtX%n`-{2IxGDabsR6hW)6*w+urID$1Lw=RCZgGcp5o zBGSyUwS9O*>N($R|E1`9PWPq!19TcTFB*J+PU~9py_0dwwsN((#Zb7PgCmn3%SkYH z8jPO-6JLSt=fLCzu;U__`Wo!|226hk_IwX!egLzVz}ywE??=#)0i8dAt}Ix49dzeF z&rQ(V2iDgdk@Pvh#zmmt2{tVT11>PQ3=GwR;W{wl2BWLNm@PgeiORalxz~1NWhB??fU4QRX zy!R>V#J^2@^R;QuSHJvPb+uiYJ@1uy^IxB*K1k7N%^n_H(Wz-1py=T5PiVluU zsw^LY$&bN~4Pfe1uxk^T{tWE-9L#J1vtNL@tzh4F(9r@qL!fIHSQ`f2t)M3YdiR6% zQP6h?Y>a{aqhM1U477p41Q1Aks5wfup=>XWX4a@|= zY|Bw8>EuF3IY)NfC(@GCcp%kKvyx-I;kI z$P1x}l1>%86HT!S)Ny+2s^zWQo zY++|sL59u)J(>skHN)Ni3Va-E|6>-OQ_mT5dd`-i^qi5Ip_7qjj;+RZOzJt#8{wks zIW3pVtW$Igrvf_ABulqqlYOPuEIs|y@bnz3)N|ULro#0c9GTQuc7wS+VBcQQu^)6s zLDwO$HU_$nf}S|&Z3F8Qpzjpe*be&7f=x*oB|`?g3&H8)(ytfVB#{^ z-UB9k!Hx`=`U&jHg6ZpEPY%r71hai$uI89zp96Hf13KS5CiNWGl4I0!O271?#vn@j zkG}vDU;<2l2`~XBzyz286JP>NfC(@GCcp%kK*S7=7`NfrGZ^$9tNrrc(_vo)(T^smd*C-rT) zI!)gy>STRuQ6JN{6Xe&5oKa&?(+2&CQEkR|8f`|^B)2BHHR>;z{{4dK&I=~BOn&~+TF zJpsB;f}YdxbG&Eh=NRbcFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPahkg$I z9Qrx*b9k$CHX)mkO~@u>6S4`}gls}KA)Am*$R=bHjbM*sP#R(fQK-ZqD6%=8lY~Qz1+RxH*#+icx)3;o~?vXUw@dn@i*7 zxTTh3YiNv1H|J1WPto0+#aHsXIc?`IKRKkEW7mi7Ov2C|#UiAe)9^p61NIr!&gZR< z>F@g!-eOLml zA52gwmad}>)6bbTD(1{9_a6JKq1>c}!uarySMWndRYQ^=CruT0_(d)_Pa=D$8qebDTjnp9XF)X!=5nD2LS zn*Zm$`!Kag{vP@L92}YSST=%wKiISx3^akkW-t^0!`r|}5RA5fu@D&F1t!8^dn=fX zfF1k6R21ww1g2wP&rvWF2eWNpE&=wP0v%^S=U1TX99Vk+bYBELUxVIn!20h%-}hkS z51{`N*mMO9{0IiG!p{j^qo2dsIZS{FFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zKZkw}{T%u^^mBNhbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa%;V>9MgI6%kx#!u z5&Ub?xHIxXkQYJ`{T$`L*tZ$3Yg8fqrG7c*{wI|qrt8cRh63j2JkS)X95HjIvKY$E zaHX9pDmqrhVz8Oj^An}^W0X%Ua%+-XO@E$PYwmxKe6O|4Zjw5UwNf{0CS&4^%@UAyj;%FvTq-)QjrB!WbhtHG=X%&h_YyrAO)(D@;|KIj!Q=;w)Z$^=x~M(6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e(feFAiJxC)7@khltIjIBcAt*Uve>XP58nkOMoLY{=29Kp#EY(h36n~+V& zCS((`2}|CDc^n;+BDkoZi~7y9TX-QTf)~R0xe$uz=*;BYoPuWt&6G2P77D}dhn&lG zzov8dp1DO;cI3wupy?Deq(&b@V_qnV@R)@kQqYGM@oj%lM`yCsValZzGpb{B{wY7y zXe(FCf3U4n!Z)k(NuH-Lygh7JwB({icUh>IkwIYC^|=fJ63c>r}Y72bb4>DQ>}m1 z=lQ5|p3l>dS7g5K?5{SMIgCD^zV^zR0n z_JD!CU~nH8Isk?bf|0{u^avO`2F8zri4$P^NicaD>^K9az5=_>LD5NHprT`-qQeB3 z025#WOn?b60Vco%m;e)C0!)AjFaajO1WKO(6&)@@;vyt2LKka)Lh-mSXicdO<} z$diyKA%{nBcm$h}O~@u>6S4`}glxi+H(?${r%VxC)6X^i<&;}^A;>8llv~Elg-}FA zrwZ1ICR$cYG_05CRy5J}sN6F}?wG2H#WOToJi5U&T^dYs8jOayG5JNE+xzDhTi2OY zke~A~n$4Z>Rp9Da3V+PRo2lrGIYnp7P>RkQ(i<{JPvy3%GeKQoH(G@L-z@hV?O>Ts zrGKxv|2^`()-tM+(y-He$mpW1Dfl(chfy_-)+Md$NN-xXcaX}MfxouYqU1wL<* zrCYJd%2I2Vo_=b0ijKu*Rh#3(QFL%$8tl6aI(`J5S3%b` zu=Z!r{R`;10eXK0>u-U+-@wMVPDuLS2Akdi1Mh;tC17YN7+wxWR)Eo!U~Cl_e-BK& z54Nublk32a55UxiVAn@r`eU$X1DN>~%x*d%6`kB?Cpbl?^vf?wev4NA0!)AjFaajO z1egF5U;<2l2`~XBzyz286JP=*Pk@RJ6&)%%RCK85@Q&$hLN+0rkWI)YWD~Lp*@SFD zHX)mkO~@u3V@;Sx(cz+e+9}ku1nm^L@_yXdDg0~7zh)5?ok~v58P?RG`<0Z6QBigZ zn4vQax26yaXAb-u>mZN0c$I?Am=koGhZ1x~W`<5ix;eI*Um@t!p1DY1S6-x=ov8f6&ODUCN6;O7s2G$V8=IL>N~LO zdocY2*mDWYTmiE`g1M_;-!;&Y1)bL+=(utebT~tY2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?cDfdnY%a0wEZAaM!O7+ZqGTUGN`)g`}GHBUmGgggm3If9cT*o16CHX)mk zO~@u>6PCOQ^9VX-MbOWnpJAk*!3#kVyb#9Eg-}F6heLA;8yHk6I$s)GN0~b^qE3Yn z1;{yri8O;aHU+*k9V*VG0gnpop`SD6{G820`8gvqK*u4~99!Mtg!FTMjJ)@MS|8V^ z-8>?v-8?iQqz4_X-=c`|St-9K-%%1nk zy!o%sQx{*!@8`6gyZq$v{2U?uoSMjeevVmf_A%RIFrvp|nIzxXdCDX`7I|pW zW4Q&oegkXYN=Ulj20ib9-gm+JC7^F9*ti_@uK=4?f`L_F@I5f}J{VpLM%IDR55U-m zVEiL6@iExG0Ze`hc5DJupMhPUgXt|`&lg~3E12C5=5~O6UxJQZpfe1*TEW@~=-v-{ zq6z8ecn>8wK!*c#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz2;=@X!zLqCUp4*eYZ zIlNgqn~+V&CS((`3E6~fLN+0rkWI)YWD~Lp$5<2Q@pHH$fBdY-r(dB6{xxabnRp?{ z3!#X9&UERWXyQ<{CYoQ>#F_<~sQH(4b`}{l(Y8#YWwk`ZdWmjD?s`=2nId;g)x_c% zn&^E-6K9{*MC)_ff59N0DtYV+h3J3G!t=olm0~G7T0xG^K!Ij}PgBG%7eLJs^0ylN z@QeHE0z_@rgt>?ZQs9rO5|gauQp6FeqHhiAU-Yd}eL~-w?rhEK4E^ge^+|nOu1?dp ziaJ@}TGYq&?F9L?VlmhZYTBSbF{;h@PNU7Jn&j3bw?_R1)4yLZ-Fd;Jmgz5;)$;6hWAfOL8qN zfC(@GCcp%k025#WOn?b6fs!XcKZh%jxB`hQkjB^wB;KZ)x2Z1qZK`<^@+9O*$f*&W z8o?%H6S4`}gls}KA)Bz|O_;~eDOUs+^m9SKfo2OY1V!*d7(W+65&azHzu31Ku4_~w z{iS|6=l&;^L-{JdTi2N*Oa;u)`Hwhs|L-;ud#Z4}(v}(3N7WzG&v{HbH03tS9ZzRm z`Z+$S=GdCQgP-$-_dwD8obF5c2k10xUNrasonD{WYS;aoN$_(N>E{^L-oM}L;tVe6 z;LxPVvKI8O1Dieo10RCHkHFB!V0Z%<`4o(90%M78KDZZNY4%>dKtYEIFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d70SY=4bSUUh(4nBi zo29b}*@SFDHX)mkO~@u>6S4`}gls}KA)9cFHDMk>hfDIu&ysu!7K-3slgeEgF9dlZ z6j9Kb$)Py~4-A?q2L>$^rrQsBA`Wi7%q^;~BR{TiM@LSXD(>)_L4MjyeXb2PopRgI zV{~qh&lzP+=b!qVopL#N2a|cq)j!Lv!KSEh=v#}rK;KSK7uby!q5n6_*Nt|tOsCSn z*WCXe`Ce<8-6WM4Yo%_#Ooshpvjn7}V{6@dN@_Z8=Y>(zbXv2A2iJ7k)|>BDa*T3j zjvNto@6N$J9UPnVS-uBDKY-y&VB`uI{Sk~^1>@Ji#Lr;+FJSTp*zqfvx&?Oq2BzOS zCE4>fn0W`xz6<7-fPG6rM=j{A16^*gb~WhsfSxs=*9+FK2YvNmV*}{-flZ%)fkrU6 z5e)gk@MbX51V)=rNlzyhIK@FarC)?rwxDtsgk+>3RjIBiC{i=Dt>XP5DnkOMoLY{;iAHnevY(h36n~+V&CS((` z2}|CDc|0AXBDk=h3;P8P7hVX8;Ds=LE`%a_Iu+7A(ZtzjHPQNn@H_x;XtA(QNR_hC(~nurf1f}9A9iK-yjSMUe|?@>S2w`XG0RArPYlb^!Ldn~Wjh$# z0mi=s6Fb57-C%ML*s&K(?E||Gfa!x^&tWig1k4@-bH~BH6QJW1=xhgFXTjPe=spj6 zIzVqHSf2uY--3-@puZbzN`ryRV6X=a^@8CH82Jf|X2IBXFrEVwH^KHkI6BFi({yy` z=r92$zyz286JP>NfC(@GCcp%k025#WOn?b6fzl^HM~99M9UVG4baZ&%bT%QIkWI)Y zWD~Lp*@SFDHX)mkO~@u>6OOSa%;V^ARsQ%{l~2b)&WE6EMA?WJg1iul=;%~IJJCeT zYKeyR65Wa>+8&jArpO&rHL-YxMvX^Tn5IjGX`zr2<=1rjm7Tf8_H||zB(R0FO;cUhH`X9W{yrm>N&Q~ z<8X9NU0C*iSs&NEnMdHwOtAj{?41vM6K8(rNB$?Hu*r*)ER(Ri(6p4c+iu#lo0k5` zwN;vJuO+=+n{w?n*QU2zdQFq;rI+q*_qGuN95BQv0vs^JC<-`Wh!G+fV~7z7IKl)+ zC}6;V5rPmDf(hYwxw)_Q0ecJ~!#%8BabpW5!U`rty5SZM^cbaagx!Y-zVuP-MxD z`3LVc*3m&%%2ichWE~y6H|ny?0z2k_oiBp1d0^Lku;(SPcOjT~8SGmG_AdbkUICre zVEGELq6Vz=fNn2XwF<1R1#8xT-a4>$Jy=%{*84!eA8dFFY-#|T8^K@`*s>jLZ3f!{ zU?d2(?*cnoz|IgDYX!UBJ4hX!o;L32Sn@yrBJUSzWdZB}JHQUG1MC1hzz(nj>;OB! z4zL6406V}AumgE_fIB)o3yEhT@hqfKHVcU_SIw8J&im!6c_ZYFkT*hJJ%U${;3VWE z9ii}+*%5Ns(V4(&=NMWyh)Oz| z+qj$3*~wxR9b@gA=h3ezHFLA$$GM?HtK$^PUsYDgDUzos_r>x(%6*AENx8SmH**yo z-}M?gH)a)`7?pEG{MB}9=xpAzD(4NIcu(erPGH}r@wyG2z%tD$I;FxU$2$x)bnxD& z$+92p`2g%a2qq4JeMi9lqu{`C&>02GV_?N;u(AtucY{?uVD)*hrWf?a!P*2^_X$|v z2l_9A4gFx#6|i{#3_9A0El#j?HrQ4UM&^R;6=25#u(J}3xxg+r*s~bytpXEE!M^Ht zYUuQ@Xy;XQxS_)iumkJ>JHQUG1MC1hzz(nj>;OB!4zL6406V}A(3 z;f4-hF`bi;laP~;laP~;laP~;laP~;laP~;laP~elqF$?h7Qlkzjfy1bHll8HPsx=w@=)drp`F2Ad7a6`KpcW1&0s?SYzl(SyTD)z*b)L; zTfw&Xz(^a|9tJxiVCMla)(&1Z!0jBKf5h{Tc>d8Sn}5WYspiX6 z=lwF(ybJ`FT2jG51<{N3cq~ zBiuSWLJr$Glc{xLmqIh_lJ9A|wDftqthk6^0lRO#>mf?6_iTK|3&n8K^a)A(3*O*DD@h8 zt*yAMuypPe+XP#IB$eGwUzUkdC-rlLtLYGxbS`-g+dK0lGf{tNOs|%V14E=)D5g4uExz zL&SO~=${QXl!Hxk!R87uxBzUa1Y2ETn;VQQ2HUH^j-_B{H5gj~cGZAA9;OB!4zL6406V}AumkKs-W}kQ4wrPeq{Ag0F6r>)(m4q^2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2{{QzSrTR_>F~7tTW4B6mn^Ij|7&vXu7Gz0+7WVC(y{&&mu`#}j>b`m zDb>(P|6;-o>+DRIq%4-vxqheSx}}?pFL(k?94Y-qs%~6E=LvLwO44uB@!*Ymrz8Ez zQ>C#~YAF$tY*p?p^4FF70{N@Tec|AJkvv8DzF59Txi66?Dfd?SZsopo@O}*4Td4+P zk$Wu47X`8pKUp9OA}lI_Y9#qxC< zI@g*xxo+sVshlG`a}QHP=kS8_IdAB6pUb?Aj%WRB{bh6E#SX!DY=-{nUk!3qr-3-F8;7>j{jr@@{su(umb^niWm!Tw%wAPzb&g5{rr z6_>!u&p`L*VAWNy`U|jT=3%0D7FavyFcoy_UOdbN9WLmw1MC1hzz(nj>;OB!4zL64 z06V}AumkJ>JHQUG1NnD=3pzXniKig(6r@o$1&J?K%@?cA`^BnxBjk;cH$q-Jg4d4V zB;+LIB;+LIB;+LIB+PpfW+>q)j;nvv^a#+y0pI6Swdexx& zY1N=qPlYliy`!UkOhEn$FhCv;NR_m(qB-ofrCDYHzIU5{!WeYngr9gj{{UON%5z1Mcz1CxIE+@yzRJo(&=8D-YTo_Xq-XTSf9 zJW@?3(6>jwrW4+*sp+5$MSth8Mvi_>2XBq)EEQng0OTm_E zuyqC4Rs%*nV7nLWSOs?0g0VGVR~^{19_+0L6F#ud5B9$W4m5zytzh{!u;Oj7atG+% z30A!WR__LD-UYpTz}mfF-9E7XebB!jZ1?~*ou-3a)3M}#3P#>9(#itZ0d{~LU7TytPN62AKXB<^e?9$-}?NZ<& zyR_yJyX1P*E?s`iE=3==OQ9Kd$@jFKt321LJXR^Zz>=}O96HI*wmS+AsXGb=j&0;P&mqEvATsG6g#=-ilZ(urSLF5AALRD5JqeD1UredpO{zdO|JiF=EN*KrC&k?dNgSv5z$ql5QGU6#XO zQwP|53=DRHEm5#F2DY6BBVAy7H`vhwcAf`gy`8#VpMZ%zuw zbXs3O!b|CJM~59?2iO62fE{25*a3Ec9bgC80d{~LUcGf)u)QAa@PVCvF!mPM)d2Q1g1t>(VmsK^4E6`WfgtGI4VJ$P zR_p;Q_k!+yVAcCz^?tDC1JHXAtUUzQ9Rcf)g8t)R!wInIB-nfk41NfJHQUG1MC1hzz(nj>;OB!4zL6406V}Ai~=m3OYO`|JIq3&jkys z#Q&OHxs!NDpdBHH1)T}JaE_rxgQ%gSxsAIig`F&x(9w2t#@+Y^&E%Wj(4j&QmvoA4 z)=~>C1as^*S}X@E!j#f|A>G@RCyHg?^-4NFP*&L~m9MR)Ge&;Cte{j<{x70Gxu6Uz zR%lHBM_7mvjRAHjURU z>9|~)Rdh;3sT}qe3}0GDe-$0PH!8Dy0(SI)otME_KiG8z>=^)i9Ua7k6YQG}_LqYL zb3x~Pu>2*kVj)=hGU#3eRxJUmUjb{Df!^g{?W+A?QEa~vF zIoT~4G(n0!u4(6JzJ21xly$~QmMjW7daKzyfhJ99Zg11^2kQN$H{w}t=Ulg)v;KPR zoNKL|T(@&#RLl|a%A?fIdHao1IdA98y^y(`6Fz$W9%JpCQel&QXAHG-@ZPA!5(Im8 zfxRtYA_Vreg8lD-18tylKUn?&SaA@nJOsLrfK^Aq>f>O|3DA2ItUU$ReF)Z{0sUve zhI3%k1+e)eF!(XpauIC(6l}W$Mm_`EKLJHQUG1MC1hzz(nj>;OB!4zL6HcYxbD+|J>44!3i- zox_()=OpAL{4ikUGhC`mzF+nmmEK)#?EYuT?#KCG%hB1RuH=XqN&H*?$U_;+abC{RPRAx=q8^+7)%wkL0SbW6!FgW1xU@Kgu;W z^EWI05H9JImWV?6tF$Uj2`ytYMY%7Q?@{hc%esg+`Xp!F!`Jiwo>`g9D2}=PO|OGO%JfSotdGUI|vc23Efg)~p7- zZ-BLH!Mb%|{hOeF1K6+;Y}y1iZw7-~z?Q9G>o&0MZ7{L}Y~KlXyaRUb24nAnU389VqEk9OIIXCI1sJ@_v0*7Qha$1MC1hzz(nj>;OB! z4zL6406V}AumkJ>JCJt=xTM3=ka!vrPeU4I(~$T|)qJJuykDuBH$vVBc_ZY7BY5Em zPC`yXPC`yXPC`yXPQtt=VTO`UiB;lR{XDDR!u1y35v&sL2)E9Tki(LW^{2RWW3+HI zj!I0ahEDny6K+^%XS!67#WFf(N;hL)kPP3Z;zzil^OM)mxiQP=IH{Z?TZZDlX<-RdC%Oho$X|n54&7%*OS*GLB$;xXd;;q7r@$&z`BpY`ir3dQ?TI@ z*z_6L{5crB3buR!w$408Y?}o}=78-lf*td~&iP>MC9rED*z+>jy9i7y0sCG7`4BhZ{QF(BVs_a}shAauRY9auRY9auRY9auRY9auRY9auSZR zB+Ssz;W_!Y&YXO1SXd?g*W}J!G4BYpBjm85b3ZSgll8Je_tUaLFGvN~Zq$2BFX^Zs zc_OEU9s29CDCrn(*<>u{OhffX%I5YIUY47Z&NTcvS*-;06ke8_cbfg5Lv&kt>;PN@6bQzr=k;np3LJRL;Wu6lGYjkfBR{6%1bllX>5uVRb(uwxe-L#TU zpl^?UNykxJWGhqpIAhVrv6f1-NKv$Gct@uc9i7IJIy!i3)Ma@Stlj|DYy`cVz}n4V z-4?KZE9l<_HoOfs?Ess1g28vdmfc|MyI|WMFtQhH-v@TQ4|eVcV;_KB2f?00VDAwy zaTM%34)&h_2Tp>{(_ncQSkVnu_JHp5U{xZ-l&d1g{;zNytgaNytgaNytgaNtpK}%+S#(uu43spC|Q8+-~6=!7A~NaO>;{ zIqc|+quz;KI{ctr3Or<&);waDT#wqN%a7Tm=;L-NG{Y|Wp0;z3=UR`)DuowVQY{_z z7qrfex}TH&waLcL1WC%GqLWpxCY$b!`Ze@dtKX^S>j`+{SygLt__C=#nJt~WlvYov ze7ABxhVHFYn6bz`7Uhcq**ArrDWLvHA>9|!eSz{sVe*N>!6yo-bAul(l8@b{{6vvl zs{BNeY*(Hrmi-0vfQ4E)KTuwf-qOM27OJIOYwL`WpD!yYm6ZRB=<$Lwuvnor^=MJ* zHS}6raakcXUThPT^2^;Qzlc&V6?BB}-Q(2K+1Vb&(mNmW5T~e@*V)74eQhJ3%ZmV4i-$9Rzrw1n5rMXk=T;x$oOp~d^MD-QBZfk`~EKlURu`?~Jiq7B)KbjUz z4!?a0@2ejz-lK`?R%Y(D~a90fa%gRv7}*GaJF6xjPAm^cIWodx^PfddynXB;d~fEAyB zm3^T5GFa6QR$l>Y20*W)lUVBn>t=)X<)D8q*iZpBEdZM;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRq96eOO4#8Z$)*%TzcS~Xv- zI`3Dj=8cdyLf!~@@d#c#f|HPwkdu&;kdu&;kdrX)NtmIaW3fs+qn~H=7jm_QcLb}% zJHoBABjm84Gl7@RF|=%udV!g1Uyr*fg`F%G(HZK_q)Ie5^oyz7(2>7Ni_v7$(7Cp9 z&dpgt$CsamPLRquB76!BomusXoHumhJ((Lifqk3C>o#=a-lE}UoDy1G$VW>T4Xf>x z(uzWX#)9Dwjum6&=z&6Nve0{@Cd(^e>oTxyIT(2rY+ng>yasl@4#rl4U2lLrYr)=i zVB$@%Zv)uB5ggbAIvc?9MzEp@tlSQ|o5898SRDjwc7fg&ur>tNwSx8Uf&MnIAq+M} zz~%#BupMkU47PTFZO6b!C)gfEL#HFg4INAVXJ6#~`m8K~9bgC80d{~LU70a|gq(z&gq(z&gq(z&gq(z&gq(z& zgq(zWq^LvuNj(ydd4w+f@8@ynfE8Yv-)bPdg_<#T*gcc!JtFrs6z}}C+#6__0Q?UOM zIPe+hyaJXFfEA7t#7ZaVo()!&gVl4vnhMan0IaP9>s(;H8}u&*8>+yjrC@V47+e9i z)PSuXu+0lbR)OucV8;OB!4zL6406V}A zumkJ>JHQUG1MC1hkbeibox}5wc>WR3KN@B8kN7gxe3|OJU#6NjLf!~@Bji;hc-06_ zLQXdcY>QnAFCa87CU{m5x~=rk9rVa}dSIemnmffV#lK~j4nJs@0uR}L z3fAG7O7HyzsW5e$hOZ@^TdoQ_c7qL_X*Z=i^VChR=TISND*n>3H~BVI8mp|6Bgt0f z-Xecpxi8@Lb0n$kZYtK4iP9LAb42_LG<0^)@#egtGt&Awj##s-+tBe(7)yX%pMX7mVDDux z(GT`rK|`m1fEzlzehxdp4zL6406V}AumkJ>JHQUG1MC1hzz(nj>;OA33LM~u4mWhT zp~DRwZs_o}(m4q^2{{Ql2{{Ql2{{Ql2{{Ql2{{Ql2{{QzSrTSw=uW&22W;?yO{>7>S}?cJHQUG1MC1hzz(nj>_FZf;F1ncL*i*jJPm1-O+(_VRrA%V^M18z-UxXk zS1AnT9`yOFE?` zqHu6Uog#S(^$X}y-i?9xsb8=nMaZ*1=xK^B`lFp0A_usUV zPM~j(eo4o(NK?|0sHEdKY0;K+Y_jX5W_g{l^vdXgLMpS+d!sT-7_5kZl?Om~J6LrX ztnL77j)C4zur>;OB!4zL6406V}AumkJ>JHQU)-vKV^a7l+tI$YA>k`7-ros*E0kdu&; zkdu&;kdu&;kdu&;kdu&;kdttfC1Hk=4o}O!b*ANW$-*k}zb4o2EW9Joj*!EW&NwQb z*rme{+NHolc4^HccFFaqUAp|3U5Y+#mqIh_lJ99d7kRD~d8|@+!C*%xRo1!o#dPV$ zR(2*x1zB`-zD)CGDtb1OiQAXy=qQWfWYs7dVf~*k^G?k)lgK5ix0DD;wkr1)`RmGk z!C*IMbaiz6*IP*E+5$Sc?&!Fwo+CUx=;(BZ&gQ(M6BuD3o%ogIvh5qnias(e?sMOr zwxaJm`|NkitdBkO)HBb1{~0;3tjJcTJU^D6FSC|P)PV9#9A43}37Z@)*DS7+s_5XY zQI(|ztoDF4UeLP=tgQv>)`0bOpnpBsP!Bfwz-B)fd<$%809zZuwk9yL9c*s~I|5*5 z5RB~tyIR1W5ZK!aCf)=4+Q9xWI1mAy2f^|~V8s!z@+jy&4pyB2t51S8r$FzAVC@-H zbn4D>MaPo==@)swNGl6q2iO62fE{25*a3Ec9bgC80d{~LUsloJuRA841H2oaSw@>7{t}`vGevX-%&EU_{dSmnkv%lSs7n^C?e3_q}n!9Cy zvM=qJujt&RRC;dCGCCWsSJC+aiLr=WS9HA8&Jn)G7*%xkyfyQWYC2k$jj(J2r`vg5l(Dmr*;RAo62*7bt*anPRt8$JP>`oQMPV6Y!-xdOHh zfNhQ#G2#T;pm!Zu`zBbo0j%E$`Zs|Mn`2bbY1$IwRdl$b!w#?m>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz*c!0j}t9MTaXoT+!i*4qq{ylaP~;laP~;laP~;laP~;laP~; zlaP~;lW>$JVTOthPs-?j&u#wijda&(Pl_=Eb3=-|Ck zlcfo4+7321gTVmU5(Ha!fo&~dBm}m%f*tRHoo!$&40c7po&#WSJD4~O_H}^$$H0M3 z(0LLpKLu8N2v(i}-Dkn7b71uau;wGs`!QI15v=f&0m1QnWu;? zvrbV%r*+OLZs_EH_C?;W(8>bX0d{~LU1Rr5y38zFCmymSOF9l=S+NytgaNytgaNytf<_aw~F z&?&M?x757!TV@Rm?u>9}g!fHav4eMnFSjG)u%W}N=47{G&;%*^czTOP{m2tHrl~Vd zlCo&$=rw6{N;X+OH5px;63uNK9^rycX^AM5zpAW|QzTDO?u+Gnl=~8Sl5%gA?^f<_ z&I&r~uUF8yF)Qc;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRopL5B-ET+rcy z4qq;vlaP~;laP~;laP~;laP~;laP~;laP~;lW>$JVTOVZ&wnrYa`WH0I$)L9I_|d> z@{TY%cZ3`kbS6{r#4d$q*d^c7c4_JJcFFN$D(%d+*ro6SLgQkBX9c0#N)O#ZkBp}W zCfcRBQ|wawTXyO2gLWzKko~7%DW0kH-d~WU)NLBR)?2tHSDhVuRwW&6gGN)R$>F!p zqnlG|mrA*zqcm?uS3@UHOXx(XoFk&%)6~#8e<_snhR#S!=s04{vTj4ivtF}=PAM8X zjRi)Q(7}77CQCcmaTx6E0At6%u1>Hg3iigp#A&dv3+(R(2YNu~1+e@hu;OE|@*?Q| z6s)=gR(}T8d=7fAg0)|Obu&*B>t})fIbg$!VADLXc|I6?32a#iw!RFuEdnD;!1h-mlNf0@wj|fE{25*a3Ec9bgC80d{~LU6IY^^y4iaCfny*xy_bXNNM#vi>Z-l&X1TP%HNytgaNytga zNytgaNtpK}%+Sy&;WcNh61NO2qqAkeQ5vIij)=$6(Ajq?n)8NE_qoh#=y=x8 z)?Y&>?oDgx*kqrh&`3iE?~R%)tH7RGuy+lZr~~`fgZ=g3fDd$T0?Rjp6uE_VISD^KG?h;41NH%90Xerfo(^?$WgHUIM{Il z>^up^PJvw?f<0%z-m_rh9N2dO4W0gvxS_*q=&%Fq06V}AumkJ>JHQUG1MC1hzz(nj z>;OB!4zL5GzyWUPa6^Y1I^59Vh7MmZos*E0kdu&;kdu&;kdu&;kdu&;kdu&;kdttf zC1Hk!4wqB7oWd=wVs2?!CEgKksU0DQ4W0XW-JGn~47#7z40=H-ymq5lNiXTBA9*6D zg&q3qvM=daZ(?`me#yow>5S%;bbdtQ3Pg!qO!tL!|24X|2&?=9<-SyYUb!D5KVMc* zDk=XL(enjmV6j4L>d~UqYv{GM;*)mg_UQL?!YedA9h(>{yLKDu>EOLlpCtkIe*zBlfzHps^3TDFt6=39pnK+r#Hv|f z^&GI~MbJABtep?my#&@T1pP0A4U530C1CR_U~n1OvK(xE6>M7xMqUHkUk5u@gPm`H zv9(~=IF`QA>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz(nj>_Gk<;GPc8MBO5jHB*} zT{`@rT?#y8m)1ODmt2q9rOS`mrRd{!DKx__`JT3Om*-lS$0~&vSW-0&IHMlMNKC|V`e&PG-Gej?FszJ8UE56`ryMSKflC8?UMgF>S zUoa}y((xB4-J24ni<7CNbFG_`>yD0->N&#oQWte}7H)GrV!KNz@Z3%Xp8LkwERU2~ zrpc~HHIF`AW|@vhCo8X=h}Yh0yX}EVKYVV|!!w?IZpMr<>toM6^~|&1e?|_Bu$E5z z%5vHE4dv%${QRA^qVGKW?01KL(B;x}bR_ENII1k#wRFbH(QTUI&RBY7^gtnXS?Ik{ zmt`kd@eWwI8+5-5R_y_+_kuP1K=1ot?S8QC1F-%e=syHD908k-g3ZUl;0dthB-nZi zZ2J(5oB`X#7Ae2~u>f{}9bgC80d{~LU>@_%3RY)Rd$}pbzf&%Rt25w7jY(|QX}Q?HV%K<^*cI~bKR;Lcf(poBXxA> z@t>@k&W7vNbdpQyT)WTJQaWyG=m^hI)O6ytYjR%GX*^$S8?Ur+#$oB6v8CNsL8&D> z<{!NGx;33LgefOX5j z`sJYiRj^?t*z_9M{5lw14Ys@iwyp)+)`5{X!S)Sc$40Po6ByeJc5MNBwt~Iez{J~N z-wv>UCphp9=xhPYLtsTKSot34ZUd{rV08qnIRJXw!P>)U>C|;_OUJ@39d>{nUQ1*`dj)p@^QHE)Ex z5%NaJ>qqeV5uAjagq(z&gq(z&gq(zVPr?i>9Scu_|8kSyx!c3to-fgz;LV0No13%Q zI_c#X361`Zv#`VN+n{X2*|TzZ;a>*HBj`w|`(Zf13G% z&Rxp7JEij7%KaF+x0YIjMeeaEUlho`DfCPMRYwZxzL4$lJiv%sM(=>gEXF+A~zp+3ybIyr2{B$z0G0?AtV6x1bYP zR%9Di&?yi_a^QOzyEqviq#_HwH7c?k2kTCN^(R69DX`%~u;~ogd=?Cz16wYDtsjAH zAA^yLVEdJHQUG1MC1hzz(nj>_FZf;DQbpbhx0y1syKv@FmkZ2{{Ql2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2}fBHW+>=N#J*>N%yQMWRIZY_?=9;$(b~S}gS5sKv4lY_{;XK&X3pU5WU;=FU1nr#GK5pmm z@;U4PJHQUG1MC1hzz(nj>;OB!4zL6406V}AumkMCC~$z=IXwS}=O6L>qfs{hh_6!3 zSE{56Ep>Z+6vx3lVrHAgIN5<0w z6YbL6DRwFTExUC1LAw-q$o^BX49`@0@9OntreXaU?P@yeM{-rzp@nL)Xy_Oy*Ic)W zGY;*WlIz|k<278;DJ>C&@>gjUn-W@?1?%9Hh+_F3<-SCoq}*EvSI!wFE9m@yL|H_s zd~GG2G4k_e1*MYme-S-iPzDw&w5A>{O1*|&Yb!1*q{54Bg3^7t8{HRC8l--X2=DHu zlFp)@k8)np8EFL_N32=aFX{M;hF5P&=&d{Y4V858-l)v-8QA(c*mf0+d;zx4>?U^1 z0z2n`u@}Lvd0@|cu=gb}u@LNg8SGyK4lDtkOTqGLuwn&RSp&K~V3ilFUIo_Fg5EV? zZ5>#*9;~ki{XVe44>r98HaCF5MzEy`Y~2pFHG`1=*dFYrl1|62ZeBu%m(XDc*a3Ec z9bgC80d{~LU-?81L)OT*D`p$i0Y?epLEYoDyqnbw_F0)L>qmz}_PQ+{PwcYl> zq#r&v>ERhqJ~v}Vnf0+}o_gll?>{4VpUb?Aj%WRB{bh6~12J7dLI zIeMTlq4!2jmUqF9Jz(cvFt!iudLQiB5B7clCJusqhrs?L;J{JP*$I|M!HO7I zc^Y(gfmPjLbq`o`9`yEtwQ;a60oH#4`uo6!%V1MK*n9;H4uCC=v&2>>*ftxCl!NVa z!Hx>Ba{(Bu1iM^ksiD*3KFbZA{Lj8{Sfu>^#{$>^c7PpV2iO62fE{25*a3Ec9bgC8 z0d{~LUahQ!-mfNymU_1%Ld&~%Lct5 zN!M<4aMd()G~Ygv)4~q@by<{jGW2NDn>FSixgXse+qIh&UvN)n^!9XqL@VYLh!VM& z?hEPuYsv~bR{6%Pq!Xixj)Z2YU{Ky&YiU7}(c| zo=$(1dpZ{G>97Or06V}AumkJ>JHQUG1MC1hzz(nj>;OB!4zL6HcYu33+|%Kn4)=7p zr^8oG=OpAL%(wax?lIu~sbonv66n)$-g=W|#-_v&P@?7ikSf%g+ zORA}({({!J(bRM_-*>mlBX!&7t8l3 z_a*Wq<=!gat=yLm-jAVsYpF$8ApaDqA>YH;ouX6)VRTq z7RkqMQ+}dIE>(V_NVY3a6wCesdcZ=R9CUFqb#!jbS~>};=Lm`N-+ZTf!Rbb~*FjftAtpIy!z+MlS@Pd7-!2VirU=8S82bRCtLmi!p4L!V;PX6a# zeZ-l%N@E#!VDdqVs1`wbAp=_H?29r ze-SU1N2xGPrV7&wQlVWc zRJU`|zkVXueVu996?E)3w=whlOHjxiu{<)3&ztdiGd^z?9X`!w__QNj45~n~ zJUs0$ADjNub{->#A8;moi3<;;kLkzJQkKb)Mf%c_<1&p(M|v>Jq2$jiuN&%m-=}*To6^=RsQWyDC|PsH>0jxbh?_Y~X4;ZGb}+HF;aiGh8a#@j|N zaNmX(my}90!k26#e^WoGcJR-3KdSuMq01+KHlJ_kvb$_~`k2bo7peUMuM> zkfRUl2=)yMF2n1;QtDk{5lVyp$iCk(r#RhVN32;ML9wr@$o3T|D5ZbkA~`V6lD%Xa zbc;Y^fgEbu`j_S;cT>ZE_(tJbKYJvTU9ObmBHG|RZ3ZL<8w&(2(kX|QuYM}ml9eSx zGjGS9MJdT8v`;z~73jFUad5-96aOr?mqyA9Mf3nkzT%o?F{c?9QW$QtatVdZngh#H znoDVE)94`snqvnF1zmhW4jsQzZBldNd97y0{Db$7ur;eR+vtO2U`3&h&;H>)C%i?% z@iA#`?xtsMUdB0gqu?1VRXjVYEJ@q55bZx&A_8lzawzzUyTIHQoi17@?ZPWkq7l&14kMz2Ypo)DiA29X ztInioLvg1%svx2RD+>%b9l2x`(cPtTX!b`tO^P;@cPi1yXwPXQqGP*DMSRg1IrL%2 z5tE|R_d}gQohGCmYjwjzvm+)JjS-F*iEe56*raGffu|C!q;#{&=_1+n;^g>WX&zY{D}@qUYAUW>R!|dgu)A zREHLfNPT~$D|*xLgSdCVCVcPRCWp?>+nRIHdXYz~!4Kwi(p$~9!;Xj=)&BQx6M^cn za;Wvwuu0X1GEa4MQAW~dEZR4rr#e_YR)o7r^}3VYCRG~>J(X%@FulIOi0Vl9SP^ab ziX2+7&ry`?eNiX%G*KOm8&Vx>_=D>81Nv$G+8x645s7wPIgxYGI>G0tO^HTY$NxcBv?lkwAKf8*Zwoo}@tI2|MH|XJ z)kiMmUU+qC`^aduMz#NKAp-Nq$)U&x3yaMqh@sq5sZPoqJjNuy)lKX9<3#ufsou2X z4U?+V_eGuDQ>Aq&#?_6f+{;FF|*sXE>3>~arj?FgpQIvcMOpOy*7t~=#WyYnNHstx6y>KLP>^%~XQSStHgXwExn z$JV><6fXB&a%f9$d5O8bF7WOXYZR}Y`l)NepN)5 zj+a9#<_(xso$j?x?%~Yd;3i>%>he_XWurQ_bi9ank?O0TEV1S~L3BRos*W+r*t$($ zwI)ImUE_sgvn+=?qi^P1wNCEgI4)f$tT)TTHR~QZwBx{ zJse_EqUi+nx09lin~QE(&$>r=_LFE&<8hOs4dtFnv@*2*p{{66?0NU!BYZ3Gl|wUE zeQHv4`mU%Gdz!EgH>YARqtzPK{+0KNzy(qrTli9`xn0pv?5R}al!9+oD)zEb9lUU_ z2ydGphu-;ol}XjuiC z?%@!V$~{tDHYEDmh4A%q&sqC*;rfI`zgstZjJciBQ0@&AO=qqDASpU&{4mj)1aW`z zb>RtqLk_K}T47Rj`o5@>d#VpH%E-Drm3!H!_6ENpe6ERdXrA*eld28no=UYcwBBn- zwcj;S1UgA|&xO4vRi}HclY6Sudr0f}!c^{M<8`oeq6n{_B!~7MIb~9{q1;m)W0axw z0Yj=I>nDk5KdD~7`}3Tu*2z7b-%Aax$I8BCsocxx^;^S#9TV%HB;ui;kwY(STx2sB zA%=2Kb&SC&1jSnuZ*!J=VA>`xk-e6MmS!%PY(62SZ7kUq2z|MG3BxsOj*9LN?-QO)KP!iPpLCei zY$)|qAH0--wOey)U01Ys)6WXu%%77(FGmw5MW-8`UFa$2tPdFx?VtH`BJe(mPVB8P zIe*ho=BbV^%D7s4E>G9#;QK!(!mm!2LmzDM=3KN+A*fv^HJ@p9MMqwpETZQ~ z^johtnG|g(@l;0_WmLUdGxKe@Xw9H0c5bqWZ~b{Wv}9q#q-aBVrxLB?^8VDM3QSjs z-<=%04kn8xJ`!7hUO47Xkwbm`XHBY3-xGBr4=FupCuGtLz2W8#|4t*?Id_V1wUg*W zT{Fj;J0cp&Je6o?k z*O*joDEOdy&`zYaY`y3kI=7en zJ^A3L#jIrePyLDrZ2XoSa?YK5ySe?*Q1+>gGs@t4bLxabR2G1-wXz9pjj?w3QmPHZtLJAIGTNk2_y$D3296tZ$Vw(ousU;e9dX!YLr zP0BVDe=6C^5PO?ym!#jN(+NM_V@+cDuL?&G$u8e=#-!|Yw{`Llr}$K<9hjUM~tHRazYjWs}_Y0G%4dtKeSfdQG|5;zPHnkgnO?b*5kVB!$SMD&EBk8Jj;!kyo z52@|gklI8to2`-UEq_4x4v_4%{q-hg8;U>G!3Oacy=Y9f|G)zx;Qe(uRMFL9QnsP^ zQ^`(FP8dAcWaW0y`|BbcC)wxrcbb%)zC-H7pDMMJzwk05I}-nO5pDhrIrQG!eI{ia zioap9X{5b3mD-uz)(rn*&A%by3%)If-dtNL%dGaxCRL{giOz3!aja2=w&Q`M>g0y5AKFL1EnI6KltUjmnscsNC;n6u z`IMpdzv-*iq@R23gTix(REN(WG^yH9`l&vQDMRg{<9cSZHL|^z9u&UazbS_{9=>2w zwxRS>$wp{<9I3%yRUoq zS53+`lzuANIM0A;@nC3Y<#y!cZ;9xJsdDJR^35h?8%jUbX}{zz8FgiA;xD#gs)%1D z*_&V7XHvGI_`|U#6@L`ksWB@)=J*`ib37W=iK|nEfez=mpZSQRBb5!R3B@Ukv3f!VQ>@4O18WDcZ8>#WFKx_ zTxKpu48@;HwldTX8I$ep{vF|K_+2@)wPC$U+37o^PW)+7JGvkhe_6ThZ}?pinEiWl zsM@>Bq-;a+r;?3bFhIXvGP02bXaAlEM@aVPFC8-}JKb%a!JlecA5z;>l}hcb+>S(k zPeeTr$)RJPT{J1%Q2eP5HaOGZ*`ShrZJOWkB+>k0EarJg#Cu8h+aJ!q)7&nZE?Y1E z=;Yt9X%>TxxBrtIs@ea#NzsPlZ`h%RPHO*Ya>LH7S{vGL|0m&^ z|NC<2i|yM?sy38X5Sha&p2T@IZ-{8X8pRHkRO zMz#CO!@|@0M{=mU+Z#|N8)jGkan&hVp zvT3xc=e1@86?yABBI^8OIdp#W$0k)9%01OlM#*P7rmJGmGcg(7$z4^mJI0)UEaG93 zz3;VoUp2Qo8cIIMR_RQe*r&fcCc=L#94r4s4%NT(nn~5^yQ5C@4a@5yIsCWD_b`~= z8r7Oj#JTcMgzEySUiev~N!5m;PxUbjrx<)IwZGxjm+ijrC&IJsKg*%`>99%J>1OL> zpX!`n=HJZd%l2;j&%!tFPvy}5aJNa>hQd#Epiu_d{?rbc)kfl<_opIoh-CY>JI0&a zA=72+q@QZCpEAg%+HPu`r~F1q-|gU`KNaECencxB1ydQ`YNN%kv; zE}4{_?zT<CZ3zlny zYQELz_OAbP;p->eN9y0mxo*7(9Gdi({AHL%sODRZZh!xui$LhV(8aH-cbL>|C<9eT z9A&sYD|MnF86Wz(gQ5Q-!c~7kSHLbmWKy@G5LD?_2HYWox+7J8A)=>A_mz)(P3lhH zFLhE-8{R{|X;OagK6ElLX}>nTPydC8`~NFl3j28ZJ?8dHLoo>5>M{4XlDlPcc%|7) zG_n)^|0*0a|B|kTy{E>c>~yh`|+SHgFWbe~+% zWm31H6jU8`kb-f#>hj?BoSonPbAKfQTOXrKV_&{%Qn#TLROwFsGS86i;MT`P_{G1b zt7D&Da<92GN#8AXQc#uN$@AIzx;1eUdGW7B^dRY;9e&fKZbK;u-Ki9$UrPSg&}Kqg zJFUj2pMtT2e=Xv#Pp3;{Z`*BBcDmi!rJ!|`ENzE<-gN4v+K{B)Sd3PP7LCtV5YP3nnPz&wdR-;Yx(aYzWDFxn%T={f6d%p zX($DU9doG0{+(oak04tc-i!ZEI8Kr5wmB5J?9DG zI6$)7Pj#4-Z72m*AJCMc_p($9W~bYE;0fXK{v%yR`~8GT-G)+7rCS+#A2+Dm?fpmL ziIeW7+bX`1>%CGZ1vTj%_NMmA?ELn||55mw|0i8cyVh$`x1khN=~jl`XAJ80H~&u& zSnyrCoVL5kq;5kgsG1gt*mJE^gx8N#vqf6)cCcP=w2+E4TdzS4nsoGHb4O4yX1fT)c-jh!X--dspYiiFq zX;O8%*E%CWRrV<-yDw&*VVC@O&}K)(kSEY#>_M9lBzWy z-h)^FNrd5t6Q2C-+pR0+oQKnWL&099rcgDO+Q9WbgkL(PjTk7u61} zF)7R3+@x$n*{70?^9_Lu#$>x^ z|BLWMNcNgbFaC_V9Wvc*o$yml3{0M6HR856@-M>Yd77@QeQKpi+37i=^9y0skwzJ3 zr`3NozdqA!B!17+BG5~+d)l_-T((a9sZIha!|dH7%&OON(VkiMs zpT+QL%R{mT{mo9d^NsHb*QccW{j>EZb*I~{6M?ETfJkl!tw3%_xBJuY3C}zKhpw}I zu*IZqLm8+#;vfT~v}Up)-QIWpkMOxJOZ)W8u=4QqBw&Gzko`8g3d zLAn#``%LOKl!B^bjxyrjpE^e}!uG96!QhGKMEK2rr3-E^t^8SY`=y~2gzi)d(yu2| zZD!3G!I8rI&3_fq&q?-MvujPtPTwtcQcx4#>bWJ?)@#*lB^tM5pZ}|f@A~)rVkg7FfhrBUJOP zMz?e6^TO3dx(~g3!K7|OA*ed&D8p{LYOY48=39+!ch~d6^VYx7wYN9T`8jhtWx8&i z5LC?!M0k7l8r1E5>)(XW`F*NXUDs)G(f(BVpjV0PQL-}!wJ2$Syqip?f< z8wx>{Zp|6(5p-+LJOsnv7vY*8&=t7*-ZQB?eZSO6L47j zCz^gJ9P|F2F2cQa@#I{GiO#2C)#o#1|Pzb6HI?BlVkU`zv)jtxxi=_Ltt4B@hHWY#? z-3V{hx=q>n?Z5aV5!m_v=u+HgFPhYyzF+Eupenp^{$bdK_J&<9Fq!3=7!2nDHdw z2(q>5ef-BFzV1KhTHNcmnUtMAq|!-2)#o#1D(!0ca*wA zF9=VS!$Mc&{>VAS+-_+o235k9vG>qbc}5VvWl;Ft5H3o+BwF}RL;2dH)h3mv@0mJ5 zs0nb#+Uq5XM!BCvi-3P7DX)0fZ&JCTC{!s|B7Ep%m2xg+XtrQ73zHkmpyxpnEy4~b zDL?0LHK{z^bDcEA=|NS9hwn&^tw#79p%ES-%_75$iwjJ%omDF ze)`ntPLO2bI0oHS2f`+G8%jf!?&Nqx?-nI z-JA>8$w8bkRK+(<7|eL0B5AlrcZ4L1sBZ!3KGt!-@SKJ4=)4^!g{M2NlZC38gvrw=MudAw zv+!+MNWyo1+HO*~p)gb(e3W5$+X$2LlA)s6#{HyO1m?I%xc6jl&V}oQAx<0)TCAk_ zbusLAngbLs=z2$pM< zCrGq#G`mT8t*gzXazlBjQjQOkzFDb{_t`0Tl4#*tu!xkuc;!Qr%F{j92}ISrLL|8> zWKg-AL<>&`ly{%GYErqOL{uGvkcjaGsS^{~dG00A!nbBIDc|$Kl3y{GEa}SiB5~-{ zq2?UQ2+Fl#PNGGi56bIyylGOop-jY4D3yuyJI=D9Or*KALnjhP$iyIN7U5k>NcjA9 zyG;r=l!?QH7s%m1ND5DWa@Nnp2x%5kcNGagUv=E1@bvvtClgg4f|N0Nc`6gLQywGH zA|8eEea=r!DmRpgD&@+koR%5Z%mdQ*JVBy`W5X+?yuNqAx6JLI>7MIkqUyw9^3 zZ&@nMvs3OR(ZaWE87bet@qkI?hB8rg9KrxsHD3iAk7`NcWGHGr@B2x#2%Ld(|EuRs zDmN60P_B38O|yS$jzU3_EW(?sN%x|8v+vLK{;89RsxTjZh4rMv8sVB@VuUn{=&a== z{F6_Xn-p#+5>+4Kl##hR6^Yp?kCA8*-w)*nqnk`B|FV&&oFhMEP@VE7yrc)x<+(XnxY5ayN+UquZmKeBMqubPV%Ly@Rbu7vr}a*_Ba$Z(NDO(^+#vk2F&B;m`?1WgJz6pAY0 z2u06dTt~Pz%}KL}egffFA{{1$r#r4QOjMmq)O>Cm!E%lA7>O3~pof$nZ%&w0ZYUE~ z$022Cema%r*(pzuXyJJIHB#POSMh7+_Rn`NhNuk9*Q7ErJLOIiEnFv{d}Wo_ zq;f-B*w=(OLB| z4w)G6rZO=*^U77k}EDQ}s($E0#YnW#Dr zDP!}^sZ7jHxsyZ-R~X9Q_#|plxuHx{DObkk{RWl0Nwo0Pyg|z6MlYLGp1y(|Xii=j+ZDObkk+fsX|=38Z`P8T)QOpNoBXc1^yL&{sX ztTCxP-E*B##0f=Jng=FlJnfP^nrdVRNwEmeT}!gpy&f?)1Gt@0-+}?sv8+NGHm_ zr_MxQYI&0pbo)uN2&{XPbVrt)F{#^74C1(>N^i#l$?t}rd>%n}kR*%nXVATA<`+5F zt&@YQX^6^D{NMC-Yd-r#NV15&TTi+dU#R-FxjZqHgQ`z$%1E4!^Mm$hHe4e-Mw&%@ z$p#Ys>EU{l!VLwXN;m@C;}~I9Tyjg%2v3k^;W!Q9?Ymk`3QylNb&_ycf?w+@8*$u8 znuW{nBjH5l7Uq3YaWa@cJ|xR*2w zUmJvf_G0CO=5|g)VTgmzpi*VjJvn(|0uke=N)5?J*v9>&Sp-)2NqEP_)j1cglZM0Q zA=3HrhmyiaICrZ#F%cxqBHRPv+fKY~Qn;ZwRDJAGM&UyjGR#hSghY#I<0ewRV(&qd z$_?eAO1Ux+52dz_?3Bkyw1}6#Mar)>TrjCTefQJ}L`{-M7o7MIEqG~20vfPutsL9}Ou#(ZD z*+bnVT6jK&a!>zTCY2k?MAcCUndsS&I;EmHlbBSl9g>r1;oH$b%4c@HmviNMp-86= zGkxaKoJUOR)(lwuBv}Mr+CsWx`%jzHZ736m9fb;1R|_0LxMoucl4cP;3gPd(eZ{13 zLy@TZ?52##7p5XHJLM4)Euw21N%^X^i+{^pu%z#vI+3Ue^S}Xv%3~y2#4kbloF(f` zDmN5~D&@+sd|4{Yv-3PbqJ^VnD=F`C>@ul5-E*BtR82%gm^(TRDtD4-;ac28%HKVI z)TDAlk*GQhDFbtFDiX8v+)bi|CkEwf4qwc*mzWnBaaY@UJ@;Q zzHOv@-mdvm&F!IvLJ^0d)X=;@_Kiu7aFb`{G{;YEnv-S`7=ZAe4X>LNZYUI0!iRst zA3?Y_%}KKezqg%)?_Ivlr113pQzsKuX|5SvWv4tsqD8d&ZBoAe#Scs>Hb2kTNvyF{s>0qJ?YD4pRPpYjw_*>tv#8CZaMl9~$6h=ee6i3(p5o?rZRyRBk8} zRmUME&4-Tt_$P=CT@!JnG$+x*_ga9IzwB)_soYQ~Lb)o<<98=_(4Yc>e8mglm!|NSa0X?VTk2gU>!QDcn#dsy@cybDM8vDicSr zT%$ZfqD6Fmkd(jm;mf~cE?Lqo*U3cH*~AfsSDKAQqdZ2UMf?bqFWJA=q;f-7A}!X}j)%0$(12$>j;8&n=6(IVdZ zE-8O?LwC-V>t*85OhnDVFgwo^Bw9F@g-H28wd429?VpA+QFRaqE^GM+?FZaEs+eo`y~`yt!edcvgabi;LqiK=Oc z$nx;N>dMyqk|IcoMR?_2l6`9PC6lrZ<)P{`oH8zN%sihjX}3mqgd~gT1?b-O+QNs- zWlFkkoitRPNlczP%=EjK5p>5$vWRbck94no=?#;*4P~L~_=7Br@63FiyAgCJNV0HL z>?7UfpEaA*Z72y8GmEixswKI>4;ymMcz-v8{ z4929!w2iSxY}2&nPVJG)Xxm(E+B7|rHi=2wq}tl1NqS{jPC;M+0Re?&<&^tUQPE}P z^5B$9E|*L08wCWR&-;$M`^>Z54f7fQz|H>h%=3KT=Y8jy*_n6VG3QhUcMu>8y7+S7 zF0C9=$Stc1GHy#1gxj7^TEF49shHbD_Dek=3mFrPJL!byQ_A*B+k7vq46=m;VdmkD zR!A>Co4M(!I3NoKV+C;gBt|LZwyh@)o=`!@OMWV@wlgoz4!9ARg-UHDfcFOg;?Ni_@tW7XW(YF??g)Wn75yd9v05&tG2&rj%3NG_`;B65CS4i72cxbd?v z@-cuG)GdrWDtJaAxoyrJ)I>BDkyUeTisdr%!pJoMEohUQfqZ^0=((F+S{(drM#mvu z&8;sDVy6NI>zW{TieeBz3%Y4WZt6R9)5#qaMRoxZRdefVt9EX#ki(E3n1zgz7631( zGb#j^RTR--h!sV{>*i5z2b|QzEMOK4A*}$eH#RB+msJzduQ+m&vM_QZKns-}jNC14 zOd+}L{^_75qH0cd$c2$t1GHeuyaeQ(;XYfG?Vqx0qJ$h~*rr=*;=;(=0a_TDXXHg5 zdWGb+IbT>!55zFBeO zOu!Df9+-uUqwN5GCdc)8Wjm;>qKLp*HCMm$8Q`><1G8Z8>;UjXx^RWyw)>}pnkcE} zOV^s<1wv$+4M{m6Hv+U!S;NSE_hc$0msJxHIX_7mlu2F<(1JOB6BkB42GD}~R2Pt^RZQJ70r}z7BMQl7)kJh0;^*d7mTGP%=loV8TuUF)}Oe{a|J?W>o)w&VRB zJ3rd7Hu2*XF3jyJRpM5GE#GcBQnh5oI=0E3E6i3_bNHpdikUA{F;?{pj=rfD386=F zo!8w4{{g317v5U$=e*JT{&lyjir*EWqF*)(f+`xOxWIm^dHaTmpbE`#CW>F=BBr=n zZQL}A1sHiqxz@v#d8_B)iX{SM3fXiYUdpyYHq8f?5v9sq#kTUW#wzRQ3i4A_y_|T? z&s)J13>xdGNd=K&L8T5Ikp#02EaRs$>{L&Z5R(-`^WD4r{?$mop ztV?^>6QzEJE68(M`+dw0YP;dg2Bx5v9?SR=iJlquWfDDYjw`6`30+@d(NVkhpZ_XT zkmr!>`$=@~h>aw=#`S9?`gAu}ke~7LuQMCe@58wd2m_Mnm-l~z?UktQjcyOJy$QAb zT;GF?5ZM$|eUmUBJNhl6WJhme3Tk_@=R-s}IQS50MMTZR%m(Er;P@ji6kkuAt~+b&oR}r1a-MLHxAGKFJi+_FCU>GX-U}VvH+@r}*M#mQ@tJ(C{6m zpjampzDqKn>;FBbAkP_BxPqd`Hh-UZ)}MZgFoz#{nh*vDKEt*`{jQzj3d*Y2rDvH9 zYQ=ofb4)?e$5XZtoBrVESuE7jo@-n|t!QuC%4|^En@YDaKS-%f{{it_7WzY`Ai~1y zTtS2X(V1=x(l{wg;EL!fa5x{BwUyw({Kn z6Q-c(UT&|F5kBAhD&s-zx>EU5;(6rA&zORucSgNNY%Y2HoGB<9^#fc%gw+>bXEr|8 zY}6k61=*X<#{81(O$P5@F+vnQdzdRI)}gvzGaD2uHvcz7iHiL#Q&2C5`M$w;&^X*X z_6D;-o_#OA$rO|ySHtf}OQ++1Pn1!=x0r&);qU}kQ1srWx0wx6I*jiSKTS#7nS$D0 zp?#MrsHJBo-xUTW%%@x5V+x9HEcydeP^|ovKQaZiD>wL0OhKMAuWQ}&Vgh9AzOvU{Fg-E>-{NHP}}{7KV@54+U271tW!HlT)U_=`L0~D z#CK6m9@J2)aUVA>S+eG$8WRdH_FDCPfLk4{L>2dqV~hs(+Cm zmynfb3cS@BtdKMp1n*16L%~HgJ1t6<-f&S}HNiW6p*v7=CQ1USuYojpxhwcWnJ|R} z)9z@V6(s~w+X1i31oxKl=%T7Q8^R?7QIQGnNtBL>l6~+RKPi3DyeJ`vyvzHz70P{1 z8NAaCd?AR`oCq!>8Xy9E_Qi$kYQ9{1jXK&7)j ziB7^0I ^F$&VK(^|dxVCte8Y)Il#id70R9O4_sTcyJZ$y`EKE&i}FGWbfgTDe{o zr8ujkZ%O4=Nmi+(5LU{SLUbj=Qm!B&E1zIkBo!nCk#Y?r=rm+i-UbVMf|P7^KUNA0 zPr{f$5P=!lTtdDw!(hQdutJ$An-e8uHP{797G{p%l&q6j6x% zlcI#I3R;UPL`P8xmynfrYAKh{m`LAyhD!*dat0*mA=Dis?eMH2BnOQNzq9Z-oOBeX zXTpDJvn^bpM2feL{7F*gt< zl}PJE{S-*h+h|P8UV>>5AR&mz5)&O0$J5%lgfdYd0(X|BBWH3}lu)w|barqn)F8vT zPA;J_(Vo@CQ-~yw2)IWh&1$@BHwFC5&+i zbw`H(IJK(1B1#BivU!49?JvB-r8MhcDo(yiAub0^atZnJx(1STV8*pgaS2(Klupx# z1L@bO6c{$cC6tNWIZ;AmwYh7SLR>$8ouVGdnWGS=BIl{prcpaHdNfvd_lXjMNUd_^ zR!FKks^Jo9<9M{2GaF2(JG5TzTtZ_aX9y&Sh&HUN^`H=QCp>9HeXJLk(6~SC>&+$P zzGqC75QJyrZf8;f5(|8|gdj>1eVy4TM85h0_HYU1K44Om5JXz5AGboj%8UHDgdqA- z0;rYGUM-hU#MBv(AcaV(Y7gWR%KgNdAS#7r26G8P9No8pi*R7B9~AmPK6}Vd5CX_P-D*}a|u}$XQfc7Cp?u) zXiR8b)3}5na(YAwC8wz(opT}gvxhUdgt{XkB7yNN5qdu z#gE6tkH^K2C&Z5@H#n;n!QV3UXa3GU`QIj8)Q6iFtJuG5*uSf9Q#s$bK^kA0mht{( zxO5Qp;f9swTcmL7)GCR0NttvXx4Os9jZ#BYq08nwxcggGJpom + flink-orc flink-jdbc flink-hadoop-compatibility flink-hbase From 35517f1291293f73f4466a4fdbed4296b2dd80a5 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 13 Nov 2017 14:54:54 +0100 Subject: [PATCH 100/367] [FLINK-2170] [connectors] Add OrcRowInputFormat and OrcTableSource. This closes #5043. --- docs/dev/table/sourceSinks.md | 49 + flink-connectors/flink-orc/pom.xml | 87 +- .../apache/flink/orc/OrcRowInputFormat.java | 745 ++ .../org/apache/flink/orc/OrcTableSource.java | 455 +- .../java/org/apache/flink/orc/OrcUtils.java | 2379 ++-- .../apache/flink/orc/RowOrcInputFormat.java | 241 - .../flink/orc/OrcRowInputFormatTest.java | 795 ++ .../flink/orc/OrcTableSourceITCase.java | 134 +- .../apache/flink/orc/OrcTableSourceTest.java | 266 +- .../org/apache/flink/orc/OrcUtilsTest.java | 148 + .../flink/orc/RowOrcInputFormatTest.java | 472 - .../test/resources/TestOrcFile.emptyFile.orc | Bin 523 -> 0 bytes .../test/resources/TestOrcFile.listlong.orc | Bin 627 -> 0 bytes .../test/resources/TestOrcFile.liststring.orc | Bin 1298 -> 0 bytes .../resources/TestOrcFile.testDate1900.dat | 10000 ---------------- .../flink-orc/src/test/resources/decimal.dat | 6000 ---------- .../src/test/resources/demo-11-none.orc | Bin 5147970 -> 0 bytes .../{decimal.orc => test-data-decimal.orc} | Bin .../src/test/resources/test-data-flat.orc | Bin 0 -> 408522 bytes ...OrcFile.test1.orc => test-data-nested.orc} | Bin ...tructlong.orc => test-data-nestedlist.orc} | Bin ...stDate1900.orc => test-data-timetypes.orc} | Bin .../flink/api/java/typeutils/RowTypeInfo.java | 17 + .../logical/FlinkLogicalTableSourceScan.scala | 16 +- .../table/plan/util/RexProgramExtractor.scala | 12 + 25 files changed, 3305 insertions(+), 18511 deletions(-) create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java delete mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java delete mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat delete mode 100644 flink-connectors/flink-orc/src/test/resources/decimal.dat delete mode 100644 flink-connectors/flink-orc/src/test/resources/demo-11-none.orc rename flink-connectors/flink-orc/src/test/resources/{decimal.orc => test-data-decimal.orc} (100%) create mode 100644 flink-connectors/flink-orc/src/test/resources/test-data-flat.orc rename flink-connectors/flink-orc/src/test/resources/{TestOrcFile.test1.orc => test-data-nested.orc} (100%) rename flink-connectors/flink-orc/src/test/resources/{TestOrcFile.listliststructlong.orc => test-data-nestedlist.orc} (100%) rename flink-connectors/flink-orc/src/test/resources/{TestOrcFile.testDate1900.orc => test-data-timetypes.orc} (100%) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 0b4bdbede0d3d..7387358beb931 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -47,6 +47,7 @@ A custom `TableSource` can be defined by implementing the `BatchTableSource` or | `Kafka08AvroTableSource` | `flink-connector-kafka-0.8` | N | Y | A `TableSource` for Avro-encoded Kafka 0.8 topics. | `Kafka08JsonTableSource` | `flink-connector-kafka-0.8` | N | Y | A `TableSource` for flat Json-encoded Kafka 0.8 topics. | `CsvTableSource` | `flink-table` | Y | Y | A simple `TableSource` for CSV files. +| `OrcTableSource` | `flink-orc` | Y | N | A `TableSource` for ORC files. All sources that come with the `flink-table` dependency are directly available for Table API or SQL programs. For all other table sources, you have to add the respective dependency in addition to the `flink-table` dependency. @@ -485,6 +486,54 @@ val csvTableSource = CsvTableSource {% top %} +### OrcTableSource + +The `OrcTableSource` reads [ORC files](https://orc.apache.org). ORC is a file format for structured data and stores the data in a compressed, columnar representation. ORC is very storage efficient and supports projection and filter push-down. + +An `OrcTableSource` is created as shown below: + +

+
+{% highlight java %} + +// create Hadoop Configuration +Configuration config = new Configuration(); + +OrcTableSource orcTableSource = OrcTableSource.builder() + // path to ORC file(s) + .path("file:///path/to/data") + // schema of ORC files + .forOrcSchema("struct>>") + // Hadoop configuration + .withConfiguration(config) + // build OrcTableSource + .build(); +{% endhighlight %} +
+ +
+{% highlight scala %} + +// create Hadoop Configuration +val config = new Configuration() + +val orcTableSource = OrcTableSource.builder() + // path to ORC file(s) + .path("file:///path/to/data") + // schema of ORC files + .forOrcSchema("struct>>") + // Hadoop configuration + .withConfiguration(config) + // build OrcTableSource + .build() +{% endhighlight %} +
+
+ +**Note:** The `OrcTableSource` does not support ORC's `Union` type yet. + +{% top %} + Provided TableSinks ------------------- diff --git a/flink-connectors/flink-orc/pom.xml b/flink-connectors/flink-orc/pom.xml index 1ac7eaa7a9f2f..4866cd78d419e 100644 --- a/flink-connectors/flink-orc/pom.xml +++ b/flink-connectors/flink-orc/pom.xml @@ -40,22 +40,39 @@ under the License. org.apache.flink - flink-table_${scala.binary.version} + flink-streaming-scala_${scala.binary.version} ${project.version} - compile + provided org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-table_${scala.binary.version} ${project.version} - compile + provided + + true org.apache.orc orc-core - 1.4.0 + 1.4.1 + + + + org.apache.hadoop + hadoop-common + + + + + + + org.apache.flink + flink-shaded-hadoop2 + ${project.version} + provided @@ -88,65 +105,7 @@ under the License. test test-jar - - - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-assembly-plugin - [2.4,) - - single - - - - - - - - - org.apache.maven.plugins - maven-clean-plugin - [1,) - - clean - - - - - - - - - org.apache.avro - avro-maven-plugin - [1.7.7,) - - schema - - - - - - - - - - - - - + diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java new file mode 100644 index 0000000000000..4353cbc66dfb2 --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java @@ -0,0 +1,745 @@ +/* + * 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.flink.orc; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.io.FileInputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.StripeInformation; +import org.apache.orc.TypeDescription; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.orc.OrcUtils.fillRows; + +/** + * InputFormat to read ORC files. + */ +public class OrcRowInputFormat extends FileInputFormat implements ResultTypeQueryable { + + private static final Logger LOG = LoggerFactory.getLogger(OrcRowInputFormat.class); + // the number of rows read in a batch + private static final int DEFAULT_BATCH_SIZE = 1000; + + // the number of fields rows to read in a batch + private int batchSize; + // the configuration to read with + private Configuration conf; + // the schema of the ORC files to read + private TypeDescription schema; + + // the fields of the ORC schema that the returned Rows are composed of. + private int[] selectedFields; + // the type information of the Rows returned by this InputFormat. + private transient RowTypeInfo rowType; + + // the ORC reader + private transient RecordReader orcRowsReader; + // the vectorized row data to be read in a batch + private transient VectorizedRowBatch rowBatch; + // the vector of rows that is read in a batch + private transient Row[] rows; + + // the number of rows in the current batch + private transient int rowsInBatch; + // the index of the next row to return + private transient int nextRow; + + private ArrayList conjunctPredicates = new ArrayList<>(); + + /** + * Creates an OrcRowInputFormat. + * + * @param path The path to read ORC files from. + * @param schemaString The schema of the ORC files as String. + * @param orcConfig The configuration to read the ORC files with. + */ + public OrcRowInputFormat(String path, String schemaString, Configuration orcConfig) { + this(path, TypeDescription.fromString(schemaString), orcConfig, DEFAULT_BATCH_SIZE); + } + + /** + * Creates an OrcRowInputFormat. + * + * @param path The path to read ORC files from. + * @param schemaString The schema of the ORC files as String. + * @param orcConfig The configuration to read the ORC files with. + * @param batchSize The number of Row objects to read in a batch. + */ + public OrcRowInputFormat(String path, String schemaString, Configuration orcConfig, int batchSize) { + this(path, TypeDescription.fromString(schemaString), orcConfig, batchSize); + } + + /** + * Creates an OrcRowInputFormat. + * + * @param path The path to read ORC files from. + * @param orcSchema The schema of the ORC files as ORC TypeDescription. + * @param orcConfig The configuration to read the ORC files with. + * @param batchSize The number of Row objects to read in a batch. + */ + public OrcRowInputFormat(String path, TypeDescription orcSchema, Configuration orcConfig, int batchSize) { + super(new Path(path)); + + // configure OrcRowInputFormat + this.schema = orcSchema; + this.rowType = (RowTypeInfo) OrcUtils.schemaToTypeInfo(schema); + this.conf = orcConfig; + this.batchSize = batchSize; + + // set default selection mask, i.e., all fields. + this.selectedFields = new int[this.schema.getChildren().size()]; + for (int i = 0; i < selectedFields.length; i++) { + this.selectedFields[i] = i; + } + } + + /** + * Adds a filter predicate to reduce the number of rows to be returned by the input format. + * Multiple conjunctive predicates can be added by calling this method multiple times. + * + *

Note: Predicates can significantly reduce the amount of data that is read. + * However, the OrcRowInputFormat does not guarantee that all returned rows qualify the + * predicates. Moreover, predicates are only applied if the referenced field is among the + * selected fields. + * + * @param predicate The filter predicate. + */ + public void addPredicate(Predicate predicate) { + // validate + validatePredicate(predicate); + // add predicate + this.conjunctPredicates.add(predicate); + } + + private void validatePredicate(Predicate pred) { + if (pred instanceof ColumnPredicate) { + // check column name + String colName = ((ColumnPredicate) pred).columnName; + if (!this.schema.getFieldNames().contains(colName)) { + throw new IllegalArgumentException("Predicate cannot be applied. " + + "Column '" + colName + "' does not exist in ORC schema."); + } + } else if (pred instanceof Not) { + validatePredicate(((Not) pred).child()); + } else if (pred instanceof Or) { + for (Predicate p : ((Or) pred).children()) { + validatePredicate(p); + } + } + } + + /** + * Selects the fields from the ORC schema that are returned by InputFormat. + * + * @param selectedFields The indices of the fields of the ORC schema that are returned by the InputFormat. + */ + public void selectFields(int... selectedFields) { + // set field mapping + this.selectedFields = selectedFields; + // adapt result type + this.rowType = RowTypeInfo.projectFields(this.rowType, selectedFields); + } + + /** + * Computes the ORC projection mask of the fields to include from the selected fields.rowOrcInputFormat.nextRecord(null). + * + * @return The ORC projection mask. + */ + private boolean[] computeProjectionMask() { + // mask with all fields of the schema + boolean[] projectionMask = new boolean[schema.getMaximumId() + 1]; + // for each selected field + for (int inIdx : selectedFields) { + // set all nested fields of a selected field to true + TypeDescription fieldSchema = schema.getChildren().get(inIdx); + for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) { + projectionMask[i] = true; + } + } + return projectionMask; + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + // create and initialize the row batch + this.rows = new Row[batchSize]; + for (int i = 0; i < batchSize; i++) { + rows[i] = new Row(selectedFields.length); + } + } + + @Override + public void open(FileInputSplit fileSplit) throws IOException { + + LOG.debug("Opening ORC file {}", fileSplit.getPath()); + + // open ORC file and create reader + org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(fileSplit.getPath().getPath()); + Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(conf)); + + // get offset and length for the stripes that start in the split + Tuple2 offsetAndLength = getOffsetAndLengthForSplit(fileSplit, getStripes(orcReader)); + + // create ORC row reader configuration + Reader.Options options = getOptions(orcReader) + .schema(schema) + .range(offsetAndLength.f0, offsetAndLength.f1) + .useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(conf)) + .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf)) + .tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(conf)); + + // configure filters + if (!conjunctPredicates.isEmpty()) { + SearchArgument.Builder b = SearchArgumentFactory.newBuilder(); + b = b.startAnd(); + for (Predicate predicate : conjunctPredicates) { + predicate.add(b); + } + b = b.end(); + options.searchArgument(b.build(), new String[]{}); + } + + // configure selected fields + options.include(computeProjectionMask()); + + // create ORC row reader + this.orcRowsReader = orcReader.rows(options); + + // assign ids + this.schema.getId(); + // create row batch + this.rowBatch = schema.createRowBatch(batchSize); + rowsInBatch = 0; + nextRow = 0; + } + + @VisibleForTesting + Reader.Options getOptions(Reader orcReader) { + return orcReader.options(); + } + + @VisibleForTesting + List getStripes(Reader orcReader) { + return orcReader.getStripes(); + } + + private Tuple2 getOffsetAndLengthForSplit(FileInputSplit split, List stripes) { + long splitStart = split.getStart(); + long splitEnd = splitStart + split.getLength(); + + long readStart = Long.MAX_VALUE; + long readEnd = Long.MIN_VALUE; + + for (StripeInformation s : stripes) { + if (splitStart <= s.getOffset() && s.getOffset() < splitEnd) { + // stripe starts in split, so it is included + readStart = Math.min(readStart, s.getOffset()); + readEnd = Math.max(readEnd, s.getOffset() + s.getLength()); + } + } + + if (readStart < Long.MAX_VALUE) { + // at least one split is included + return Tuple2.of(readStart, readEnd - readStart); + } else { + return Tuple2.of(0L, 0L); + } + } + + @Override + public void close() throws IOException { + if (orcRowsReader != null) { + this.orcRowsReader.close(); + } + this.orcRowsReader = null; + } + + @Override + public void closeInputFormat() throws IOException { + this.rows = null; + this.rows = null; + this.schema = null; + this.rowBatch = null; + } + + @Override + public boolean reachedEnd() throws IOException { + return !ensureBatch(); + } + + /** + * Checks if there is at least one row left in the batch to return. + * If no more row are available, it reads another batch of rows. + * + * @return Returns true if there is one more row to return, false otherwise. + * @throws IOException throw if an exception happens while reading a batch. + */ + private boolean ensureBatch() throws IOException { + + if (nextRow >= rowsInBatch) { + // No more rows available in the Rows array. + nextRow = 0; + // Try to read the next batch if rows from the ORC file. + boolean moreRows = orcRowsReader.nextBatch(rowBatch); + + if (moreRows) { + // Load the data into the Rows array. + rowsInBatch = fillRows(rows, schema, rowBatch, selectedFields); + } + return moreRows; + } + // there is at least one Row left in the Rows array. + return true; + } + + @Override + public Row nextRecord(Row reuse) throws IOException { + // return the next row + return rows[this.nextRow++]; + } + + @Override + public TypeInformation getProducedType() { + return rowType; + } + + // -------------------------------------------------------------------------------------------- + // Custom serialization methods + // -------------------------------------------------------------------------------------------- + + private void writeObject(ObjectOutputStream out) throws IOException { + out.writeInt(batchSize); + this.conf.write(out); + out.writeUTF(schema.toString()); + + out.writeInt(selectedFields.length); + for (int f : selectedFields) { + out.writeInt(f); + } + + out.writeInt(conjunctPredicates.size()); + for (Predicate p : conjunctPredicates) { + out.writeObject(p); + } + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + batchSize = in.readInt(); + org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); + configuration.readFields(in); + + if (this.conf == null) { + this.conf = configuration; + } + this.schema = TypeDescription.fromString(in.readUTF()); + + this.selectedFields = new int[in.readInt()]; + for (int i = 0; i < selectedFields.length; i++) { + this.selectedFields[i] = in.readInt(); + } + + this.conjunctPredicates = new ArrayList<>(); + int numPreds = in.readInt(); + for (int i = 0; i < numPreds; i++) { + conjunctPredicates.add((Predicate) in.readObject()); + } + } + + // -------------------------------------------------------------------------------------------- + // Classes to define predicates + // -------------------------------------------------------------------------------------------- + + /** + * A filter predicate that can be evaluated by the OrcRowInputFormat. + */ + public abstract static class Predicate implements Serializable { + protected abstract SearchArgument.Builder add(SearchArgument.Builder builder); + } + + abstract static class ColumnPredicate extends Predicate { + final String columnName; + final PredicateLeaf.Type literalType; + + ColumnPredicate(String columnName, PredicateLeaf.Type literalType) { + this.columnName = columnName; + this.literalType = literalType; + } + + Object castLiteral(Serializable literal) { + + switch (literalType) { + case LONG: + if (literal instanceof Byte) { + return new Long((Byte) literal); + } else if (literal instanceof Short) { + return new Long((Short) literal); + } else if (literal instanceof Integer) { + return new Long((Integer) literal); + } else if (literal instanceof Long) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a LONG column requires an integer " + + "literal, i.e., Byte, Short, Integer, or Long."); + } + case FLOAT: + if (literal instanceof Float) { + return new Double((Float) literal); + } else if (literal instanceof Double) { + return literal; + } else if (literal instanceof BigDecimal) { + return ((BigDecimal) literal).doubleValue(); + } else { + throw new IllegalArgumentException("A predicate on a FLOAT column requires a floating " + + "literal, i.e., Float or Double."); + } + case STRING: + if (literal instanceof String) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a STRING column requires a floating " + + "literal, i.e., Float or Double."); + } + case BOOLEAN: + if (literal instanceof Boolean) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a BOOLEAN column requires a Boolean literal."); + } + case DATE: + if (literal instanceof Date) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a DATE column requires a java.sql.Date literal."); + } + case TIMESTAMP: + if (literal instanceof Timestamp) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a TIMESTAMP column requires a java.sql.Timestamp literal."); + } + case DECIMAL: + if (literal instanceof BigDecimal) { + return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) literal)); + } else { + throw new IllegalArgumentException("A predicate on a DECIMAL column requires a BigDecimal literal."); + } + default: + throw new IllegalArgumentException("Unknown literal type " + literalType); + } + } + } + + abstract static class BinaryPredicate extends ColumnPredicate { + final Serializable literal; + + BinaryPredicate(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType); + this.literal = literal; + } + } + + /** + * An EQUALS predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Equals extends BinaryPredicate { + /** + * Creates an EQUALS predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public Equals(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.equals(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " = " + literal; + } + } + + /** + * An EQUALS predicate that can be evaluated with Null safety by the OrcRowInputFormat. + */ + public static class NullSafeEquals extends BinaryPredicate { + /** + * Creates a null-safe EQUALS predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public NullSafeEquals(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.nullSafeEquals(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " = " + literal; + } + } + + /** + * A LESS_THAN predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class LessThan extends BinaryPredicate { + /** + * Creates a LESS_THAN predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public LessThan(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.lessThan(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " < " + literal; + } + } + + /** + * A LESS_THAN_EQUALS predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class LessThanEquals extends BinaryPredicate { + /** + * Creates a LESS_THAN_EQUALS predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public LessThanEquals(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.lessThanEquals(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " <= " + literal; + } + } + + /** + * An IS_NULL predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class IsNull extends ColumnPredicate { + /** + * Creates an IS_NULL predicate. + * + * @param columnName The column to check for null. + * @param literalType The type of the column to check for null. + */ + public IsNull(String columnName, PredicateLeaf.Type literalType) { + super(columnName, literalType); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.isNull(columnName, literalType); + } + + @Override + public String toString() { + return columnName + " IS NULL"; + } + } + + /** + * An BETWEEN predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Between extends ColumnPredicate { + private Serializable lowerBound; + private Serializable upperBound; + + /** + * Creates an BETWEEN predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literals. + * @param lowerBound The literal value of the (inclusive) lower bound to check the column against. + * @param upperBound The literal value of the (inclusive) upper bound to check the column against. + */ + public Between(String columnName, PredicateLeaf.Type literalType, Serializable lowerBound, Serializable upperBound) { + super(columnName, literalType); + this.lowerBound = lowerBound; + this.upperBound = upperBound; + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.between(columnName, literalType, castLiteral(lowerBound), castLiteral(upperBound)); + } + + @Override + public String toString() { + return lowerBound + " <= " + columnName + " <= " + upperBound; + } + } + + /** + * An IN predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class In extends ColumnPredicate { + private Serializable[] literals; + + /** + * Creates an IN predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literals. + * @param literals The literal values to check the column against. + */ + public In(String columnName, PredicateLeaf.Type literalType, Serializable... literals) { + super(columnName, literalType); + this.literals = literals; + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + Object[] castedLiterals = new Object[literals.length]; + for (int i = 0; i < literals.length; i++) { + castedLiterals[i] = castLiteral(literals[i]); + } + return builder.in(columnName, literalType, (Object[]) castedLiterals); + } + + @Override + public String toString() { + return columnName + " IN " + Arrays.toString(literals); + } + } + + /** + * A NOT predicate to negate a predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Not extends Predicate { + private final Predicate pred; + + /** + * Creates a NOT predicate. + * + * @param predicate The predicate to negate. + */ + public Not(Predicate predicate) { + this.pred = predicate; + } + + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return pred.add(builder.startNot()).end(); + } + + protected Predicate child() { + return pred; + } + + @Override + public String toString() { + return "NOT(" + pred.toString() + ")"; + } + } + + /** + * An OR predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Or extends Predicate { + private final Predicate[] preds; + + /** + * Creates an OR predicate. + * + * @param predicates The disjunctive predicates. + */ + public Or(Predicate... predicates) { + this.preds = predicates; + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + SearchArgument.Builder withOr = builder.startOr(); + for (Predicate p : preds) { + withOr = p.add(withOr); + } + return withOr.end(); + } + + protected Iterable children() { + return Arrays.asList(preds); + } + + @Override + public String toString() { + return "OR(" + Arrays.toString(preds) + ")"; + } + } +} diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java index 0454ba45af16a..b7c537865764d 100644 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java @@ -18,111 +18,474 @@ package org.apache.flink.orc; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.orc.OrcRowInputFormat.Predicate; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.Attribute; +import org.apache.flink.table.expressions.BinaryComparison; +import org.apache.flink.table.expressions.EqualTo; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.GreaterThan; +import org.apache.flink.table.expressions.GreaterThanOrEqual; +import org.apache.flink.table.expressions.IsNotNull; +import org.apache.flink.table.expressions.IsNull; +import org.apache.flink.table.expressions.LessThan; +import org.apache.flink.table.expressions.LessThanOrEqual; +import org.apache.flink.table.expressions.Literal; +import org.apache.flink.table.expressions.Not; +import org.apache.flink.table.expressions.NotEqualTo; +import org.apache.flink.table.expressions.Or; +import org.apache.flink.table.expressions.UnaryExpression; import org.apache.flink.table.sources.BatchTableSource; +import org.apache.flink.table.sources.FilterableTableSource; import org.apache.flink.table.sources.ProjectableTableSource; import org.apache.flink.table.sources.TableSource; import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.apache.orc.TypeDescription; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + /** - * Creates a TableSource to read ORC file. + * A TableSource to read ORC files. * - *

The ORC file path and schema is passed during {@link OrcTableSource} construction. configuration is optional. + *

The {@link OrcTableSource} supports projection and filter push-down.

* - *

The OrcTableSource is used as shown in the example below. + *

An {@link OrcTableSource} is used as shown in the example below. * *

  * {@code
- * String path = testInputURL.getPath();
- * String schema = "struct"
- * OrcTableSource orcSrc = new OrcTableSource(path, schema);
+ * OrcTableSource orcSrc = OrcTableSource.builder()
+ *   .path("file:///my/data/file.orc")
+ *   .forOrcSchema("struct")
+ *   .build();
+ *
  * tEnv.registerTableSource("orcTable", orcSrc);
  * Table res = tableEnv.sql("SELECT * FROM orcTable");
  * }
  * 
*/ -public class OrcTableSource implements BatchTableSource, ProjectableTableSource { +public class OrcTableSource + implements BatchTableSource, ProjectableTableSource, FilterableTableSource { - private String path; - private TypeDescription orcSchema; - private RowTypeInfo typeInfo; - private Configuration orcConfig; - private int[] fieldMapping; + private static final int DEFAULT_BATCH_SIZE = 1000; - /** - * The ORC file path and schema. - * - * @param path the path of orc file - * @param orcSchema schema of orc file - */ - public OrcTableSource(String path, String orcSchema) { - this(path, orcSchema, new Configuration()); - } + // path to read ORC files from + private final String path; + // schema of the ORC file + private final TypeDescription orcSchema; + // the schema of the Table + private final TableSchema tableSchema; + // the configuration to read the file + private final Configuration orcConfig; + // the number of rows to read in a batch + private final int batchSize; + + // type information of the data returned by the InputFormat + private final RowTypeInfo typeInfo; + // list of selected ORC fields to return + private final int[] selectedFields; + // list of predicates to apply + private final Predicate[] predicates; /** - * The file path and schema of orc file, and configuration to read orc file . + * Creates an OrcTableSouce from an ORC TypeDescription. * - * @param path the path of orc file - * @param orcSchema schema of orc file - * @param orcConfig configuration to read orc file + * @param path The path to read the ORC files from. + * @param orcSchema The schema of the ORC files as TypeDescription. + * @param orcConfig The configuration to read the ORC files. + * @param batchSize The number of Rows to read in a batch, default is 1000. */ - public OrcTableSource(String path, String orcSchema, Configuration orcConfig) { - this(path, TypeDescription.fromString(orcSchema), orcConfig); + private OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig, int batchSize) { + this(path, orcSchema, orcConfig, batchSize, null, null); } - public OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig) { + private OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig, + int batchSize, int[] selectedFields, Predicate[] predicates) { + + Preconditions.checkNotNull(path, "Path must not be null."); + Preconditions.checkNotNull(orcSchema, "OrcSchema must not be null."); + Preconditions.checkNotNull(path, "Configuration must not be null."); + Preconditions.checkArgument(batchSize > 0, "Batch size must be larger than null."); this.path = path; this.orcSchema = orcSchema; this.orcConfig = orcConfig; + this.batchSize = batchSize; + this.selectedFields = selectedFields; + this.predicates = predicates; - this.typeInfo = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema); + // determine the type information from the ORC schema + RowTypeInfo typeInfoFromSchema = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema); + + // set return type info + if (selectedFields == null) { + this.typeInfo = typeInfoFromSchema; + } else { + this.typeInfo = RowTypeInfo.projectFields(typeInfoFromSchema, selectedFields); + } + // create a TableSchema that corresponds to the ORC schema + this.tableSchema = new TableSchema( + typeInfoFromSchema.getFieldNames(), + typeInfoFromSchema.getFieldTypes() + ); } @Override public DataSet getDataSet(ExecutionEnvironment execEnv) { - - RowOrcInputFormat orcIF = new RowOrcInputFormat(path, orcSchema, orcConfig); - if (fieldMapping != null) { - orcIF.setFieldMapping(fieldMapping); + OrcRowInputFormat orcIF = buildOrcInputFormat(); + if (selectedFields != null) { + orcIF.selectFields(selectedFields); + } + if (predicates != null) { + for (OrcRowInputFormat.Predicate pred : predicates) { + orcIF.addPredicate(pred); + } } return execEnv.createInput(orcIF); } + @VisibleForTesting + protected OrcRowInputFormat buildOrcInputFormat() { + return new OrcRowInputFormat(path, orcSchema, orcConfig, batchSize); + } + @Override public TypeInformation getReturnType() { return typeInfo; } @Override - public TableSource projectFields(int[] fields) { + public TableSchema getTableSchema() { + return this.tableSchema; + } - OrcTableSource copy = new OrcTableSource(path, orcSchema, orcConfig); + @Override + public TableSource projectFields(int[] selectedFields) { + // create a copy of the OrcTableSouce with new selected fields + return new OrcTableSource(path, orcSchema, orcConfig, batchSize, selectedFields, predicates); + } - // set field mapping - copy.fieldMapping = fields; + @Override + public TableSource applyPredicate(List predicates) { + ArrayList orcPredicates = new ArrayList<>(); - // adapt TypeInfo - TypeInformation[] fieldTypes = new TypeInformation[fields.length]; - String[] fieldNames = new String[fields.length]; - for (int i = 0; i < fields.length; i++) { - fieldTypes[i] = this.typeInfo.getTypeAt(fields[i]); - fieldNames[i] = this.typeInfo.getFieldNames()[fields[i]]; + // we do not remove any predicates from the list because ORC does not fully apply predicates + for (Expression pred : predicates) { + Predicate orcPred = toOrcPredicate(pred); + if (orcPred != null) { + orcPredicates.add(orcPred); + } } - copy.typeInfo = new RowTypeInfo(fieldTypes, fieldNames); - return copy; + return new OrcTableSource(path, orcSchema, orcConfig, batchSize, selectedFields, orcPredicates.toArray(new Predicate[]{})); + } + + @Override + public boolean isFilterPushedDown() { + return this.predicates != null; } @Override public String explainSource() { - return "ORC Source file at path " + this.path + " with schema " + this.orcSchema; + return "OrcFile[path=" + path + ", schema=" + orcSchema + ", filter=" + predicateString() + "]"; + } + + private String predicateString() { + if (predicates != null) { + return "AND(" + Arrays.toString(predicates) + ")"; + } else { + return "TRUE"; + } + } + + // Predicate conversion for filter push-down. + + private Predicate toOrcPredicate(Expression pred) { + if (pred instanceof Or) { + Predicate c1 = toOrcPredicate(((Or) pred).left()); + Predicate c2 = toOrcPredicate(((Or) pred).right()); + if (c1 == null || c2 == null) { + return null; + } else { + return new OrcRowInputFormat.Or(c1, c2); + } + } else if (pred instanceof Not) { + Predicate c = toOrcPredicate(((Not) pred).child()); + if (c == null) { + return null; + } else { + return new OrcRowInputFormat.Not(c); + } + } else if (pred instanceof BinaryComparison) { + + BinaryComparison binComp = (BinaryComparison) pred; + + if (!isValid(binComp)) { + // not a valid predicate + return null; + } + PredicateLeaf.Type litType = getLiteralType(binComp); + if (litType == null) { + // unsupported literal type + return null; + } + + boolean literalOnRight = literalOnRight(binComp); + String colName = getColumnName(binComp); + Serializable literal = (Serializable) getLiteral(binComp); + + if (pred instanceof EqualTo) { + return new OrcRowInputFormat.Equals(colName, litType, literal); + } else if (pred instanceof NotEqualTo) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.Equals(colName, litType, literal)); + } else if (pred instanceof GreaterThan) { + if (literalOnRight) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThanEquals(colName, litType, literal)); + } else { + return new OrcRowInputFormat.LessThan(colName, litType, literal); + } + } else if (pred instanceof GreaterThanOrEqual) { + if (literalOnRight) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThan(colName, litType, literal)); + } else { + return new OrcRowInputFormat.LessThanEquals(colName, litType, literal); + } + } else if (pred instanceof LessThan) { + if (literalOnRight) { + return new OrcRowInputFormat.LessThan(colName, litType, literal); + } else { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThanEquals(colName, litType, literal)); + } + } else if (pred instanceof LessThanOrEqual) { + if (literalOnRight) { + return new OrcRowInputFormat.LessThanEquals(colName, litType, literal); + } else { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThan(colName, litType, literal)); + } + } else { + // unsupported predicate + return null; + } + } else if (pred instanceof UnaryExpression) { + + UnaryExpression unary = (UnaryExpression) pred; + if (!isValid(unary)) { + // not a valid predicate + return null; + } + PredicateLeaf.Type colType = toOrcType(((UnaryExpression) pred).child().resultType()); + if (colType == null) { + // unsupported type + return null; + } + + String colName = getColumnName(unary); + + if (pred instanceof IsNull) { + return new OrcRowInputFormat.IsNull(colName, colType); + } else if (pred instanceof IsNotNull) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.IsNull(colName, colType)); + } else { + // unsupported predicate + return null; + } + } else { + // unsupported predicate + return null; + } + } + + private boolean isValid(UnaryExpression unary) { + return unary.child() instanceof Attribute; + } + + private boolean isValid(BinaryComparison comp) { + return (comp.left() instanceof Literal && comp.right() instanceof Attribute) || + (comp.left() instanceof Attribute && comp.right() instanceof Literal); + } + + private boolean literalOnRight(BinaryComparison comp) { + if (comp.left() instanceof Literal && comp.right() instanceof Attribute) { + return false; + } else if (comp.left() instanceof Attribute && comp.right() instanceof Literal) { + return true; + } else { + throw new RuntimeException("Invalid binary comparison."); + } + } + + private String getColumnName(UnaryExpression unary) { + return ((Attribute) unary.child()).name(); + } + + private String getColumnName(BinaryComparison comp) { + if (literalOnRight(comp)) { + return ((Attribute) comp.left()).name(); + } else { + return ((Attribute) comp.right()).name(); + } + } + + private PredicateLeaf.Type getLiteralType(BinaryComparison comp) { + if (literalOnRight(comp)) { + return toOrcType(((Literal) comp.right()).resultType()); + } else { + return toOrcType(((Literal) comp.left()).resultType()); + } + } + + private Object getLiteral(BinaryComparison comp) { + if (literalOnRight(comp)) { + return ((Literal) comp.right()).value(); + } else { + return ((Literal) comp.left()).value(); + } + } + + private PredicateLeaf.Type toOrcType(TypeInformation type) { + if (type == BasicTypeInfo.BYTE_TYPE_INFO || + type == BasicTypeInfo.SHORT_TYPE_INFO || + type == BasicTypeInfo.INT_TYPE_INFO || + type == BasicTypeInfo.LONG_TYPE_INFO) { + return PredicateLeaf.Type.LONG; + } else if (type == BasicTypeInfo.FLOAT_TYPE_INFO || + type == BasicTypeInfo.DOUBLE_TYPE_INFO) { + return PredicateLeaf.Type.FLOAT; + } else if (type == BasicTypeInfo.BOOLEAN_TYPE_INFO) { + return PredicateLeaf.Type.BOOLEAN; + } else if (type == BasicTypeInfo.STRING_TYPE_INFO) { + return PredicateLeaf.Type.STRING; + } else if (type == SqlTimeTypeInfo.TIMESTAMP) { + return PredicateLeaf.Type.TIMESTAMP; + } else if (type == SqlTimeTypeInfo.DATE) { + return PredicateLeaf.Type.DATE; + } else if (type == BasicTypeInfo.BIG_DEC_TYPE_INFO) { + return PredicateLeaf.Type.DECIMAL; + } else { + // unsupported type + return null; + } + } + + // Builder + + public static Builder builder() { + return new Builder(); + } + + /** + * Constructs an {@link OrcTableSource}. + */ + public static class Builder { + + private String path; + + private TypeDescription schema; + + private Configuration config; + + private int batchSize = 0; + + /** + * Sets the path of the ORC file(s). + * + * @param path The path of the ORC file(s). + * @return The builder. + */ + public Builder path(String path) { + Preconditions.checkNotNull(path, "Path must not be null."); + Preconditions.checkArgument(!path.isEmpty(), "Path must not be empty."); + this.path = path; + return this; + } + + /** + * Sets the ORC schema of the files to read as a String. + * + * @param orcSchema The ORC schema of the files to read as a String. + * @return The builder. + */ + public Builder forOrcSchema(String orcSchema) { + Preconditions.checkNotNull(orcSchema, "ORC schema must not be null."); + this.schema = TypeDescription.fromString(orcSchema); + return this; + } + + /** + * Sets the ORC schema of the files to read as a {@link TypeDescription}. + * + * @param orcSchema The ORC schema of the files to read as a String. + * @return The builder. + */ + public Builder forOrcSchema(TypeDescription orcSchema) { + Preconditions.checkNotNull(orcSchema, "ORC Schema must not be null."); + this.schema = orcSchema; + return this; + } + + /** + * Sets a Hadoop {@link Configuration} for the ORC reader. If no configuration is configured, + * an empty configuration is used. + * + * @param config The Hadoop Configuration for the ORC reader. + * @return The builder. + */ + public Builder withConfiguration(Configuration config) { + Preconditions.checkNotNull(config, "Configuration must not be null."); + this.config = config; + return this; + } + + /** + * Sets the number of rows that are read in a batch. If not configured, the ORC files are + * read with a batch size of 1000. + * + * @param batchSize The number of rows that are read in a batch. + * @return The builder. + */ + public Builder withBatchSize(int batchSize) { + Preconditions.checkArgument(batchSize > 0, "Batch size must be greater than zero."); + this.batchSize = batchSize; + return this; + } + + /** + * Builds the OrcTableSource for this builder. + * + * @return The OrcTableSource for this builder. + */ + public OrcTableSource build() { + Preconditions.checkNotNull(this.path, "Path must not be null."); + Preconditions.checkNotNull(this.schema, "ORC schema must not be null."); + if (this.config == null) { + this.config = new Configuration(); + } + if (this.batchSize == 0) { + // set default batch size + this.batchSize = DEFAULT_BATCH_SIZE; + } + return new OrcTableSource(this.path, this.schema, this.config, this.batchSize); + } + } } diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java index c7557c7477be1..cfb4e0e66a818 100644 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java @@ -39,29 +39,36 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; - import org.apache.orc.TypeDescription; import java.lang.reflect.Array; import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; import java.sql.Date; import java.sql.Timestamp; import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.TimeZone; +import java.util.function.DoubleFunction; +import java.util.function.IntFunction; +import java.util.function.LongFunction; /** * A class that provides utility methods for orc file reading. */ -public class OrcUtils { +class OrcUtils { + + private static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); /** - * Convert ORC schema types to Flink types. - * - * @param schema schema of orc file + * Converts an ORC schema to a Flink TypeInformation. * + * @param schema The ORC schema. + * @return The TypeInformation that corresponds to the ORC schema. */ - public static TypeInformation schemaToTypeInfo(TypeDescription schema) { + static TypeInformation schemaToTypeInfo(TypeDescription schema) { switch (schema.getCategory()) { case BOOLEAN: return BasicTypeInfo.BOOLEAN_TYPE_INFO; @@ -77,6 +84,8 @@ public static TypeInformation schemaToTypeInfo(TypeDescription schema) { return BasicTypeInfo.FLOAT_TYPE_INFO; case DOUBLE: return BasicTypeInfo.DOUBLE_TYPE_INFO; + case DECIMAL: + return BasicTypeInfo.BIG_DEC_TYPE_INFO; case STRING: case CHAR: case VARCHAR: @@ -97,154 +106,164 @@ public static TypeInformation schemaToTypeInfo(TypeDescription schema) { return new RowTypeInfo(fieldTypes, fieldNames); case LIST: TypeDescription elementSchema = schema.getChildren().get(0); - TypeInformation elementType = schemaToTypeInfo(elementSchema); + TypeInformation elementType = schemaToTypeInfo(elementSchema); + // arrays of primitive types are handled as object arrays to support null values return ObjectArrayTypeInfo.getInfoFor(elementType); case MAP: TypeDescription keySchema = schema.getChildren().get(0); TypeDescription valSchema = schema.getChildren().get(1); - TypeInformation keyType = schemaToTypeInfo(keySchema); - TypeInformation valType = schemaToTypeInfo(valSchema); - return new MapTypeInfo(keyType, valType); - case DECIMAL: - return BasicTypeInfo.BIG_DEC_TYPE_INFO; + TypeInformation keyType = schemaToTypeInfo(keySchema); + TypeInformation valType = schemaToTypeInfo(valSchema); + return new MapTypeInfo<>(keyType, valType); case UNION: - throw new UnsupportedOperationException("UNION type not supported yet."); + throw new UnsupportedOperationException("UNION type is not supported yet."); default: throw new IllegalArgumentException("Unknown type " + schema); } } /** - * Fill rows from orc batch. - * - * @param rows the batch of rows need to be filled - * @param schema schema of orc file - * @param batch current orc batch data used to fill the rows - * @param fieldMapping field mapping + * Fills an ORC batch into an array of Row. * + * @param rows The batch of rows need to be filled. + * @param schema The schema of the ORC data. + * @param batch The ORC data. + * @param selectedFields The list of selected ORC fields. + * @return The number of rows that were filled. */ - public static void fillRows(Object[] rows, TypeDescription schema, VectorizedRowBatch batch, int[] fieldMapping) { + static int fillRows(Row[] rows, TypeDescription schema, VectorizedRowBatch batch, int[] selectedFields) { - int totalRowsInBatch = (int) batch.count(); + int rowsToRead = Math.min((int) batch.count(), rows.length); List fieldTypes = schema.getChildren(); - for (int outIdx = 0; outIdx < fieldMapping.length; outIdx++) { - int inIdx = fieldMapping[outIdx]; - readField(rows, outIdx, fieldTypes.get(inIdx), batch.cols[inIdx], null, Math.min((int) totalRowsInBatch, rows.length)); + // read each selected field + for (int rowIdx = 0; rowIdx < selectedFields.length; rowIdx++) { + int orcIdx = selectedFields[rowIdx]; + readField(rows, rowIdx, fieldTypes.get(orcIdx), batch.cols[orcIdx], null, rowsToRead); } + return rowsToRead; } - private static void readField(Object[] rows, int fieldIdx, TypeDescription schema, ColumnVector vector, long[] lengthVector, int childCount) { + /** + * Reads a vector of data into an array of objects. + * + * @param vals The array that needs to be filled. + * @param fieldIdx If the vals array is an array of Row, the index of the field that needs to be filled. + * Otherwise a -1 must be passed and the data is directly filled into the array. + * @param schema The schema of the vector to read. + * @param vector The vector to read. + * @param lengthVector If the vector is of type List or Map, the number of sub-elements to read for each field. Otherwise, it must be null. + * @param childCount The number of vector entries to read. + */ + private static void readField(Object[] vals, int fieldIdx, TypeDescription schema, ColumnVector vector, long[] lengthVector, int childCount) { + // check the type of the vector to decide how to read it. switch (schema.getCategory()) { case BOOLEAN: if (vector.noNulls) { - readNonNullBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readBoolean, OrcUtils::boolArray); } else { - readBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readBoolean, OrcUtils::boolArray); } break; case BYTE: if (vector.noNulls) { - readNonNullByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readByte, OrcUtils::byteArray); } else { - readByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readByte, OrcUtils::byteArray); } break; case SHORT: if (vector.noNulls) { - readNonNullShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readShort, OrcUtils::shortArray); } else { - readShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readShort, OrcUtils::shortArray); } break; case INT: if (vector.noNulls) { - readNonNullIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readInt, OrcUtils::intArray); } else { - readIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readInt, OrcUtils::intArray); } break; case LONG: if (vector.noNulls) { - readNonNullLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readLong, OrcUtils::longArray); } else { - readLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readLong, OrcUtils::longArray); } break; case FLOAT: if (vector.noNulls) { - readNonNullFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readNonNullDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readFloat, OrcUtils::floatArray); } else { - readFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readFloat, OrcUtils::floatArray); } break; case DOUBLE: if (vector.noNulls) { - readNonNullDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readNonNullDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readDouble, OrcUtils::doubleArray); } else { - readDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readDouble, OrcUtils::doubleArray); } break; case CHAR: case VARCHAR: case STRING: if (vector.noNulls) { - readNonNullStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readNonNullBytesColumnAsString(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } else { - readStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readBytesColumnAsString(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } break; case DATE: if (vector.noNulls) { - readNonNullDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumnAsDate(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); } else { - readDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumnAsDate(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); } break; case TIMESTAMP: if (vector.noNulls) { - readNonNullTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + readNonNullTimestampColumn(vals, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); } else { - readTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + readTimestampColumn(vals, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); } break; case BINARY: if (vector.noNulls) { - readNonNullBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readNonNullBytesColumnAsBinary(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } else { - readBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readBytesColumnAsBinary(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } break; case DECIMAL: if (vector.noNulls) { - readNonNullDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); - } - else { - readDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + readNonNullDecimalColumn(vals, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + } else { + readDecimalColumn(vals, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); } break; case STRUCT: if (vector.noNulls) { - readNonNullStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + readNonNullStructColumn(vals, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); } else { - readStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + readStructColumn(vals, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); } break; case LIST: if (vector.noNulls) { - readNonNullListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); - } - else { - readListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + readNonNullListColumn(vals, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + } else { + readListColumn(vals, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); } break; case MAP: if (vector.noNulls) { - readNonNullMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); - } - else { - readMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + readNonNullMapColumn(vals, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + } else { + readMapColumn(vals, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); } break; case UNION: @@ -254,1870 +273,1013 @@ private static void readField(Object[] rows, int fieldIdx, TypeDescription schem } } - private static void readNonNullBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullLongColumn(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount, + LongFunction reader, IntFunction array) { - // check if boolean is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - boolean repeatingValue = vector.vector[0] != 0; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + T repeatingValue = reader.apply(vector.vector[0]); + fillColumnWithRepeatingValue(vals, fieldIdx, repeatingValue, childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = vector.vector[i] != 0; + vals[i] = reader.apply(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } } } } else { // in a list - boolean[] temp; + T[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - boolean repeatingValue = vector.vector[0] != 0; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + T repeatingValue = reader.apply(vector.vector[0]); + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++] != 0; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + temp[j] = reader.apply(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++] != 0; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullDoubleColumn(Object[] vals, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount, + DoubleFunction reader, IntFunction array) { - // check if byte is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - byte repeatingValue = (byte) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + T repeatingValue = reader.apply(vector.vector[0]); + fillColumnWithRepeatingValue(vals, fieldIdx, repeatingValue, childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (byte) vector.vector[i]; + vals[i] = reader.apply(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } } } } else { // in a list - byte[] temp; + T[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - byte repeatingValue = (byte) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + T repeatingValue = reader.apply(vector.vector[0]); + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (byte) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + temp[j] = reader.apply(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (byte) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if short is directly in a list or not, e.g, array + private static void readNonNullBytesColumnAsString(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + // check if the values need to be read into lists or as single values if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - short repeatingValue = (short) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (bytes.isRepeating) { // fill complete column with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); + fillColumnWithRepeatingValue(vals, fieldIdx, repeatingValue, childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (short) vector.vector[i]; + vals[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i], StandardCharsets.UTF_8); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); + rows[i].setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i], StandardCharsets.UTF_8)); } } } - } else { // in a list - short[] temp; + } else { + String[] temp; int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - short repeatingValue = (short) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + if (bytes.isRepeating) { // fill complete list with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0], StandardCharsets.UTF_8); + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (short) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset], StandardCharsets.UTF_8); + offset++; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (short) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if int is directly in a list or not, e.g, array + private static void readNonNullBytesColumnAsBinary(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + // check if the values need to be read into lists or as single values if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - int repeatingValue = (int) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (bytes.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + // don't reuse repeating val to avoid object mutation + vals[i] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + } else { // set as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + // don't reuse repeating val to avoid object mutation + rows[i].setField(fieldIdx, readBinary(bytes.vector[0], bytes.start[0], bytes.length[0])); + } + } } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (int) vector.vector[i]; + vals[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); + rows[i].setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); } } } - } else { // in a list - int[] temp; + } else { + byte[][] temp; int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - int repeatingValue = (int) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + if (bytes.isRepeating) { // fill complete list with first value + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (int) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (int) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullLongColumnAsDate(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - // check if long is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - long repeatingValue = vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + // do not reuse repeated value due to mutability of Date + vals[i] = readDate(vector.vector[0]); + } + } else { // set as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + // do not reuse repeated value due to mutability of Date + rows[i].setField(fieldIdx, readDate(vector.vector[0])); + } + } } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = vector.vector[i]; + vals[i] = readDate(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (Long) vector.vector[i]); + rows[i].setField(fieldIdx, readDate(vector.vector[i])); } } } } else { // in a list - long[] temp; + Date[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - long repeatingValue = vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[0]); + } + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullTimestampColumn(Object[] vals, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { - // check if float is directly in a list or not, e.g, array + // check if the timestamps need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - float repeatingValue = (float) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + // do not reuse value to prevent object mutation + vals[i] = readTimestamp(vector.time[0], vector.nanos[0]); + } + } else { // set as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + // do not reuse value to prevent object mutation + rows[i].setField(fieldIdx, readTimestamp(vector.time[0], vector.nanos[0])); + } + } } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (float) vector.vector[i]; + vals[i] = readTimestamp(vector.time[i], vector.nanos[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); + rows[i].setField(fieldIdx, readTimestamp(vector.time[i], vector.nanos[i])); } } } - } else { // in a list - float[] temp; + } else { + Timestamp[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - float repeatingValue = (float) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + // do not reuse value to prevent object mutation + temp[j] = readTimestamp(vector.time[0], vector.nanos[0]); + } + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (float) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimestamp(vector.time[offset], vector.nanos[offset]); + offset++; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (float) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullDecimalColumn(Object[] vals, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { - // check if double is directly in a list or not, e.g, array + // check if the decimals need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - double repeatingValue = vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + fillColumnWithRepeatingValue(vals, fieldIdx, readBigDecimal(vector.vector[0]), childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = vector.vector[i]; + vals[i] = readBigDecimal(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + rows[i].setField(fieldIdx, readBigDecimal(vector.vector[i])); } } } - } else { // in a list - double[] temp; + } else { + BigDecimal[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - double repeatingValue = vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + BigDecimal repeatingValue = readBigDecimal(vector.vector[0]); + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBigDecimal(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } + } - private static void readNonNullStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + private static void readNonNullStructColumn(Object[] vals, int fieldIdx, StructColumnVector structVector, TypeDescription schema, long[] lengthVector, int childCount) { + + List childrenTypes = schema.getChildren(); + + int numFields = childrenTypes.size(); + // create a batch of Rows to read the structs + Row[] structs = new Row[childCount]; + // TODO: possible improvement: reuse existing Row objects + for (int i = 0; i < childCount; i++) { + structs[i] = new Row(numFields); + } + + // read struct fields + for (int i = 0; i < numFields; i++) { + readField(structs, i, childrenTypes.get(i), structVector.fields[i], null, childCount); + } - // check if string is directly in a list or not, e.g, array + // check if the structs need to be read into lists or as single values if (lengthVector == null) { - if (bytes.isRepeating) { // fill complete column with first value - String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); - } else { - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); - } + if (fieldIdx == -1) { // set struct as an object + System.arraycopy(structs, 0, vals, 0, childCount); + } else { // set struct as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + rows[i].setField(fieldIdx, structs[i]); } } - } - else { // in a list - String[] temp; + } else { // struct in a list int offset = 0; - if (bytes.isRepeating) { // fill list with first value - String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); - if (fieldIdx == -1) { // set list as an object - for (int i = 0; i < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field - for (int i = 0; i < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; - } + Row[] temp; + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + System.arraycopy(structs, offset, temp, 0, temp.length); + offset = offset + temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } - } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } + } + } + } + + private static void readNonNullListColumn(Object[] vals, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + + TypeDescription fieldType = schema.getChildren().get(0); + // check if the list need to be read into lists or as single values + if (lengthVector == null) { + long[] lengthVectorNested = list.lengths; + readField(vals, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } else { // list in a list + Object[] nestedLists = new Object[childCount]; + // length vector for nested list + long[] lengthVectorNested = list.lengths; + // read nested list + readField(nestedLists, -1, fieldType, list.child, lengthVectorNested, list.childCount); + // get type of nestedList + Class classType = nestedLists[0].getClass(); + + // fill outer list with nested list + int offset = 0; + int length; + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object[] temp = (Object[]) Array.newInstance(classType, length); + System.arraycopy(nestedLists, offset, temp, 0, length); + offset = offset + length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } + } + + private static void readNonNullMapColumn(Object[] vals, int fieldIdx, MapColumnVector mapsVector, TypeDescription schema, long[] lengthVector, int childCount) { + + List fieldType = schema.getChildren(); + TypeDescription keyType = fieldType.get(0); + TypeDescription valueType = fieldType.get(1); + + ColumnVector keys = mapsVector.keys; + ColumnVector values = mapsVector.values; + Object[] keyRows = new Object[mapsVector.childCount]; + Object[] valueRows = new Object[mapsVector.childCount]; + + // read map keys and values + readField(keyRows, -1, keyType, keys, null, keyRows.length); + readField(valueRows, -1, valueType, values, null, valueRows.length); + + // check if the maps need to be read into lists or as single values + if (lengthVector == null) { + long[] lengthVectorMap = mapsVector.lengths; + int offset = 0; + + for (int i = 0; i < childCount; i++) { + long numMapEntries = lengthVectorMap[i]; + HashMap map = readHashMap(keyRows, valueRows, offset, numMapEntries); + offset += numMapEntries; + + if (fieldIdx == -1) { + vals[i] = map; + } else { + ((Row) vals[i]).setField(fieldIdx, map); + } + } + } else { // list of map + + long[] lengthVectorMap = mapsVector.lengths; + int mapOffset = 0; // offset of map element + int offset = 0; // offset of map + HashMap[] temp; + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + long numMapEntries = lengthVectorMap[offset]; + temp[j] = readHashMap(keyRows, valueRows, mapOffset, numMapEntries); + mapOffset += numMapEntries; + offset++; + } + if (fieldIdx == 1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); + } + } + } } - private static void readNonNullDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readLongColumn(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount, + LongFunction reader, IntFunction array) { - // check if date is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readDate(vector.vector[0]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[0])); - } - } + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { + boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = readDate(vector.vector[i]); + if (isNullVector[i]) { + vals[i] = null; + } else { + vals[i] = reader.apply(vector.vector[i]); + } } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); - } - } - } - } else { - Date[] temp; - int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[0]); - } - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[0]); + if (isNullVector[i]) { + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; } } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, array); } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[offset++]); + // column contain null values + int offset = 0; + T[] temp; + boolean[] isNullVector = vector.isNull; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { + temp[j] = reader.apply(vector.vector[offset++]); } - rows[i] = temp; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[offset++]); - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + private static void readDoubleColumn(Object[] vals, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount, + DoubleFunction reader, IntFunction array) { - // check if timestamp is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readTimeStamp(vector.time[0], vector.nanos[0]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[0], vector.nanos[0])); - } - } + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { + boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = readTimeStamp(vector.time[i], vector.nanos[i]); + if (isNullVector[i]) { + vals[i] = null; + } else { + vals[i] = reader.apply(vector.vector[i]); + } } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[i], vector.nanos[i])); - } - } - } - } else { - Timestamp[] temp; - int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); - } - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); + if (isNullVector[i]) { + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; } } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, array); } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); + // column contain null values + int offset = 0; + T[] temp; + boolean[] isNullVector = vector.isNull; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { offset++; + } else { + temp[j] = reader.apply(vector.vector[offset++]); } - rows[i] = temp; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + private static void readBytesColumnAsString(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { - // check if string is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (bytes.isRepeating) { // fill complete column with first value - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[0], bytes.start[0], bytes.length[0])); - } - } - } else { - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); - } - } - } - } else { - byte[][] temp; - int offset = 0; - if (bytes.isRepeating) { // fill complete list with first value - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); - } - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); - } - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; - } - } - } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - } - - private static void readNonNullDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { - - // check if decimal is directly in a list or not, e.g, array - if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - fillColumnWithRepeatingValue(rows, fieldIdx, readBigDecimal(vector.vector[0]), childCount); - } else { - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readBigDecimal(vector.vector[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); - } - } - } - } else { - BigDecimal[] temp; - int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - BigDecimal repeatingValue = readBigDecimal(vector.vector[0]); - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; - } - } - } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBigDecimal(vector.vector[offset++]); - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBigDecimal(vector.vector[offset++]); - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - } - - private static void readNonNullStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { - - List childrenTypes = schema.getChildren(); - - int numChildren = childrenTypes.size(); - Row[] nestedFields = new Row[childCount]; - for (int i = 0; i < childCount; i++) { - nestedFields[i] = new Row(numChildren); - } - for (int i = 0; i < numChildren; i++) { - readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); - } - - // check if struct is directly in a list or not, e.g, array> - if (lengthVector == null) { - if (fieldIdx == -1) { // set struct as an object - System.arraycopy(nestedFields, 0, rows, 0, childCount); - } - else { // set struct as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); - } - } - } - else { // struct in a list - int offset = 0; - Row[] temp; - if (fieldIdx == -1) { // set list of struct as an object - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - System.arraycopy(nestedFields, offset, temp, 0, temp.length); - offset = offset + temp.length; - rows[i] = temp; - } - } - else { // set list of struct as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - System.arraycopy(nestedFields, offset, temp, 0, temp.length); - offset = offset + temp.length; - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - private static void readNonNullListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { - - TypeDescription fieldType = schema.getChildren().get(0); - if (lengthVector == null) { - long[] lengthVectorNested = list.lengths; - readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); - } - else { // list in a list - - Object[] nestedList = new Object[childCount]; - - // length vector for nested list - long[] lengthVectorNested = list.lengths; - - // read nested list - readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); - - // get type of nestedList - Class classType = nestedList[0].getClass(); - - // fill outer list with nested list - int offset = 0; - int length; - if (fieldIdx == -1) { // set list of list as an object - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - rows[i] = temp; - - } - } else { // set list of list as an field on Row - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - - } - - private static void readNonNullMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { - - List fieldType = schema.getChildren(); - TypeDescription keyType = fieldType.get(0); - TypeDescription valueType = fieldType.get(1); - - ColumnVector keys = map.keys; - ColumnVector values = map.values; - Object[] keyRows = new Object[map.childCount]; - Object[] valueRows = new Object[map.childCount]; - - // read map kes and values - readField(keyRows, -1, keyType, keys, null, keyRows.length); - readField(valueRows, -1, valueType, values, null, valueRows.length); - - // check if map is directly in a list or not, e.g, array> - if (lengthVector == null) { - long[] lengthVectorMap = map.lengths; - int offset = 0; - if (fieldIdx == -1) { - for (int i = 0; i < childCount; i++) { - rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); - offset += lengthVectorMap[i]; - } - } else { - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); - offset += lengthVectorMap[i]; - } - } - } else { // list of map - - long[] lengthVectorMap = map.lengths; - int mapOffset = 0; // offset of map element - int offset = 0; // offset of map - HashMap[] temp; - if (fieldIdx == -1) { // set map list as an object - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); - mapOffset += lengthVectorMap[offset]; - offset++; - } - rows[i] = temp; - } - } else { // set map list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); - mapOffset += lengthVectorMap[offset]; - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - private static void fillColumnWithRepeatingValue(Object[] rows, int fieldIdx, Object repeatingValue, int childCount) { - - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = repeatingValue; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, repeatingValue); - } - } - } - - private static void fillListWithRepeatingNull(Object[] rows, int fieldIdx, Class classType, long[] lengthVector, int childCount) { - - int length; - if (fieldIdx == -1) { - for (int i = 0; i < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - rows[i] = temp; - } - } else { - for (int i = 0; i < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - - private static void readBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = vector.vector[i] != 0; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, boolean[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - boolean[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++] != 0; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++] != 0; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = (byte) vector.vector[i]; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, byte[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - byte[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (byte) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (byte) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = (short) vector.vector[i]; - } - } else { // set as field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, short[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - short[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (short) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (short) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = (int) vector.vector[i]; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, int[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - int[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (int) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (int) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = vector.vector[i]; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, long[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - long[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { - boolean[] isNullVector = vector.isNull; + boolean[] isNullVector = bytes.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); } - rows[i] = (float) vector.vector[i]; } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); } - ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); } } } } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, float[].class, lengthVector, childCount); + if (bytes.isRepeating) { // fill list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::stringArray); } else { - // column contain null values int offset = 0; - float[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (float) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; i < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (float) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = vector.vector[i]; - } - } else { // set as field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + String[] temp; + boolean[] isNullVector = bytes.isNull; + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; } - ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, double[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - double[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + private static void readBytesColumnAsBinary(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { - // check if string is directly in a list or not, e.g, array + // check if the binary need to be read into lists or as single values if (lengthVector == null) { if (bytes.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = bytes.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); } - rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); } - ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); } } } - } else { // in a list - if (bytes.isRepeating) { // fill list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, String[].class, lengthVector, childCount); + } else { + if (bytes.isRepeating) { // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::binaryArray); } else { int offset = 0; - String[] temp; + byte[][] temp; boolean[] isNullVector = bytes.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; // skip null value - } - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; // skip null value - } - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + } else { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); offset++; } - ((Row) rows[i]).setField(fieldIdx, temp); + } + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - } - private static void readDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readLongColumnAsDate(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - // check if date is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readDate(vector.vector[i]); } - rows[i] = readDate(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readDate(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); } } } - } else { - if (vector.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, Date[].class, lengthVector, childCount); + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::dateArray); } else { + // column contain null values int offset = 0; Date[] temp; boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readDate(vector.vector[offset++]); - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { temp[j] = readDate(vector.vector[offset++]); } - ((Row) rows[i]).setField(fieldIdx, temp); + } + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - } - private static void readTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + private static void readTimestampColumn(Object[] vals, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { - // check if timestamp is directly in a list or not, e.g, array + // check if the timestamps need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + Timestamp ts = readTimestamp(vector.time[i], vector.nanos[i]); + vals[i] = ts; } - Timestamp ts = new Timestamp(vector.time[i]); - ts.setNanos(vector.nanos[i]); - rows[i] = ts; } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + Timestamp ts = readTimestamp(vector.time[i], vector.nanos[i]); + rows[i].setField(fieldIdx, ts); } - Timestamp ts = new Timestamp(vector.time[i]); - ts.setNanos(vector.nanos[i]); - ((Row) rows[i]).setField(fieldIdx, ts); } } } - } - else { + } else { if (vector.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, Timestamp[].class, lengthVector, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::timestampArray); } else { int offset = 0; Timestamp[] temp; boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = new Timestamp(vector.time[offset]); - temp[j].setNanos(vector.nanos[offset]); + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { offset++; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = new Timestamp(vector.time[offset]); - temp[j].setNanos(vector.nanos[offset]); + } else { + temp[j] = readTimestamp(vector.time[offset], vector.nanos[offset]); offset++; } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { - - // check if string is directly in a list or not, e.g, array - if (lengthVector == null) { - if (bytes.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVectorIndex = bytes.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVectorIndex[i]) { - rows[i] = null; - continue; - } - rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVectorIndex[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); } - } - } - } else { - if (bytes.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, byte[][].class, lengthVector, childCount); - } else { - int offset = 0; - byte[][] temp; - boolean[] isNullVector = bytes.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { + private static void readDecimalColumn(Object[] vals, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { - // check if decimal is directly in a list or not, e.g, array + // check if the decimals need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readBigDecimal(vector.vector[i]); } - rows[i] = readBigDecimal(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readBigDecimal(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); } } } } else { if (vector.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, BigDecimal[].class, lengthVector, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::decimalArray); } else { int offset = 0; BigDecimal[] temp; boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readBigDecimal(vector.vector[offset++]); - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { temp[j] = readBigDecimal(vector.vector[offset++]); } - ((Row) rows[i]).setField(fieldIdx, temp); + } + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { + private static void readStructColumn(Object[] vals, int fieldIdx, StructColumnVector structVector, TypeDescription schema, long[] lengthVector, int childCount) { List childrenTypes = schema.getChildren(); - int numChildren = childrenTypes.size(); - Row[] nestedFields = new Row[childCount]; + int numFields = childrenTypes.size(); + // create a batch of Rows to read the structs + Row[] structs = new Row[childCount]; + // TODO: possible improvement: reuse existing Row objects for (int i = 0; i < childCount; i++) { - nestedFields[i] = new Row(numChildren); + structs[i] = new Row(numFields); } - for (int i = 0; i < numChildren; i++) { - readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); + + // read struct fields + for (int i = 0; i < numFields; i++) { + readField(structs, i, childrenTypes.get(i), structVector.fields[i], null, childCount); } - boolean[] isNullVector = struct.isNull; + boolean[] isNullVector = structVector.isNull; - // check if struct is directly in a list or not, e.g, array> + // check if the structs need to be read into lists or as single values if (lengthVector == null) { if (fieldIdx == -1) { // set struct as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = structs[i]; } - rows[i] = nestedFields[i]; } } else { // set struct as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, structs[i]); } - ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); } } } else { // struct in a list int offset = 0; Row[] temp; - if (fieldIdx == -1) { // set list of struct as an object - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } - temp[j] = nestedFields[offset++]; + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + } else { + temp[j] = structs[offset++]; } - rows[i] = temp; } - } - else { // set list of struct as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } - temp[j] = nestedFields[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { // set list of structs as an object + vals[i] = temp; + } else { // set list of structs as field of row + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - private static void readListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + private static void readListColumn(Object[] vals, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { TypeDescription fieldType = schema.getChildren().get(0); + // check if the lists need to be read into lists or as single values if (lengthVector == null) { long[] lengthVectorNested = list.lengths; - readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); - } - else { // list in a list - + readField(vals, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } else { // list in a list Object[] nestedList = new Object[childCount]; - // length vector for nested list long[] lengthVectorNested = list.lengths; - // read nested list readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); - // get type of nestedList - Class classType = nestedList[0].getClass(); - // fill outer list with nested list int offset = 0; int length; - if (fieldIdx == -1) { // set list of list as an object - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - rows[i] = temp; - - } - } else { // set list of list as an field on Row - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - ((Row) rows[i]).setField(fieldIdx, temp); + // get type of nestedList + Class classType = nestedList[0].getClass(); + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object[] temp = (Object[]) Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + if (fieldIdx == -1) { // set list of list as an object + vals[i] = temp; + } else { // set list of list as field of row + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { + private static void readMapColumn(Object[] vals, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { List fieldType = schema.getChildren(); TypeDescription keyType = fieldType.get(0); @@ -2134,27 +1296,28 @@ private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector m boolean[] isNullVector = map.isNull; - // check if map is directly in a list or not, e.g, array> + // check if the maps need to be read into lists or as single values if (lengthVector == null) { long[] lengthVectorMap = map.lengths; int offset = 0; if (fieldIdx == -1) { // set map as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); + offset += lengthVectorMap[i]; } - rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); - offset += lengthVectorMap[i]; } } else { // set map as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); + offset += lengthVectorMap[i]; } - ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); - offset += lengthVectorMap[i]; } } } else { // list of map @@ -2162,41 +1325,110 @@ private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector m int mapOffset = 0; // offset of map element int offset = 0; // offset of map HashMap[] temp; - if (fieldIdx == -1) { // set map list as an object - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } + + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + } else { temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); mapOffset += lengthVectorMap[offset]; offset++; } - rows[i] = temp; } - } else { // set map list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } - temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); - mapOffset += lengthVectorMap[offset]; - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - private static BigDecimal readBigDecimal(HiveDecimalWritable hiveDecimalWritable) { - HiveDecimal hiveDecimal = hiveDecimalWritable.getHiveDecimal(); - return hiveDecimal.bigDecimalValue(); + /** + * Sets a repeating value to all objects or row fields of the passed vals array. + * + * @param vals The array of objects or Rows. + * @param fieldIdx If the objs array is an array of Row, the index of the field that needs to be filled. + * Otherwise a -1 must be passed and the data is directly filled into the array. + * @param repeatingValue The value that is set. + * @param childCount The number of times the value is set. + */ + private static void fillColumnWithRepeatingValue(Object[] vals, int fieldIdx, Object repeatingValue, int childCount) { + + if (fieldIdx == -1) { + // set value as an object + Arrays.fill(vals, 0, childCount, repeatingValue); + } else { + // set value as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + rows[i].setField(fieldIdx, repeatingValue); + } + } + } + + /** + * Sets arrays containing only null values to all objects or row fields of the passed vals array. + * + * @param vals The array of objects or Rows to which the empty arrays are set. + * @param fieldIdx If the objs array is an array of Row, the index of the field that needs to be filled. + * Otherwise a -1 must be passed and the data is directly filled into the array. + * @param lengthVector The vector containing the lengths of the individual empty arrays. + * @param childCount The number of objects or Rows to fill. + * @param array A method to create arrays of the appropriate type. + * @param The type of the arrays to create. + */ + private static void fillListWithRepeatingNull(Object[] vals, int fieldIdx, long[] lengthVector, int childCount, IntFunction array) { + + if (fieldIdx == -1) { + // set empty array as object + for (int i = 0; i < childCount; i++) { + vals[i] = array.apply((int) lengthVector[i]); + } + } else { + // set empty array as field in Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + rows[i].setField(fieldIdx, array.apply((int) lengthVector[i])); + } + } + } + + private static Boolean readBoolean(long l) { + return l != 0; + } + + private static Byte readByte(long l) { + return (byte) l; + } + + private static Short readShort(long l) { + return (short) l; + } + + private static Integer readInt(long l) { + return (int) l; + } + + private static Long readLong(long l) { + return l; + } + + private static Float readFloat(double d) { + return (float) d; + } + + private static Double readDouble(double d) { + return d; + } + + private static Date readDate(long l) { + // day to milliseconds + final long t = l * MILLIS_PER_DAY; + // adjust by local timezone + return new java.sql.Date(t - LOCAL_TZ.getOffset(t)); } private static byte[] readBinary(byte[] src, int srcPos, int length) { @@ -2205,19 +1437,18 @@ private static byte[] readBinary(byte[] src, int srcPos, int length) { return result; } - private static Timestamp readTimeStamp(long time, int nanos) { + private static BigDecimal readBigDecimal(HiveDecimalWritable hiveDecimalWritable) { + HiveDecimal hiveDecimal = hiveDecimalWritable.getHiveDecimal(); + return hiveDecimal.bigDecimalValue(); + } + + private static Timestamp readTimestamp(long time, int nanos) { Timestamp ts = new Timestamp(time); ts.setNanos(nanos); return ts; } - private static Date readDate(long days) { - // day to milliseconds - return new Date(days * 24 * 60 * 60 * 1000); - } - private static HashMap readHashMap(Object[] keyRows, Object[] valueRows, int offset, long length) { - HashMap resultMap = new HashMap<>(); for (int j = 0; j < length; j++) { resultMap.put(keyRows[offset], valueRows[offset]); @@ -2226,4 +1457,52 @@ private static HashMap readHashMap(Object[] keyRows, Object[] valueRows, int off return resultMap; } + private static Boolean[] boolArray(int len) { + return new Boolean[len]; + } + + private static Byte[] byteArray(int len) { + return new Byte[len]; + } + + private static Short[] shortArray(int len) { + return new Short[len]; + } + + private static Integer[] intArray(int len) { + return new Integer[len]; + } + + private static Long[] longArray(int len) { + return new Long[len]; + } + + private static Float[] floatArray(int len) { + return new Float[len]; + } + + private static Double[] doubleArray(int len) { + return new Double[len]; + } + + private static Date[] dateArray(int len) { + return new Date[len]; + } + + private static byte[][] binaryArray(int len) { + return new byte[len][]; + } + + private static String[] stringArray(int len) { + return new String[len]; + } + + private static BigDecimal[] decimalArray(int len) { + return new BigDecimal[len]; + } + + private static Timestamp[] timestampArray(int len) { + return new Timestamp[len]; + } + } diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java deleted file mode 100644 index 0c9c549aed2cf..0000000000000 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * 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.flink.orc; - -import org.apache.flink.api.common.io.FileInputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; - -import org.apache.orc.OrcConf; -import org.apache.orc.OrcFile; -import org.apache.orc.Reader; -import org.apache.orc.RecordReader; -import org.apache.orc.TypeDescription; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; - -import static org.apache.flink.orc.OrcUtils.fillRows; - -/** - * InputFormat to read ORC data. - * For Optimization, reading is done in batch instead of a single row. - */ -public class RowOrcInputFormat - extends FileInputFormat - implements ResultTypeQueryable { - - private static final Logger LOG = LoggerFactory.getLogger(RowOrcInputFormat.class); - private static final int BATCH_SIZE = 1024; - - private org.apache.hadoop.conf.Configuration config; - private TypeDescription schema; - private int[] fieldMapping; - - private transient RowTypeInfo rowType; - private transient RecordReader orcRowsReader; - private transient VectorizedRowBatch rowBatch; - private transient Row[] rows; - - private transient int rowInBatch; - - public RowOrcInputFormat(String path, String schemaString, Configuration orcConfig) { - this(path, TypeDescription.fromString(schemaString), orcConfig); - } - - public RowOrcInputFormat(String path, TypeDescription orcSchema, Configuration orcConfig) { - super(new Path(path)); - this.unsplittable = false; - this.schema = orcSchema; - this.rowType = (RowTypeInfo) OrcUtils.schemaToTypeInfo(schema); - this.config = orcConfig; - - this.fieldMapping = new int[this.schema.getChildren().size()]; - for (int i = 0; i < fieldMapping.length; i++) { - this.fieldMapping[i] = i; - } - - } - - public void setFieldMapping(int[] fieldMapping) { - this.fieldMapping = fieldMapping; - // adapt result type - - TypeInformation[] fieldTypes = new TypeInformation[fieldMapping.length]; - String[] fieldNames = new String[fieldMapping.length]; - for (int i = 0; i < fieldMapping.length; i++) { - fieldTypes[i] = this.rowType.getTypeAt(fieldMapping[i]); - fieldNames[i] = this.rowType.getFieldNames()[fieldMapping[i]]; - } - this.rowType = new RowTypeInfo(fieldTypes, fieldNames); - } - - private boolean[] computeProjectionMask() { - boolean[] projectionMask = new boolean[schema.getMaximumId() + 1]; - for (int inIdx : fieldMapping) { - TypeDescription fieldSchema = schema.getChildren().get(inIdx); - for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) { - projectionMask[i] = true; - } - } - return projectionMask; - } - - @Override - public void openInputFormat() throws IOException { - super.openInputFormat(); - this.rows = new Row[BATCH_SIZE]; - for (int i = 0; i < BATCH_SIZE; i++) { - rows[i] = new Row(fieldMapping.length); - } - } - - @Override - public void open(FileInputSplit fileSplit) throws IOException { - - this.currentSplit = fileSplit; - Preconditions.checkArgument(this.splitStart == 0, "ORC files must be read from the start."); - - if (LOG.isDebugEnabled()) { - LOG.debug("Opening ORC file " + fileSplit.getPath()); - } - - org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(fileSplit.getPath().getPath()); - - Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(config)); - - Reader.Options options = orcReader.options() - .range(fileSplit.getStart(), fileSplit.getLength()) - .useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(config)) - .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(config)) - .tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(config)); - - options.include(computeProjectionMask()); - - // check that schema of file is as expected - if (!this.schema.equals(orcReader.getSchema())) { - - throw new RuntimeException("Invalid schema for file at " + this.filePath + - " Expected:" + this.schema + " Actual: " + orcReader.getSchema()); - } - - this.orcRowsReader = orcReader.rows(options); - - // assign ids - this.schema.getId(); - - this.rowBatch = schema.createRowBatch(BATCH_SIZE); - rowInBatch = 0; - } - - @Override - public void close() throws IOException { - - if (orcRowsReader != null) { - this.orcRowsReader.close(); - } - this.orcRowsReader = null; - - } - - @Override - public void closeInputFormat() throws IOException { - this.rows = null; - this.rows = null; - this.schema = null; - this.rowBatch = null; - } - - @Override - public boolean reachedEnd() throws IOException { - return !ensureBatch(); - } - - private boolean ensureBatch() throws IOException { - - if (rowInBatch >= rowBatch.size) { - rowInBatch = 0; - boolean moreRows = orcRowsReader.nextBatch(rowBatch); - - if (moreRows) { - // read rows - fillRows(rows, schema, rowBatch, fieldMapping); - } - return moreRows; - } - - return true; - } - - @Override - public Row nextRecord(Row reuse) throws IOException { - return rows[this.rowInBatch++]; - } - - @Override - public TypeInformation getProducedType() { - return rowType; - } - - // -------------------------------------------------------------------------------------------- - // Custom serialization methods - // -------------------------------------------------------------------------------------------- - - private void writeObject(ObjectOutputStream out) throws IOException { - this.config.write(out); - out.writeUTF(schema.toString()); - - out.writeInt(fieldMapping.length); - for (int f : fieldMapping) { - out.writeInt(f); - } - - } - - @SuppressWarnings("unchecked") - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - - org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); - configuration.readFields(in); - - if (this.config == null) { - this.config = configuration; - } - this.schema = TypeDescription.fromString(in.readUTF()); - - this.fieldMapping = new int[in.readInt()]; - for (int i = 0; i < fieldMapping.length; i++) { - this.fieldMapping[i] = in.readInt(); - } - - } - -} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java new file mode 100644 index 0000000000000..0efe41fd1525d --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java @@ -0,0 +1,795 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.orc.Reader; +import org.apache.orc.StripeInformation; +import org.junit.After; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link OrcRowInputFormat}. + * + */ +public class OrcRowInputFormatTest { + + private OrcRowInputFormat rowOrcInputFormat; + + @After + public void tearDown() throws IOException { + if (rowOrcInputFormat != null) { + rowOrcInputFormat.close(); + rowOrcInputFormat.closeInputFormat(); + } + rowOrcInputFormat = null; + } + + private static final String TEST_FILE_FLAT = "test-data-flat.orc"; + private static final String TEST_SCHEMA_FLAT = + "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int,_col5:string,_col6:int,_col7:int,_col8:int>"; + + private static final String TEST_FILE_NESTED = "test-data-nested.orc"; + private static final String TEST_SCHEMA_NESTED = + "struct<" + + "boolean1:boolean," + + "byte1:tinyint," + + "short1:smallint," + + "int1:int," + + "long1:bigint," + + "float1:float," + + "double1:double," + + "bytes1:binary," + + "string1:string," + + "middle:struct<" + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">," + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">," + + "map:map<" + + "string," + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">"; + + private static final String TEST_FILE_TIMETYPES = "test-data-timetypes.orc"; + private static final String TEST_SCHEMA_TIMETYPES = "struct"; + + private static final String TEST_FILE_DECIMAL = "test-data-decimal.orc"; + private static final String TEST_SCHEMA_DECIMAL = "struct<_col0:decimal(10,5)>"; + + private static final String TEST_FILE_NESTEDLIST = "test-data-nestedlist.orc"; + private static final String TEST_SCHEMA_NESTEDLIST = "struct>>>"; + + @Test(expected = FileNotFoundException.class) + public void testInvalidPath() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat("/does/not/exist", TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.open(inputSplits[0]); + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testInvalidProjection1() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + int[] projectionMask = {1, 2, 3, -1}; + rowOrcInputFormat.selectFields(projectionMask); + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testInvalidProjection2() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + int[] projectionMask = {1, 2, 3, 9}; + rowOrcInputFormat.selectFields(projectionMask); + } + + @Test + public void testProjectionMaskNested() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + OrcRowInputFormat spy = spy(rowOrcInputFormat); + + // mock options to check configuration of ORC reader + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.selectFields(9, 11, 2); + spy.openInputFormat(); + FileInputSplit[] splits = spy.createInputSplits(1); + spy.open(splits[0]); + + // top-level struct is false + boolean[] expected = new boolean[]{ + false, // top level + false, false, // flat fields 0, 1 are out + true, // flat field 2 is in + false, false, false, false, false, false, // flat fields 3, 4, 5, 6, 7, 8 are out + true, true, true, true, true, // nested field 9 is in + false, false, false, false, // nested field 10 is out + true, true, true, true, true}; // nested field 11 is in + assertArrayEquals(expected, options.getInclude()); + } + + @Test + public void testSplitStripesGivenSplits() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + + OrcRowInputFormat spy = spy(rowOrcInputFormat); + + // mock options to check configuration of ORC reader + Reader.Options options = spy(new Reader.Options()); + doReturn(options).when(spy).getOptions(any()); + + FileInputSplit[] splits = spy.createInputSplits(3); + + spy.openInputFormat(); + spy.open(splits[0]); + verify(options).range(eq(3L), eq(137005L)); + spy.open(splits[1]); + verify(options).range(eq(137008L), eq(136182L)); + spy.open(splits[2]); + verify(options).range(eq(273190L), eq(123633L)); + } + + @Test + public void testSplitStripesCustomSplits() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + + OrcRowInputFormat spy = spy(rowOrcInputFormat); + + // mock list of stripes + List stripes = new ArrayList<>(); + StripeInformation stripe1 = mock(StripeInformation.class); + when(stripe1.getOffset()).thenReturn(10L); + when(stripe1.getLength()).thenReturn(90L); + StripeInformation stripe2 = mock(StripeInformation.class); + when(stripe2.getOffset()).thenReturn(100L); + when(stripe2.getLength()).thenReturn(100L); + StripeInformation stripe3 = mock(StripeInformation.class); + when(stripe3.getOffset()).thenReturn(200L); + when(stripe3.getLength()).thenReturn(100L); + StripeInformation stripe4 = mock(StripeInformation.class); + when(stripe4.getOffset()).thenReturn(300L); + when(stripe4.getLength()).thenReturn(100L); + StripeInformation stripe5 = mock(StripeInformation.class); + when(stripe5.getOffset()).thenReturn(400L); + when(stripe5.getLength()).thenReturn(100L); + stripes.add(stripe1); + stripes.add(stripe2); + stripes.add(stripe3); + stripes.add(stripe4); + stripes.add(stripe5); + doReturn(stripes).when(spy).getStripes(any()); + + // mock options to check configuration of ORC reader + Reader.Options options = spy(new Reader.Options()); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + // split ranging 2 stripes + spy.open(new FileInputSplit(0, new Path(getPath(TEST_FILE_FLAT)), 0, 150, new String[]{})); + verify(options).range(eq(10L), eq(190L)); + // split ranging 0 stripes + spy.open(new FileInputSplit(1, new Path(getPath(TEST_FILE_FLAT)), 150, 10, new String[]{})); + verify(options).range(eq(0L), eq(0L)); + // split ranging 1 stripe + spy.open(new FileInputSplit(2, new Path(getPath(TEST_FILE_FLAT)), 160, 41, new String[]{})); + verify(options).range(eq(200L), eq(100L)); + // split ranging 2 stripe + spy.open(new FileInputSplit(3, new Path(getPath(TEST_FILE_FLAT)), 201, 299, new String[]{})); + verify(options).range(eq(300L), eq(200L)); + } + + @Test + public void testProducedType() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + assertTrue(rowOrcInputFormat.getProducedType() instanceof RowTypeInfo); + RowTypeInfo producedType = (RowTypeInfo) rowOrcInputFormat.getProducedType(); + + assertArrayEquals( + new TypeInformation[]{ + // primitives + Types.BOOLEAN, Types.BYTE, Types.SHORT, Types.INT, Types.LONG, Types.FLOAT, Types.DOUBLE, + // binary + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + // string + Types.STRING, + // struct + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING))), + // list + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING)), + // map + new MapTypeInfo<>(Types.STRING, Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING)) + }, + producedType.getFieldTypes()); + assertArrayEquals( + new String[]{"boolean1", "byte1", "short1", "int1", "long1", "float1", "double1", "bytes1", "string1", "middle", "list", "map"}, + producedType.getFieldNames()); + } + + @Test + public void testProducedTypeWithProjection() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.selectFields(9, 3, 7, 10); + + assertTrue(rowOrcInputFormat.getProducedType() instanceof RowTypeInfo); + RowTypeInfo producedType = (RowTypeInfo) rowOrcInputFormat.getProducedType(); + + assertArrayEquals( + new TypeInformation[]{ + // struct + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING))), + // int + Types.INT, + // binary + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + // list + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING)) + }, + producedType.getFieldTypes()); + assertArrayEquals( + new String[]{"middle", "int1", "bytes1", "list"}, + producedType.getFieldNames()); + } + + @Test + public void testSerialization() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + + rowOrcInputFormat.selectFields(0, 4, 1); + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("_col1", PredicateLeaf.Type.STRING, "M")); + + byte[] bytes = InstantiationUtil.serializeObject(rowOrcInputFormat); + OrcRowInputFormat copy = InstantiationUtil.deserializeObject(bytes, getClass().getClassLoader()); + + FileInputSplit[] splits = copy.createInputSplits(1); + copy.openInputFormat(); + copy.open(splits[0]); + assertFalse(copy.reachedEnd()); + Row row = copy.nextRecord(null); + + assertNotNull(row); + assertEquals(3, row.getArity()); + // check first row + assertEquals(1, row.getField(0)); + assertEquals(500, row.getField(1)); + assertEquals("M", row.getField(2)); + } + + @Test + public void testNumericBooleanStringPredicates() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.selectFields(0, 1, 2, 3, 4, 5, 6, 8); + + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("boolean1", PredicateLeaf.Type.BOOLEAN, false)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.LessThan("byte1", PredicateLeaf.Type.LONG, 1)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.LessThanEquals("short1", PredicateLeaf.Type.LONG, 1024)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Between("int1", PredicateLeaf.Type.LONG, -1, 65536)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("long1", PredicateLeaf.Type.LONG, 9223372036854775807L)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("float1", PredicateLeaf.Type.FLOAT, 1.0)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("double1", PredicateLeaf.Type.FLOAT, -15.0)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.IsNull("string1", PredicateLeaf.Type.STRING)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("string1", PredicateLeaf.Type.STRING, "hello")); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.openInputFormat(); + + // mock options to check configuration of ORC reader + OrcRowInputFormat spy = spy(rowOrcInputFormat); + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + spy.open(splits[0]); + + // verify predicate configuration + SearchArgument sarg = options.getSearchArgument(); + assertNotNull(sarg); + assertEquals("(and leaf-0 leaf-1 leaf-2 leaf-3 leaf-4 leaf-5 leaf-6 leaf-7 leaf-8)", sarg.getExpression().toString()); + assertEquals(9, sarg.getLeaves().size()); + List leaves = sarg.getLeaves(); + assertEquals("(EQUALS boolean1 false)", leaves.get(0).toString()); + assertEquals("(LESS_THAN byte1 1)", leaves.get(1).toString()); + assertEquals("(LESS_THAN_EQUALS short1 1024)", leaves.get(2).toString()); + assertEquals("(BETWEEN int1 -1 65536)", leaves.get(3).toString()); + assertEquals("(EQUALS long1 9223372036854775807)", leaves.get(4).toString()); + assertEquals("(EQUALS float1 1.0)", leaves.get(5).toString()); + assertEquals("(EQUALS double1 -15.0)", leaves.get(6).toString()); + assertEquals("(IS_NULL string1)", leaves.get(7).toString()); + assertEquals("(EQUALS string1 hello)", leaves.get(8).toString()); + } + + @Test + public void testTimePredicates() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_TIMETYPES), TEST_SCHEMA_TIMETYPES, new Configuration()); + + rowOrcInputFormat.addPredicate( + // OR + new OrcRowInputFormat.Or( + // timestamp pred + new OrcRowInputFormat.Equals("time", PredicateLeaf.Type.TIMESTAMP, Timestamp.valueOf("1900-05-05 12:34:56.100")), + // date pred + new OrcRowInputFormat.Equals("date", PredicateLeaf.Type.DATE, Date.valueOf("1900-12-25"))) + ); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.openInputFormat(); + + // mock options to check configuration of ORC reader + OrcRowInputFormat spy = spy(rowOrcInputFormat); + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + spy.open(splits[0]); + + // verify predicate configuration + SearchArgument sarg = options.getSearchArgument(); + assertNotNull(sarg); + assertEquals("(or leaf-0 leaf-1)", sarg.getExpression().toString()); + assertEquals(2, sarg.getLeaves().size()); + List leaves = sarg.getLeaves(); + assertEquals("(EQUALS time 1900-05-05 12:34:56.1)", leaves.get(0).toString()); + assertEquals("(EQUALS date 1900-12-25)", leaves.get(1).toString()); + } + + @Test + public void testDecimalPredicate() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_DECIMAL), TEST_SCHEMA_DECIMAL, new Configuration()); + + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Not( + // decimal pred + new OrcRowInputFormat.Equals("_col0", PredicateLeaf.Type.DECIMAL, BigDecimal.valueOf(-1000.5)))); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.openInputFormat(); + + // mock options to check configuration of ORC reader + OrcRowInputFormat spy = spy(rowOrcInputFormat); + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + spy.open(splits[0]); + + // verify predicate configuration + SearchArgument sarg = options.getSearchArgument(); + assertNotNull(sarg); + assertEquals("(not leaf-0)", sarg.getExpression().toString()); + assertEquals(1, sarg.getLeaves().size()); + List leaves = sarg.getLeaves(); + assertEquals("(EQUALS _col0 -1000.5)", leaves.get(0).toString()); + } + + @Test(expected = IllegalArgumentException.class) + public void testPredicateWithInvalidColumn() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("unknown", PredicateLeaf.Type.LONG, 42)); + } + + @Test + public void testReadNestedFile() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(12, row.getArity()); + assertEquals(false, row.getField(0)); + assertEquals((byte) 1, row.getField(1)); + assertEquals((short) 1024, row.getField(2)); + assertEquals(65536, row.getField(3)); + assertEquals(9223372036854775807L, row.getField(4)); + assertEquals(1.0f, row.getField(5)); + assertEquals(-15.0d, row.getField(6)); + assertArrayEquals(new byte[]{0, 1, 2, 3, 4}, (byte[]) row.getField(7)); + assertEquals("hi", row.getField(8)); + // check nested field + assertTrue(row.getField(9) instanceof Row); + Row nested1 = (Row) row.getField(9); + assertEquals(1, nested1.getArity()); + assertTrue(nested1.getField(0) instanceof Object[]); + Object[] nestedList1 = (Object[]) nested1.getField(0); + assertEquals(2, nestedList1.length); + assertEquals(Row.of(1, "bye"), nestedList1[0]); + assertEquals(Row.of(2, "sigh"), nestedList1[1]); + // check list + assertTrue(row.getField(10) instanceof Object[]); + Object[] list1 = (Object[]) row.getField(10); + assertEquals(2, list1.length); + assertEquals(Row.of(3, "good"), list1[0]); + assertEquals(Row.of(4, "bad"), list1[1]); + // check map + assertTrue(row.getField(11) instanceof HashMap); + HashMap map1 = (HashMap) row.getField(11); + assertEquals(0, map1.size()); + + // read second row + assertFalse(rowOrcInputFormat.reachedEnd()); + row = rowOrcInputFormat.nextRecord(null); + + // validate second row + assertNotNull(row); + assertEquals(12, row.getArity()); + assertEquals(true, row.getField(0)); + assertEquals((byte) 100, row.getField(1)); + assertEquals((short) 2048, row.getField(2)); + assertEquals(65536, row.getField(3)); + assertEquals(9223372036854775807L, row.getField(4)); + assertEquals(2.0f, row.getField(5)); + assertEquals(-5.0d, row.getField(6)); + assertArrayEquals(new byte[]{}, (byte[]) row.getField(7)); + assertEquals("bye", row.getField(8)); + // check nested field + assertTrue(row.getField(9) instanceof Row); + Row nested2 = (Row) row.getField(9); + assertEquals(1, nested2.getArity()); + assertTrue(nested2.getField(0) instanceof Object[]); + Object[] nestedList2 = (Object[]) nested2.getField(0); + assertEquals(2, nestedList2.length); + assertEquals(Row.of(1, "bye"), nestedList2[0]); + assertEquals(Row.of(2, "sigh"), nestedList2[1]); + // check list + assertTrue(row.getField(10) instanceof Object[]); + Object[] list2 = (Object[]) row.getField(10); + assertEquals(3, list2.length); + assertEquals(Row.of(100000000, "cat"), list2[0]); + assertEquals(Row.of(-100000, "in"), list2[1]); + assertEquals(Row.of(1234, "hat"), list2[2]); + // check map + assertTrue(row.getField(11) instanceof HashMap); + HashMap map = (HashMap) row.getField(11); + assertEquals(2, map.size()); + assertEquals(Row.of(5, "chani"), map.get("chani")); + assertEquals(Row.of(1, "mauddib"), map.get("mauddib")); + + assertTrue(rowOrcInputFormat.reachedEnd()); + } + + @Test + public void testReadTimeTypeFile() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_TIMETYPES), TEST_SCHEMA_TIMETYPES, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(2, row.getArity()); + assertEquals(Timestamp.valueOf("1900-05-05 12:34:56.1"), row.getField(0)); + assertEquals(Date.valueOf("1900-12-25"), row.getField(1)); + + // check correct number of rows + long cnt = 1; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + assertEquals(70000, cnt); + } + + @Test + public void testReadDecimalTypeFile() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_DECIMAL), TEST_SCHEMA_DECIMAL, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(1, row.getArity()); + assertEquals(BigDecimal.valueOf(-1000.5d), row.getField(0)); + + // check correct number of rows + long cnt = 1; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + assertEquals(6000, cnt); + } + + @Test + public void testReadNestedListFile() throws Exception { + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_NESTEDLIST), TEST_SCHEMA_NESTEDLIST, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + + Row row = null; + long cnt = 0; + + // read all rows + while (!rowOrcInputFormat.reachedEnd()) { + + row = rowOrcInputFormat.nextRecord(row); + assertEquals(1, row.getArity()); + + // outer list + Object[] list = (Object[]) row.getField(0); + assertEquals(1, list.length); + + // nested list of rows + Row[] nestedRows = (Row[]) list[0]; + assertEquals(1, nestedRows.length); + assertEquals(1, nestedRows[0].getArity()); + + // verify list value + assertEquals(cnt, nestedRows[0].getField(0)); + cnt++; + } + // number of rows in file + assertEquals(100, cnt); + } + + @Test + public void testReadWithProjection() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.selectFields(7, 0, 10, 8); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(4, row.getArity()); + // check binary + assertArrayEquals(new byte[]{0, 1, 2, 3, 4}, (byte[]) row.getField(0)); + // check boolean + assertEquals(false, row.getField(1)); + // check list + assertTrue(row.getField(2) instanceof Object[]); + Object[] list1 = (Object[]) row.getField(2); + assertEquals(2, list1.length); + assertEquals(Row.of(3, "good"), list1[0]); + assertEquals(Row.of(4, "bad"), list1[1]); + // check string + assertEquals("hi", row.getField(3)); + + // check that there is a second row with four fields + assertFalse(rowOrcInputFormat.reachedEnd()); + row = rowOrcInputFormat.nextRecord(null); + assertNotNull(row); + assertEquals(4, row.getArity()); + assertTrue(rowOrcInputFormat.reachedEnd()); + } + + @Test + public void testReadFileInSplits() throws IOException{ + + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.selectFields(0, 1); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(4); + assertEquals(4, splits.length); + rowOrcInputFormat.openInputFormat(); + + long cnt = 0; + // read all splits + for (FileInputSplit split : splits) { + + // open split + rowOrcInputFormat.open(split); + // read and count all rows + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + } + // check that all rows have been read + assertEquals(1920800, cnt); + } + + @Test + public void testReadFileWithFilter() throws IOException{ + + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.selectFields(0, 1); + + // read head and tail of file + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Or( + new OrcRowInputFormat.LessThan("_col0", PredicateLeaf.Type.LONG, 10L), + new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThanEquals("_col0", PredicateLeaf.Type.LONG, 1920000L)) + )); + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("_col1", PredicateLeaf.Type.STRING, "M")); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + + // open split + rowOrcInputFormat.open(splits[0]); + + // read and count all rows + long cnt = 0; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + // check that only the first and last stripes of the file have been read. + // Each stripe has 5000 rows, except the last which has 800 rows. + assertEquals(5800, cnt); + } + + @Test + public void testReadFileWithEvolvedSchema() throws IOException{ + + rowOrcInputFormat = new OrcRowInputFormat( + getPath(TEST_FILE_FLAT), + "struct<_col0:int,_col1:string,_col4:string,_col3:string>", // previous version of schema + new Configuration()); + rowOrcInputFormat.selectFields(3, 0, 2); + + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.LessThan("_col0", PredicateLeaf.Type.LONG, 10L)); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + + // open split + rowOrcInputFormat.open(splits[0]); + + // read and validate first row + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + assertNotNull(row); + assertEquals(3, row.getArity()); + assertEquals("Primary", row.getField(0)); + assertEquals(1, row.getField(1)); + assertEquals("M", row.getField(2)); + + // read and count remaining rows + long cnt = 1; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + // check that only the first and last stripes of the file have been read. + // Each stripe has 5000 rows, except the last which has 800 rows. + assertEquals(5000, cnt); + } + + private String getPath(String fileName) { + return getClass().getClassLoader().getResource(fileName).getPath(); + } +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java index 3de6ab31010a5..e6ef1e16f8697 100644 --- a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java @@ -18,125 +18,101 @@ package org.apache.flink.orc; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.typeutils.MapTypeInfo; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.java.BatchTableEnvironment; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.types.Row; -import org.hamcrest.CoreMatchers; -import org.junit.Assert; import org.junit.Test; -import java.net.URL; -import java.util.ArrayList; import java.util.List; +import static org.junit.Assert.assertEquals; + /** * Tests for {@link OrcTableSource}. */ public class OrcTableSourceITCase extends MultipleProgramsTestBase { - private static final String TEST1_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; - - private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); - - - private static final String[] TEST1_DATA = new String[] { - "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", - "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + - "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; + private static final String TEST_FILE_FLAT = "test-data-flat.orc"; + private static final String TEST_SCHEMA_FLAT = + "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int,_col5:string,_col6:int,_col7:int,_col8:int>"; public OrcTableSourceITCase() { super(TestExecutionMode.COLLECTION); } @Test - public void testOrcTableSource() throws Exception { + public void testFullScan() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); - assert (test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); - - tEnv.registerTableSource("orcTable", orc); - - String query = "Select * from orcTable"; - Table t = tEnv.sql(query); + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_FLAT)) + .forOrcSchema(TEST_SCHEMA_FLAT) + .build(); + tEnv.registerTableSource("OrcTable", orc); + + String query = + "SELECT COUNT(*), " + + "MIN(_col0), MAX(_col0), " + + "MIN(_col1), MAX(_col1), " + + "MIN(_col2), MAX(_col2), " + + "MIN(_col3), MAX(_col3), " + + "MIN(_col4), MAX(_col4), " + + "MIN(_col5), MAX(_col5), " + + "MIN(_col6), MAX(_col6), " + + "MIN(_col7), MAX(_col7), " + + "MIN(_col8), MAX(_col8) " + + "FROM OrcTable"; + Table t = tEnv.sqlQuery(query); DataSet dataSet = tEnv.toDataSet(t, Row.class); - List records = dataSet.collect(); - - Assert.assertEquals(records.size(), 2); + List result = dataSet.collect(); - List actualRecords = new ArrayList<>(); - for (Row record : records) { - Assert.assertEquals(record.getArity(), 12); - actualRecords.add(record.toString()); - } - - Assert.assertThat(actualRecords, CoreMatchers.hasItems(TEST1_DATA)); + assertEquals(1, result.size()); + assertEquals( + "1920800,1,1920800,F,M,D,W,2 yr Degree,Unknown,500,10000,Good,Unknown,0,6,0,6,0,6", + result.get(0).toString()); } @Test - public void testOrcTableProjection() throws Exception { + public void testScanWithProjectionAndFilter() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); - assert(test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); - - tEnv.registerTableSource("orcTable", orc); - - String query = "Select middle,list,map from orcTable"; - Table t = tEnv.sql(query); - - String[] colNames = new String[] {"middle", "list", "map"}; - - RowTypeInfo rowTypeInfo = new RowTypeInfo( - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO}, - new String[] {"int1", "string1"}); - - RowTypeInfo structTypeInfo = new RowTypeInfo( - new TypeInformation[] {ObjectArrayTypeInfo.getInfoFor(rowTypeInfo)}, - new String[] {"list"}); - - TypeInformation[] colTypes = new TypeInformation[] { - structTypeInfo, - ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), - new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) - }; - - TableSchema actualTableSchema = new TableSchema(colNames, colTypes); - - Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); - Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); - Assert.assertEquals(actualTableSchema.toString(), t.getSchema().toString()); + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_FLAT)) + .forOrcSchema(TEST_SCHEMA_FLAT) + .build(); + tEnv.registerTableSource("OrcTable", orc); + + String query = + "SELECT " + + "MIN(_col4), MAX(_col4), " + + "MIN(_col3), MAX(_col3), " + + "MIN(_col0), MAX(_col0), " + + "MIN(_col2), MAX(_col2), " + + "COUNT(*) " + + "FROM OrcTable " + + "WHERE (_col0 BETWEEN 4975 and 5024 OR _col0 BETWEEN 9975 AND 10024) AND _col1 = 'F'"; + Table t = tEnv.sqlQuery(query); DataSet dataSet = tEnv.toDataSet(t, Row.class); - List records = dataSet.collect(); - - Assert.assertEquals(records.size(), 2); - for (Row record: records) { - Assert.assertEquals(record.getArity(), 3); - } + List result = dataSet.collect(); + assertEquals(1, result.size()); + assertEquals( + "1500,6000,2 yr Degree,Unknown,4976,10024,D,W,50", + result.get(0).toString()); } + private String getPath(String fileName) { + return getClass().getClassLoader().getResource(fileName).getPath(); + } } diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java index c285054c6e155..b654f76819a79 100644 --- a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java @@ -18,96 +18,248 @@ package org.apache.flink.orc; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.table.expressions.ArrayElementAt; +import org.apache.flink.table.expressions.EqualTo; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.GetCompositeField; +import org.apache.flink.table.expressions.GreaterThan; +import org.apache.flink.table.expressions.Literal; +import org.apache.flink.table.expressions.ResolvedFieldReference; +import org.apache.flink.types.Row; -import org.junit.Assert; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.junit.Test; +import org.mockito.ArgumentCaptor; -import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Unit Tests for {@link OrcTableSource}. */ public class OrcTableSourceTest { - private static final String TEST1_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; + private static final String TEST_FILE_NESTED = "test-data-nested.orc"; + private static final String TEST_SCHEMA_NESTED = + "struct<" + + "boolean1:boolean," + + "byte1:tinyint," + + "short1:smallint," + + "int1:int," + + "long1:bigint," + + "float1:float," + + "double1:double," + + "bytes1:binary," + + "string1:string," + + "middle:struct<" + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">," + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">," + + "map:map<" + + "string," + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">"; + + @Test + public void testGetReturnType() throws Exception { + + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); - private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + TypeInformation returnType = orc.getReturnType(); + assertNotNull(returnType); + assertTrue(returnType instanceof RowTypeInfo); + RowTypeInfo rowType = (RowTypeInfo) returnType; + + RowTypeInfo expected = Types.ROW_NAMED(getNestedFieldNames(), getNestedFieldTypes()); + assertEquals(expected, rowType); + } @Test - public void testOrcSchema() throws Exception { + public void testGetTableSchema() throws Exception { + + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); + + TableSchema schema = orc.getTableSchema(); + assertNotNull(schema); + assertArrayEquals(getNestedFieldNames(), schema.getColumnNames()); + assertArrayEquals(getNestedFieldTypes(), schema.getTypes()); + } + + @Test + public void testProjectFields() throws Exception { + + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); + + OrcTableSource projected = (OrcTableSource) orc.projectFields(new int[]{3, 5, 1, 0}); - assert(test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + // ensure copy is returned + assertTrue(orc != projected); - String expectedSchema = "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer, long1: Long, " + - "float1: Float, double1: Double, bytes1: byte[], string1: String, " + - "middle: Row(list: ObjectArrayTypeInfo), " + - "list: ObjectArrayTypeInfo, " + - "map: Map)"; + // ensure table schema is identical + assertEquals(orc.getTableSchema(), projected.getTableSchema()); - Assert.assertEquals(expectedSchema, orc.getReturnType().toString()); + // ensure return type was adapted + String[] fieldNames = getNestedFieldNames(); + TypeInformation[] fieldTypes = getNestedFieldTypes(); + assertEquals( + Types.ROW_NAMED( + new String[] {fieldNames[3], fieldNames[5], fieldNames[1], fieldNames[0]}, + new TypeInformation[] {fieldTypes[3], fieldTypes[5], fieldTypes[1], fieldTypes[0]}), + projected.getReturnType()); + // ensure IF is configured with selected fields + OrcTableSource spyTS = spy(projected); + OrcRowInputFormat mockIF = mock(OrcRowInputFormat.class); + doReturn(mockIF).when(spyTS).buildOrcInputFormat(); + spyTS.getDataSet(mock(ExecutionEnvironment.class)); + verify(mockIF).selectFields(eq(3), eq(5), eq(1), eq(0)); } @Test - public void testOrcTableSchema() throws Exception { + public void testApplyPredicate() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); - assert(test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + // expressions for predicates + Expression pred1 = new GreaterThan( + new ResolvedFieldReference("int1", Types.INT), + new Literal(100, Types.INT)); + Expression pred2 = new EqualTo( + new ResolvedFieldReference("string1", Types.STRING), + new Literal("hello", Types.STRING)); + Expression pred3 = new EqualTo( + new GetCompositeField( + new ArrayElementAt( + new ResolvedFieldReference( + "list", + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[] {"int1", "string1"}, Types.INT, Types.STRING))), + new Literal(1, Types.INT)), + "int1"), + new Literal(1, Types.INT) + ); + ArrayList preds = new ArrayList<>(); + preds.add(pred1); + preds.add(pred2); + preds.add(pred3); - tEnv.registerTableSource("orcTable", orc); - String query = "Select * from orcTable"; - Table t = tEnv.sql(query); + // apply predicates on TableSource + OrcTableSource projected = (OrcTableSource) orc.applyPredicate(preds); - String[] colNames = new String[] { - "boolean1", "byte1", "short1", "int1", "long1", "float1", - "double1", "bytes1", "string1", "list", "list0", "map" - }; + // ensure copy is returned + assertTrue(orc != projected); - RowTypeInfo rowTypeInfo = new RowTypeInfo( - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO}, - new String[] {"int1", "string1"}); - - TypeInformation[] colTypes = new TypeInformation[] { - BasicTypeInfo.BOOLEAN_TYPE_INFO, - BasicTypeInfo.BYTE_TYPE_INFO, - BasicTypeInfo.SHORT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.FLOAT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), - ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), - new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) - }; - TableSchema expectedTableSchema = new TableSchema(colNames, colTypes); + // ensure table schema is identical + assertEquals(orc.getTableSchema(), projected.getTableSchema()); + + // ensure return type is identical + assertEquals( + Types.ROW_NAMED(getNestedFieldNames(), getNestedFieldTypes()), + projected.getReturnType()); + + // ensure IF is configured with supported predicates + OrcTableSource spyTS = spy(projected); + OrcRowInputFormat mockIF = mock(OrcRowInputFormat.class); + doReturn(mockIF).when(spyTS).buildOrcInputFormat(); + spyTS.getDataSet(mock(ExecutionEnvironment.class)); + + ArgumentCaptor arguments = ArgumentCaptor.forClass(OrcRowInputFormat.Predicate.class); + verify(mockIF, times(2)).addPredicate(arguments.capture()); + List values = arguments.getAllValues().stream().map(Object::toString).collect(Collectors.toList()); + assertTrue(values.contains( + new OrcRowInputFormat.Not(new OrcRowInputFormat.LessThanEquals("int1", PredicateLeaf.Type.LONG, 100)).toString())); + assertTrue(values.contains( + new OrcRowInputFormat.Equals("string1", PredicateLeaf.Type.STRING, "hello").toString())); + + // ensure filter pushdown is correct + assertTrue(spyTS.isFilterPushedDown()); + assertFalse(orc.isFilterPushedDown()); + } + + private String getPath(String fileName) { + return getClass().getClassLoader().getResource(fileName).getPath(); + } - Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); - Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); - Assert.assertEquals(expectedTableSchema.toString(), t.getSchema().toString()); + private String[] getNestedFieldNames() { + return new String[] { + "boolean1", "byte1", "short1", "int1", "long1", "float1", "double1", "bytes1", "string1", "middle", "list", "map" + }; + } + private TypeInformation[] getNestedFieldTypes() { + return new TypeInformation[]{ + Types.BOOLEAN, Types.BYTE, Types.SHORT, Types.INT, Types.LONG, Types.FLOAT, Types.DOUBLE, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, Types.STRING, + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + ), + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ), + new MapTypeInfo<>( + Types.STRING, + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + }; } } diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java new file mode 100644 index 0000000000000..2cb17150bf306 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java @@ -0,0 +1,148 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import org.apache.orc.TypeDescription; +import org.junit.Assert; +import org.junit.Test; + +/** + * Unit tests for {@link OrcUtils}. + * + */ +public class OrcUtilsTest { + + @Test + public void testFlatSchemaToTypeInfo1() { + + String schema = + "struct<" + + "boolean1:boolean," + + "byte1:tinyint," + + "short1:smallint," + + "int1:int," + + "long1:bigint," + + "float1:float," + + "double1:double," + + "bytes1:binary," + + "string1:string," + + "date1:date," + + "timestamp1:timestamp," + + "decimal1:decimal(5,2)" + + ">"; + TypeInformation typeInfo = OrcUtils.schemaToTypeInfo(TypeDescription.fromString(schema)); + + Assert.assertNotNull(typeInfo); + Assert.assertTrue(typeInfo instanceof RowTypeInfo); + RowTypeInfo rowTypeInfo = (RowTypeInfo) typeInfo; + + // validate field types + Assert.assertArrayEquals( + new TypeInformation[]{ + Types.BOOLEAN, Types.BYTE, Types.SHORT, Types.INT, Types.LONG, Types.FLOAT, Types.DOUBLE, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, Types.STRING, + Types.SQL_DATE, Types.SQL_TIMESTAMP, BasicTypeInfo.BIG_DEC_TYPE_INFO + }, + rowTypeInfo.getFieldTypes()); + + // validate field names + Assert.assertArrayEquals( + new String[] { + "boolean1", "byte1", "short1", "int1", "long1", "float1", "double1", + "bytes1", "string1", "date1", "timestamp1", "decimal1" + }, + rowTypeInfo.getFieldNames()); + + } + + @Test + public void testNestedSchemaToTypeInfo1() { + + String schema = + "struct<" + + "middle:struct<" + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">," + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">," + + "map:map<" + + "string," + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">"; + TypeInformation typeInfo = OrcUtils.schemaToTypeInfo(TypeDescription.fromString(schema)); + + Assert.assertNotNull(typeInfo); + Assert.assertTrue(typeInfo instanceof RowTypeInfo); + RowTypeInfo rowTypeInfo = (RowTypeInfo) typeInfo; + + // validate field types + Assert.assertArrayEquals( + new TypeInformation[]{ + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + ), + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ), + new MapTypeInfo<>( + Types.STRING, + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + }, + rowTypeInfo.getFieldTypes()); + + // validate field names + Assert.assertArrayEquals( + new String[] {"middle", "list", "map"}, + rowTypeInfo.getFieldNames()); + } +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java deleted file mode 100644 index 60008a0838612..0000000000000 --- a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java +++ /dev/null @@ -1,472 +0,0 @@ -/* - * 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.flink.orc; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Tests for the {@link RowOrcInputFormat}. - */ - -public class RowOrcInputFormatTest { - - private RowOrcInputFormat rowOrcInputFormat; - - @After - public void tearDown() throws IOException { - if (rowOrcInputFormat != null) { - rowOrcInputFormat.close(); - rowOrcInputFormat.closeInputFormat(); - } - rowOrcInputFormat = null; - } - - private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); - - private static final String TEST1_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; - - private static final String[] TEST1_DATA = new String[] { - "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", - "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + - "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; - - private static final String[] TEST1_PROJECTED_DATA = new String[] { - "{},[3,good, 4,bad],[1,bye, 2,sigh],hi,[0, 1, 2, 3, 4],-15.0,1.0,9223372036854775807,65536,1024,1,false", - "{chani=5,chani, mauddib=1,mauddib},[100000000,cat, -100000,in, 1234,hat],[1,bye, 2,sigh],bye," + - "[],-5.0,2.0,9223372036854775807,65536,2048,100,true" }; - - private static final String TEST1_INVALID_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; - - @Test(expected = FileNotFoundException.class) - public void testInvalidPath() throws IOException{ - - rowOrcInputFormat = new RowOrcInputFormat("TestOrcFile.test2.orc", TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - rowOrcInputFormat.open(inputSplits[0]); - - } - - @Test(expected = RuntimeException.class) - public void testInvalidSchema() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_INVALID_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - rowOrcInputFormat.open(inputSplits[0]); - - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testInvalidProjection() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - int[] projectionMask = {14}; - rowOrcInputFormat.setFieldMapping(projectionMask); - } - - @Test - public void testMajorDataTypes() throws IOException{ - - // test for boolean,byte,short,int,long,float,double,bytes,string,struct,list,map - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.toString(), TEST1_DATA[count++]); - } - } - } - - @Test - public void testProjection() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - int[] projectionMask = {11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}; - rowOrcInputFormat.setFieldMapping(projectionMask); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.toString(), TEST1_PROJECTED_DATA[count++]); - } - } - - } - - @Test - public void testTimeStampAndDate() throws IOException{ - - URL expectedDataURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.dat"); - assert(expectedDataURL != null); - List expectedTimeStampAndDate = Files.readAllLines(Paths.get(expectedDataURL.getPath())); - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct"; - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - List actualTimeStampAndDate = new ArrayList<>(); - - Row row = null; - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - count++; - if (count <= 10000) { - actualTimeStampAndDate.add(row.getField(0) + "," + row.getField(1)); - } - - } - } - Assert.assertEquals(count, 70000); - Assert.assertEquals(expectedTimeStampAndDate.size(), actualTimeStampAndDate.size()); - Assert.assertEquals(expectedTimeStampAndDate.toString(), actualTimeStampAndDate.toString()); - - } - - @Test - public void testDecimal() throws IOException{ - - URL expectedDataURL = getClass().getClassLoader().getResource("decimal.dat"); - List expectedDecimal = Files.readAllLines(Paths.get(expectedDataURL.getPath())); - - URL testInputURL = getClass().getClassLoader().getResource("decimal.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct<_col0:decimal(10,5)>"; - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - List actualDecimal = new ArrayList<>(); - - Row row = null; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - actualDecimal.add(row.getField(0)); - } - } - - Assert.assertEquals(expectedDecimal.size(), actualDecimal.size()); - Assert.assertEquals(expectedDecimal.toString(), actualDecimal.toString()); - - } - - @Test - public void testEmptyFile() throws IOException{ - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.emptyFile.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - - rowOrcInputFormat = new RowOrcInputFormat(path, TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = new Row(1); - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - count++; - } - } - - Assert.assertEquals(count, 0); - } - - @Test - public void testLargeFile() throws IOException{ - - URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + - "_col5:string,_col6:int,_col7:int,_col8:int>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = new Row(1); - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - count++; - } - } - - Assert.assertEquals(count, 1920800); - } - - @Test - public void testProducedType() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - rowOrcInputFormat.open(inputSplits[0]); - - TypeInformation type = rowOrcInputFormat.getProducedType(); - Assert.assertEquals(type.toString(), "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer," + - " long1: Long, float1: Float, double1: Double, bytes1: byte[], string1: String," + - " middle: Row(list: ObjectArrayTypeInfo)," + - " list: ObjectArrayTypeInfo," + - " map: Map)"); - - } - - @Test - public void testProducedTypeWithProjection() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - int[] projectionMask = {9, 10, 11}; - rowOrcInputFormat.setFieldMapping(projectionMask); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - rowOrcInputFormat.open(inputSplits[0]); - - TypeInformation type = rowOrcInputFormat.getProducedType(); - Assert.assertEquals(type.toString(), "Row(middle: Row(list: ObjectArrayTypeInfo)," + - " list: ObjectArrayTypeInfo," + - " map: Map)"); - - } - - @Test - public void testLongList() throws Exception { - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listlong.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - long count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.getArity(), 1); - Object object = row.getField(0); - long[] l = (long[]) object; - - Assert.assertEquals(l.length, 2); - if (count < 50) { - Assert.assertArrayEquals(l, new long[]{count, count + 1}); - } - else { - Assert.assertArrayEquals(l, new long[]{0L, 0L}); - } - count = count + 2; - } - } - Assert.assertEquals(count, 100); - } - - @Test - public void testStringList() throws Exception { - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.liststring.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - long count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.getArity(), 1); - Object object = row.getField(0); - String[] l = (String[]) object; - - Assert.assertEquals(l.length, 2); - Assert.assertArrayEquals(l, new String[]{"hello" + count, "hello" + (count + 1) }); - count = count + 2; - } - } - Assert.assertEquals(count, 200); - } - - @Test - public void testListOfListOfStructOfLong() throws Exception { - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listliststructlong.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct>>>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - long count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.getArity(), 1); - - Object[] objects = (Object[]) row.getField(0); - Assert.assertEquals(objects.length, 1); - - Object[] objects1 = (Object[]) objects[0]; - Assert.assertEquals(objects1.length, 1); - - Row[] nestedRows = Arrays.copyOf(objects1, objects1.length, Row[].class); - Assert.assertEquals(nestedRows.length, 1); - - Assert.assertEquals(nestedRows[0].getArity(), 1); - - Assert.assertEquals(nestedRows[0].getField(0), count); - - count++; - } - } - Assert.assertEquals(count, 100); - } - - @Test - public void testSplit() throws IOException{ - - URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + - "_col5:string,_col6:int,_col7:int,_col8:int>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(10); - - Assert.assertEquals(inputSplits.length, 10); - - Row row = null; - int countTotalRecords = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - int countSplitRecords = 0; - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - countSplitRecords++; - } - Assert.assertNotEquals(countSplitRecords, 1920800); - countTotalRecords += countSplitRecords; - } - - Assert.assertEquals(countTotalRecords, 1920800); - } - -} diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc deleted file mode 100644 index ecdadcbff134615d7eefcb740d55fe710cee059b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 523 zcmaix$xg#C5Qe>EnJmx{S{5I3LY$gwPl(Ki0VfCx86O*nAn33yj{0RALJfJ%g= z-yY9=v&depmn0*ZJW?2kTu94yT=M+Mv7+18fd!+++rKoYVb8*(6m+$6w2JQ%VP zoBWuE$qYVmaJZY`s{`<^gga2tKpn+nltV!*rn5eyFL-h~g3j5(B6iCb91v JV7f4AZ$rdMGkni%gxIAM^?8)@NzR1G03_ zQ~I=$l8)`w(zKm!g}HebqmEB z$(}8Vks5ovjmb^kX-N*YbktC-?e0-&YOo*RHb1o4jT#G6Uq?2aIh-Ryw8c%=ROlA(7bVdZ!>jqI83wO`#jQ!BZ1BcvtYw$6ASzM8ou-0iYb#i<<5?Y8 zcR86{`PbuAyu-G0%+|g?R_Q}}d}YnIwm1mjJj`x;n`S?P$jk3QlgcznjWaUDr?(61 z3Bbqtmr$)JCquHRII7bMwFr!QAr#AIzr0d}oOz2A#o{B}df>n)gR+|a0pW23?LA#& zcAvMyU=0E<{s9R90KhQ<01*HHz%c*h0OOM8V&O0fV1Sam z94SKV91H?U5|e}j7yy(20OMd5_{=QD&A}|NfLTIF!A`?SiI0Ovh>=l>J-0F^v$(`i ziIsy(h>=N&iG#(!$clx-D1ZS<@^YjIv2!p8BuPvX4q$M2!43cjO%MSXfFL3Q01PM` NFb0D30|QS|Llz^%2`T^p diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc deleted file mode 100644 index 75a5f2a10a3fe703394fd3700e56d06149f2785b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1298 zcmV+t1?~DzQbQ2{015^Y1PEkM04M+e|ol`QeugC00RIb1OOdU!HN?>5PjWgV`Yg@6M_T} zQj^_<=xJ8=;;j=RGjzO2ys!ex3fVHFMPf}N0~_|J$b(oPe=BEqIe z@P+Q{3BlH(q^KS_Q66s*qbnS;<%}RL3%{1=BQ6o&kDZt%K`b7imB4z^u%)5M`$3|1 z!$5h0e8-8HVU#(PcXGQlE7Kax-Uj?& z!#wOIPOfEmByAH>txpV$-a8&5P(_YG26PHYo!MV84}7~>7RQl&sAf< zk-72I{bY7VtVnN>nrTZf<~#-2&t3rAgjOZ}g{>kyvM5fFwQb86006*a0stBV z06o(|uADFw1<@^07-PJkNPiG1`=3RjicH>&qi(4C^XvETKgZ9%%X`oFj`v>go$tNh zyWab}cfXe&_Gx?^pGN=%B+x(vl~d*HH>jK{r^=~vs+=mP%BeC`hRW!#s0@{%GE|1j zP#G$h%B6CtT)h#MOXX6zR4$cEWvWb-sWMe&FIHu$OqHoJRc@7AZk4<5qjIa< zD!0l~St?6qsVtSHvii&_OJ%7%Dv!#e@~Av2kIJL+REx@^vQ@UqR@o|BWvgtJt+G|N z3Wp4b4u=qj5{DFr7Ka#z8iyQ5-v5mAzL39?zm&h0znH(8zntrtf~KN5`+YS{QB&2F zHFZs4Q`wX@wM}s|`oHB(eN*66I3-SvQ{+@RWlo)Q_5LcIQm57_cB-9nr`{=eDxQ*O z_C1Q8s;BI!dkUY*r}U|Pil6G|?)%n14S*Iv6QB*y2xtW~1KI%%fmJU}fwn+npf%7O zXb&_9S_DmkHo;SWt%7DjyP#pvGH4pK4H^fngXY2Be*qN$00#!ZSqA{XV+a6%001?K zK?;CC36VJHcrhQ)_j>Tb#)6j#Gf5Ahjca;@_Q=nTv)p)8Qgd?h4TRXiw56rQ67c{A0IUE2 z<6su}%^}6Z!7MP5RbmOVg0n`75+4VT5F?`$dv0Y;W^sw35-SIn5F?Wk69>CNiWLh- zN&o|tl;$|Ws3gM0mXVs1lW!oz4yG+FC6FgUzm2LK3J5CIr~AR+<)3@9fs27>eh I15Z*z7I1J@L;wH) diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat deleted file mode 100644 index 59b933ddf0b5c..0000000000000 --- a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat +++ /dev/null @@ -1,10000 +0,0 @@ -1900-05-05 12:34:56.1,1900-12-25 -1900-05-05 12:34:56.1001,1900-12-25 -1900-05-05 12:34:56.1002,1900-12-25 -1900-05-05 12:34:56.1003,1900-12-25 -1900-05-05 12:34:56.1004,1900-12-25 -1900-05-05 12:34:56.1005,1900-12-25 -1900-05-05 12:34:56.1006,1900-12-25 -1900-05-05 12:34:56.1007,1900-12-25 -1900-05-05 12:34:56.1008,1900-12-25 -1900-05-05 12:34:56.1009,1900-12-25 -1900-05-05 12:34:56.101,1900-12-25 -1900-05-05 12:34:56.1011,1900-12-25 -1900-05-05 12:34:56.1012,1900-12-25 -1900-05-05 12:34:56.1013,1900-12-25 -1900-05-05 12:34:56.1014,1900-12-25 -1900-05-05 12:34:56.1015,1900-12-25 -1900-05-05 12:34:56.1016,1900-12-25 -1900-05-05 12:34:56.1017,1900-12-25 -1900-05-05 12:34:56.1018,1900-12-25 -1900-05-05 12:34:56.1019,1900-12-25 -1900-05-05 12:34:56.102,1900-12-25 -1900-05-05 12:34:56.1021,1900-12-25 -1900-05-05 12:34:56.1022,1900-12-25 -1900-05-05 12:34:56.1023,1900-12-25 -1900-05-05 12:34:56.1024,1900-12-25 -1900-05-05 12:34:56.1025,1900-12-25 -1900-05-05 12:34:56.1026,1900-12-25 -1900-05-05 12:34:56.1027,1900-12-25 -1900-05-05 12:34:56.1028,1900-12-25 -1900-05-05 12:34:56.1029,1900-12-25 -1900-05-05 12:34:56.103,1900-12-25 -1900-05-05 12:34:56.1031,1900-12-25 -1900-05-05 12:34:56.1032,1900-12-25 -1900-05-05 12:34:56.1033,1900-12-25 -1900-05-05 12:34:56.1034,1900-12-25 -1900-05-05 12:34:56.1035,1900-12-25 -1900-05-05 12:34:56.1036,1900-12-25 -1900-05-05 12:34:56.1037,1900-12-25 -1900-05-05 12:34:56.1038,1900-12-25 -1900-05-05 12:34:56.1039,1900-12-25 -1900-05-05 12:34:56.104,1900-12-25 -1900-05-05 12:34:56.1041,1900-12-25 -1900-05-05 12:34:56.1042,1900-12-25 -1900-05-05 12:34:56.1043,1900-12-25 -1900-05-05 12:34:56.1044,1900-12-25 -1900-05-05 12:34:56.1045,1900-12-25 -1900-05-05 12:34:56.1046,1900-12-25 -1900-05-05 12:34:56.1047,1900-12-25 -1900-05-05 12:34:56.1048,1900-12-25 -1900-05-05 12:34:56.1049,1900-12-25 -1900-05-05 12:34:56.105,1900-12-25 -1900-05-05 12:34:56.1051,1900-12-25 -1900-05-05 12:34:56.1052,1900-12-25 -1900-05-05 12:34:56.1053,1900-12-25 -1900-05-05 12:34:56.1054,1900-12-25 -1900-05-05 12:34:56.1055,1900-12-25 -1900-05-05 12:34:56.1056,1900-12-25 -1900-05-05 12:34:56.1057,1900-12-25 -1900-05-05 12:34:56.1058,1900-12-25 -1900-05-05 12:34:56.1059,1900-12-25 -1900-05-05 12:34:56.106,1900-12-25 -1900-05-05 12:34:56.1061,1900-12-25 -1900-05-05 12:34:56.1062,1900-12-25 -1900-05-05 12:34:56.1063,1900-12-25 -1900-05-05 12:34:56.1064,1900-12-25 -1900-05-05 12:34:56.1065,1900-12-25 -1900-05-05 12:34:56.1066,1900-12-25 -1900-05-05 12:34:56.1067,1900-12-25 -1900-05-05 12:34:56.1068,1900-12-25 -1900-05-05 12:34:56.1069,1900-12-25 -1900-05-05 12:34:56.107,1900-12-25 -1900-05-05 12:34:56.1071,1900-12-25 -1900-05-05 12:34:56.1072,1900-12-25 -1900-05-05 12:34:56.1073,1900-12-25 -1900-05-05 12:34:56.1074,1900-12-25 -1900-05-05 12:34:56.1075,1900-12-25 -1900-05-05 12:34:56.1076,1900-12-25 -1900-05-05 12:34:56.1077,1900-12-25 -1900-05-05 12:34:56.1078,1900-12-25 -1900-05-05 12:34:56.1079,1900-12-25 -1900-05-05 12:34:56.108,1900-12-25 -1900-05-05 12:34:56.1081,1900-12-25 -1900-05-05 12:34:56.1082,1900-12-25 -1900-05-05 12:34:56.1083,1900-12-25 -1900-05-05 12:34:56.1084,1900-12-25 -1900-05-05 12:34:56.1085,1900-12-25 -1900-05-05 12:34:56.1086,1900-12-25 -1900-05-05 12:34:56.1087,1900-12-25 -1900-05-05 12:34:56.1088,1900-12-25 -1900-05-05 12:34:56.1089,1900-12-25 -1900-05-05 12:34:56.109,1900-12-25 -1900-05-05 12:34:56.1091,1900-12-25 -1900-05-05 12:34:56.1092,1900-12-25 -1900-05-05 12:34:56.1093,1900-12-25 -1900-05-05 12:34:56.1094,1900-12-25 -1900-05-05 12:34:56.1095,1900-12-25 -1900-05-05 12:34:56.1096,1900-12-25 -1900-05-05 12:34:56.1097,1900-12-25 -1900-05-05 12:34:56.1098,1900-12-25 -1900-05-05 12:34:56.1099,1900-12-25 -1900-05-05 12:34:56.11,1900-12-25 -1900-05-05 12:34:56.1101,1900-12-25 -1900-05-05 12:34:56.1102,1900-12-25 -1900-05-05 12:34:56.1103,1900-12-25 -1900-05-05 12:34:56.1104,1900-12-25 -1900-05-05 12:34:56.1105,1900-12-25 -1900-05-05 12:34:56.1106,1900-12-25 -1900-05-05 12:34:56.1107,1900-12-25 -1900-05-05 12:34:56.1108,1900-12-25 -1900-05-05 12:34:56.1109,1900-12-25 -1900-05-05 12:34:56.111,1900-12-25 -1900-05-05 12:34:56.1111,1900-12-25 -1900-05-05 12:34:56.1112,1900-12-25 -1900-05-05 12:34:56.1113,1900-12-25 -1900-05-05 12:34:56.1114,1900-12-25 -1900-05-05 12:34:56.1115,1900-12-25 -1900-05-05 12:34:56.1116,1900-12-25 -1900-05-05 12:34:56.1117,1900-12-25 -1900-05-05 12:34:56.1118,1900-12-25 -1900-05-05 12:34:56.1119,1900-12-25 -1900-05-05 12:34:56.112,1900-12-25 -1900-05-05 12:34:56.1121,1900-12-25 -1900-05-05 12:34:56.1122,1900-12-25 -1900-05-05 12:34:56.1123,1900-12-25 -1900-05-05 12:34:56.1124,1900-12-25 -1900-05-05 12:34:56.1125,1900-12-25 -1900-05-05 12:34:56.1126,1900-12-25 -1900-05-05 12:34:56.1127,1900-12-25 -1900-05-05 12:34:56.1128,1900-12-25 -1900-05-05 12:34:56.1129,1900-12-25 -1900-05-05 12:34:56.113,1900-12-25 -1900-05-05 12:34:56.1131,1900-12-25 -1900-05-05 12:34:56.1132,1900-12-25 -1900-05-05 12:34:56.1133,1900-12-25 -1900-05-05 12:34:56.1134,1900-12-25 -1900-05-05 12:34:56.1135,1900-12-25 -1900-05-05 12:34:56.1136,1900-12-25 -1900-05-05 12:34:56.1137,1900-12-25 -1900-05-05 12:34:56.1138,1900-12-25 -1900-05-05 12:34:56.1139,1900-12-25 -1900-05-05 12:34:56.114,1900-12-25 -1900-05-05 12:34:56.1141,1900-12-25 -1900-05-05 12:34:56.1142,1900-12-25 -1900-05-05 12:34:56.1143,1900-12-25 -1900-05-05 12:34:56.1144,1900-12-25 -1900-05-05 12:34:56.1145,1900-12-25 -1900-05-05 12:34:56.1146,1900-12-25 -1900-05-05 12:34:56.1147,1900-12-25 -1900-05-05 12:34:56.1148,1900-12-25 -1900-05-05 12:34:56.1149,1900-12-25 -1900-05-05 12:34:56.115,1900-12-25 -1900-05-05 12:34:56.1151,1900-12-25 -1900-05-05 12:34:56.1152,1900-12-25 -1900-05-05 12:34:56.1153,1900-12-25 -1900-05-05 12:34:56.1154,1900-12-25 -1900-05-05 12:34:56.1155,1900-12-25 -1900-05-05 12:34:56.1156,1900-12-25 -1900-05-05 12:34:56.1157,1900-12-25 -1900-05-05 12:34:56.1158,1900-12-25 -1900-05-05 12:34:56.1159,1900-12-25 -1900-05-05 12:34:56.116,1900-12-25 -1900-05-05 12:34:56.1161,1900-12-25 -1900-05-05 12:34:56.1162,1900-12-25 -1900-05-05 12:34:56.1163,1900-12-25 -1900-05-05 12:34:56.1164,1900-12-25 -1900-05-05 12:34:56.1165,1900-12-25 -1900-05-05 12:34:56.1166,1900-12-25 -1900-05-05 12:34:56.1167,1900-12-25 -1900-05-05 12:34:56.1168,1900-12-25 -1900-05-05 12:34:56.1169,1900-12-25 -1900-05-05 12:34:56.117,1900-12-25 -1900-05-05 12:34:56.1171,1900-12-25 -1900-05-05 12:34:56.1172,1900-12-25 -1900-05-05 12:34:56.1173,1900-12-25 -1900-05-05 12:34:56.1174,1900-12-25 -1900-05-05 12:34:56.1175,1900-12-25 -1900-05-05 12:34:56.1176,1900-12-25 -1900-05-05 12:34:56.1177,1900-12-25 -1900-05-05 12:34:56.1178,1900-12-25 -1900-05-05 12:34:56.1179,1900-12-25 -1900-05-05 12:34:56.118,1900-12-25 -1900-05-05 12:34:56.1181,1900-12-25 -1900-05-05 12:34:56.1182,1900-12-25 -1900-05-05 12:34:56.1183,1900-12-25 -1900-05-05 12:34:56.1184,1900-12-25 -1900-05-05 12:34:56.1185,1900-12-25 -1900-05-05 12:34:56.1186,1900-12-25 -1900-05-05 12:34:56.1187,1900-12-25 -1900-05-05 12:34:56.1188,1900-12-25 -1900-05-05 12:34:56.1189,1900-12-25 -1900-05-05 12:34:56.119,1900-12-25 -1900-05-05 12:34:56.1191,1900-12-25 -1900-05-05 12:34:56.1192,1900-12-25 -1900-05-05 12:34:56.1193,1900-12-25 -1900-05-05 12:34:56.1194,1900-12-25 -1900-05-05 12:34:56.1195,1900-12-25 -1900-05-05 12:34:56.1196,1900-12-25 -1900-05-05 12:34:56.1197,1900-12-25 -1900-05-05 12:34:56.1198,1900-12-25 -1900-05-05 12:34:56.1199,1900-12-25 -1900-05-05 12:34:56.12,1900-12-25 -1900-05-05 12:34:56.1201,1900-12-25 -1900-05-05 12:34:56.1202,1900-12-25 -1900-05-05 12:34:56.1203,1900-12-25 -1900-05-05 12:34:56.1204,1900-12-25 -1900-05-05 12:34:56.1205,1900-12-25 -1900-05-05 12:34:56.1206,1900-12-25 -1900-05-05 12:34:56.1207,1900-12-25 -1900-05-05 12:34:56.1208,1900-12-25 -1900-05-05 12:34:56.1209,1900-12-25 -1900-05-05 12:34:56.121,1900-12-25 -1900-05-05 12:34:56.1211,1900-12-25 -1900-05-05 12:34:56.1212,1900-12-25 -1900-05-05 12:34:56.1213,1900-12-25 -1900-05-05 12:34:56.1214,1900-12-25 -1900-05-05 12:34:56.1215,1900-12-25 -1900-05-05 12:34:56.1216,1900-12-25 -1900-05-05 12:34:56.1217,1900-12-25 -1900-05-05 12:34:56.1218,1900-12-25 -1900-05-05 12:34:56.1219,1900-12-25 -1900-05-05 12:34:56.122,1900-12-25 -1900-05-05 12:34:56.1221,1900-12-25 -1900-05-05 12:34:56.1222,1900-12-25 -1900-05-05 12:34:56.1223,1900-12-25 -1900-05-05 12:34:56.1224,1900-12-25 -1900-05-05 12:34:56.1225,1900-12-25 -1900-05-05 12:34:56.1226,1900-12-25 -1900-05-05 12:34:56.1227,1900-12-25 -1900-05-05 12:34:56.1228,1900-12-25 -1900-05-05 12:34:56.1229,1900-12-25 -1900-05-05 12:34:56.123,1900-12-25 -1900-05-05 12:34:56.1231,1900-12-25 -1900-05-05 12:34:56.1232,1900-12-25 -1900-05-05 12:34:56.1233,1900-12-25 -1900-05-05 12:34:56.1234,1900-12-25 -1900-05-05 12:34:56.1235,1900-12-25 -1900-05-05 12:34:56.1236,1900-12-25 -1900-05-05 12:34:56.1237,1900-12-25 -1900-05-05 12:34:56.1238,1900-12-25 -1900-05-05 12:34:56.1239,1900-12-25 -1900-05-05 12:34:56.124,1900-12-25 -1900-05-05 12:34:56.1241,1900-12-25 -1900-05-05 12:34:56.1242,1900-12-25 -1900-05-05 12:34:56.1243,1900-12-25 -1900-05-05 12:34:56.1244,1900-12-25 -1900-05-05 12:34:56.1245,1900-12-25 -1900-05-05 12:34:56.1246,1900-12-25 -1900-05-05 12:34:56.1247,1900-12-25 -1900-05-05 12:34:56.1248,1900-12-25 -1900-05-05 12:34:56.1249,1900-12-25 -1900-05-05 12:34:56.125,1900-12-25 -1900-05-05 12:34:56.1251,1900-12-25 -1900-05-05 12:34:56.1252,1900-12-25 -1900-05-05 12:34:56.1253,1900-12-25 -1900-05-05 12:34:56.1254,1900-12-25 -1900-05-05 12:34:56.1255,1900-12-25 -1900-05-05 12:34:56.1256,1900-12-25 -1900-05-05 12:34:56.1257,1900-12-25 -1900-05-05 12:34:56.1258,1900-12-25 -1900-05-05 12:34:56.1259,1900-12-25 -1900-05-05 12:34:56.126,1900-12-25 -1900-05-05 12:34:56.1261,1900-12-25 -1900-05-05 12:34:56.1262,1900-12-25 -1900-05-05 12:34:56.1263,1900-12-25 -1900-05-05 12:34:56.1264,1900-12-25 -1900-05-05 12:34:56.1265,1900-12-25 -1900-05-05 12:34:56.1266,1900-12-25 -1900-05-05 12:34:56.1267,1900-12-25 -1900-05-05 12:34:56.1268,1900-12-25 -1900-05-05 12:34:56.1269,1900-12-25 -1900-05-05 12:34:56.127,1900-12-25 -1900-05-05 12:34:56.1271,1900-12-25 -1900-05-05 12:34:56.1272,1900-12-25 -1900-05-05 12:34:56.1273,1900-12-25 -1900-05-05 12:34:56.1274,1900-12-25 -1900-05-05 12:34:56.1275,1900-12-25 -1900-05-05 12:34:56.1276,1900-12-25 -1900-05-05 12:34:56.1277,1900-12-25 -1900-05-05 12:34:56.1278,1900-12-25 -1900-05-05 12:34:56.1279,1900-12-25 -1900-05-05 12:34:56.128,1900-12-25 -1900-05-05 12:34:56.1281,1900-12-25 -1900-05-05 12:34:56.1282,1900-12-25 -1900-05-05 12:34:56.1283,1900-12-25 -1900-05-05 12:34:56.1284,1900-12-25 -1900-05-05 12:34:56.1285,1900-12-25 -1900-05-05 12:34:56.1286,1900-12-25 -1900-05-05 12:34:56.1287,1900-12-25 -1900-05-05 12:34:56.1288,1900-12-25 -1900-05-05 12:34:56.1289,1900-12-25 -1900-05-05 12:34:56.129,1900-12-25 -1900-05-05 12:34:56.1291,1900-12-25 -1900-05-05 12:34:56.1292,1900-12-25 -1900-05-05 12:34:56.1293,1900-12-25 -1900-05-05 12:34:56.1294,1900-12-25 -1900-05-05 12:34:56.1295,1900-12-25 -1900-05-05 12:34:56.1296,1900-12-25 -1900-05-05 12:34:56.1297,1900-12-25 -1900-05-05 12:34:56.1298,1900-12-25 -1900-05-05 12:34:56.1299,1900-12-25 -1900-05-05 12:34:56.13,1900-12-25 -1900-05-05 12:34:56.1301,1900-12-25 -1900-05-05 12:34:56.1302,1900-12-25 -1900-05-05 12:34:56.1303,1900-12-25 -1900-05-05 12:34:56.1304,1900-12-25 -1900-05-05 12:34:56.1305,1900-12-25 -1900-05-05 12:34:56.1306,1900-12-25 -1900-05-05 12:34:56.1307,1900-12-25 -1900-05-05 12:34:56.1308,1900-12-25 -1900-05-05 12:34:56.1309,1900-12-25 -1900-05-05 12:34:56.131,1900-12-25 -1900-05-05 12:34:56.1311,1900-12-25 -1900-05-05 12:34:56.1312,1900-12-25 -1900-05-05 12:34:56.1313,1900-12-25 -1900-05-05 12:34:56.1314,1900-12-25 -1900-05-05 12:34:56.1315,1900-12-25 -1900-05-05 12:34:56.1316,1900-12-25 -1900-05-05 12:34:56.1317,1900-12-25 -1900-05-05 12:34:56.1318,1900-12-25 -1900-05-05 12:34:56.1319,1900-12-25 -1900-05-05 12:34:56.132,1900-12-25 -1900-05-05 12:34:56.1321,1900-12-25 -1900-05-05 12:34:56.1322,1900-12-25 -1900-05-05 12:34:56.1323,1900-12-25 -1900-05-05 12:34:56.1324,1900-12-25 -1900-05-05 12:34:56.1325,1900-12-25 -1900-05-05 12:34:56.1326,1900-12-25 -1900-05-05 12:34:56.1327,1900-12-25 -1900-05-05 12:34:56.1328,1900-12-25 -1900-05-05 12:34:56.1329,1900-12-25 -1900-05-05 12:34:56.133,1900-12-25 -1900-05-05 12:34:56.1331,1900-12-25 -1900-05-05 12:34:56.1332,1900-12-25 -1900-05-05 12:34:56.1333,1900-12-25 -1900-05-05 12:34:56.1334,1900-12-25 -1900-05-05 12:34:56.1335,1900-12-25 -1900-05-05 12:34:56.1336,1900-12-25 -1900-05-05 12:34:56.1337,1900-12-25 -1900-05-05 12:34:56.1338,1900-12-25 -1900-05-05 12:34:56.1339,1900-12-25 -1900-05-05 12:34:56.134,1900-12-25 -1900-05-05 12:34:56.1341,1900-12-25 -1900-05-05 12:34:56.1342,1900-12-25 -1900-05-05 12:34:56.1343,1900-12-25 -1900-05-05 12:34:56.1344,1900-12-25 -1900-05-05 12:34:56.1345,1900-12-25 -1900-05-05 12:34:56.1346,1900-12-25 -1900-05-05 12:34:56.1347,1900-12-25 -1900-05-05 12:34:56.1348,1900-12-25 -1900-05-05 12:34:56.1349,1900-12-25 -1900-05-05 12:34:56.135,1900-12-25 -1900-05-05 12:34:56.1351,1900-12-25 -1900-05-05 12:34:56.1352,1900-12-25 -1900-05-05 12:34:56.1353,1900-12-25 -1900-05-05 12:34:56.1354,1900-12-25 -1900-05-05 12:34:56.1355,1900-12-25 -1900-05-05 12:34:56.1356,1900-12-25 -1900-05-05 12:34:56.1357,1900-12-25 -1900-05-05 12:34:56.1358,1900-12-25 -1900-05-05 12:34:56.1359,1900-12-25 -1900-05-05 12:34:56.136,1900-12-25 -1900-05-05 12:34:56.1361,1900-12-25 -1900-05-05 12:34:56.1362,1900-12-25 -1900-05-05 12:34:56.1363,1900-12-25 -1900-05-05 12:34:56.1364,1900-12-25 -1900-05-05 12:34:56.1365,1900-12-25 -1900-05-05 12:34:56.1366,1900-12-25 -1900-05-05 12:34:56.1367,1900-12-25 -1900-05-05 12:34:56.1368,1900-12-25 -1900-05-05 12:34:56.1369,1900-12-25 -1900-05-05 12:34:56.137,1900-12-25 -1900-05-05 12:34:56.1371,1900-12-25 -1900-05-05 12:34:56.1372,1900-12-25 -1900-05-05 12:34:56.1373,1900-12-25 -1900-05-05 12:34:56.1374,1900-12-25 -1900-05-05 12:34:56.1375,1900-12-25 -1900-05-05 12:34:56.1376,1900-12-25 -1900-05-05 12:34:56.1377,1900-12-25 -1900-05-05 12:34:56.1378,1900-12-25 -1900-05-05 12:34:56.1379,1900-12-25 -1900-05-05 12:34:56.138,1900-12-25 -1900-05-05 12:34:56.1381,1900-12-25 -1900-05-05 12:34:56.1382,1900-12-25 -1900-05-05 12:34:56.1383,1900-12-25 -1900-05-05 12:34:56.1384,1900-12-25 -1900-05-05 12:34:56.1385,1900-12-25 -1900-05-05 12:34:56.1386,1900-12-25 -1900-05-05 12:34:56.1387,1900-12-25 -1900-05-05 12:34:56.1388,1900-12-25 -1900-05-05 12:34:56.1389,1900-12-25 -1900-05-05 12:34:56.139,1900-12-25 -1900-05-05 12:34:56.1391,1900-12-25 -1900-05-05 12:34:56.1392,1900-12-25 -1900-05-05 12:34:56.1393,1900-12-25 -1900-05-05 12:34:56.1394,1900-12-25 -1900-05-05 12:34:56.1395,1900-12-25 -1900-05-05 12:34:56.1396,1900-12-25 -1900-05-05 12:34:56.1397,1900-12-25 -1900-05-05 12:34:56.1398,1900-12-25 -1900-05-05 12:34:56.1399,1900-12-25 -1900-05-05 12:34:56.14,1900-12-25 -1900-05-05 12:34:56.1401,1900-12-25 -1900-05-05 12:34:56.1402,1900-12-25 -1900-05-05 12:34:56.1403,1900-12-25 -1900-05-05 12:34:56.1404,1900-12-25 -1900-05-05 12:34:56.1405,1900-12-25 -1900-05-05 12:34:56.1406,1900-12-25 -1900-05-05 12:34:56.1407,1900-12-25 -1900-05-05 12:34:56.1408,1900-12-25 -1900-05-05 12:34:56.1409,1900-12-25 -1900-05-05 12:34:56.141,1900-12-25 -1900-05-05 12:34:56.1411,1900-12-25 -1900-05-05 12:34:56.1412,1900-12-25 -1900-05-05 12:34:56.1413,1900-12-25 -1900-05-05 12:34:56.1414,1900-12-25 -1900-05-05 12:34:56.1415,1900-12-25 -1900-05-05 12:34:56.1416,1900-12-25 -1900-05-05 12:34:56.1417,1900-12-25 -1900-05-05 12:34:56.1418,1900-12-25 -1900-05-05 12:34:56.1419,1900-12-25 -1900-05-05 12:34:56.142,1900-12-25 -1900-05-05 12:34:56.1421,1900-12-25 -1900-05-05 12:34:56.1422,1900-12-25 -1900-05-05 12:34:56.1423,1900-12-25 -1900-05-05 12:34:56.1424,1900-12-25 -1900-05-05 12:34:56.1425,1900-12-25 -1900-05-05 12:34:56.1426,1900-12-25 -1900-05-05 12:34:56.1427,1900-12-25 -1900-05-05 12:34:56.1428,1900-12-25 -1900-05-05 12:34:56.1429,1900-12-25 -1900-05-05 12:34:56.143,1900-12-25 -1900-05-05 12:34:56.1431,1900-12-25 -1900-05-05 12:34:56.1432,1900-12-25 -1900-05-05 12:34:56.1433,1900-12-25 -1900-05-05 12:34:56.1434,1900-12-25 -1900-05-05 12:34:56.1435,1900-12-25 -1900-05-05 12:34:56.1436,1900-12-25 -1900-05-05 12:34:56.1437,1900-12-25 -1900-05-05 12:34:56.1438,1900-12-25 -1900-05-05 12:34:56.1439,1900-12-25 -1900-05-05 12:34:56.144,1900-12-25 -1900-05-05 12:34:56.1441,1900-12-25 -1900-05-05 12:34:56.1442,1900-12-25 -1900-05-05 12:34:56.1443,1900-12-25 -1900-05-05 12:34:56.1444,1900-12-25 -1900-05-05 12:34:56.1445,1900-12-25 -1900-05-05 12:34:56.1446,1900-12-25 -1900-05-05 12:34:56.1447,1900-12-25 -1900-05-05 12:34:56.1448,1900-12-25 -1900-05-05 12:34:56.1449,1900-12-25 -1900-05-05 12:34:56.145,1900-12-25 -1900-05-05 12:34:56.1451,1900-12-25 -1900-05-05 12:34:56.1452,1900-12-25 -1900-05-05 12:34:56.1453,1900-12-25 -1900-05-05 12:34:56.1454,1900-12-25 -1900-05-05 12:34:56.1455,1900-12-25 -1900-05-05 12:34:56.1456,1900-12-25 -1900-05-05 12:34:56.1457,1900-12-25 -1900-05-05 12:34:56.1458,1900-12-25 -1900-05-05 12:34:56.1459,1900-12-25 -1900-05-05 12:34:56.146,1900-12-25 -1900-05-05 12:34:56.1461,1900-12-25 -1900-05-05 12:34:56.1462,1900-12-25 -1900-05-05 12:34:56.1463,1900-12-25 -1900-05-05 12:34:56.1464,1900-12-25 -1900-05-05 12:34:56.1465,1900-12-25 -1900-05-05 12:34:56.1466,1900-12-25 -1900-05-05 12:34:56.1467,1900-12-25 -1900-05-05 12:34:56.1468,1900-12-25 -1900-05-05 12:34:56.1469,1900-12-25 -1900-05-05 12:34:56.147,1900-12-25 -1900-05-05 12:34:56.1471,1900-12-25 -1900-05-05 12:34:56.1472,1900-12-25 -1900-05-05 12:34:56.1473,1900-12-25 -1900-05-05 12:34:56.1474,1900-12-25 -1900-05-05 12:34:56.1475,1900-12-25 -1900-05-05 12:34:56.1476,1900-12-25 -1900-05-05 12:34:56.1477,1900-12-25 -1900-05-05 12:34:56.1478,1900-12-25 -1900-05-05 12:34:56.1479,1900-12-25 -1900-05-05 12:34:56.148,1900-12-25 -1900-05-05 12:34:56.1481,1900-12-25 -1900-05-05 12:34:56.1482,1900-12-25 -1900-05-05 12:34:56.1483,1900-12-25 -1900-05-05 12:34:56.1484,1900-12-25 -1900-05-05 12:34:56.1485,1900-12-25 -1900-05-05 12:34:56.1486,1900-12-25 -1900-05-05 12:34:56.1487,1900-12-25 -1900-05-05 12:34:56.1488,1900-12-25 -1900-05-05 12:34:56.1489,1900-12-25 -1900-05-05 12:34:56.149,1900-12-25 -1900-05-05 12:34:56.1491,1900-12-25 -1900-05-05 12:34:56.1492,1900-12-25 -1900-05-05 12:34:56.1493,1900-12-25 -1900-05-05 12:34:56.1494,1900-12-25 -1900-05-05 12:34:56.1495,1900-12-25 -1900-05-05 12:34:56.1496,1900-12-25 -1900-05-05 12:34:56.1497,1900-12-25 -1900-05-05 12:34:56.1498,1900-12-25 -1900-05-05 12:34:56.1499,1900-12-25 -1900-05-05 12:34:56.15,1900-12-25 -1900-05-05 12:34:56.1501,1900-12-25 -1900-05-05 12:34:56.1502,1900-12-25 -1900-05-05 12:34:56.1503,1900-12-25 -1900-05-05 12:34:56.1504,1900-12-25 -1900-05-05 12:34:56.1505,1900-12-25 -1900-05-05 12:34:56.1506,1900-12-25 -1900-05-05 12:34:56.1507,1900-12-25 -1900-05-05 12:34:56.1508,1900-12-25 -1900-05-05 12:34:56.1509,1900-12-25 -1900-05-05 12:34:56.151,1900-12-25 -1900-05-05 12:34:56.1511,1900-12-25 -1900-05-05 12:34:56.1512,1900-12-25 -1900-05-05 12:34:56.1513,1900-12-25 -1900-05-05 12:34:56.1514,1900-12-25 -1900-05-05 12:34:56.1515,1900-12-25 -1900-05-05 12:34:56.1516,1900-12-25 -1900-05-05 12:34:56.1517,1900-12-25 -1900-05-05 12:34:56.1518,1900-12-25 -1900-05-05 12:34:56.1519,1900-12-25 -1900-05-05 12:34:56.152,1900-12-25 -1900-05-05 12:34:56.1521,1900-12-25 -1900-05-05 12:34:56.1522,1900-12-25 -1900-05-05 12:34:56.1523,1900-12-25 -1900-05-05 12:34:56.1524,1900-12-25 -1900-05-05 12:34:56.1525,1900-12-25 -1900-05-05 12:34:56.1526,1900-12-25 -1900-05-05 12:34:56.1527,1900-12-25 -1900-05-05 12:34:56.1528,1900-12-25 -1900-05-05 12:34:56.1529,1900-12-25 -1900-05-05 12:34:56.153,1900-12-25 -1900-05-05 12:34:56.1531,1900-12-25 -1900-05-05 12:34:56.1532,1900-12-25 -1900-05-05 12:34:56.1533,1900-12-25 -1900-05-05 12:34:56.1534,1900-12-25 -1900-05-05 12:34:56.1535,1900-12-25 -1900-05-05 12:34:56.1536,1900-12-25 -1900-05-05 12:34:56.1537,1900-12-25 -1900-05-05 12:34:56.1538,1900-12-25 -1900-05-05 12:34:56.1539,1900-12-25 -1900-05-05 12:34:56.154,1900-12-25 -1900-05-05 12:34:56.1541,1900-12-25 -1900-05-05 12:34:56.1542,1900-12-25 -1900-05-05 12:34:56.1543,1900-12-25 -1900-05-05 12:34:56.1544,1900-12-25 -1900-05-05 12:34:56.1545,1900-12-25 -1900-05-05 12:34:56.1546,1900-12-25 -1900-05-05 12:34:56.1547,1900-12-25 -1900-05-05 12:34:56.1548,1900-12-25 -1900-05-05 12:34:56.1549,1900-12-25 -1900-05-05 12:34:56.155,1900-12-25 -1900-05-05 12:34:56.1551,1900-12-25 -1900-05-05 12:34:56.1552,1900-12-25 -1900-05-05 12:34:56.1553,1900-12-25 -1900-05-05 12:34:56.1554,1900-12-25 -1900-05-05 12:34:56.1555,1900-12-25 -1900-05-05 12:34:56.1556,1900-12-25 -1900-05-05 12:34:56.1557,1900-12-25 -1900-05-05 12:34:56.1558,1900-12-25 -1900-05-05 12:34:56.1559,1900-12-25 -1900-05-05 12:34:56.156,1900-12-25 -1900-05-05 12:34:56.1561,1900-12-25 -1900-05-05 12:34:56.1562,1900-12-25 -1900-05-05 12:34:56.1563,1900-12-25 -1900-05-05 12:34:56.1564,1900-12-25 -1900-05-05 12:34:56.1565,1900-12-25 -1900-05-05 12:34:56.1566,1900-12-25 -1900-05-05 12:34:56.1567,1900-12-25 -1900-05-05 12:34:56.1568,1900-12-25 -1900-05-05 12:34:56.1569,1900-12-25 -1900-05-05 12:34:56.157,1900-12-25 -1900-05-05 12:34:56.1571,1900-12-25 -1900-05-05 12:34:56.1572,1900-12-25 -1900-05-05 12:34:56.1573,1900-12-25 -1900-05-05 12:34:56.1574,1900-12-25 -1900-05-05 12:34:56.1575,1900-12-25 -1900-05-05 12:34:56.1576,1900-12-25 -1900-05-05 12:34:56.1577,1900-12-25 -1900-05-05 12:34:56.1578,1900-12-25 -1900-05-05 12:34:56.1579,1900-12-25 -1900-05-05 12:34:56.158,1900-12-25 -1900-05-05 12:34:56.1581,1900-12-25 -1900-05-05 12:34:56.1582,1900-12-25 -1900-05-05 12:34:56.1583,1900-12-25 -1900-05-05 12:34:56.1584,1900-12-25 -1900-05-05 12:34:56.1585,1900-12-25 -1900-05-05 12:34:56.1586,1900-12-25 -1900-05-05 12:34:56.1587,1900-12-25 -1900-05-05 12:34:56.1588,1900-12-25 -1900-05-05 12:34:56.1589,1900-12-25 -1900-05-05 12:34:56.159,1900-12-25 -1900-05-05 12:34:56.1591,1900-12-25 -1900-05-05 12:34:56.1592,1900-12-25 -1900-05-05 12:34:56.1593,1900-12-25 -1900-05-05 12:34:56.1594,1900-12-25 -1900-05-05 12:34:56.1595,1900-12-25 -1900-05-05 12:34:56.1596,1900-12-25 -1900-05-05 12:34:56.1597,1900-12-25 -1900-05-05 12:34:56.1598,1900-12-25 -1900-05-05 12:34:56.1599,1900-12-25 -1900-05-05 12:34:56.16,1900-12-25 -1900-05-05 12:34:56.1601,1900-12-25 -1900-05-05 12:34:56.1602,1900-12-25 -1900-05-05 12:34:56.1603,1900-12-25 -1900-05-05 12:34:56.1604,1900-12-25 -1900-05-05 12:34:56.1605,1900-12-25 -1900-05-05 12:34:56.1606,1900-12-25 -1900-05-05 12:34:56.1607,1900-12-25 -1900-05-05 12:34:56.1608,1900-12-25 -1900-05-05 12:34:56.1609,1900-12-25 -1900-05-05 12:34:56.161,1900-12-25 -1900-05-05 12:34:56.1611,1900-12-25 -1900-05-05 12:34:56.1612,1900-12-25 -1900-05-05 12:34:56.1613,1900-12-25 -1900-05-05 12:34:56.1614,1900-12-25 -1900-05-05 12:34:56.1615,1900-12-25 -1900-05-05 12:34:56.1616,1900-12-25 -1900-05-05 12:34:56.1617,1900-12-25 -1900-05-05 12:34:56.1618,1900-12-25 -1900-05-05 12:34:56.1619,1900-12-25 -1900-05-05 12:34:56.162,1900-12-25 -1900-05-05 12:34:56.1621,1900-12-25 -1900-05-05 12:34:56.1622,1900-12-25 -1900-05-05 12:34:56.1623,1900-12-25 -1900-05-05 12:34:56.1624,1900-12-25 -1900-05-05 12:34:56.1625,1900-12-25 -1900-05-05 12:34:56.1626,1900-12-25 -1900-05-05 12:34:56.1627,1900-12-25 -1900-05-05 12:34:56.1628,1900-12-25 -1900-05-05 12:34:56.1629,1900-12-25 -1900-05-05 12:34:56.163,1900-12-25 -1900-05-05 12:34:56.1631,1900-12-25 -1900-05-05 12:34:56.1632,1900-12-25 -1900-05-05 12:34:56.1633,1900-12-25 -1900-05-05 12:34:56.1634,1900-12-25 -1900-05-05 12:34:56.1635,1900-12-25 -1900-05-05 12:34:56.1636,1900-12-25 -1900-05-05 12:34:56.1637,1900-12-25 -1900-05-05 12:34:56.1638,1900-12-25 -1900-05-05 12:34:56.1639,1900-12-25 -1900-05-05 12:34:56.164,1900-12-25 -1900-05-05 12:34:56.1641,1900-12-25 -1900-05-05 12:34:56.1642,1900-12-25 -1900-05-05 12:34:56.1643,1900-12-25 -1900-05-05 12:34:56.1644,1900-12-25 -1900-05-05 12:34:56.1645,1900-12-25 -1900-05-05 12:34:56.1646,1900-12-25 -1900-05-05 12:34:56.1647,1900-12-25 -1900-05-05 12:34:56.1648,1900-12-25 -1900-05-05 12:34:56.1649,1900-12-25 -1900-05-05 12:34:56.165,1900-12-25 -1900-05-05 12:34:56.1651,1900-12-25 -1900-05-05 12:34:56.1652,1900-12-25 -1900-05-05 12:34:56.1653,1900-12-25 -1900-05-05 12:34:56.1654,1900-12-25 -1900-05-05 12:34:56.1655,1900-12-25 -1900-05-05 12:34:56.1656,1900-12-25 -1900-05-05 12:34:56.1657,1900-12-25 -1900-05-05 12:34:56.1658,1900-12-25 -1900-05-05 12:34:56.1659,1900-12-25 -1900-05-05 12:34:56.166,1900-12-25 -1900-05-05 12:34:56.1661,1900-12-25 -1900-05-05 12:34:56.1662,1900-12-25 -1900-05-05 12:34:56.1663,1900-12-25 -1900-05-05 12:34:56.1664,1900-12-25 -1900-05-05 12:34:56.1665,1900-12-25 -1900-05-05 12:34:56.1666,1900-12-25 -1900-05-05 12:34:56.1667,1900-12-25 -1900-05-05 12:34:56.1668,1900-12-25 -1900-05-05 12:34:56.1669,1900-12-25 -1900-05-05 12:34:56.167,1900-12-25 -1900-05-05 12:34:56.1671,1900-12-25 -1900-05-05 12:34:56.1672,1900-12-25 -1900-05-05 12:34:56.1673,1900-12-25 -1900-05-05 12:34:56.1674,1900-12-25 -1900-05-05 12:34:56.1675,1900-12-25 -1900-05-05 12:34:56.1676,1900-12-25 -1900-05-05 12:34:56.1677,1900-12-25 -1900-05-05 12:34:56.1678,1900-12-25 -1900-05-05 12:34:56.1679,1900-12-25 -1900-05-05 12:34:56.168,1900-12-25 -1900-05-05 12:34:56.1681,1900-12-25 -1900-05-05 12:34:56.1682,1900-12-25 -1900-05-05 12:34:56.1683,1900-12-25 -1900-05-05 12:34:56.1684,1900-12-25 -1900-05-05 12:34:56.1685,1900-12-25 -1900-05-05 12:34:56.1686,1900-12-25 -1900-05-05 12:34:56.1687,1900-12-25 -1900-05-05 12:34:56.1688,1900-12-25 -1900-05-05 12:34:56.1689,1900-12-25 -1900-05-05 12:34:56.169,1900-12-25 -1900-05-05 12:34:56.1691,1900-12-25 -1900-05-05 12:34:56.1692,1900-12-25 -1900-05-05 12:34:56.1693,1900-12-25 -1900-05-05 12:34:56.1694,1900-12-25 -1900-05-05 12:34:56.1695,1900-12-25 -1900-05-05 12:34:56.1696,1900-12-25 -1900-05-05 12:34:56.1697,1900-12-25 -1900-05-05 12:34:56.1698,1900-12-25 -1900-05-05 12:34:56.1699,1900-12-25 -1900-05-05 12:34:56.17,1900-12-25 -1900-05-05 12:34:56.1701,1900-12-25 -1900-05-05 12:34:56.1702,1900-12-25 -1900-05-05 12:34:56.1703,1900-12-25 -1900-05-05 12:34:56.1704,1900-12-25 -1900-05-05 12:34:56.1705,1900-12-25 -1900-05-05 12:34:56.1706,1900-12-25 -1900-05-05 12:34:56.1707,1900-12-25 -1900-05-05 12:34:56.1708,1900-12-25 -1900-05-05 12:34:56.1709,1900-12-25 -1900-05-05 12:34:56.171,1900-12-25 -1900-05-05 12:34:56.1711,1900-12-25 -1900-05-05 12:34:56.1712,1900-12-25 -1900-05-05 12:34:56.1713,1900-12-25 -1900-05-05 12:34:56.1714,1900-12-25 -1900-05-05 12:34:56.1715,1900-12-25 -1900-05-05 12:34:56.1716,1900-12-25 -1900-05-05 12:34:56.1717,1900-12-25 -1900-05-05 12:34:56.1718,1900-12-25 -1900-05-05 12:34:56.1719,1900-12-25 -1900-05-05 12:34:56.172,1900-12-25 -1900-05-05 12:34:56.1721,1900-12-25 -1900-05-05 12:34:56.1722,1900-12-25 -1900-05-05 12:34:56.1723,1900-12-25 -1900-05-05 12:34:56.1724,1900-12-25 -1900-05-05 12:34:56.1725,1900-12-25 -1900-05-05 12:34:56.1726,1900-12-25 -1900-05-05 12:34:56.1727,1900-12-25 -1900-05-05 12:34:56.1728,1900-12-25 -1900-05-05 12:34:56.1729,1900-12-25 -1900-05-05 12:34:56.173,1900-12-25 -1900-05-05 12:34:56.1731,1900-12-25 -1900-05-05 12:34:56.1732,1900-12-25 -1900-05-05 12:34:56.1733,1900-12-25 -1900-05-05 12:34:56.1734,1900-12-25 -1900-05-05 12:34:56.1735,1900-12-25 -1900-05-05 12:34:56.1736,1900-12-25 -1900-05-05 12:34:56.1737,1900-12-25 -1900-05-05 12:34:56.1738,1900-12-25 -1900-05-05 12:34:56.1739,1900-12-25 -1900-05-05 12:34:56.174,1900-12-25 -1900-05-05 12:34:56.1741,1900-12-25 -1900-05-05 12:34:56.1742,1900-12-25 -1900-05-05 12:34:56.1743,1900-12-25 -1900-05-05 12:34:56.1744,1900-12-25 -1900-05-05 12:34:56.1745,1900-12-25 -1900-05-05 12:34:56.1746,1900-12-25 -1900-05-05 12:34:56.1747,1900-12-25 -1900-05-05 12:34:56.1748,1900-12-25 -1900-05-05 12:34:56.1749,1900-12-25 -1900-05-05 12:34:56.175,1900-12-25 -1900-05-05 12:34:56.1751,1900-12-25 -1900-05-05 12:34:56.1752,1900-12-25 -1900-05-05 12:34:56.1753,1900-12-25 -1900-05-05 12:34:56.1754,1900-12-25 -1900-05-05 12:34:56.1755,1900-12-25 -1900-05-05 12:34:56.1756,1900-12-25 -1900-05-05 12:34:56.1757,1900-12-25 -1900-05-05 12:34:56.1758,1900-12-25 -1900-05-05 12:34:56.1759,1900-12-25 -1900-05-05 12:34:56.176,1900-12-25 -1900-05-05 12:34:56.1761,1900-12-25 -1900-05-05 12:34:56.1762,1900-12-25 -1900-05-05 12:34:56.1763,1900-12-25 -1900-05-05 12:34:56.1764,1900-12-25 -1900-05-05 12:34:56.1765,1900-12-25 -1900-05-05 12:34:56.1766,1900-12-25 -1900-05-05 12:34:56.1767,1900-12-25 -1900-05-05 12:34:56.1768,1900-12-25 -1900-05-05 12:34:56.1769,1900-12-25 -1900-05-05 12:34:56.177,1900-12-25 -1900-05-05 12:34:56.1771,1900-12-25 -1900-05-05 12:34:56.1772,1900-12-25 -1900-05-05 12:34:56.1773,1900-12-25 -1900-05-05 12:34:56.1774,1900-12-25 -1900-05-05 12:34:56.1775,1900-12-25 -1900-05-05 12:34:56.1776,1900-12-25 -1900-05-05 12:34:56.1777,1900-12-25 -1900-05-05 12:34:56.1778,1900-12-25 -1900-05-05 12:34:56.1779,1900-12-25 -1900-05-05 12:34:56.178,1900-12-25 -1900-05-05 12:34:56.1781,1900-12-25 -1900-05-05 12:34:56.1782,1900-12-25 -1900-05-05 12:34:56.1783,1900-12-25 -1900-05-05 12:34:56.1784,1900-12-25 -1900-05-05 12:34:56.1785,1900-12-25 -1900-05-05 12:34:56.1786,1900-12-25 -1900-05-05 12:34:56.1787,1900-12-25 -1900-05-05 12:34:56.1788,1900-12-25 -1900-05-05 12:34:56.1789,1900-12-25 -1900-05-05 12:34:56.179,1900-12-25 -1900-05-05 12:34:56.1791,1900-12-25 -1900-05-05 12:34:56.1792,1900-12-25 -1900-05-05 12:34:56.1793,1900-12-25 -1900-05-05 12:34:56.1794,1900-12-25 -1900-05-05 12:34:56.1795,1900-12-25 -1900-05-05 12:34:56.1796,1900-12-25 -1900-05-05 12:34:56.1797,1900-12-25 -1900-05-05 12:34:56.1798,1900-12-25 -1900-05-05 12:34:56.1799,1900-12-25 -1900-05-05 12:34:56.18,1900-12-25 -1900-05-05 12:34:56.1801,1900-12-25 -1900-05-05 12:34:56.1802,1900-12-25 -1900-05-05 12:34:56.1803,1900-12-25 -1900-05-05 12:34:56.1804,1900-12-25 -1900-05-05 12:34:56.1805,1900-12-25 -1900-05-05 12:34:56.1806,1900-12-25 -1900-05-05 12:34:56.1807,1900-12-25 -1900-05-05 12:34:56.1808,1900-12-25 -1900-05-05 12:34:56.1809,1900-12-25 -1900-05-05 12:34:56.181,1900-12-25 -1900-05-05 12:34:56.1811,1900-12-25 -1900-05-05 12:34:56.1812,1900-12-25 -1900-05-05 12:34:56.1813,1900-12-25 -1900-05-05 12:34:56.1814,1900-12-25 -1900-05-05 12:34:56.1815,1900-12-25 -1900-05-05 12:34:56.1816,1900-12-25 -1900-05-05 12:34:56.1817,1900-12-25 -1900-05-05 12:34:56.1818,1900-12-25 -1900-05-05 12:34:56.1819,1900-12-25 -1900-05-05 12:34:56.182,1900-12-25 -1900-05-05 12:34:56.1821,1900-12-25 -1900-05-05 12:34:56.1822,1900-12-25 -1900-05-05 12:34:56.1823,1900-12-25 -1900-05-05 12:34:56.1824,1900-12-25 -1900-05-05 12:34:56.1825,1900-12-25 -1900-05-05 12:34:56.1826,1900-12-25 -1900-05-05 12:34:56.1827,1900-12-25 -1900-05-05 12:34:56.1828,1900-12-25 -1900-05-05 12:34:56.1829,1900-12-25 -1900-05-05 12:34:56.183,1900-12-25 -1900-05-05 12:34:56.1831,1900-12-25 -1900-05-05 12:34:56.1832,1900-12-25 -1900-05-05 12:34:56.1833,1900-12-25 -1900-05-05 12:34:56.1834,1900-12-25 -1900-05-05 12:34:56.1835,1900-12-25 -1900-05-05 12:34:56.1836,1900-12-25 -1900-05-05 12:34:56.1837,1900-12-25 -1900-05-05 12:34:56.1838,1900-12-25 -1900-05-05 12:34:56.1839,1900-12-25 -1900-05-05 12:34:56.184,1900-12-25 -1900-05-05 12:34:56.1841,1900-12-25 -1900-05-05 12:34:56.1842,1900-12-25 -1900-05-05 12:34:56.1843,1900-12-25 -1900-05-05 12:34:56.1844,1900-12-25 -1900-05-05 12:34:56.1845,1900-12-25 -1900-05-05 12:34:56.1846,1900-12-25 -1900-05-05 12:34:56.1847,1900-12-25 -1900-05-05 12:34:56.1848,1900-12-25 -1900-05-05 12:34:56.1849,1900-12-25 -1900-05-05 12:34:56.185,1900-12-25 -1900-05-05 12:34:56.1851,1900-12-25 -1900-05-05 12:34:56.1852,1900-12-25 -1900-05-05 12:34:56.1853,1900-12-25 -1900-05-05 12:34:56.1854,1900-12-25 -1900-05-05 12:34:56.1855,1900-12-25 -1900-05-05 12:34:56.1856,1900-12-25 -1900-05-05 12:34:56.1857,1900-12-25 -1900-05-05 12:34:56.1858,1900-12-25 -1900-05-05 12:34:56.1859,1900-12-25 -1900-05-05 12:34:56.186,1900-12-25 -1900-05-05 12:34:56.1861,1900-12-25 -1900-05-05 12:34:56.1862,1900-12-25 -1900-05-05 12:34:56.1863,1900-12-25 -1900-05-05 12:34:56.1864,1900-12-25 -1900-05-05 12:34:56.1865,1900-12-25 -1900-05-05 12:34:56.1866,1900-12-25 -1900-05-05 12:34:56.1867,1900-12-25 -1900-05-05 12:34:56.1868,1900-12-25 -1900-05-05 12:34:56.1869,1900-12-25 -1900-05-05 12:34:56.187,1900-12-25 -1900-05-05 12:34:56.1871,1900-12-25 -1900-05-05 12:34:56.1872,1900-12-25 -1900-05-05 12:34:56.1873,1900-12-25 -1900-05-05 12:34:56.1874,1900-12-25 -1900-05-05 12:34:56.1875,1900-12-25 -1900-05-05 12:34:56.1876,1900-12-25 -1900-05-05 12:34:56.1877,1900-12-25 -1900-05-05 12:34:56.1878,1900-12-25 -1900-05-05 12:34:56.1879,1900-12-25 -1900-05-05 12:34:56.188,1900-12-25 -1900-05-05 12:34:56.1881,1900-12-25 -1900-05-05 12:34:56.1882,1900-12-25 -1900-05-05 12:34:56.1883,1900-12-25 -1900-05-05 12:34:56.1884,1900-12-25 -1900-05-05 12:34:56.1885,1900-12-25 -1900-05-05 12:34:56.1886,1900-12-25 -1900-05-05 12:34:56.1887,1900-12-25 -1900-05-05 12:34:56.1888,1900-12-25 -1900-05-05 12:34:56.1889,1900-12-25 -1900-05-05 12:34:56.189,1900-12-25 -1900-05-05 12:34:56.1891,1900-12-25 -1900-05-05 12:34:56.1892,1900-12-25 -1900-05-05 12:34:56.1893,1900-12-25 -1900-05-05 12:34:56.1894,1900-12-25 -1900-05-05 12:34:56.1895,1900-12-25 -1900-05-05 12:34:56.1896,1900-12-25 -1900-05-05 12:34:56.1897,1900-12-25 -1900-05-05 12:34:56.1898,1900-12-25 -1900-05-05 12:34:56.1899,1900-12-25 -1900-05-05 12:34:56.19,1900-12-25 -1900-05-05 12:34:56.1901,1900-12-25 -1900-05-05 12:34:56.1902,1900-12-25 -1900-05-05 12:34:56.1903,1900-12-25 -1900-05-05 12:34:56.1904,1900-12-25 -1900-05-05 12:34:56.1905,1900-12-25 -1900-05-05 12:34:56.1906,1900-12-25 -1900-05-05 12:34:56.1907,1900-12-25 -1900-05-05 12:34:56.1908,1900-12-25 -1900-05-05 12:34:56.1909,1900-12-25 -1900-05-05 12:34:56.191,1900-12-25 -1900-05-05 12:34:56.1911,1900-12-25 -1900-05-05 12:34:56.1912,1900-12-25 -1900-05-05 12:34:56.1913,1900-12-25 -1900-05-05 12:34:56.1914,1900-12-25 -1900-05-05 12:34:56.1915,1900-12-25 -1900-05-05 12:34:56.1916,1900-12-25 -1900-05-05 12:34:56.1917,1900-12-25 -1900-05-05 12:34:56.1918,1900-12-25 -1900-05-05 12:34:56.1919,1900-12-25 -1900-05-05 12:34:56.192,1900-12-25 -1900-05-05 12:34:56.1921,1900-12-25 -1900-05-05 12:34:56.1922,1900-12-25 -1900-05-05 12:34:56.1923,1900-12-25 -1900-05-05 12:34:56.1924,1900-12-25 -1900-05-05 12:34:56.1925,1900-12-25 -1900-05-05 12:34:56.1926,1900-12-25 -1900-05-05 12:34:56.1927,1900-12-25 -1900-05-05 12:34:56.1928,1900-12-25 -1900-05-05 12:34:56.1929,1900-12-25 -1900-05-05 12:34:56.193,1900-12-25 -1900-05-05 12:34:56.1931,1900-12-25 -1900-05-05 12:34:56.1932,1900-12-25 -1900-05-05 12:34:56.1933,1900-12-25 -1900-05-05 12:34:56.1934,1900-12-25 -1900-05-05 12:34:56.1935,1900-12-25 -1900-05-05 12:34:56.1936,1900-12-25 -1900-05-05 12:34:56.1937,1900-12-25 -1900-05-05 12:34:56.1938,1900-12-25 -1900-05-05 12:34:56.1939,1900-12-25 -1900-05-05 12:34:56.194,1900-12-25 -1900-05-05 12:34:56.1941,1900-12-25 -1900-05-05 12:34:56.1942,1900-12-25 -1900-05-05 12:34:56.1943,1900-12-25 -1900-05-05 12:34:56.1944,1900-12-25 -1900-05-05 12:34:56.1945,1900-12-25 -1900-05-05 12:34:56.1946,1900-12-25 -1900-05-05 12:34:56.1947,1900-12-25 -1900-05-05 12:34:56.1948,1900-12-25 -1900-05-05 12:34:56.1949,1900-12-25 -1900-05-05 12:34:56.195,1900-12-25 -1900-05-05 12:34:56.1951,1900-12-25 -1900-05-05 12:34:56.1952,1900-12-25 -1900-05-05 12:34:56.1953,1900-12-25 -1900-05-05 12:34:56.1954,1900-12-25 -1900-05-05 12:34:56.1955,1900-12-25 -1900-05-05 12:34:56.1956,1900-12-25 -1900-05-05 12:34:56.1957,1900-12-25 -1900-05-05 12:34:56.1958,1900-12-25 -1900-05-05 12:34:56.1959,1900-12-25 -1900-05-05 12:34:56.196,1900-12-25 -1900-05-05 12:34:56.1961,1900-12-25 -1900-05-05 12:34:56.1962,1900-12-25 -1900-05-05 12:34:56.1963,1900-12-25 -1900-05-05 12:34:56.1964,1900-12-25 -1900-05-05 12:34:56.1965,1900-12-25 -1900-05-05 12:34:56.1966,1900-12-25 -1900-05-05 12:34:56.1967,1900-12-25 -1900-05-05 12:34:56.1968,1900-12-25 -1900-05-05 12:34:56.1969,1900-12-25 -1900-05-05 12:34:56.197,1900-12-25 -1900-05-05 12:34:56.1971,1900-12-25 -1900-05-05 12:34:56.1972,1900-12-25 -1900-05-05 12:34:56.1973,1900-12-25 -1900-05-05 12:34:56.1974,1900-12-25 -1900-05-05 12:34:56.1975,1900-12-25 -1900-05-05 12:34:56.1976,1900-12-25 -1900-05-05 12:34:56.1977,1900-12-25 -1900-05-05 12:34:56.1978,1900-12-25 -1900-05-05 12:34:56.1979,1900-12-25 -1900-05-05 12:34:56.198,1900-12-25 -1900-05-05 12:34:56.1981,1900-12-25 -1900-05-05 12:34:56.1982,1900-12-25 -1900-05-05 12:34:56.1983,1900-12-25 -1900-05-05 12:34:56.1984,1900-12-25 -1900-05-05 12:34:56.1985,1900-12-25 -1900-05-05 12:34:56.1986,1900-12-25 -1900-05-05 12:34:56.1987,1900-12-25 -1900-05-05 12:34:56.1988,1900-12-25 -1900-05-05 12:34:56.1989,1900-12-25 -1900-05-05 12:34:56.199,1900-12-25 -1900-05-05 12:34:56.1991,1900-12-25 -1900-05-05 12:34:56.1992,1900-12-25 -1900-05-05 12:34:56.1993,1900-12-25 -1900-05-05 12:34:56.1994,1900-12-25 -1900-05-05 12:34:56.1995,1900-12-25 -1900-05-05 12:34:56.1996,1900-12-25 -1900-05-05 12:34:56.1997,1900-12-25 -1900-05-05 12:34:56.1998,1900-12-25 -1900-05-05 12:34:56.1999,1900-12-25 -1901-05-05 12:34:56.1,1901-12-25 -1901-05-05 12:34:56.1001,1901-12-25 -1901-05-05 12:34:56.1002,1901-12-25 -1901-05-05 12:34:56.1003,1901-12-25 -1901-05-05 12:34:56.1004,1901-12-25 -1901-05-05 12:34:56.1005,1901-12-25 -1901-05-05 12:34:56.1006,1901-12-25 -1901-05-05 12:34:56.1007,1901-12-25 -1901-05-05 12:34:56.1008,1901-12-25 -1901-05-05 12:34:56.1009,1901-12-25 -1901-05-05 12:34:56.101,1901-12-25 -1901-05-05 12:34:56.1011,1901-12-25 -1901-05-05 12:34:56.1012,1901-12-25 -1901-05-05 12:34:56.1013,1901-12-25 -1901-05-05 12:34:56.1014,1901-12-25 -1901-05-05 12:34:56.1015,1901-12-25 -1901-05-05 12:34:56.1016,1901-12-25 -1901-05-05 12:34:56.1017,1901-12-25 -1901-05-05 12:34:56.1018,1901-12-25 -1901-05-05 12:34:56.1019,1901-12-25 -1901-05-05 12:34:56.102,1901-12-25 -1901-05-05 12:34:56.1021,1901-12-25 -1901-05-05 12:34:56.1022,1901-12-25 -1901-05-05 12:34:56.1023,1901-12-25 -1901-05-05 12:34:56.1024,1901-12-25 -1901-05-05 12:34:56.1025,1901-12-25 -1901-05-05 12:34:56.1026,1901-12-25 -1901-05-05 12:34:56.1027,1901-12-25 -1901-05-05 12:34:56.1028,1901-12-25 -1901-05-05 12:34:56.1029,1901-12-25 -1901-05-05 12:34:56.103,1901-12-25 -1901-05-05 12:34:56.1031,1901-12-25 -1901-05-05 12:34:56.1032,1901-12-25 -1901-05-05 12:34:56.1033,1901-12-25 -1901-05-05 12:34:56.1034,1901-12-25 -1901-05-05 12:34:56.1035,1901-12-25 -1901-05-05 12:34:56.1036,1901-12-25 -1901-05-05 12:34:56.1037,1901-12-25 -1901-05-05 12:34:56.1038,1901-12-25 -1901-05-05 12:34:56.1039,1901-12-25 -1901-05-05 12:34:56.104,1901-12-25 -1901-05-05 12:34:56.1041,1901-12-25 -1901-05-05 12:34:56.1042,1901-12-25 -1901-05-05 12:34:56.1043,1901-12-25 -1901-05-05 12:34:56.1044,1901-12-25 -1901-05-05 12:34:56.1045,1901-12-25 -1901-05-05 12:34:56.1046,1901-12-25 -1901-05-05 12:34:56.1047,1901-12-25 -1901-05-05 12:34:56.1048,1901-12-25 -1901-05-05 12:34:56.1049,1901-12-25 -1901-05-05 12:34:56.105,1901-12-25 -1901-05-05 12:34:56.1051,1901-12-25 -1901-05-05 12:34:56.1052,1901-12-25 -1901-05-05 12:34:56.1053,1901-12-25 -1901-05-05 12:34:56.1054,1901-12-25 -1901-05-05 12:34:56.1055,1901-12-25 -1901-05-05 12:34:56.1056,1901-12-25 -1901-05-05 12:34:56.1057,1901-12-25 -1901-05-05 12:34:56.1058,1901-12-25 -1901-05-05 12:34:56.1059,1901-12-25 -1901-05-05 12:34:56.106,1901-12-25 -1901-05-05 12:34:56.1061,1901-12-25 -1901-05-05 12:34:56.1062,1901-12-25 -1901-05-05 12:34:56.1063,1901-12-25 -1901-05-05 12:34:56.1064,1901-12-25 -1901-05-05 12:34:56.1065,1901-12-25 -1901-05-05 12:34:56.1066,1901-12-25 -1901-05-05 12:34:56.1067,1901-12-25 -1901-05-05 12:34:56.1068,1901-12-25 -1901-05-05 12:34:56.1069,1901-12-25 -1901-05-05 12:34:56.107,1901-12-25 -1901-05-05 12:34:56.1071,1901-12-25 -1901-05-05 12:34:56.1072,1901-12-25 -1901-05-05 12:34:56.1073,1901-12-25 -1901-05-05 12:34:56.1074,1901-12-25 -1901-05-05 12:34:56.1075,1901-12-25 -1901-05-05 12:34:56.1076,1901-12-25 -1901-05-05 12:34:56.1077,1901-12-25 -1901-05-05 12:34:56.1078,1901-12-25 -1901-05-05 12:34:56.1079,1901-12-25 -1901-05-05 12:34:56.108,1901-12-25 -1901-05-05 12:34:56.1081,1901-12-25 -1901-05-05 12:34:56.1082,1901-12-25 -1901-05-05 12:34:56.1083,1901-12-25 -1901-05-05 12:34:56.1084,1901-12-25 -1901-05-05 12:34:56.1085,1901-12-25 -1901-05-05 12:34:56.1086,1901-12-25 -1901-05-05 12:34:56.1087,1901-12-25 -1901-05-05 12:34:56.1088,1901-12-25 -1901-05-05 12:34:56.1089,1901-12-25 -1901-05-05 12:34:56.109,1901-12-25 -1901-05-05 12:34:56.1091,1901-12-25 -1901-05-05 12:34:56.1092,1901-12-25 -1901-05-05 12:34:56.1093,1901-12-25 -1901-05-05 12:34:56.1094,1901-12-25 -1901-05-05 12:34:56.1095,1901-12-25 -1901-05-05 12:34:56.1096,1901-12-25 -1901-05-05 12:34:56.1097,1901-12-25 -1901-05-05 12:34:56.1098,1901-12-25 -1901-05-05 12:34:56.1099,1901-12-25 -1901-05-05 12:34:56.11,1901-12-25 -1901-05-05 12:34:56.1101,1901-12-25 -1901-05-05 12:34:56.1102,1901-12-25 -1901-05-05 12:34:56.1103,1901-12-25 -1901-05-05 12:34:56.1104,1901-12-25 -1901-05-05 12:34:56.1105,1901-12-25 -1901-05-05 12:34:56.1106,1901-12-25 -1901-05-05 12:34:56.1107,1901-12-25 -1901-05-05 12:34:56.1108,1901-12-25 -1901-05-05 12:34:56.1109,1901-12-25 -1901-05-05 12:34:56.111,1901-12-25 -1901-05-05 12:34:56.1111,1901-12-25 -1901-05-05 12:34:56.1112,1901-12-25 -1901-05-05 12:34:56.1113,1901-12-25 -1901-05-05 12:34:56.1114,1901-12-25 -1901-05-05 12:34:56.1115,1901-12-25 -1901-05-05 12:34:56.1116,1901-12-25 -1901-05-05 12:34:56.1117,1901-12-25 -1901-05-05 12:34:56.1118,1901-12-25 -1901-05-05 12:34:56.1119,1901-12-25 -1901-05-05 12:34:56.112,1901-12-25 -1901-05-05 12:34:56.1121,1901-12-25 -1901-05-05 12:34:56.1122,1901-12-25 -1901-05-05 12:34:56.1123,1901-12-25 -1901-05-05 12:34:56.1124,1901-12-25 -1901-05-05 12:34:56.1125,1901-12-25 -1901-05-05 12:34:56.1126,1901-12-25 -1901-05-05 12:34:56.1127,1901-12-25 -1901-05-05 12:34:56.1128,1901-12-25 -1901-05-05 12:34:56.1129,1901-12-25 -1901-05-05 12:34:56.113,1901-12-25 -1901-05-05 12:34:56.1131,1901-12-25 -1901-05-05 12:34:56.1132,1901-12-25 -1901-05-05 12:34:56.1133,1901-12-25 -1901-05-05 12:34:56.1134,1901-12-25 -1901-05-05 12:34:56.1135,1901-12-25 -1901-05-05 12:34:56.1136,1901-12-25 -1901-05-05 12:34:56.1137,1901-12-25 -1901-05-05 12:34:56.1138,1901-12-25 -1901-05-05 12:34:56.1139,1901-12-25 -1901-05-05 12:34:56.114,1901-12-25 -1901-05-05 12:34:56.1141,1901-12-25 -1901-05-05 12:34:56.1142,1901-12-25 -1901-05-05 12:34:56.1143,1901-12-25 -1901-05-05 12:34:56.1144,1901-12-25 -1901-05-05 12:34:56.1145,1901-12-25 -1901-05-05 12:34:56.1146,1901-12-25 -1901-05-05 12:34:56.1147,1901-12-25 -1901-05-05 12:34:56.1148,1901-12-25 -1901-05-05 12:34:56.1149,1901-12-25 -1901-05-05 12:34:56.115,1901-12-25 -1901-05-05 12:34:56.1151,1901-12-25 -1901-05-05 12:34:56.1152,1901-12-25 -1901-05-05 12:34:56.1153,1901-12-25 -1901-05-05 12:34:56.1154,1901-12-25 -1901-05-05 12:34:56.1155,1901-12-25 -1901-05-05 12:34:56.1156,1901-12-25 -1901-05-05 12:34:56.1157,1901-12-25 -1901-05-05 12:34:56.1158,1901-12-25 -1901-05-05 12:34:56.1159,1901-12-25 -1901-05-05 12:34:56.116,1901-12-25 -1901-05-05 12:34:56.1161,1901-12-25 -1901-05-05 12:34:56.1162,1901-12-25 -1901-05-05 12:34:56.1163,1901-12-25 -1901-05-05 12:34:56.1164,1901-12-25 -1901-05-05 12:34:56.1165,1901-12-25 -1901-05-05 12:34:56.1166,1901-12-25 -1901-05-05 12:34:56.1167,1901-12-25 -1901-05-05 12:34:56.1168,1901-12-25 -1901-05-05 12:34:56.1169,1901-12-25 -1901-05-05 12:34:56.117,1901-12-25 -1901-05-05 12:34:56.1171,1901-12-25 -1901-05-05 12:34:56.1172,1901-12-25 -1901-05-05 12:34:56.1173,1901-12-25 -1901-05-05 12:34:56.1174,1901-12-25 -1901-05-05 12:34:56.1175,1901-12-25 -1901-05-05 12:34:56.1176,1901-12-25 -1901-05-05 12:34:56.1177,1901-12-25 -1901-05-05 12:34:56.1178,1901-12-25 -1901-05-05 12:34:56.1179,1901-12-25 -1901-05-05 12:34:56.118,1901-12-25 -1901-05-05 12:34:56.1181,1901-12-25 -1901-05-05 12:34:56.1182,1901-12-25 -1901-05-05 12:34:56.1183,1901-12-25 -1901-05-05 12:34:56.1184,1901-12-25 -1901-05-05 12:34:56.1185,1901-12-25 -1901-05-05 12:34:56.1186,1901-12-25 -1901-05-05 12:34:56.1187,1901-12-25 -1901-05-05 12:34:56.1188,1901-12-25 -1901-05-05 12:34:56.1189,1901-12-25 -1901-05-05 12:34:56.119,1901-12-25 -1901-05-05 12:34:56.1191,1901-12-25 -1901-05-05 12:34:56.1192,1901-12-25 -1901-05-05 12:34:56.1193,1901-12-25 -1901-05-05 12:34:56.1194,1901-12-25 -1901-05-05 12:34:56.1195,1901-12-25 -1901-05-05 12:34:56.1196,1901-12-25 -1901-05-05 12:34:56.1197,1901-12-25 -1901-05-05 12:34:56.1198,1901-12-25 -1901-05-05 12:34:56.1199,1901-12-25 -1901-05-05 12:34:56.12,1901-12-25 -1901-05-05 12:34:56.1201,1901-12-25 -1901-05-05 12:34:56.1202,1901-12-25 -1901-05-05 12:34:56.1203,1901-12-25 -1901-05-05 12:34:56.1204,1901-12-25 -1901-05-05 12:34:56.1205,1901-12-25 -1901-05-05 12:34:56.1206,1901-12-25 -1901-05-05 12:34:56.1207,1901-12-25 -1901-05-05 12:34:56.1208,1901-12-25 -1901-05-05 12:34:56.1209,1901-12-25 -1901-05-05 12:34:56.121,1901-12-25 -1901-05-05 12:34:56.1211,1901-12-25 -1901-05-05 12:34:56.1212,1901-12-25 -1901-05-05 12:34:56.1213,1901-12-25 -1901-05-05 12:34:56.1214,1901-12-25 -1901-05-05 12:34:56.1215,1901-12-25 -1901-05-05 12:34:56.1216,1901-12-25 -1901-05-05 12:34:56.1217,1901-12-25 -1901-05-05 12:34:56.1218,1901-12-25 -1901-05-05 12:34:56.1219,1901-12-25 -1901-05-05 12:34:56.122,1901-12-25 -1901-05-05 12:34:56.1221,1901-12-25 -1901-05-05 12:34:56.1222,1901-12-25 -1901-05-05 12:34:56.1223,1901-12-25 -1901-05-05 12:34:56.1224,1901-12-25 -1901-05-05 12:34:56.1225,1901-12-25 -1901-05-05 12:34:56.1226,1901-12-25 -1901-05-05 12:34:56.1227,1901-12-25 -1901-05-05 12:34:56.1228,1901-12-25 -1901-05-05 12:34:56.1229,1901-12-25 -1901-05-05 12:34:56.123,1901-12-25 -1901-05-05 12:34:56.1231,1901-12-25 -1901-05-05 12:34:56.1232,1901-12-25 -1901-05-05 12:34:56.1233,1901-12-25 -1901-05-05 12:34:56.1234,1901-12-25 -1901-05-05 12:34:56.1235,1901-12-25 -1901-05-05 12:34:56.1236,1901-12-25 -1901-05-05 12:34:56.1237,1901-12-25 -1901-05-05 12:34:56.1238,1901-12-25 -1901-05-05 12:34:56.1239,1901-12-25 -1901-05-05 12:34:56.124,1901-12-25 -1901-05-05 12:34:56.1241,1901-12-25 -1901-05-05 12:34:56.1242,1901-12-25 -1901-05-05 12:34:56.1243,1901-12-25 -1901-05-05 12:34:56.1244,1901-12-25 -1901-05-05 12:34:56.1245,1901-12-25 -1901-05-05 12:34:56.1246,1901-12-25 -1901-05-05 12:34:56.1247,1901-12-25 -1901-05-05 12:34:56.1248,1901-12-25 -1901-05-05 12:34:56.1249,1901-12-25 -1901-05-05 12:34:56.125,1901-12-25 -1901-05-05 12:34:56.1251,1901-12-25 -1901-05-05 12:34:56.1252,1901-12-25 -1901-05-05 12:34:56.1253,1901-12-25 -1901-05-05 12:34:56.1254,1901-12-25 -1901-05-05 12:34:56.1255,1901-12-25 -1901-05-05 12:34:56.1256,1901-12-25 -1901-05-05 12:34:56.1257,1901-12-25 -1901-05-05 12:34:56.1258,1901-12-25 -1901-05-05 12:34:56.1259,1901-12-25 -1901-05-05 12:34:56.126,1901-12-25 -1901-05-05 12:34:56.1261,1901-12-25 -1901-05-05 12:34:56.1262,1901-12-25 -1901-05-05 12:34:56.1263,1901-12-25 -1901-05-05 12:34:56.1264,1901-12-25 -1901-05-05 12:34:56.1265,1901-12-25 -1901-05-05 12:34:56.1266,1901-12-25 -1901-05-05 12:34:56.1267,1901-12-25 -1901-05-05 12:34:56.1268,1901-12-25 -1901-05-05 12:34:56.1269,1901-12-25 -1901-05-05 12:34:56.127,1901-12-25 -1901-05-05 12:34:56.1271,1901-12-25 -1901-05-05 12:34:56.1272,1901-12-25 -1901-05-05 12:34:56.1273,1901-12-25 -1901-05-05 12:34:56.1274,1901-12-25 -1901-05-05 12:34:56.1275,1901-12-25 -1901-05-05 12:34:56.1276,1901-12-25 -1901-05-05 12:34:56.1277,1901-12-25 -1901-05-05 12:34:56.1278,1901-12-25 -1901-05-05 12:34:56.1279,1901-12-25 -1901-05-05 12:34:56.128,1901-12-25 -1901-05-05 12:34:56.1281,1901-12-25 -1901-05-05 12:34:56.1282,1901-12-25 -1901-05-05 12:34:56.1283,1901-12-25 -1901-05-05 12:34:56.1284,1901-12-25 -1901-05-05 12:34:56.1285,1901-12-25 -1901-05-05 12:34:56.1286,1901-12-25 -1901-05-05 12:34:56.1287,1901-12-25 -1901-05-05 12:34:56.1288,1901-12-25 -1901-05-05 12:34:56.1289,1901-12-25 -1901-05-05 12:34:56.129,1901-12-25 -1901-05-05 12:34:56.1291,1901-12-25 -1901-05-05 12:34:56.1292,1901-12-25 -1901-05-05 12:34:56.1293,1901-12-25 -1901-05-05 12:34:56.1294,1901-12-25 -1901-05-05 12:34:56.1295,1901-12-25 -1901-05-05 12:34:56.1296,1901-12-25 -1901-05-05 12:34:56.1297,1901-12-25 -1901-05-05 12:34:56.1298,1901-12-25 -1901-05-05 12:34:56.1299,1901-12-25 -1901-05-05 12:34:56.13,1901-12-25 -1901-05-05 12:34:56.1301,1901-12-25 -1901-05-05 12:34:56.1302,1901-12-25 -1901-05-05 12:34:56.1303,1901-12-25 -1901-05-05 12:34:56.1304,1901-12-25 -1901-05-05 12:34:56.1305,1901-12-25 -1901-05-05 12:34:56.1306,1901-12-25 -1901-05-05 12:34:56.1307,1901-12-25 -1901-05-05 12:34:56.1308,1901-12-25 -1901-05-05 12:34:56.1309,1901-12-25 -1901-05-05 12:34:56.131,1901-12-25 -1901-05-05 12:34:56.1311,1901-12-25 -1901-05-05 12:34:56.1312,1901-12-25 -1901-05-05 12:34:56.1313,1901-12-25 -1901-05-05 12:34:56.1314,1901-12-25 -1901-05-05 12:34:56.1315,1901-12-25 -1901-05-05 12:34:56.1316,1901-12-25 -1901-05-05 12:34:56.1317,1901-12-25 -1901-05-05 12:34:56.1318,1901-12-25 -1901-05-05 12:34:56.1319,1901-12-25 -1901-05-05 12:34:56.132,1901-12-25 -1901-05-05 12:34:56.1321,1901-12-25 -1901-05-05 12:34:56.1322,1901-12-25 -1901-05-05 12:34:56.1323,1901-12-25 -1901-05-05 12:34:56.1324,1901-12-25 -1901-05-05 12:34:56.1325,1901-12-25 -1901-05-05 12:34:56.1326,1901-12-25 -1901-05-05 12:34:56.1327,1901-12-25 -1901-05-05 12:34:56.1328,1901-12-25 -1901-05-05 12:34:56.1329,1901-12-25 -1901-05-05 12:34:56.133,1901-12-25 -1901-05-05 12:34:56.1331,1901-12-25 -1901-05-05 12:34:56.1332,1901-12-25 -1901-05-05 12:34:56.1333,1901-12-25 -1901-05-05 12:34:56.1334,1901-12-25 -1901-05-05 12:34:56.1335,1901-12-25 -1901-05-05 12:34:56.1336,1901-12-25 -1901-05-05 12:34:56.1337,1901-12-25 -1901-05-05 12:34:56.1338,1901-12-25 -1901-05-05 12:34:56.1339,1901-12-25 -1901-05-05 12:34:56.134,1901-12-25 -1901-05-05 12:34:56.1341,1901-12-25 -1901-05-05 12:34:56.1342,1901-12-25 -1901-05-05 12:34:56.1343,1901-12-25 -1901-05-05 12:34:56.1344,1901-12-25 -1901-05-05 12:34:56.1345,1901-12-25 -1901-05-05 12:34:56.1346,1901-12-25 -1901-05-05 12:34:56.1347,1901-12-25 -1901-05-05 12:34:56.1348,1901-12-25 -1901-05-05 12:34:56.1349,1901-12-25 -1901-05-05 12:34:56.135,1901-12-25 -1901-05-05 12:34:56.1351,1901-12-25 -1901-05-05 12:34:56.1352,1901-12-25 -1901-05-05 12:34:56.1353,1901-12-25 -1901-05-05 12:34:56.1354,1901-12-25 -1901-05-05 12:34:56.1355,1901-12-25 -1901-05-05 12:34:56.1356,1901-12-25 -1901-05-05 12:34:56.1357,1901-12-25 -1901-05-05 12:34:56.1358,1901-12-25 -1901-05-05 12:34:56.1359,1901-12-25 -1901-05-05 12:34:56.136,1901-12-25 -1901-05-05 12:34:56.1361,1901-12-25 -1901-05-05 12:34:56.1362,1901-12-25 -1901-05-05 12:34:56.1363,1901-12-25 -1901-05-05 12:34:56.1364,1901-12-25 -1901-05-05 12:34:56.1365,1901-12-25 -1901-05-05 12:34:56.1366,1901-12-25 -1901-05-05 12:34:56.1367,1901-12-25 -1901-05-05 12:34:56.1368,1901-12-25 -1901-05-05 12:34:56.1369,1901-12-25 -1901-05-05 12:34:56.137,1901-12-25 -1901-05-05 12:34:56.1371,1901-12-25 -1901-05-05 12:34:56.1372,1901-12-25 -1901-05-05 12:34:56.1373,1901-12-25 -1901-05-05 12:34:56.1374,1901-12-25 -1901-05-05 12:34:56.1375,1901-12-25 -1901-05-05 12:34:56.1376,1901-12-25 -1901-05-05 12:34:56.1377,1901-12-25 -1901-05-05 12:34:56.1378,1901-12-25 -1901-05-05 12:34:56.1379,1901-12-25 -1901-05-05 12:34:56.138,1901-12-25 -1901-05-05 12:34:56.1381,1901-12-25 -1901-05-05 12:34:56.1382,1901-12-25 -1901-05-05 12:34:56.1383,1901-12-25 -1901-05-05 12:34:56.1384,1901-12-25 -1901-05-05 12:34:56.1385,1901-12-25 -1901-05-05 12:34:56.1386,1901-12-25 -1901-05-05 12:34:56.1387,1901-12-25 -1901-05-05 12:34:56.1388,1901-12-25 -1901-05-05 12:34:56.1389,1901-12-25 -1901-05-05 12:34:56.139,1901-12-25 -1901-05-05 12:34:56.1391,1901-12-25 -1901-05-05 12:34:56.1392,1901-12-25 -1901-05-05 12:34:56.1393,1901-12-25 -1901-05-05 12:34:56.1394,1901-12-25 -1901-05-05 12:34:56.1395,1901-12-25 -1901-05-05 12:34:56.1396,1901-12-25 -1901-05-05 12:34:56.1397,1901-12-25 -1901-05-05 12:34:56.1398,1901-12-25 -1901-05-05 12:34:56.1399,1901-12-25 -1901-05-05 12:34:56.14,1901-12-25 -1901-05-05 12:34:56.1401,1901-12-25 -1901-05-05 12:34:56.1402,1901-12-25 -1901-05-05 12:34:56.1403,1901-12-25 -1901-05-05 12:34:56.1404,1901-12-25 -1901-05-05 12:34:56.1405,1901-12-25 -1901-05-05 12:34:56.1406,1901-12-25 -1901-05-05 12:34:56.1407,1901-12-25 -1901-05-05 12:34:56.1408,1901-12-25 -1901-05-05 12:34:56.1409,1901-12-25 -1901-05-05 12:34:56.141,1901-12-25 -1901-05-05 12:34:56.1411,1901-12-25 -1901-05-05 12:34:56.1412,1901-12-25 -1901-05-05 12:34:56.1413,1901-12-25 -1901-05-05 12:34:56.1414,1901-12-25 -1901-05-05 12:34:56.1415,1901-12-25 -1901-05-05 12:34:56.1416,1901-12-25 -1901-05-05 12:34:56.1417,1901-12-25 -1901-05-05 12:34:56.1418,1901-12-25 -1901-05-05 12:34:56.1419,1901-12-25 -1901-05-05 12:34:56.142,1901-12-25 -1901-05-05 12:34:56.1421,1901-12-25 -1901-05-05 12:34:56.1422,1901-12-25 -1901-05-05 12:34:56.1423,1901-12-25 -1901-05-05 12:34:56.1424,1901-12-25 -1901-05-05 12:34:56.1425,1901-12-25 -1901-05-05 12:34:56.1426,1901-12-25 -1901-05-05 12:34:56.1427,1901-12-25 -1901-05-05 12:34:56.1428,1901-12-25 -1901-05-05 12:34:56.1429,1901-12-25 -1901-05-05 12:34:56.143,1901-12-25 -1901-05-05 12:34:56.1431,1901-12-25 -1901-05-05 12:34:56.1432,1901-12-25 -1901-05-05 12:34:56.1433,1901-12-25 -1901-05-05 12:34:56.1434,1901-12-25 -1901-05-05 12:34:56.1435,1901-12-25 -1901-05-05 12:34:56.1436,1901-12-25 -1901-05-05 12:34:56.1437,1901-12-25 -1901-05-05 12:34:56.1438,1901-12-25 -1901-05-05 12:34:56.1439,1901-12-25 -1901-05-05 12:34:56.144,1901-12-25 -1901-05-05 12:34:56.1441,1901-12-25 -1901-05-05 12:34:56.1442,1901-12-25 -1901-05-05 12:34:56.1443,1901-12-25 -1901-05-05 12:34:56.1444,1901-12-25 -1901-05-05 12:34:56.1445,1901-12-25 -1901-05-05 12:34:56.1446,1901-12-25 -1901-05-05 12:34:56.1447,1901-12-25 -1901-05-05 12:34:56.1448,1901-12-25 -1901-05-05 12:34:56.1449,1901-12-25 -1901-05-05 12:34:56.145,1901-12-25 -1901-05-05 12:34:56.1451,1901-12-25 -1901-05-05 12:34:56.1452,1901-12-25 -1901-05-05 12:34:56.1453,1901-12-25 -1901-05-05 12:34:56.1454,1901-12-25 -1901-05-05 12:34:56.1455,1901-12-25 -1901-05-05 12:34:56.1456,1901-12-25 -1901-05-05 12:34:56.1457,1901-12-25 -1901-05-05 12:34:56.1458,1901-12-25 -1901-05-05 12:34:56.1459,1901-12-25 -1901-05-05 12:34:56.146,1901-12-25 -1901-05-05 12:34:56.1461,1901-12-25 -1901-05-05 12:34:56.1462,1901-12-25 -1901-05-05 12:34:56.1463,1901-12-25 -1901-05-05 12:34:56.1464,1901-12-25 -1901-05-05 12:34:56.1465,1901-12-25 -1901-05-05 12:34:56.1466,1901-12-25 -1901-05-05 12:34:56.1467,1901-12-25 -1901-05-05 12:34:56.1468,1901-12-25 -1901-05-05 12:34:56.1469,1901-12-25 -1901-05-05 12:34:56.147,1901-12-25 -1901-05-05 12:34:56.1471,1901-12-25 -1901-05-05 12:34:56.1472,1901-12-25 -1901-05-05 12:34:56.1473,1901-12-25 -1901-05-05 12:34:56.1474,1901-12-25 -1901-05-05 12:34:56.1475,1901-12-25 -1901-05-05 12:34:56.1476,1901-12-25 -1901-05-05 12:34:56.1477,1901-12-25 -1901-05-05 12:34:56.1478,1901-12-25 -1901-05-05 12:34:56.1479,1901-12-25 -1901-05-05 12:34:56.148,1901-12-25 -1901-05-05 12:34:56.1481,1901-12-25 -1901-05-05 12:34:56.1482,1901-12-25 -1901-05-05 12:34:56.1483,1901-12-25 -1901-05-05 12:34:56.1484,1901-12-25 -1901-05-05 12:34:56.1485,1901-12-25 -1901-05-05 12:34:56.1486,1901-12-25 -1901-05-05 12:34:56.1487,1901-12-25 -1901-05-05 12:34:56.1488,1901-12-25 -1901-05-05 12:34:56.1489,1901-12-25 -1901-05-05 12:34:56.149,1901-12-25 -1901-05-05 12:34:56.1491,1901-12-25 -1901-05-05 12:34:56.1492,1901-12-25 -1901-05-05 12:34:56.1493,1901-12-25 -1901-05-05 12:34:56.1494,1901-12-25 -1901-05-05 12:34:56.1495,1901-12-25 -1901-05-05 12:34:56.1496,1901-12-25 -1901-05-05 12:34:56.1497,1901-12-25 -1901-05-05 12:34:56.1498,1901-12-25 -1901-05-05 12:34:56.1499,1901-12-25 -1901-05-05 12:34:56.15,1901-12-25 -1901-05-05 12:34:56.1501,1901-12-25 -1901-05-05 12:34:56.1502,1901-12-25 -1901-05-05 12:34:56.1503,1901-12-25 -1901-05-05 12:34:56.1504,1901-12-25 -1901-05-05 12:34:56.1505,1901-12-25 -1901-05-05 12:34:56.1506,1901-12-25 -1901-05-05 12:34:56.1507,1901-12-25 -1901-05-05 12:34:56.1508,1901-12-25 -1901-05-05 12:34:56.1509,1901-12-25 -1901-05-05 12:34:56.151,1901-12-25 -1901-05-05 12:34:56.1511,1901-12-25 -1901-05-05 12:34:56.1512,1901-12-25 -1901-05-05 12:34:56.1513,1901-12-25 -1901-05-05 12:34:56.1514,1901-12-25 -1901-05-05 12:34:56.1515,1901-12-25 -1901-05-05 12:34:56.1516,1901-12-25 -1901-05-05 12:34:56.1517,1901-12-25 -1901-05-05 12:34:56.1518,1901-12-25 -1901-05-05 12:34:56.1519,1901-12-25 -1901-05-05 12:34:56.152,1901-12-25 -1901-05-05 12:34:56.1521,1901-12-25 -1901-05-05 12:34:56.1522,1901-12-25 -1901-05-05 12:34:56.1523,1901-12-25 -1901-05-05 12:34:56.1524,1901-12-25 -1901-05-05 12:34:56.1525,1901-12-25 -1901-05-05 12:34:56.1526,1901-12-25 -1901-05-05 12:34:56.1527,1901-12-25 -1901-05-05 12:34:56.1528,1901-12-25 -1901-05-05 12:34:56.1529,1901-12-25 -1901-05-05 12:34:56.153,1901-12-25 -1901-05-05 12:34:56.1531,1901-12-25 -1901-05-05 12:34:56.1532,1901-12-25 -1901-05-05 12:34:56.1533,1901-12-25 -1901-05-05 12:34:56.1534,1901-12-25 -1901-05-05 12:34:56.1535,1901-12-25 -1901-05-05 12:34:56.1536,1901-12-25 -1901-05-05 12:34:56.1537,1901-12-25 -1901-05-05 12:34:56.1538,1901-12-25 -1901-05-05 12:34:56.1539,1901-12-25 -1901-05-05 12:34:56.154,1901-12-25 -1901-05-05 12:34:56.1541,1901-12-25 -1901-05-05 12:34:56.1542,1901-12-25 -1901-05-05 12:34:56.1543,1901-12-25 -1901-05-05 12:34:56.1544,1901-12-25 -1901-05-05 12:34:56.1545,1901-12-25 -1901-05-05 12:34:56.1546,1901-12-25 -1901-05-05 12:34:56.1547,1901-12-25 -1901-05-05 12:34:56.1548,1901-12-25 -1901-05-05 12:34:56.1549,1901-12-25 -1901-05-05 12:34:56.155,1901-12-25 -1901-05-05 12:34:56.1551,1901-12-25 -1901-05-05 12:34:56.1552,1901-12-25 -1901-05-05 12:34:56.1553,1901-12-25 -1901-05-05 12:34:56.1554,1901-12-25 -1901-05-05 12:34:56.1555,1901-12-25 -1901-05-05 12:34:56.1556,1901-12-25 -1901-05-05 12:34:56.1557,1901-12-25 -1901-05-05 12:34:56.1558,1901-12-25 -1901-05-05 12:34:56.1559,1901-12-25 -1901-05-05 12:34:56.156,1901-12-25 -1901-05-05 12:34:56.1561,1901-12-25 -1901-05-05 12:34:56.1562,1901-12-25 -1901-05-05 12:34:56.1563,1901-12-25 -1901-05-05 12:34:56.1564,1901-12-25 -1901-05-05 12:34:56.1565,1901-12-25 -1901-05-05 12:34:56.1566,1901-12-25 -1901-05-05 12:34:56.1567,1901-12-25 -1901-05-05 12:34:56.1568,1901-12-25 -1901-05-05 12:34:56.1569,1901-12-25 -1901-05-05 12:34:56.157,1901-12-25 -1901-05-05 12:34:56.1571,1901-12-25 -1901-05-05 12:34:56.1572,1901-12-25 -1901-05-05 12:34:56.1573,1901-12-25 -1901-05-05 12:34:56.1574,1901-12-25 -1901-05-05 12:34:56.1575,1901-12-25 -1901-05-05 12:34:56.1576,1901-12-25 -1901-05-05 12:34:56.1577,1901-12-25 -1901-05-05 12:34:56.1578,1901-12-25 -1901-05-05 12:34:56.1579,1901-12-25 -1901-05-05 12:34:56.158,1901-12-25 -1901-05-05 12:34:56.1581,1901-12-25 -1901-05-05 12:34:56.1582,1901-12-25 -1901-05-05 12:34:56.1583,1901-12-25 -1901-05-05 12:34:56.1584,1901-12-25 -1901-05-05 12:34:56.1585,1901-12-25 -1901-05-05 12:34:56.1586,1901-12-25 -1901-05-05 12:34:56.1587,1901-12-25 -1901-05-05 12:34:56.1588,1901-12-25 -1901-05-05 12:34:56.1589,1901-12-25 -1901-05-05 12:34:56.159,1901-12-25 -1901-05-05 12:34:56.1591,1901-12-25 -1901-05-05 12:34:56.1592,1901-12-25 -1901-05-05 12:34:56.1593,1901-12-25 -1901-05-05 12:34:56.1594,1901-12-25 -1901-05-05 12:34:56.1595,1901-12-25 -1901-05-05 12:34:56.1596,1901-12-25 -1901-05-05 12:34:56.1597,1901-12-25 -1901-05-05 12:34:56.1598,1901-12-25 -1901-05-05 12:34:56.1599,1901-12-25 -1901-05-05 12:34:56.16,1901-12-25 -1901-05-05 12:34:56.1601,1901-12-25 -1901-05-05 12:34:56.1602,1901-12-25 -1901-05-05 12:34:56.1603,1901-12-25 -1901-05-05 12:34:56.1604,1901-12-25 -1901-05-05 12:34:56.1605,1901-12-25 -1901-05-05 12:34:56.1606,1901-12-25 -1901-05-05 12:34:56.1607,1901-12-25 -1901-05-05 12:34:56.1608,1901-12-25 -1901-05-05 12:34:56.1609,1901-12-25 -1901-05-05 12:34:56.161,1901-12-25 -1901-05-05 12:34:56.1611,1901-12-25 -1901-05-05 12:34:56.1612,1901-12-25 -1901-05-05 12:34:56.1613,1901-12-25 -1901-05-05 12:34:56.1614,1901-12-25 -1901-05-05 12:34:56.1615,1901-12-25 -1901-05-05 12:34:56.1616,1901-12-25 -1901-05-05 12:34:56.1617,1901-12-25 -1901-05-05 12:34:56.1618,1901-12-25 -1901-05-05 12:34:56.1619,1901-12-25 -1901-05-05 12:34:56.162,1901-12-25 -1901-05-05 12:34:56.1621,1901-12-25 -1901-05-05 12:34:56.1622,1901-12-25 -1901-05-05 12:34:56.1623,1901-12-25 -1901-05-05 12:34:56.1624,1901-12-25 -1901-05-05 12:34:56.1625,1901-12-25 -1901-05-05 12:34:56.1626,1901-12-25 -1901-05-05 12:34:56.1627,1901-12-25 -1901-05-05 12:34:56.1628,1901-12-25 -1901-05-05 12:34:56.1629,1901-12-25 -1901-05-05 12:34:56.163,1901-12-25 -1901-05-05 12:34:56.1631,1901-12-25 -1901-05-05 12:34:56.1632,1901-12-25 -1901-05-05 12:34:56.1633,1901-12-25 -1901-05-05 12:34:56.1634,1901-12-25 -1901-05-05 12:34:56.1635,1901-12-25 -1901-05-05 12:34:56.1636,1901-12-25 -1901-05-05 12:34:56.1637,1901-12-25 -1901-05-05 12:34:56.1638,1901-12-25 -1901-05-05 12:34:56.1639,1901-12-25 -1901-05-05 12:34:56.164,1901-12-25 -1901-05-05 12:34:56.1641,1901-12-25 -1901-05-05 12:34:56.1642,1901-12-25 -1901-05-05 12:34:56.1643,1901-12-25 -1901-05-05 12:34:56.1644,1901-12-25 -1901-05-05 12:34:56.1645,1901-12-25 -1901-05-05 12:34:56.1646,1901-12-25 -1901-05-05 12:34:56.1647,1901-12-25 -1901-05-05 12:34:56.1648,1901-12-25 -1901-05-05 12:34:56.1649,1901-12-25 -1901-05-05 12:34:56.165,1901-12-25 -1901-05-05 12:34:56.1651,1901-12-25 -1901-05-05 12:34:56.1652,1901-12-25 -1901-05-05 12:34:56.1653,1901-12-25 -1901-05-05 12:34:56.1654,1901-12-25 -1901-05-05 12:34:56.1655,1901-12-25 -1901-05-05 12:34:56.1656,1901-12-25 -1901-05-05 12:34:56.1657,1901-12-25 -1901-05-05 12:34:56.1658,1901-12-25 -1901-05-05 12:34:56.1659,1901-12-25 -1901-05-05 12:34:56.166,1901-12-25 -1901-05-05 12:34:56.1661,1901-12-25 -1901-05-05 12:34:56.1662,1901-12-25 -1901-05-05 12:34:56.1663,1901-12-25 -1901-05-05 12:34:56.1664,1901-12-25 -1901-05-05 12:34:56.1665,1901-12-25 -1901-05-05 12:34:56.1666,1901-12-25 -1901-05-05 12:34:56.1667,1901-12-25 -1901-05-05 12:34:56.1668,1901-12-25 -1901-05-05 12:34:56.1669,1901-12-25 -1901-05-05 12:34:56.167,1901-12-25 -1901-05-05 12:34:56.1671,1901-12-25 -1901-05-05 12:34:56.1672,1901-12-25 -1901-05-05 12:34:56.1673,1901-12-25 -1901-05-05 12:34:56.1674,1901-12-25 -1901-05-05 12:34:56.1675,1901-12-25 -1901-05-05 12:34:56.1676,1901-12-25 -1901-05-05 12:34:56.1677,1901-12-25 -1901-05-05 12:34:56.1678,1901-12-25 -1901-05-05 12:34:56.1679,1901-12-25 -1901-05-05 12:34:56.168,1901-12-25 -1901-05-05 12:34:56.1681,1901-12-25 -1901-05-05 12:34:56.1682,1901-12-25 -1901-05-05 12:34:56.1683,1901-12-25 -1901-05-05 12:34:56.1684,1901-12-25 -1901-05-05 12:34:56.1685,1901-12-25 -1901-05-05 12:34:56.1686,1901-12-25 -1901-05-05 12:34:56.1687,1901-12-25 -1901-05-05 12:34:56.1688,1901-12-25 -1901-05-05 12:34:56.1689,1901-12-25 -1901-05-05 12:34:56.169,1901-12-25 -1901-05-05 12:34:56.1691,1901-12-25 -1901-05-05 12:34:56.1692,1901-12-25 -1901-05-05 12:34:56.1693,1901-12-25 -1901-05-05 12:34:56.1694,1901-12-25 -1901-05-05 12:34:56.1695,1901-12-25 -1901-05-05 12:34:56.1696,1901-12-25 -1901-05-05 12:34:56.1697,1901-12-25 -1901-05-05 12:34:56.1698,1901-12-25 -1901-05-05 12:34:56.1699,1901-12-25 -1901-05-05 12:34:56.17,1901-12-25 -1901-05-05 12:34:56.1701,1901-12-25 -1901-05-05 12:34:56.1702,1901-12-25 -1901-05-05 12:34:56.1703,1901-12-25 -1901-05-05 12:34:56.1704,1901-12-25 -1901-05-05 12:34:56.1705,1901-12-25 -1901-05-05 12:34:56.1706,1901-12-25 -1901-05-05 12:34:56.1707,1901-12-25 -1901-05-05 12:34:56.1708,1901-12-25 -1901-05-05 12:34:56.1709,1901-12-25 -1901-05-05 12:34:56.171,1901-12-25 -1901-05-05 12:34:56.1711,1901-12-25 -1901-05-05 12:34:56.1712,1901-12-25 -1901-05-05 12:34:56.1713,1901-12-25 -1901-05-05 12:34:56.1714,1901-12-25 -1901-05-05 12:34:56.1715,1901-12-25 -1901-05-05 12:34:56.1716,1901-12-25 -1901-05-05 12:34:56.1717,1901-12-25 -1901-05-05 12:34:56.1718,1901-12-25 -1901-05-05 12:34:56.1719,1901-12-25 -1901-05-05 12:34:56.172,1901-12-25 -1901-05-05 12:34:56.1721,1901-12-25 -1901-05-05 12:34:56.1722,1901-12-25 -1901-05-05 12:34:56.1723,1901-12-25 -1901-05-05 12:34:56.1724,1901-12-25 -1901-05-05 12:34:56.1725,1901-12-25 -1901-05-05 12:34:56.1726,1901-12-25 -1901-05-05 12:34:56.1727,1901-12-25 -1901-05-05 12:34:56.1728,1901-12-25 -1901-05-05 12:34:56.1729,1901-12-25 -1901-05-05 12:34:56.173,1901-12-25 -1901-05-05 12:34:56.1731,1901-12-25 -1901-05-05 12:34:56.1732,1901-12-25 -1901-05-05 12:34:56.1733,1901-12-25 -1901-05-05 12:34:56.1734,1901-12-25 -1901-05-05 12:34:56.1735,1901-12-25 -1901-05-05 12:34:56.1736,1901-12-25 -1901-05-05 12:34:56.1737,1901-12-25 -1901-05-05 12:34:56.1738,1901-12-25 -1901-05-05 12:34:56.1739,1901-12-25 -1901-05-05 12:34:56.174,1901-12-25 -1901-05-05 12:34:56.1741,1901-12-25 -1901-05-05 12:34:56.1742,1901-12-25 -1901-05-05 12:34:56.1743,1901-12-25 -1901-05-05 12:34:56.1744,1901-12-25 -1901-05-05 12:34:56.1745,1901-12-25 -1901-05-05 12:34:56.1746,1901-12-25 -1901-05-05 12:34:56.1747,1901-12-25 -1901-05-05 12:34:56.1748,1901-12-25 -1901-05-05 12:34:56.1749,1901-12-25 -1901-05-05 12:34:56.175,1901-12-25 -1901-05-05 12:34:56.1751,1901-12-25 -1901-05-05 12:34:56.1752,1901-12-25 -1901-05-05 12:34:56.1753,1901-12-25 -1901-05-05 12:34:56.1754,1901-12-25 -1901-05-05 12:34:56.1755,1901-12-25 -1901-05-05 12:34:56.1756,1901-12-25 -1901-05-05 12:34:56.1757,1901-12-25 -1901-05-05 12:34:56.1758,1901-12-25 -1901-05-05 12:34:56.1759,1901-12-25 -1901-05-05 12:34:56.176,1901-12-25 -1901-05-05 12:34:56.1761,1901-12-25 -1901-05-05 12:34:56.1762,1901-12-25 -1901-05-05 12:34:56.1763,1901-12-25 -1901-05-05 12:34:56.1764,1901-12-25 -1901-05-05 12:34:56.1765,1901-12-25 -1901-05-05 12:34:56.1766,1901-12-25 -1901-05-05 12:34:56.1767,1901-12-25 -1901-05-05 12:34:56.1768,1901-12-25 -1901-05-05 12:34:56.1769,1901-12-25 -1901-05-05 12:34:56.177,1901-12-25 -1901-05-05 12:34:56.1771,1901-12-25 -1901-05-05 12:34:56.1772,1901-12-25 -1901-05-05 12:34:56.1773,1901-12-25 -1901-05-05 12:34:56.1774,1901-12-25 -1901-05-05 12:34:56.1775,1901-12-25 -1901-05-05 12:34:56.1776,1901-12-25 -1901-05-05 12:34:56.1777,1901-12-25 -1901-05-05 12:34:56.1778,1901-12-25 -1901-05-05 12:34:56.1779,1901-12-25 -1901-05-05 12:34:56.178,1901-12-25 -1901-05-05 12:34:56.1781,1901-12-25 -1901-05-05 12:34:56.1782,1901-12-25 -1901-05-05 12:34:56.1783,1901-12-25 -1901-05-05 12:34:56.1784,1901-12-25 -1901-05-05 12:34:56.1785,1901-12-25 -1901-05-05 12:34:56.1786,1901-12-25 -1901-05-05 12:34:56.1787,1901-12-25 -1901-05-05 12:34:56.1788,1901-12-25 -1901-05-05 12:34:56.1789,1901-12-25 -1901-05-05 12:34:56.179,1901-12-25 -1901-05-05 12:34:56.1791,1901-12-25 -1901-05-05 12:34:56.1792,1901-12-25 -1901-05-05 12:34:56.1793,1901-12-25 -1901-05-05 12:34:56.1794,1901-12-25 -1901-05-05 12:34:56.1795,1901-12-25 -1901-05-05 12:34:56.1796,1901-12-25 -1901-05-05 12:34:56.1797,1901-12-25 -1901-05-05 12:34:56.1798,1901-12-25 -1901-05-05 12:34:56.1799,1901-12-25 -1901-05-05 12:34:56.18,1901-12-25 -1901-05-05 12:34:56.1801,1901-12-25 -1901-05-05 12:34:56.1802,1901-12-25 -1901-05-05 12:34:56.1803,1901-12-25 -1901-05-05 12:34:56.1804,1901-12-25 -1901-05-05 12:34:56.1805,1901-12-25 -1901-05-05 12:34:56.1806,1901-12-25 -1901-05-05 12:34:56.1807,1901-12-25 -1901-05-05 12:34:56.1808,1901-12-25 -1901-05-05 12:34:56.1809,1901-12-25 -1901-05-05 12:34:56.181,1901-12-25 -1901-05-05 12:34:56.1811,1901-12-25 -1901-05-05 12:34:56.1812,1901-12-25 -1901-05-05 12:34:56.1813,1901-12-25 -1901-05-05 12:34:56.1814,1901-12-25 -1901-05-05 12:34:56.1815,1901-12-25 -1901-05-05 12:34:56.1816,1901-12-25 -1901-05-05 12:34:56.1817,1901-12-25 -1901-05-05 12:34:56.1818,1901-12-25 -1901-05-05 12:34:56.1819,1901-12-25 -1901-05-05 12:34:56.182,1901-12-25 -1901-05-05 12:34:56.1821,1901-12-25 -1901-05-05 12:34:56.1822,1901-12-25 -1901-05-05 12:34:56.1823,1901-12-25 -1901-05-05 12:34:56.1824,1901-12-25 -1901-05-05 12:34:56.1825,1901-12-25 -1901-05-05 12:34:56.1826,1901-12-25 -1901-05-05 12:34:56.1827,1901-12-25 -1901-05-05 12:34:56.1828,1901-12-25 -1901-05-05 12:34:56.1829,1901-12-25 -1901-05-05 12:34:56.183,1901-12-25 -1901-05-05 12:34:56.1831,1901-12-25 -1901-05-05 12:34:56.1832,1901-12-25 -1901-05-05 12:34:56.1833,1901-12-25 -1901-05-05 12:34:56.1834,1901-12-25 -1901-05-05 12:34:56.1835,1901-12-25 -1901-05-05 12:34:56.1836,1901-12-25 -1901-05-05 12:34:56.1837,1901-12-25 -1901-05-05 12:34:56.1838,1901-12-25 -1901-05-05 12:34:56.1839,1901-12-25 -1901-05-05 12:34:56.184,1901-12-25 -1901-05-05 12:34:56.1841,1901-12-25 -1901-05-05 12:34:56.1842,1901-12-25 -1901-05-05 12:34:56.1843,1901-12-25 -1901-05-05 12:34:56.1844,1901-12-25 -1901-05-05 12:34:56.1845,1901-12-25 -1901-05-05 12:34:56.1846,1901-12-25 -1901-05-05 12:34:56.1847,1901-12-25 -1901-05-05 12:34:56.1848,1901-12-25 -1901-05-05 12:34:56.1849,1901-12-25 -1901-05-05 12:34:56.185,1901-12-25 -1901-05-05 12:34:56.1851,1901-12-25 -1901-05-05 12:34:56.1852,1901-12-25 -1901-05-05 12:34:56.1853,1901-12-25 -1901-05-05 12:34:56.1854,1901-12-25 -1901-05-05 12:34:56.1855,1901-12-25 -1901-05-05 12:34:56.1856,1901-12-25 -1901-05-05 12:34:56.1857,1901-12-25 -1901-05-05 12:34:56.1858,1901-12-25 -1901-05-05 12:34:56.1859,1901-12-25 -1901-05-05 12:34:56.186,1901-12-25 -1901-05-05 12:34:56.1861,1901-12-25 -1901-05-05 12:34:56.1862,1901-12-25 -1901-05-05 12:34:56.1863,1901-12-25 -1901-05-05 12:34:56.1864,1901-12-25 -1901-05-05 12:34:56.1865,1901-12-25 -1901-05-05 12:34:56.1866,1901-12-25 -1901-05-05 12:34:56.1867,1901-12-25 -1901-05-05 12:34:56.1868,1901-12-25 -1901-05-05 12:34:56.1869,1901-12-25 -1901-05-05 12:34:56.187,1901-12-25 -1901-05-05 12:34:56.1871,1901-12-25 -1901-05-05 12:34:56.1872,1901-12-25 -1901-05-05 12:34:56.1873,1901-12-25 -1901-05-05 12:34:56.1874,1901-12-25 -1901-05-05 12:34:56.1875,1901-12-25 -1901-05-05 12:34:56.1876,1901-12-25 -1901-05-05 12:34:56.1877,1901-12-25 -1901-05-05 12:34:56.1878,1901-12-25 -1901-05-05 12:34:56.1879,1901-12-25 -1901-05-05 12:34:56.188,1901-12-25 -1901-05-05 12:34:56.1881,1901-12-25 -1901-05-05 12:34:56.1882,1901-12-25 -1901-05-05 12:34:56.1883,1901-12-25 -1901-05-05 12:34:56.1884,1901-12-25 -1901-05-05 12:34:56.1885,1901-12-25 -1901-05-05 12:34:56.1886,1901-12-25 -1901-05-05 12:34:56.1887,1901-12-25 -1901-05-05 12:34:56.1888,1901-12-25 -1901-05-05 12:34:56.1889,1901-12-25 -1901-05-05 12:34:56.189,1901-12-25 -1901-05-05 12:34:56.1891,1901-12-25 -1901-05-05 12:34:56.1892,1901-12-25 -1901-05-05 12:34:56.1893,1901-12-25 -1901-05-05 12:34:56.1894,1901-12-25 -1901-05-05 12:34:56.1895,1901-12-25 -1901-05-05 12:34:56.1896,1901-12-25 -1901-05-05 12:34:56.1897,1901-12-25 -1901-05-05 12:34:56.1898,1901-12-25 -1901-05-05 12:34:56.1899,1901-12-25 -1901-05-05 12:34:56.19,1901-12-25 -1901-05-05 12:34:56.1901,1901-12-25 -1901-05-05 12:34:56.1902,1901-12-25 -1901-05-05 12:34:56.1903,1901-12-25 -1901-05-05 12:34:56.1904,1901-12-25 -1901-05-05 12:34:56.1905,1901-12-25 -1901-05-05 12:34:56.1906,1901-12-25 -1901-05-05 12:34:56.1907,1901-12-25 -1901-05-05 12:34:56.1908,1901-12-25 -1901-05-05 12:34:56.1909,1901-12-25 -1901-05-05 12:34:56.191,1901-12-25 -1901-05-05 12:34:56.1911,1901-12-25 -1901-05-05 12:34:56.1912,1901-12-25 -1901-05-05 12:34:56.1913,1901-12-25 -1901-05-05 12:34:56.1914,1901-12-25 -1901-05-05 12:34:56.1915,1901-12-25 -1901-05-05 12:34:56.1916,1901-12-25 -1901-05-05 12:34:56.1917,1901-12-25 -1901-05-05 12:34:56.1918,1901-12-25 -1901-05-05 12:34:56.1919,1901-12-25 -1901-05-05 12:34:56.192,1901-12-25 -1901-05-05 12:34:56.1921,1901-12-25 -1901-05-05 12:34:56.1922,1901-12-25 -1901-05-05 12:34:56.1923,1901-12-25 -1901-05-05 12:34:56.1924,1901-12-25 -1901-05-05 12:34:56.1925,1901-12-25 -1901-05-05 12:34:56.1926,1901-12-25 -1901-05-05 12:34:56.1927,1901-12-25 -1901-05-05 12:34:56.1928,1901-12-25 -1901-05-05 12:34:56.1929,1901-12-25 -1901-05-05 12:34:56.193,1901-12-25 -1901-05-05 12:34:56.1931,1901-12-25 -1901-05-05 12:34:56.1932,1901-12-25 -1901-05-05 12:34:56.1933,1901-12-25 -1901-05-05 12:34:56.1934,1901-12-25 -1901-05-05 12:34:56.1935,1901-12-25 -1901-05-05 12:34:56.1936,1901-12-25 -1901-05-05 12:34:56.1937,1901-12-25 -1901-05-05 12:34:56.1938,1901-12-25 -1901-05-05 12:34:56.1939,1901-12-25 -1901-05-05 12:34:56.194,1901-12-25 -1901-05-05 12:34:56.1941,1901-12-25 -1901-05-05 12:34:56.1942,1901-12-25 -1901-05-05 12:34:56.1943,1901-12-25 -1901-05-05 12:34:56.1944,1901-12-25 -1901-05-05 12:34:56.1945,1901-12-25 -1901-05-05 12:34:56.1946,1901-12-25 -1901-05-05 12:34:56.1947,1901-12-25 -1901-05-05 12:34:56.1948,1901-12-25 -1901-05-05 12:34:56.1949,1901-12-25 -1901-05-05 12:34:56.195,1901-12-25 -1901-05-05 12:34:56.1951,1901-12-25 -1901-05-05 12:34:56.1952,1901-12-25 -1901-05-05 12:34:56.1953,1901-12-25 -1901-05-05 12:34:56.1954,1901-12-25 -1901-05-05 12:34:56.1955,1901-12-25 -1901-05-05 12:34:56.1956,1901-12-25 -1901-05-05 12:34:56.1957,1901-12-25 -1901-05-05 12:34:56.1958,1901-12-25 -1901-05-05 12:34:56.1959,1901-12-25 -1901-05-05 12:34:56.196,1901-12-25 -1901-05-05 12:34:56.1961,1901-12-25 -1901-05-05 12:34:56.1962,1901-12-25 -1901-05-05 12:34:56.1963,1901-12-25 -1901-05-05 12:34:56.1964,1901-12-25 -1901-05-05 12:34:56.1965,1901-12-25 -1901-05-05 12:34:56.1966,1901-12-25 -1901-05-05 12:34:56.1967,1901-12-25 -1901-05-05 12:34:56.1968,1901-12-25 -1901-05-05 12:34:56.1969,1901-12-25 -1901-05-05 12:34:56.197,1901-12-25 -1901-05-05 12:34:56.1971,1901-12-25 -1901-05-05 12:34:56.1972,1901-12-25 -1901-05-05 12:34:56.1973,1901-12-25 -1901-05-05 12:34:56.1974,1901-12-25 -1901-05-05 12:34:56.1975,1901-12-25 -1901-05-05 12:34:56.1976,1901-12-25 -1901-05-05 12:34:56.1977,1901-12-25 -1901-05-05 12:34:56.1978,1901-12-25 -1901-05-05 12:34:56.1979,1901-12-25 -1901-05-05 12:34:56.198,1901-12-25 -1901-05-05 12:34:56.1981,1901-12-25 -1901-05-05 12:34:56.1982,1901-12-25 -1901-05-05 12:34:56.1983,1901-12-25 -1901-05-05 12:34:56.1984,1901-12-25 -1901-05-05 12:34:56.1985,1901-12-25 -1901-05-05 12:34:56.1986,1901-12-25 -1901-05-05 12:34:56.1987,1901-12-25 -1901-05-05 12:34:56.1988,1901-12-25 -1901-05-05 12:34:56.1989,1901-12-25 -1901-05-05 12:34:56.199,1901-12-25 -1901-05-05 12:34:56.1991,1901-12-25 -1901-05-05 12:34:56.1992,1901-12-25 -1901-05-05 12:34:56.1993,1901-12-25 -1901-05-05 12:34:56.1994,1901-12-25 -1901-05-05 12:34:56.1995,1901-12-25 -1901-05-05 12:34:56.1996,1901-12-25 -1901-05-05 12:34:56.1997,1901-12-25 -1901-05-05 12:34:56.1998,1901-12-25 -1901-05-05 12:34:56.1999,1901-12-25 -1902-05-05 12:34:56.1,1902-12-25 -1902-05-05 12:34:56.1001,1902-12-25 -1902-05-05 12:34:56.1002,1902-12-25 -1902-05-05 12:34:56.1003,1902-12-25 -1902-05-05 12:34:56.1004,1902-12-25 -1902-05-05 12:34:56.1005,1902-12-25 -1902-05-05 12:34:56.1006,1902-12-25 -1902-05-05 12:34:56.1007,1902-12-25 -1902-05-05 12:34:56.1008,1902-12-25 -1902-05-05 12:34:56.1009,1902-12-25 -1902-05-05 12:34:56.101,1902-12-25 -1902-05-05 12:34:56.1011,1902-12-25 -1902-05-05 12:34:56.1012,1902-12-25 -1902-05-05 12:34:56.1013,1902-12-25 -1902-05-05 12:34:56.1014,1902-12-25 -1902-05-05 12:34:56.1015,1902-12-25 -1902-05-05 12:34:56.1016,1902-12-25 -1902-05-05 12:34:56.1017,1902-12-25 -1902-05-05 12:34:56.1018,1902-12-25 -1902-05-05 12:34:56.1019,1902-12-25 -1902-05-05 12:34:56.102,1902-12-25 -1902-05-05 12:34:56.1021,1902-12-25 -1902-05-05 12:34:56.1022,1902-12-25 -1902-05-05 12:34:56.1023,1902-12-25 -1902-05-05 12:34:56.1024,1902-12-25 -1902-05-05 12:34:56.1025,1902-12-25 -1902-05-05 12:34:56.1026,1902-12-25 -1902-05-05 12:34:56.1027,1902-12-25 -1902-05-05 12:34:56.1028,1902-12-25 -1902-05-05 12:34:56.1029,1902-12-25 -1902-05-05 12:34:56.103,1902-12-25 -1902-05-05 12:34:56.1031,1902-12-25 -1902-05-05 12:34:56.1032,1902-12-25 -1902-05-05 12:34:56.1033,1902-12-25 -1902-05-05 12:34:56.1034,1902-12-25 -1902-05-05 12:34:56.1035,1902-12-25 -1902-05-05 12:34:56.1036,1902-12-25 -1902-05-05 12:34:56.1037,1902-12-25 -1902-05-05 12:34:56.1038,1902-12-25 -1902-05-05 12:34:56.1039,1902-12-25 -1902-05-05 12:34:56.104,1902-12-25 -1902-05-05 12:34:56.1041,1902-12-25 -1902-05-05 12:34:56.1042,1902-12-25 -1902-05-05 12:34:56.1043,1902-12-25 -1902-05-05 12:34:56.1044,1902-12-25 -1902-05-05 12:34:56.1045,1902-12-25 -1902-05-05 12:34:56.1046,1902-12-25 -1902-05-05 12:34:56.1047,1902-12-25 -1902-05-05 12:34:56.1048,1902-12-25 -1902-05-05 12:34:56.1049,1902-12-25 -1902-05-05 12:34:56.105,1902-12-25 -1902-05-05 12:34:56.1051,1902-12-25 -1902-05-05 12:34:56.1052,1902-12-25 -1902-05-05 12:34:56.1053,1902-12-25 -1902-05-05 12:34:56.1054,1902-12-25 -1902-05-05 12:34:56.1055,1902-12-25 -1902-05-05 12:34:56.1056,1902-12-25 -1902-05-05 12:34:56.1057,1902-12-25 -1902-05-05 12:34:56.1058,1902-12-25 -1902-05-05 12:34:56.1059,1902-12-25 -1902-05-05 12:34:56.106,1902-12-25 -1902-05-05 12:34:56.1061,1902-12-25 -1902-05-05 12:34:56.1062,1902-12-25 -1902-05-05 12:34:56.1063,1902-12-25 -1902-05-05 12:34:56.1064,1902-12-25 -1902-05-05 12:34:56.1065,1902-12-25 -1902-05-05 12:34:56.1066,1902-12-25 -1902-05-05 12:34:56.1067,1902-12-25 -1902-05-05 12:34:56.1068,1902-12-25 -1902-05-05 12:34:56.1069,1902-12-25 -1902-05-05 12:34:56.107,1902-12-25 -1902-05-05 12:34:56.1071,1902-12-25 -1902-05-05 12:34:56.1072,1902-12-25 -1902-05-05 12:34:56.1073,1902-12-25 -1902-05-05 12:34:56.1074,1902-12-25 -1902-05-05 12:34:56.1075,1902-12-25 -1902-05-05 12:34:56.1076,1902-12-25 -1902-05-05 12:34:56.1077,1902-12-25 -1902-05-05 12:34:56.1078,1902-12-25 -1902-05-05 12:34:56.1079,1902-12-25 -1902-05-05 12:34:56.108,1902-12-25 -1902-05-05 12:34:56.1081,1902-12-25 -1902-05-05 12:34:56.1082,1902-12-25 -1902-05-05 12:34:56.1083,1902-12-25 -1902-05-05 12:34:56.1084,1902-12-25 -1902-05-05 12:34:56.1085,1902-12-25 -1902-05-05 12:34:56.1086,1902-12-25 -1902-05-05 12:34:56.1087,1902-12-25 -1902-05-05 12:34:56.1088,1902-12-25 -1902-05-05 12:34:56.1089,1902-12-25 -1902-05-05 12:34:56.109,1902-12-25 -1902-05-05 12:34:56.1091,1902-12-25 -1902-05-05 12:34:56.1092,1902-12-25 -1902-05-05 12:34:56.1093,1902-12-25 -1902-05-05 12:34:56.1094,1902-12-25 -1902-05-05 12:34:56.1095,1902-12-25 -1902-05-05 12:34:56.1096,1902-12-25 -1902-05-05 12:34:56.1097,1902-12-25 -1902-05-05 12:34:56.1098,1902-12-25 -1902-05-05 12:34:56.1099,1902-12-25 -1902-05-05 12:34:56.11,1902-12-25 -1902-05-05 12:34:56.1101,1902-12-25 -1902-05-05 12:34:56.1102,1902-12-25 -1902-05-05 12:34:56.1103,1902-12-25 -1902-05-05 12:34:56.1104,1902-12-25 -1902-05-05 12:34:56.1105,1902-12-25 -1902-05-05 12:34:56.1106,1902-12-25 -1902-05-05 12:34:56.1107,1902-12-25 -1902-05-05 12:34:56.1108,1902-12-25 -1902-05-05 12:34:56.1109,1902-12-25 -1902-05-05 12:34:56.111,1902-12-25 -1902-05-05 12:34:56.1111,1902-12-25 -1902-05-05 12:34:56.1112,1902-12-25 -1902-05-05 12:34:56.1113,1902-12-25 -1902-05-05 12:34:56.1114,1902-12-25 -1902-05-05 12:34:56.1115,1902-12-25 -1902-05-05 12:34:56.1116,1902-12-25 -1902-05-05 12:34:56.1117,1902-12-25 -1902-05-05 12:34:56.1118,1902-12-25 -1902-05-05 12:34:56.1119,1902-12-25 -1902-05-05 12:34:56.112,1902-12-25 -1902-05-05 12:34:56.1121,1902-12-25 -1902-05-05 12:34:56.1122,1902-12-25 -1902-05-05 12:34:56.1123,1902-12-25 -1902-05-05 12:34:56.1124,1902-12-25 -1902-05-05 12:34:56.1125,1902-12-25 -1902-05-05 12:34:56.1126,1902-12-25 -1902-05-05 12:34:56.1127,1902-12-25 -1902-05-05 12:34:56.1128,1902-12-25 -1902-05-05 12:34:56.1129,1902-12-25 -1902-05-05 12:34:56.113,1902-12-25 -1902-05-05 12:34:56.1131,1902-12-25 -1902-05-05 12:34:56.1132,1902-12-25 -1902-05-05 12:34:56.1133,1902-12-25 -1902-05-05 12:34:56.1134,1902-12-25 -1902-05-05 12:34:56.1135,1902-12-25 -1902-05-05 12:34:56.1136,1902-12-25 -1902-05-05 12:34:56.1137,1902-12-25 -1902-05-05 12:34:56.1138,1902-12-25 -1902-05-05 12:34:56.1139,1902-12-25 -1902-05-05 12:34:56.114,1902-12-25 -1902-05-05 12:34:56.1141,1902-12-25 -1902-05-05 12:34:56.1142,1902-12-25 -1902-05-05 12:34:56.1143,1902-12-25 -1902-05-05 12:34:56.1144,1902-12-25 -1902-05-05 12:34:56.1145,1902-12-25 -1902-05-05 12:34:56.1146,1902-12-25 -1902-05-05 12:34:56.1147,1902-12-25 -1902-05-05 12:34:56.1148,1902-12-25 -1902-05-05 12:34:56.1149,1902-12-25 -1902-05-05 12:34:56.115,1902-12-25 -1902-05-05 12:34:56.1151,1902-12-25 -1902-05-05 12:34:56.1152,1902-12-25 -1902-05-05 12:34:56.1153,1902-12-25 -1902-05-05 12:34:56.1154,1902-12-25 -1902-05-05 12:34:56.1155,1902-12-25 -1902-05-05 12:34:56.1156,1902-12-25 -1902-05-05 12:34:56.1157,1902-12-25 -1902-05-05 12:34:56.1158,1902-12-25 -1902-05-05 12:34:56.1159,1902-12-25 -1902-05-05 12:34:56.116,1902-12-25 -1902-05-05 12:34:56.1161,1902-12-25 -1902-05-05 12:34:56.1162,1902-12-25 -1902-05-05 12:34:56.1163,1902-12-25 -1902-05-05 12:34:56.1164,1902-12-25 -1902-05-05 12:34:56.1165,1902-12-25 -1902-05-05 12:34:56.1166,1902-12-25 -1902-05-05 12:34:56.1167,1902-12-25 -1902-05-05 12:34:56.1168,1902-12-25 -1902-05-05 12:34:56.1169,1902-12-25 -1902-05-05 12:34:56.117,1902-12-25 -1902-05-05 12:34:56.1171,1902-12-25 -1902-05-05 12:34:56.1172,1902-12-25 -1902-05-05 12:34:56.1173,1902-12-25 -1902-05-05 12:34:56.1174,1902-12-25 -1902-05-05 12:34:56.1175,1902-12-25 -1902-05-05 12:34:56.1176,1902-12-25 -1902-05-05 12:34:56.1177,1902-12-25 -1902-05-05 12:34:56.1178,1902-12-25 -1902-05-05 12:34:56.1179,1902-12-25 -1902-05-05 12:34:56.118,1902-12-25 -1902-05-05 12:34:56.1181,1902-12-25 -1902-05-05 12:34:56.1182,1902-12-25 -1902-05-05 12:34:56.1183,1902-12-25 -1902-05-05 12:34:56.1184,1902-12-25 -1902-05-05 12:34:56.1185,1902-12-25 -1902-05-05 12:34:56.1186,1902-12-25 -1902-05-05 12:34:56.1187,1902-12-25 -1902-05-05 12:34:56.1188,1902-12-25 -1902-05-05 12:34:56.1189,1902-12-25 -1902-05-05 12:34:56.119,1902-12-25 -1902-05-05 12:34:56.1191,1902-12-25 -1902-05-05 12:34:56.1192,1902-12-25 -1902-05-05 12:34:56.1193,1902-12-25 -1902-05-05 12:34:56.1194,1902-12-25 -1902-05-05 12:34:56.1195,1902-12-25 -1902-05-05 12:34:56.1196,1902-12-25 -1902-05-05 12:34:56.1197,1902-12-25 -1902-05-05 12:34:56.1198,1902-12-25 -1902-05-05 12:34:56.1199,1902-12-25 -1902-05-05 12:34:56.12,1902-12-25 -1902-05-05 12:34:56.1201,1902-12-25 -1902-05-05 12:34:56.1202,1902-12-25 -1902-05-05 12:34:56.1203,1902-12-25 -1902-05-05 12:34:56.1204,1902-12-25 -1902-05-05 12:34:56.1205,1902-12-25 -1902-05-05 12:34:56.1206,1902-12-25 -1902-05-05 12:34:56.1207,1902-12-25 -1902-05-05 12:34:56.1208,1902-12-25 -1902-05-05 12:34:56.1209,1902-12-25 -1902-05-05 12:34:56.121,1902-12-25 -1902-05-05 12:34:56.1211,1902-12-25 -1902-05-05 12:34:56.1212,1902-12-25 -1902-05-05 12:34:56.1213,1902-12-25 -1902-05-05 12:34:56.1214,1902-12-25 -1902-05-05 12:34:56.1215,1902-12-25 -1902-05-05 12:34:56.1216,1902-12-25 -1902-05-05 12:34:56.1217,1902-12-25 -1902-05-05 12:34:56.1218,1902-12-25 -1902-05-05 12:34:56.1219,1902-12-25 -1902-05-05 12:34:56.122,1902-12-25 -1902-05-05 12:34:56.1221,1902-12-25 -1902-05-05 12:34:56.1222,1902-12-25 -1902-05-05 12:34:56.1223,1902-12-25 -1902-05-05 12:34:56.1224,1902-12-25 -1902-05-05 12:34:56.1225,1902-12-25 -1902-05-05 12:34:56.1226,1902-12-25 -1902-05-05 12:34:56.1227,1902-12-25 -1902-05-05 12:34:56.1228,1902-12-25 -1902-05-05 12:34:56.1229,1902-12-25 -1902-05-05 12:34:56.123,1902-12-25 -1902-05-05 12:34:56.1231,1902-12-25 -1902-05-05 12:34:56.1232,1902-12-25 -1902-05-05 12:34:56.1233,1902-12-25 -1902-05-05 12:34:56.1234,1902-12-25 -1902-05-05 12:34:56.1235,1902-12-25 -1902-05-05 12:34:56.1236,1902-12-25 -1902-05-05 12:34:56.1237,1902-12-25 -1902-05-05 12:34:56.1238,1902-12-25 -1902-05-05 12:34:56.1239,1902-12-25 -1902-05-05 12:34:56.124,1902-12-25 -1902-05-05 12:34:56.1241,1902-12-25 -1902-05-05 12:34:56.1242,1902-12-25 -1902-05-05 12:34:56.1243,1902-12-25 -1902-05-05 12:34:56.1244,1902-12-25 -1902-05-05 12:34:56.1245,1902-12-25 -1902-05-05 12:34:56.1246,1902-12-25 -1902-05-05 12:34:56.1247,1902-12-25 -1902-05-05 12:34:56.1248,1902-12-25 -1902-05-05 12:34:56.1249,1902-12-25 -1902-05-05 12:34:56.125,1902-12-25 -1902-05-05 12:34:56.1251,1902-12-25 -1902-05-05 12:34:56.1252,1902-12-25 -1902-05-05 12:34:56.1253,1902-12-25 -1902-05-05 12:34:56.1254,1902-12-25 -1902-05-05 12:34:56.1255,1902-12-25 -1902-05-05 12:34:56.1256,1902-12-25 -1902-05-05 12:34:56.1257,1902-12-25 -1902-05-05 12:34:56.1258,1902-12-25 -1902-05-05 12:34:56.1259,1902-12-25 -1902-05-05 12:34:56.126,1902-12-25 -1902-05-05 12:34:56.1261,1902-12-25 -1902-05-05 12:34:56.1262,1902-12-25 -1902-05-05 12:34:56.1263,1902-12-25 -1902-05-05 12:34:56.1264,1902-12-25 -1902-05-05 12:34:56.1265,1902-12-25 -1902-05-05 12:34:56.1266,1902-12-25 -1902-05-05 12:34:56.1267,1902-12-25 -1902-05-05 12:34:56.1268,1902-12-25 -1902-05-05 12:34:56.1269,1902-12-25 -1902-05-05 12:34:56.127,1902-12-25 -1902-05-05 12:34:56.1271,1902-12-25 -1902-05-05 12:34:56.1272,1902-12-25 -1902-05-05 12:34:56.1273,1902-12-25 -1902-05-05 12:34:56.1274,1902-12-25 -1902-05-05 12:34:56.1275,1902-12-25 -1902-05-05 12:34:56.1276,1902-12-25 -1902-05-05 12:34:56.1277,1902-12-25 -1902-05-05 12:34:56.1278,1902-12-25 -1902-05-05 12:34:56.1279,1902-12-25 -1902-05-05 12:34:56.128,1902-12-25 -1902-05-05 12:34:56.1281,1902-12-25 -1902-05-05 12:34:56.1282,1902-12-25 -1902-05-05 12:34:56.1283,1902-12-25 -1902-05-05 12:34:56.1284,1902-12-25 -1902-05-05 12:34:56.1285,1902-12-25 -1902-05-05 12:34:56.1286,1902-12-25 -1902-05-05 12:34:56.1287,1902-12-25 -1902-05-05 12:34:56.1288,1902-12-25 -1902-05-05 12:34:56.1289,1902-12-25 -1902-05-05 12:34:56.129,1902-12-25 -1902-05-05 12:34:56.1291,1902-12-25 -1902-05-05 12:34:56.1292,1902-12-25 -1902-05-05 12:34:56.1293,1902-12-25 -1902-05-05 12:34:56.1294,1902-12-25 -1902-05-05 12:34:56.1295,1902-12-25 -1902-05-05 12:34:56.1296,1902-12-25 -1902-05-05 12:34:56.1297,1902-12-25 -1902-05-05 12:34:56.1298,1902-12-25 -1902-05-05 12:34:56.1299,1902-12-25 -1902-05-05 12:34:56.13,1902-12-25 -1902-05-05 12:34:56.1301,1902-12-25 -1902-05-05 12:34:56.1302,1902-12-25 -1902-05-05 12:34:56.1303,1902-12-25 -1902-05-05 12:34:56.1304,1902-12-25 -1902-05-05 12:34:56.1305,1902-12-25 -1902-05-05 12:34:56.1306,1902-12-25 -1902-05-05 12:34:56.1307,1902-12-25 -1902-05-05 12:34:56.1308,1902-12-25 -1902-05-05 12:34:56.1309,1902-12-25 -1902-05-05 12:34:56.131,1902-12-25 -1902-05-05 12:34:56.1311,1902-12-25 -1902-05-05 12:34:56.1312,1902-12-25 -1902-05-05 12:34:56.1313,1902-12-25 -1902-05-05 12:34:56.1314,1902-12-25 -1902-05-05 12:34:56.1315,1902-12-25 -1902-05-05 12:34:56.1316,1902-12-25 -1902-05-05 12:34:56.1317,1902-12-25 -1902-05-05 12:34:56.1318,1902-12-25 -1902-05-05 12:34:56.1319,1902-12-25 -1902-05-05 12:34:56.132,1902-12-25 -1902-05-05 12:34:56.1321,1902-12-25 -1902-05-05 12:34:56.1322,1902-12-25 -1902-05-05 12:34:56.1323,1902-12-25 -1902-05-05 12:34:56.1324,1902-12-25 -1902-05-05 12:34:56.1325,1902-12-25 -1902-05-05 12:34:56.1326,1902-12-25 -1902-05-05 12:34:56.1327,1902-12-25 -1902-05-05 12:34:56.1328,1902-12-25 -1902-05-05 12:34:56.1329,1902-12-25 -1902-05-05 12:34:56.133,1902-12-25 -1902-05-05 12:34:56.1331,1902-12-25 -1902-05-05 12:34:56.1332,1902-12-25 -1902-05-05 12:34:56.1333,1902-12-25 -1902-05-05 12:34:56.1334,1902-12-25 -1902-05-05 12:34:56.1335,1902-12-25 -1902-05-05 12:34:56.1336,1902-12-25 -1902-05-05 12:34:56.1337,1902-12-25 -1902-05-05 12:34:56.1338,1902-12-25 -1902-05-05 12:34:56.1339,1902-12-25 -1902-05-05 12:34:56.134,1902-12-25 -1902-05-05 12:34:56.1341,1902-12-25 -1902-05-05 12:34:56.1342,1902-12-25 -1902-05-05 12:34:56.1343,1902-12-25 -1902-05-05 12:34:56.1344,1902-12-25 -1902-05-05 12:34:56.1345,1902-12-25 -1902-05-05 12:34:56.1346,1902-12-25 -1902-05-05 12:34:56.1347,1902-12-25 -1902-05-05 12:34:56.1348,1902-12-25 -1902-05-05 12:34:56.1349,1902-12-25 -1902-05-05 12:34:56.135,1902-12-25 -1902-05-05 12:34:56.1351,1902-12-25 -1902-05-05 12:34:56.1352,1902-12-25 -1902-05-05 12:34:56.1353,1902-12-25 -1902-05-05 12:34:56.1354,1902-12-25 -1902-05-05 12:34:56.1355,1902-12-25 -1902-05-05 12:34:56.1356,1902-12-25 -1902-05-05 12:34:56.1357,1902-12-25 -1902-05-05 12:34:56.1358,1902-12-25 -1902-05-05 12:34:56.1359,1902-12-25 -1902-05-05 12:34:56.136,1902-12-25 -1902-05-05 12:34:56.1361,1902-12-25 -1902-05-05 12:34:56.1362,1902-12-25 -1902-05-05 12:34:56.1363,1902-12-25 -1902-05-05 12:34:56.1364,1902-12-25 -1902-05-05 12:34:56.1365,1902-12-25 -1902-05-05 12:34:56.1366,1902-12-25 -1902-05-05 12:34:56.1367,1902-12-25 -1902-05-05 12:34:56.1368,1902-12-25 -1902-05-05 12:34:56.1369,1902-12-25 -1902-05-05 12:34:56.137,1902-12-25 -1902-05-05 12:34:56.1371,1902-12-25 -1902-05-05 12:34:56.1372,1902-12-25 -1902-05-05 12:34:56.1373,1902-12-25 -1902-05-05 12:34:56.1374,1902-12-25 -1902-05-05 12:34:56.1375,1902-12-25 -1902-05-05 12:34:56.1376,1902-12-25 -1902-05-05 12:34:56.1377,1902-12-25 -1902-05-05 12:34:56.1378,1902-12-25 -1902-05-05 12:34:56.1379,1902-12-25 -1902-05-05 12:34:56.138,1902-12-25 -1902-05-05 12:34:56.1381,1902-12-25 -1902-05-05 12:34:56.1382,1902-12-25 -1902-05-05 12:34:56.1383,1902-12-25 -1902-05-05 12:34:56.1384,1902-12-25 -1902-05-05 12:34:56.1385,1902-12-25 -1902-05-05 12:34:56.1386,1902-12-25 -1902-05-05 12:34:56.1387,1902-12-25 -1902-05-05 12:34:56.1388,1902-12-25 -1902-05-05 12:34:56.1389,1902-12-25 -1902-05-05 12:34:56.139,1902-12-25 -1902-05-05 12:34:56.1391,1902-12-25 -1902-05-05 12:34:56.1392,1902-12-25 -1902-05-05 12:34:56.1393,1902-12-25 -1902-05-05 12:34:56.1394,1902-12-25 -1902-05-05 12:34:56.1395,1902-12-25 -1902-05-05 12:34:56.1396,1902-12-25 -1902-05-05 12:34:56.1397,1902-12-25 -1902-05-05 12:34:56.1398,1902-12-25 -1902-05-05 12:34:56.1399,1902-12-25 -1902-05-05 12:34:56.14,1902-12-25 -1902-05-05 12:34:56.1401,1902-12-25 -1902-05-05 12:34:56.1402,1902-12-25 -1902-05-05 12:34:56.1403,1902-12-25 -1902-05-05 12:34:56.1404,1902-12-25 -1902-05-05 12:34:56.1405,1902-12-25 -1902-05-05 12:34:56.1406,1902-12-25 -1902-05-05 12:34:56.1407,1902-12-25 -1902-05-05 12:34:56.1408,1902-12-25 -1902-05-05 12:34:56.1409,1902-12-25 -1902-05-05 12:34:56.141,1902-12-25 -1902-05-05 12:34:56.1411,1902-12-25 -1902-05-05 12:34:56.1412,1902-12-25 -1902-05-05 12:34:56.1413,1902-12-25 -1902-05-05 12:34:56.1414,1902-12-25 -1902-05-05 12:34:56.1415,1902-12-25 -1902-05-05 12:34:56.1416,1902-12-25 -1902-05-05 12:34:56.1417,1902-12-25 -1902-05-05 12:34:56.1418,1902-12-25 -1902-05-05 12:34:56.1419,1902-12-25 -1902-05-05 12:34:56.142,1902-12-25 -1902-05-05 12:34:56.1421,1902-12-25 -1902-05-05 12:34:56.1422,1902-12-25 -1902-05-05 12:34:56.1423,1902-12-25 -1902-05-05 12:34:56.1424,1902-12-25 -1902-05-05 12:34:56.1425,1902-12-25 -1902-05-05 12:34:56.1426,1902-12-25 -1902-05-05 12:34:56.1427,1902-12-25 -1902-05-05 12:34:56.1428,1902-12-25 -1902-05-05 12:34:56.1429,1902-12-25 -1902-05-05 12:34:56.143,1902-12-25 -1902-05-05 12:34:56.1431,1902-12-25 -1902-05-05 12:34:56.1432,1902-12-25 -1902-05-05 12:34:56.1433,1902-12-25 -1902-05-05 12:34:56.1434,1902-12-25 -1902-05-05 12:34:56.1435,1902-12-25 -1902-05-05 12:34:56.1436,1902-12-25 -1902-05-05 12:34:56.1437,1902-12-25 -1902-05-05 12:34:56.1438,1902-12-25 -1902-05-05 12:34:56.1439,1902-12-25 -1902-05-05 12:34:56.144,1902-12-25 -1902-05-05 12:34:56.1441,1902-12-25 -1902-05-05 12:34:56.1442,1902-12-25 -1902-05-05 12:34:56.1443,1902-12-25 -1902-05-05 12:34:56.1444,1902-12-25 -1902-05-05 12:34:56.1445,1902-12-25 -1902-05-05 12:34:56.1446,1902-12-25 -1902-05-05 12:34:56.1447,1902-12-25 -1902-05-05 12:34:56.1448,1902-12-25 -1902-05-05 12:34:56.1449,1902-12-25 -1902-05-05 12:34:56.145,1902-12-25 -1902-05-05 12:34:56.1451,1902-12-25 -1902-05-05 12:34:56.1452,1902-12-25 -1902-05-05 12:34:56.1453,1902-12-25 -1902-05-05 12:34:56.1454,1902-12-25 -1902-05-05 12:34:56.1455,1902-12-25 -1902-05-05 12:34:56.1456,1902-12-25 -1902-05-05 12:34:56.1457,1902-12-25 -1902-05-05 12:34:56.1458,1902-12-25 -1902-05-05 12:34:56.1459,1902-12-25 -1902-05-05 12:34:56.146,1902-12-25 -1902-05-05 12:34:56.1461,1902-12-25 -1902-05-05 12:34:56.1462,1902-12-25 -1902-05-05 12:34:56.1463,1902-12-25 -1902-05-05 12:34:56.1464,1902-12-25 -1902-05-05 12:34:56.1465,1902-12-25 -1902-05-05 12:34:56.1466,1902-12-25 -1902-05-05 12:34:56.1467,1902-12-25 -1902-05-05 12:34:56.1468,1902-12-25 -1902-05-05 12:34:56.1469,1902-12-25 -1902-05-05 12:34:56.147,1902-12-25 -1902-05-05 12:34:56.1471,1902-12-25 -1902-05-05 12:34:56.1472,1902-12-25 -1902-05-05 12:34:56.1473,1902-12-25 -1902-05-05 12:34:56.1474,1902-12-25 -1902-05-05 12:34:56.1475,1902-12-25 -1902-05-05 12:34:56.1476,1902-12-25 -1902-05-05 12:34:56.1477,1902-12-25 -1902-05-05 12:34:56.1478,1902-12-25 -1902-05-05 12:34:56.1479,1902-12-25 -1902-05-05 12:34:56.148,1902-12-25 -1902-05-05 12:34:56.1481,1902-12-25 -1902-05-05 12:34:56.1482,1902-12-25 -1902-05-05 12:34:56.1483,1902-12-25 -1902-05-05 12:34:56.1484,1902-12-25 -1902-05-05 12:34:56.1485,1902-12-25 -1902-05-05 12:34:56.1486,1902-12-25 -1902-05-05 12:34:56.1487,1902-12-25 -1902-05-05 12:34:56.1488,1902-12-25 -1902-05-05 12:34:56.1489,1902-12-25 -1902-05-05 12:34:56.149,1902-12-25 -1902-05-05 12:34:56.1491,1902-12-25 -1902-05-05 12:34:56.1492,1902-12-25 -1902-05-05 12:34:56.1493,1902-12-25 -1902-05-05 12:34:56.1494,1902-12-25 -1902-05-05 12:34:56.1495,1902-12-25 -1902-05-05 12:34:56.1496,1902-12-25 -1902-05-05 12:34:56.1497,1902-12-25 -1902-05-05 12:34:56.1498,1902-12-25 -1902-05-05 12:34:56.1499,1902-12-25 -1902-05-05 12:34:56.15,1902-12-25 -1902-05-05 12:34:56.1501,1902-12-25 -1902-05-05 12:34:56.1502,1902-12-25 -1902-05-05 12:34:56.1503,1902-12-25 -1902-05-05 12:34:56.1504,1902-12-25 -1902-05-05 12:34:56.1505,1902-12-25 -1902-05-05 12:34:56.1506,1902-12-25 -1902-05-05 12:34:56.1507,1902-12-25 -1902-05-05 12:34:56.1508,1902-12-25 -1902-05-05 12:34:56.1509,1902-12-25 -1902-05-05 12:34:56.151,1902-12-25 -1902-05-05 12:34:56.1511,1902-12-25 -1902-05-05 12:34:56.1512,1902-12-25 -1902-05-05 12:34:56.1513,1902-12-25 -1902-05-05 12:34:56.1514,1902-12-25 -1902-05-05 12:34:56.1515,1902-12-25 -1902-05-05 12:34:56.1516,1902-12-25 -1902-05-05 12:34:56.1517,1902-12-25 -1902-05-05 12:34:56.1518,1902-12-25 -1902-05-05 12:34:56.1519,1902-12-25 -1902-05-05 12:34:56.152,1902-12-25 -1902-05-05 12:34:56.1521,1902-12-25 -1902-05-05 12:34:56.1522,1902-12-25 -1902-05-05 12:34:56.1523,1902-12-25 -1902-05-05 12:34:56.1524,1902-12-25 -1902-05-05 12:34:56.1525,1902-12-25 -1902-05-05 12:34:56.1526,1902-12-25 -1902-05-05 12:34:56.1527,1902-12-25 -1902-05-05 12:34:56.1528,1902-12-25 -1902-05-05 12:34:56.1529,1902-12-25 -1902-05-05 12:34:56.153,1902-12-25 -1902-05-05 12:34:56.1531,1902-12-25 -1902-05-05 12:34:56.1532,1902-12-25 -1902-05-05 12:34:56.1533,1902-12-25 -1902-05-05 12:34:56.1534,1902-12-25 -1902-05-05 12:34:56.1535,1902-12-25 -1902-05-05 12:34:56.1536,1902-12-25 -1902-05-05 12:34:56.1537,1902-12-25 -1902-05-05 12:34:56.1538,1902-12-25 -1902-05-05 12:34:56.1539,1902-12-25 -1902-05-05 12:34:56.154,1902-12-25 -1902-05-05 12:34:56.1541,1902-12-25 -1902-05-05 12:34:56.1542,1902-12-25 -1902-05-05 12:34:56.1543,1902-12-25 -1902-05-05 12:34:56.1544,1902-12-25 -1902-05-05 12:34:56.1545,1902-12-25 -1902-05-05 12:34:56.1546,1902-12-25 -1902-05-05 12:34:56.1547,1902-12-25 -1902-05-05 12:34:56.1548,1902-12-25 -1902-05-05 12:34:56.1549,1902-12-25 -1902-05-05 12:34:56.155,1902-12-25 -1902-05-05 12:34:56.1551,1902-12-25 -1902-05-05 12:34:56.1552,1902-12-25 -1902-05-05 12:34:56.1553,1902-12-25 -1902-05-05 12:34:56.1554,1902-12-25 -1902-05-05 12:34:56.1555,1902-12-25 -1902-05-05 12:34:56.1556,1902-12-25 -1902-05-05 12:34:56.1557,1902-12-25 -1902-05-05 12:34:56.1558,1902-12-25 -1902-05-05 12:34:56.1559,1902-12-25 -1902-05-05 12:34:56.156,1902-12-25 -1902-05-05 12:34:56.1561,1902-12-25 -1902-05-05 12:34:56.1562,1902-12-25 -1902-05-05 12:34:56.1563,1902-12-25 -1902-05-05 12:34:56.1564,1902-12-25 -1902-05-05 12:34:56.1565,1902-12-25 -1902-05-05 12:34:56.1566,1902-12-25 -1902-05-05 12:34:56.1567,1902-12-25 -1902-05-05 12:34:56.1568,1902-12-25 -1902-05-05 12:34:56.1569,1902-12-25 -1902-05-05 12:34:56.157,1902-12-25 -1902-05-05 12:34:56.1571,1902-12-25 -1902-05-05 12:34:56.1572,1902-12-25 -1902-05-05 12:34:56.1573,1902-12-25 -1902-05-05 12:34:56.1574,1902-12-25 -1902-05-05 12:34:56.1575,1902-12-25 -1902-05-05 12:34:56.1576,1902-12-25 -1902-05-05 12:34:56.1577,1902-12-25 -1902-05-05 12:34:56.1578,1902-12-25 -1902-05-05 12:34:56.1579,1902-12-25 -1902-05-05 12:34:56.158,1902-12-25 -1902-05-05 12:34:56.1581,1902-12-25 -1902-05-05 12:34:56.1582,1902-12-25 -1902-05-05 12:34:56.1583,1902-12-25 -1902-05-05 12:34:56.1584,1902-12-25 -1902-05-05 12:34:56.1585,1902-12-25 -1902-05-05 12:34:56.1586,1902-12-25 -1902-05-05 12:34:56.1587,1902-12-25 -1902-05-05 12:34:56.1588,1902-12-25 -1902-05-05 12:34:56.1589,1902-12-25 -1902-05-05 12:34:56.159,1902-12-25 -1902-05-05 12:34:56.1591,1902-12-25 -1902-05-05 12:34:56.1592,1902-12-25 -1902-05-05 12:34:56.1593,1902-12-25 -1902-05-05 12:34:56.1594,1902-12-25 -1902-05-05 12:34:56.1595,1902-12-25 -1902-05-05 12:34:56.1596,1902-12-25 -1902-05-05 12:34:56.1597,1902-12-25 -1902-05-05 12:34:56.1598,1902-12-25 -1902-05-05 12:34:56.1599,1902-12-25 -1902-05-05 12:34:56.16,1902-12-25 -1902-05-05 12:34:56.1601,1902-12-25 -1902-05-05 12:34:56.1602,1902-12-25 -1902-05-05 12:34:56.1603,1902-12-25 -1902-05-05 12:34:56.1604,1902-12-25 -1902-05-05 12:34:56.1605,1902-12-25 -1902-05-05 12:34:56.1606,1902-12-25 -1902-05-05 12:34:56.1607,1902-12-25 -1902-05-05 12:34:56.1608,1902-12-25 -1902-05-05 12:34:56.1609,1902-12-25 -1902-05-05 12:34:56.161,1902-12-25 -1902-05-05 12:34:56.1611,1902-12-25 -1902-05-05 12:34:56.1612,1902-12-25 -1902-05-05 12:34:56.1613,1902-12-25 -1902-05-05 12:34:56.1614,1902-12-25 -1902-05-05 12:34:56.1615,1902-12-25 -1902-05-05 12:34:56.1616,1902-12-25 -1902-05-05 12:34:56.1617,1902-12-25 -1902-05-05 12:34:56.1618,1902-12-25 -1902-05-05 12:34:56.1619,1902-12-25 -1902-05-05 12:34:56.162,1902-12-25 -1902-05-05 12:34:56.1621,1902-12-25 -1902-05-05 12:34:56.1622,1902-12-25 -1902-05-05 12:34:56.1623,1902-12-25 -1902-05-05 12:34:56.1624,1902-12-25 -1902-05-05 12:34:56.1625,1902-12-25 -1902-05-05 12:34:56.1626,1902-12-25 -1902-05-05 12:34:56.1627,1902-12-25 -1902-05-05 12:34:56.1628,1902-12-25 -1902-05-05 12:34:56.1629,1902-12-25 -1902-05-05 12:34:56.163,1902-12-25 -1902-05-05 12:34:56.1631,1902-12-25 -1902-05-05 12:34:56.1632,1902-12-25 -1902-05-05 12:34:56.1633,1902-12-25 -1902-05-05 12:34:56.1634,1902-12-25 -1902-05-05 12:34:56.1635,1902-12-25 -1902-05-05 12:34:56.1636,1902-12-25 -1902-05-05 12:34:56.1637,1902-12-25 -1902-05-05 12:34:56.1638,1902-12-25 -1902-05-05 12:34:56.1639,1902-12-25 -1902-05-05 12:34:56.164,1902-12-25 -1902-05-05 12:34:56.1641,1902-12-25 -1902-05-05 12:34:56.1642,1902-12-25 -1902-05-05 12:34:56.1643,1902-12-25 -1902-05-05 12:34:56.1644,1902-12-25 -1902-05-05 12:34:56.1645,1902-12-25 -1902-05-05 12:34:56.1646,1902-12-25 -1902-05-05 12:34:56.1647,1902-12-25 -1902-05-05 12:34:56.1648,1902-12-25 -1902-05-05 12:34:56.1649,1902-12-25 -1902-05-05 12:34:56.165,1902-12-25 -1902-05-05 12:34:56.1651,1902-12-25 -1902-05-05 12:34:56.1652,1902-12-25 -1902-05-05 12:34:56.1653,1902-12-25 -1902-05-05 12:34:56.1654,1902-12-25 -1902-05-05 12:34:56.1655,1902-12-25 -1902-05-05 12:34:56.1656,1902-12-25 -1902-05-05 12:34:56.1657,1902-12-25 -1902-05-05 12:34:56.1658,1902-12-25 -1902-05-05 12:34:56.1659,1902-12-25 -1902-05-05 12:34:56.166,1902-12-25 -1902-05-05 12:34:56.1661,1902-12-25 -1902-05-05 12:34:56.1662,1902-12-25 -1902-05-05 12:34:56.1663,1902-12-25 -1902-05-05 12:34:56.1664,1902-12-25 -1902-05-05 12:34:56.1665,1902-12-25 -1902-05-05 12:34:56.1666,1902-12-25 -1902-05-05 12:34:56.1667,1902-12-25 -1902-05-05 12:34:56.1668,1902-12-25 -1902-05-05 12:34:56.1669,1902-12-25 -1902-05-05 12:34:56.167,1902-12-25 -1902-05-05 12:34:56.1671,1902-12-25 -1902-05-05 12:34:56.1672,1902-12-25 -1902-05-05 12:34:56.1673,1902-12-25 -1902-05-05 12:34:56.1674,1902-12-25 -1902-05-05 12:34:56.1675,1902-12-25 -1902-05-05 12:34:56.1676,1902-12-25 -1902-05-05 12:34:56.1677,1902-12-25 -1902-05-05 12:34:56.1678,1902-12-25 -1902-05-05 12:34:56.1679,1902-12-25 -1902-05-05 12:34:56.168,1902-12-25 -1902-05-05 12:34:56.1681,1902-12-25 -1902-05-05 12:34:56.1682,1902-12-25 -1902-05-05 12:34:56.1683,1902-12-25 -1902-05-05 12:34:56.1684,1902-12-25 -1902-05-05 12:34:56.1685,1902-12-25 -1902-05-05 12:34:56.1686,1902-12-25 -1902-05-05 12:34:56.1687,1902-12-25 -1902-05-05 12:34:56.1688,1902-12-25 -1902-05-05 12:34:56.1689,1902-12-25 -1902-05-05 12:34:56.169,1902-12-25 -1902-05-05 12:34:56.1691,1902-12-25 -1902-05-05 12:34:56.1692,1902-12-25 -1902-05-05 12:34:56.1693,1902-12-25 -1902-05-05 12:34:56.1694,1902-12-25 -1902-05-05 12:34:56.1695,1902-12-25 -1902-05-05 12:34:56.1696,1902-12-25 -1902-05-05 12:34:56.1697,1902-12-25 -1902-05-05 12:34:56.1698,1902-12-25 -1902-05-05 12:34:56.1699,1902-12-25 -1902-05-05 12:34:56.17,1902-12-25 -1902-05-05 12:34:56.1701,1902-12-25 -1902-05-05 12:34:56.1702,1902-12-25 -1902-05-05 12:34:56.1703,1902-12-25 -1902-05-05 12:34:56.1704,1902-12-25 -1902-05-05 12:34:56.1705,1902-12-25 -1902-05-05 12:34:56.1706,1902-12-25 -1902-05-05 12:34:56.1707,1902-12-25 -1902-05-05 12:34:56.1708,1902-12-25 -1902-05-05 12:34:56.1709,1902-12-25 -1902-05-05 12:34:56.171,1902-12-25 -1902-05-05 12:34:56.1711,1902-12-25 -1902-05-05 12:34:56.1712,1902-12-25 -1902-05-05 12:34:56.1713,1902-12-25 -1902-05-05 12:34:56.1714,1902-12-25 -1902-05-05 12:34:56.1715,1902-12-25 -1902-05-05 12:34:56.1716,1902-12-25 -1902-05-05 12:34:56.1717,1902-12-25 -1902-05-05 12:34:56.1718,1902-12-25 -1902-05-05 12:34:56.1719,1902-12-25 -1902-05-05 12:34:56.172,1902-12-25 -1902-05-05 12:34:56.1721,1902-12-25 -1902-05-05 12:34:56.1722,1902-12-25 -1902-05-05 12:34:56.1723,1902-12-25 -1902-05-05 12:34:56.1724,1902-12-25 -1902-05-05 12:34:56.1725,1902-12-25 -1902-05-05 12:34:56.1726,1902-12-25 -1902-05-05 12:34:56.1727,1902-12-25 -1902-05-05 12:34:56.1728,1902-12-25 -1902-05-05 12:34:56.1729,1902-12-25 -1902-05-05 12:34:56.173,1902-12-25 -1902-05-05 12:34:56.1731,1902-12-25 -1902-05-05 12:34:56.1732,1902-12-25 -1902-05-05 12:34:56.1733,1902-12-25 -1902-05-05 12:34:56.1734,1902-12-25 -1902-05-05 12:34:56.1735,1902-12-25 -1902-05-05 12:34:56.1736,1902-12-25 -1902-05-05 12:34:56.1737,1902-12-25 -1902-05-05 12:34:56.1738,1902-12-25 -1902-05-05 12:34:56.1739,1902-12-25 -1902-05-05 12:34:56.174,1902-12-25 -1902-05-05 12:34:56.1741,1902-12-25 -1902-05-05 12:34:56.1742,1902-12-25 -1902-05-05 12:34:56.1743,1902-12-25 -1902-05-05 12:34:56.1744,1902-12-25 -1902-05-05 12:34:56.1745,1902-12-25 -1902-05-05 12:34:56.1746,1902-12-25 -1902-05-05 12:34:56.1747,1902-12-25 -1902-05-05 12:34:56.1748,1902-12-25 -1902-05-05 12:34:56.1749,1902-12-25 -1902-05-05 12:34:56.175,1902-12-25 -1902-05-05 12:34:56.1751,1902-12-25 -1902-05-05 12:34:56.1752,1902-12-25 -1902-05-05 12:34:56.1753,1902-12-25 -1902-05-05 12:34:56.1754,1902-12-25 -1902-05-05 12:34:56.1755,1902-12-25 -1902-05-05 12:34:56.1756,1902-12-25 -1902-05-05 12:34:56.1757,1902-12-25 -1902-05-05 12:34:56.1758,1902-12-25 -1902-05-05 12:34:56.1759,1902-12-25 -1902-05-05 12:34:56.176,1902-12-25 -1902-05-05 12:34:56.1761,1902-12-25 -1902-05-05 12:34:56.1762,1902-12-25 -1902-05-05 12:34:56.1763,1902-12-25 -1902-05-05 12:34:56.1764,1902-12-25 -1902-05-05 12:34:56.1765,1902-12-25 -1902-05-05 12:34:56.1766,1902-12-25 -1902-05-05 12:34:56.1767,1902-12-25 -1902-05-05 12:34:56.1768,1902-12-25 -1902-05-05 12:34:56.1769,1902-12-25 -1902-05-05 12:34:56.177,1902-12-25 -1902-05-05 12:34:56.1771,1902-12-25 -1902-05-05 12:34:56.1772,1902-12-25 -1902-05-05 12:34:56.1773,1902-12-25 -1902-05-05 12:34:56.1774,1902-12-25 -1902-05-05 12:34:56.1775,1902-12-25 -1902-05-05 12:34:56.1776,1902-12-25 -1902-05-05 12:34:56.1777,1902-12-25 -1902-05-05 12:34:56.1778,1902-12-25 -1902-05-05 12:34:56.1779,1902-12-25 -1902-05-05 12:34:56.178,1902-12-25 -1902-05-05 12:34:56.1781,1902-12-25 -1902-05-05 12:34:56.1782,1902-12-25 -1902-05-05 12:34:56.1783,1902-12-25 -1902-05-05 12:34:56.1784,1902-12-25 -1902-05-05 12:34:56.1785,1902-12-25 -1902-05-05 12:34:56.1786,1902-12-25 -1902-05-05 12:34:56.1787,1902-12-25 -1902-05-05 12:34:56.1788,1902-12-25 -1902-05-05 12:34:56.1789,1902-12-25 -1902-05-05 12:34:56.179,1902-12-25 -1902-05-05 12:34:56.1791,1902-12-25 -1902-05-05 12:34:56.1792,1902-12-25 -1902-05-05 12:34:56.1793,1902-12-25 -1902-05-05 12:34:56.1794,1902-12-25 -1902-05-05 12:34:56.1795,1902-12-25 -1902-05-05 12:34:56.1796,1902-12-25 -1902-05-05 12:34:56.1797,1902-12-25 -1902-05-05 12:34:56.1798,1902-12-25 -1902-05-05 12:34:56.1799,1902-12-25 -1902-05-05 12:34:56.18,1902-12-25 -1902-05-05 12:34:56.1801,1902-12-25 -1902-05-05 12:34:56.1802,1902-12-25 -1902-05-05 12:34:56.1803,1902-12-25 -1902-05-05 12:34:56.1804,1902-12-25 -1902-05-05 12:34:56.1805,1902-12-25 -1902-05-05 12:34:56.1806,1902-12-25 -1902-05-05 12:34:56.1807,1902-12-25 -1902-05-05 12:34:56.1808,1902-12-25 -1902-05-05 12:34:56.1809,1902-12-25 -1902-05-05 12:34:56.181,1902-12-25 -1902-05-05 12:34:56.1811,1902-12-25 -1902-05-05 12:34:56.1812,1902-12-25 -1902-05-05 12:34:56.1813,1902-12-25 -1902-05-05 12:34:56.1814,1902-12-25 -1902-05-05 12:34:56.1815,1902-12-25 -1902-05-05 12:34:56.1816,1902-12-25 -1902-05-05 12:34:56.1817,1902-12-25 -1902-05-05 12:34:56.1818,1902-12-25 -1902-05-05 12:34:56.1819,1902-12-25 -1902-05-05 12:34:56.182,1902-12-25 -1902-05-05 12:34:56.1821,1902-12-25 -1902-05-05 12:34:56.1822,1902-12-25 -1902-05-05 12:34:56.1823,1902-12-25 -1902-05-05 12:34:56.1824,1902-12-25 -1902-05-05 12:34:56.1825,1902-12-25 -1902-05-05 12:34:56.1826,1902-12-25 -1902-05-05 12:34:56.1827,1902-12-25 -1902-05-05 12:34:56.1828,1902-12-25 -1902-05-05 12:34:56.1829,1902-12-25 -1902-05-05 12:34:56.183,1902-12-25 -1902-05-05 12:34:56.1831,1902-12-25 -1902-05-05 12:34:56.1832,1902-12-25 -1902-05-05 12:34:56.1833,1902-12-25 -1902-05-05 12:34:56.1834,1902-12-25 -1902-05-05 12:34:56.1835,1902-12-25 -1902-05-05 12:34:56.1836,1902-12-25 -1902-05-05 12:34:56.1837,1902-12-25 -1902-05-05 12:34:56.1838,1902-12-25 -1902-05-05 12:34:56.1839,1902-12-25 -1902-05-05 12:34:56.184,1902-12-25 -1902-05-05 12:34:56.1841,1902-12-25 -1902-05-05 12:34:56.1842,1902-12-25 -1902-05-05 12:34:56.1843,1902-12-25 -1902-05-05 12:34:56.1844,1902-12-25 -1902-05-05 12:34:56.1845,1902-12-25 -1902-05-05 12:34:56.1846,1902-12-25 -1902-05-05 12:34:56.1847,1902-12-25 -1902-05-05 12:34:56.1848,1902-12-25 -1902-05-05 12:34:56.1849,1902-12-25 -1902-05-05 12:34:56.185,1902-12-25 -1902-05-05 12:34:56.1851,1902-12-25 -1902-05-05 12:34:56.1852,1902-12-25 -1902-05-05 12:34:56.1853,1902-12-25 -1902-05-05 12:34:56.1854,1902-12-25 -1902-05-05 12:34:56.1855,1902-12-25 -1902-05-05 12:34:56.1856,1902-12-25 -1902-05-05 12:34:56.1857,1902-12-25 -1902-05-05 12:34:56.1858,1902-12-25 -1902-05-05 12:34:56.1859,1902-12-25 -1902-05-05 12:34:56.186,1902-12-25 -1902-05-05 12:34:56.1861,1902-12-25 -1902-05-05 12:34:56.1862,1902-12-25 -1902-05-05 12:34:56.1863,1902-12-25 -1902-05-05 12:34:56.1864,1902-12-25 -1902-05-05 12:34:56.1865,1902-12-25 -1902-05-05 12:34:56.1866,1902-12-25 -1902-05-05 12:34:56.1867,1902-12-25 -1902-05-05 12:34:56.1868,1902-12-25 -1902-05-05 12:34:56.1869,1902-12-25 -1902-05-05 12:34:56.187,1902-12-25 -1902-05-05 12:34:56.1871,1902-12-25 -1902-05-05 12:34:56.1872,1902-12-25 -1902-05-05 12:34:56.1873,1902-12-25 -1902-05-05 12:34:56.1874,1902-12-25 -1902-05-05 12:34:56.1875,1902-12-25 -1902-05-05 12:34:56.1876,1902-12-25 -1902-05-05 12:34:56.1877,1902-12-25 -1902-05-05 12:34:56.1878,1902-12-25 -1902-05-05 12:34:56.1879,1902-12-25 -1902-05-05 12:34:56.188,1902-12-25 -1902-05-05 12:34:56.1881,1902-12-25 -1902-05-05 12:34:56.1882,1902-12-25 -1902-05-05 12:34:56.1883,1902-12-25 -1902-05-05 12:34:56.1884,1902-12-25 -1902-05-05 12:34:56.1885,1902-12-25 -1902-05-05 12:34:56.1886,1902-12-25 -1902-05-05 12:34:56.1887,1902-12-25 -1902-05-05 12:34:56.1888,1902-12-25 -1902-05-05 12:34:56.1889,1902-12-25 -1902-05-05 12:34:56.189,1902-12-25 -1902-05-05 12:34:56.1891,1902-12-25 -1902-05-05 12:34:56.1892,1902-12-25 -1902-05-05 12:34:56.1893,1902-12-25 -1902-05-05 12:34:56.1894,1902-12-25 -1902-05-05 12:34:56.1895,1902-12-25 -1902-05-05 12:34:56.1896,1902-12-25 -1902-05-05 12:34:56.1897,1902-12-25 -1902-05-05 12:34:56.1898,1902-12-25 -1902-05-05 12:34:56.1899,1902-12-25 -1902-05-05 12:34:56.19,1902-12-25 -1902-05-05 12:34:56.1901,1902-12-25 -1902-05-05 12:34:56.1902,1902-12-25 -1902-05-05 12:34:56.1903,1902-12-25 -1902-05-05 12:34:56.1904,1902-12-25 -1902-05-05 12:34:56.1905,1902-12-25 -1902-05-05 12:34:56.1906,1902-12-25 -1902-05-05 12:34:56.1907,1902-12-25 -1902-05-05 12:34:56.1908,1902-12-25 -1902-05-05 12:34:56.1909,1902-12-25 -1902-05-05 12:34:56.191,1902-12-25 -1902-05-05 12:34:56.1911,1902-12-25 -1902-05-05 12:34:56.1912,1902-12-25 -1902-05-05 12:34:56.1913,1902-12-25 -1902-05-05 12:34:56.1914,1902-12-25 -1902-05-05 12:34:56.1915,1902-12-25 -1902-05-05 12:34:56.1916,1902-12-25 -1902-05-05 12:34:56.1917,1902-12-25 -1902-05-05 12:34:56.1918,1902-12-25 -1902-05-05 12:34:56.1919,1902-12-25 -1902-05-05 12:34:56.192,1902-12-25 -1902-05-05 12:34:56.1921,1902-12-25 -1902-05-05 12:34:56.1922,1902-12-25 -1902-05-05 12:34:56.1923,1902-12-25 -1902-05-05 12:34:56.1924,1902-12-25 -1902-05-05 12:34:56.1925,1902-12-25 -1902-05-05 12:34:56.1926,1902-12-25 -1902-05-05 12:34:56.1927,1902-12-25 -1902-05-05 12:34:56.1928,1902-12-25 -1902-05-05 12:34:56.1929,1902-12-25 -1902-05-05 12:34:56.193,1902-12-25 -1902-05-05 12:34:56.1931,1902-12-25 -1902-05-05 12:34:56.1932,1902-12-25 -1902-05-05 12:34:56.1933,1902-12-25 -1902-05-05 12:34:56.1934,1902-12-25 -1902-05-05 12:34:56.1935,1902-12-25 -1902-05-05 12:34:56.1936,1902-12-25 -1902-05-05 12:34:56.1937,1902-12-25 -1902-05-05 12:34:56.1938,1902-12-25 -1902-05-05 12:34:56.1939,1902-12-25 -1902-05-05 12:34:56.194,1902-12-25 -1902-05-05 12:34:56.1941,1902-12-25 -1902-05-05 12:34:56.1942,1902-12-25 -1902-05-05 12:34:56.1943,1902-12-25 -1902-05-05 12:34:56.1944,1902-12-25 -1902-05-05 12:34:56.1945,1902-12-25 -1902-05-05 12:34:56.1946,1902-12-25 -1902-05-05 12:34:56.1947,1902-12-25 -1902-05-05 12:34:56.1948,1902-12-25 -1902-05-05 12:34:56.1949,1902-12-25 -1902-05-05 12:34:56.195,1902-12-25 -1902-05-05 12:34:56.1951,1902-12-25 -1902-05-05 12:34:56.1952,1902-12-25 -1902-05-05 12:34:56.1953,1902-12-25 -1902-05-05 12:34:56.1954,1902-12-25 -1902-05-05 12:34:56.1955,1902-12-25 -1902-05-05 12:34:56.1956,1902-12-25 -1902-05-05 12:34:56.1957,1902-12-25 -1902-05-05 12:34:56.1958,1902-12-25 -1902-05-05 12:34:56.1959,1902-12-25 -1902-05-05 12:34:56.196,1902-12-25 -1902-05-05 12:34:56.1961,1902-12-25 -1902-05-05 12:34:56.1962,1902-12-25 -1902-05-05 12:34:56.1963,1902-12-25 -1902-05-05 12:34:56.1964,1902-12-25 -1902-05-05 12:34:56.1965,1902-12-25 -1902-05-05 12:34:56.1966,1902-12-25 -1902-05-05 12:34:56.1967,1902-12-25 -1902-05-05 12:34:56.1968,1902-12-25 -1902-05-05 12:34:56.1969,1902-12-25 -1902-05-05 12:34:56.197,1902-12-25 -1902-05-05 12:34:56.1971,1902-12-25 -1902-05-05 12:34:56.1972,1902-12-25 -1902-05-05 12:34:56.1973,1902-12-25 -1902-05-05 12:34:56.1974,1902-12-25 -1902-05-05 12:34:56.1975,1902-12-25 -1902-05-05 12:34:56.1976,1902-12-25 -1902-05-05 12:34:56.1977,1902-12-25 -1902-05-05 12:34:56.1978,1902-12-25 -1902-05-05 12:34:56.1979,1902-12-25 -1902-05-05 12:34:56.198,1902-12-25 -1902-05-05 12:34:56.1981,1902-12-25 -1902-05-05 12:34:56.1982,1902-12-25 -1902-05-05 12:34:56.1983,1902-12-25 -1902-05-05 12:34:56.1984,1902-12-25 -1902-05-05 12:34:56.1985,1902-12-25 -1902-05-05 12:34:56.1986,1902-12-25 -1902-05-05 12:34:56.1987,1902-12-25 -1902-05-05 12:34:56.1988,1902-12-25 -1902-05-05 12:34:56.1989,1902-12-25 -1902-05-05 12:34:56.199,1902-12-25 -1902-05-05 12:34:56.1991,1902-12-25 -1902-05-05 12:34:56.1992,1902-12-25 -1902-05-05 12:34:56.1993,1902-12-25 -1902-05-05 12:34:56.1994,1902-12-25 -1902-05-05 12:34:56.1995,1902-12-25 -1902-05-05 12:34:56.1996,1902-12-25 -1902-05-05 12:34:56.1997,1902-12-25 -1902-05-05 12:34:56.1998,1902-12-25 -1902-05-05 12:34:56.1999,1902-12-25 -1903-05-05 12:34:56.1,1903-12-25 -1903-05-05 12:34:56.1001,1903-12-25 -1903-05-05 12:34:56.1002,1903-12-25 -1903-05-05 12:34:56.1003,1903-12-25 -1903-05-05 12:34:56.1004,1903-12-25 -1903-05-05 12:34:56.1005,1903-12-25 -1903-05-05 12:34:56.1006,1903-12-25 -1903-05-05 12:34:56.1007,1903-12-25 -1903-05-05 12:34:56.1008,1903-12-25 -1903-05-05 12:34:56.1009,1903-12-25 -1903-05-05 12:34:56.101,1903-12-25 -1903-05-05 12:34:56.1011,1903-12-25 -1903-05-05 12:34:56.1012,1903-12-25 -1903-05-05 12:34:56.1013,1903-12-25 -1903-05-05 12:34:56.1014,1903-12-25 -1903-05-05 12:34:56.1015,1903-12-25 -1903-05-05 12:34:56.1016,1903-12-25 -1903-05-05 12:34:56.1017,1903-12-25 -1903-05-05 12:34:56.1018,1903-12-25 -1903-05-05 12:34:56.1019,1903-12-25 -1903-05-05 12:34:56.102,1903-12-25 -1903-05-05 12:34:56.1021,1903-12-25 -1903-05-05 12:34:56.1022,1903-12-25 -1903-05-05 12:34:56.1023,1903-12-25 -1903-05-05 12:34:56.1024,1903-12-25 -1903-05-05 12:34:56.1025,1903-12-25 -1903-05-05 12:34:56.1026,1903-12-25 -1903-05-05 12:34:56.1027,1903-12-25 -1903-05-05 12:34:56.1028,1903-12-25 -1903-05-05 12:34:56.1029,1903-12-25 -1903-05-05 12:34:56.103,1903-12-25 -1903-05-05 12:34:56.1031,1903-12-25 -1903-05-05 12:34:56.1032,1903-12-25 -1903-05-05 12:34:56.1033,1903-12-25 -1903-05-05 12:34:56.1034,1903-12-25 -1903-05-05 12:34:56.1035,1903-12-25 -1903-05-05 12:34:56.1036,1903-12-25 -1903-05-05 12:34:56.1037,1903-12-25 -1903-05-05 12:34:56.1038,1903-12-25 -1903-05-05 12:34:56.1039,1903-12-25 -1903-05-05 12:34:56.104,1903-12-25 -1903-05-05 12:34:56.1041,1903-12-25 -1903-05-05 12:34:56.1042,1903-12-25 -1903-05-05 12:34:56.1043,1903-12-25 -1903-05-05 12:34:56.1044,1903-12-25 -1903-05-05 12:34:56.1045,1903-12-25 -1903-05-05 12:34:56.1046,1903-12-25 -1903-05-05 12:34:56.1047,1903-12-25 -1903-05-05 12:34:56.1048,1903-12-25 -1903-05-05 12:34:56.1049,1903-12-25 -1903-05-05 12:34:56.105,1903-12-25 -1903-05-05 12:34:56.1051,1903-12-25 -1903-05-05 12:34:56.1052,1903-12-25 -1903-05-05 12:34:56.1053,1903-12-25 -1903-05-05 12:34:56.1054,1903-12-25 -1903-05-05 12:34:56.1055,1903-12-25 -1903-05-05 12:34:56.1056,1903-12-25 -1903-05-05 12:34:56.1057,1903-12-25 -1903-05-05 12:34:56.1058,1903-12-25 -1903-05-05 12:34:56.1059,1903-12-25 -1903-05-05 12:34:56.106,1903-12-25 -1903-05-05 12:34:56.1061,1903-12-25 -1903-05-05 12:34:56.1062,1903-12-25 -1903-05-05 12:34:56.1063,1903-12-25 -1903-05-05 12:34:56.1064,1903-12-25 -1903-05-05 12:34:56.1065,1903-12-25 -1903-05-05 12:34:56.1066,1903-12-25 -1903-05-05 12:34:56.1067,1903-12-25 -1903-05-05 12:34:56.1068,1903-12-25 -1903-05-05 12:34:56.1069,1903-12-25 -1903-05-05 12:34:56.107,1903-12-25 -1903-05-05 12:34:56.1071,1903-12-25 -1903-05-05 12:34:56.1072,1903-12-25 -1903-05-05 12:34:56.1073,1903-12-25 -1903-05-05 12:34:56.1074,1903-12-25 -1903-05-05 12:34:56.1075,1903-12-25 -1903-05-05 12:34:56.1076,1903-12-25 -1903-05-05 12:34:56.1077,1903-12-25 -1903-05-05 12:34:56.1078,1903-12-25 -1903-05-05 12:34:56.1079,1903-12-25 -1903-05-05 12:34:56.108,1903-12-25 -1903-05-05 12:34:56.1081,1903-12-25 -1903-05-05 12:34:56.1082,1903-12-25 -1903-05-05 12:34:56.1083,1903-12-25 -1903-05-05 12:34:56.1084,1903-12-25 -1903-05-05 12:34:56.1085,1903-12-25 -1903-05-05 12:34:56.1086,1903-12-25 -1903-05-05 12:34:56.1087,1903-12-25 -1903-05-05 12:34:56.1088,1903-12-25 -1903-05-05 12:34:56.1089,1903-12-25 -1903-05-05 12:34:56.109,1903-12-25 -1903-05-05 12:34:56.1091,1903-12-25 -1903-05-05 12:34:56.1092,1903-12-25 -1903-05-05 12:34:56.1093,1903-12-25 -1903-05-05 12:34:56.1094,1903-12-25 -1903-05-05 12:34:56.1095,1903-12-25 -1903-05-05 12:34:56.1096,1903-12-25 -1903-05-05 12:34:56.1097,1903-12-25 -1903-05-05 12:34:56.1098,1903-12-25 -1903-05-05 12:34:56.1099,1903-12-25 -1903-05-05 12:34:56.11,1903-12-25 -1903-05-05 12:34:56.1101,1903-12-25 -1903-05-05 12:34:56.1102,1903-12-25 -1903-05-05 12:34:56.1103,1903-12-25 -1903-05-05 12:34:56.1104,1903-12-25 -1903-05-05 12:34:56.1105,1903-12-25 -1903-05-05 12:34:56.1106,1903-12-25 -1903-05-05 12:34:56.1107,1903-12-25 -1903-05-05 12:34:56.1108,1903-12-25 -1903-05-05 12:34:56.1109,1903-12-25 -1903-05-05 12:34:56.111,1903-12-25 -1903-05-05 12:34:56.1111,1903-12-25 -1903-05-05 12:34:56.1112,1903-12-25 -1903-05-05 12:34:56.1113,1903-12-25 -1903-05-05 12:34:56.1114,1903-12-25 -1903-05-05 12:34:56.1115,1903-12-25 -1903-05-05 12:34:56.1116,1903-12-25 -1903-05-05 12:34:56.1117,1903-12-25 -1903-05-05 12:34:56.1118,1903-12-25 -1903-05-05 12:34:56.1119,1903-12-25 -1903-05-05 12:34:56.112,1903-12-25 -1903-05-05 12:34:56.1121,1903-12-25 -1903-05-05 12:34:56.1122,1903-12-25 -1903-05-05 12:34:56.1123,1903-12-25 -1903-05-05 12:34:56.1124,1903-12-25 -1903-05-05 12:34:56.1125,1903-12-25 -1903-05-05 12:34:56.1126,1903-12-25 -1903-05-05 12:34:56.1127,1903-12-25 -1903-05-05 12:34:56.1128,1903-12-25 -1903-05-05 12:34:56.1129,1903-12-25 -1903-05-05 12:34:56.113,1903-12-25 -1903-05-05 12:34:56.1131,1903-12-25 -1903-05-05 12:34:56.1132,1903-12-25 -1903-05-05 12:34:56.1133,1903-12-25 -1903-05-05 12:34:56.1134,1903-12-25 -1903-05-05 12:34:56.1135,1903-12-25 -1903-05-05 12:34:56.1136,1903-12-25 -1903-05-05 12:34:56.1137,1903-12-25 -1903-05-05 12:34:56.1138,1903-12-25 -1903-05-05 12:34:56.1139,1903-12-25 -1903-05-05 12:34:56.114,1903-12-25 -1903-05-05 12:34:56.1141,1903-12-25 -1903-05-05 12:34:56.1142,1903-12-25 -1903-05-05 12:34:56.1143,1903-12-25 -1903-05-05 12:34:56.1144,1903-12-25 -1903-05-05 12:34:56.1145,1903-12-25 -1903-05-05 12:34:56.1146,1903-12-25 -1903-05-05 12:34:56.1147,1903-12-25 -1903-05-05 12:34:56.1148,1903-12-25 -1903-05-05 12:34:56.1149,1903-12-25 -1903-05-05 12:34:56.115,1903-12-25 -1903-05-05 12:34:56.1151,1903-12-25 -1903-05-05 12:34:56.1152,1903-12-25 -1903-05-05 12:34:56.1153,1903-12-25 -1903-05-05 12:34:56.1154,1903-12-25 -1903-05-05 12:34:56.1155,1903-12-25 -1903-05-05 12:34:56.1156,1903-12-25 -1903-05-05 12:34:56.1157,1903-12-25 -1903-05-05 12:34:56.1158,1903-12-25 -1903-05-05 12:34:56.1159,1903-12-25 -1903-05-05 12:34:56.116,1903-12-25 -1903-05-05 12:34:56.1161,1903-12-25 -1903-05-05 12:34:56.1162,1903-12-25 -1903-05-05 12:34:56.1163,1903-12-25 -1903-05-05 12:34:56.1164,1903-12-25 -1903-05-05 12:34:56.1165,1903-12-25 -1903-05-05 12:34:56.1166,1903-12-25 -1903-05-05 12:34:56.1167,1903-12-25 -1903-05-05 12:34:56.1168,1903-12-25 -1903-05-05 12:34:56.1169,1903-12-25 -1903-05-05 12:34:56.117,1903-12-25 -1903-05-05 12:34:56.1171,1903-12-25 -1903-05-05 12:34:56.1172,1903-12-25 -1903-05-05 12:34:56.1173,1903-12-25 -1903-05-05 12:34:56.1174,1903-12-25 -1903-05-05 12:34:56.1175,1903-12-25 -1903-05-05 12:34:56.1176,1903-12-25 -1903-05-05 12:34:56.1177,1903-12-25 -1903-05-05 12:34:56.1178,1903-12-25 -1903-05-05 12:34:56.1179,1903-12-25 -1903-05-05 12:34:56.118,1903-12-25 -1903-05-05 12:34:56.1181,1903-12-25 -1903-05-05 12:34:56.1182,1903-12-25 -1903-05-05 12:34:56.1183,1903-12-25 -1903-05-05 12:34:56.1184,1903-12-25 -1903-05-05 12:34:56.1185,1903-12-25 -1903-05-05 12:34:56.1186,1903-12-25 -1903-05-05 12:34:56.1187,1903-12-25 -1903-05-05 12:34:56.1188,1903-12-25 -1903-05-05 12:34:56.1189,1903-12-25 -1903-05-05 12:34:56.119,1903-12-25 -1903-05-05 12:34:56.1191,1903-12-25 -1903-05-05 12:34:56.1192,1903-12-25 -1903-05-05 12:34:56.1193,1903-12-25 -1903-05-05 12:34:56.1194,1903-12-25 -1903-05-05 12:34:56.1195,1903-12-25 -1903-05-05 12:34:56.1196,1903-12-25 -1903-05-05 12:34:56.1197,1903-12-25 -1903-05-05 12:34:56.1198,1903-12-25 -1903-05-05 12:34:56.1199,1903-12-25 -1903-05-05 12:34:56.12,1903-12-25 -1903-05-05 12:34:56.1201,1903-12-25 -1903-05-05 12:34:56.1202,1903-12-25 -1903-05-05 12:34:56.1203,1903-12-25 -1903-05-05 12:34:56.1204,1903-12-25 -1903-05-05 12:34:56.1205,1903-12-25 -1903-05-05 12:34:56.1206,1903-12-25 -1903-05-05 12:34:56.1207,1903-12-25 -1903-05-05 12:34:56.1208,1903-12-25 -1903-05-05 12:34:56.1209,1903-12-25 -1903-05-05 12:34:56.121,1903-12-25 -1903-05-05 12:34:56.1211,1903-12-25 -1903-05-05 12:34:56.1212,1903-12-25 -1903-05-05 12:34:56.1213,1903-12-25 -1903-05-05 12:34:56.1214,1903-12-25 -1903-05-05 12:34:56.1215,1903-12-25 -1903-05-05 12:34:56.1216,1903-12-25 -1903-05-05 12:34:56.1217,1903-12-25 -1903-05-05 12:34:56.1218,1903-12-25 -1903-05-05 12:34:56.1219,1903-12-25 -1903-05-05 12:34:56.122,1903-12-25 -1903-05-05 12:34:56.1221,1903-12-25 -1903-05-05 12:34:56.1222,1903-12-25 -1903-05-05 12:34:56.1223,1903-12-25 -1903-05-05 12:34:56.1224,1903-12-25 -1903-05-05 12:34:56.1225,1903-12-25 -1903-05-05 12:34:56.1226,1903-12-25 -1903-05-05 12:34:56.1227,1903-12-25 -1903-05-05 12:34:56.1228,1903-12-25 -1903-05-05 12:34:56.1229,1903-12-25 -1903-05-05 12:34:56.123,1903-12-25 -1903-05-05 12:34:56.1231,1903-12-25 -1903-05-05 12:34:56.1232,1903-12-25 -1903-05-05 12:34:56.1233,1903-12-25 -1903-05-05 12:34:56.1234,1903-12-25 -1903-05-05 12:34:56.1235,1903-12-25 -1903-05-05 12:34:56.1236,1903-12-25 -1903-05-05 12:34:56.1237,1903-12-25 -1903-05-05 12:34:56.1238,1903-12-25 -1903-05-05 12:34:56.1239,1903-12-25 -1903-05-05 12:34:56.124,1903-12-25 -1903-05-05 12:34:56.1241,1903-12-25 -1903-05-05 12:34:56.1242,1903-12-25 -1903-05-05 12:34:56.1243,1903-12-25 -1903-05-05 12:34:56.1244,1903-12-25 -1903-05-05 12:34:56.1245,1903-12-25 -1903-05-05 12:34:56.1246,1903-12-25 -1903-05-05 12:34:56.1247,1903-12-25 -1903-05-05 12:34:56.1248,1903-12-25 -1903-05-05 12:34:56.1249,1903-12-25 -1903-05-05 12:34:56.125,1903-12-25 -1903-05-05 12:34:56.1251,1903-12-25 -1903-05-05 12:34:56.1252,1903-12-25 -1903-05-05 12:34:56.1253,1903-12-25 -1903-05-05 12:34:56.1254,1903-12-25 -1903-05-05 12:34:56.1255,1903-12-25 -1903-05-05 12:34:56.1256,1903-12-25 -1903-05-05 12:34:56.1257,1903-12-25 -1903-05-05 12:34:56.1258,1903-12-25 -1903-05-05 12:34:56.1259,1903-12-25 -1903-05-05 12:34:56.126,1903-12-25 -1903-05-05 12:34:56.1261,1903-12-25 -1903-05-05 12:34:56.1262,1903-12-25 -1903-05-05 12:34:56.1263,1903-12-25 -1903-05-05 12:34:56.1264,1903-12-25 -1903-05-05 12:34:56.1265,1903-12-25 -1903-05-05 12:34:56.1266,1903-12-25 -1903-05-05 12:34:56.1267,1903-12-25 -1903-05-05 12:34:56.1268,1903-12-25 -1903-05-05 12:34:56.1269,1903-12-25 -1903-05-05 12:34:56.127,1903-12-25 -1903-05-05 12:34:56.1271,1903-12-25 -1903-05-05 12:34:56.1272,1903-12-25 -1903-05-05 12:34:56.1273,1903-12-25 -1903-05-05 12:34:56.1274,1903-12-25 -1903-05-05 12:34:56.1275,1903-12-25 -1903-05-05 12:34:56.1276,1903-12-25 -1903-05-05 12:34:56.1277,1903-12-25 -1903-05-05 12:34:56.1278,1903-12-25 -1903-05-05 12:34:56.1279,1903-12-25 -1903-05-05 12:34:56.128,1903-12-25 -1903-05-05 12:34:56.1281,1903-12-25 -1903-05-05 12:34:56.1282,1903-12-25 -1903-05-05 12:34:56.1283,1903-12-25 -1903-05-05 12:34:56.1284,1903-12-25 -1903-05-05 12:34:56.1285,1903-12-25 -1903-05-05 12:34:56.1286,1903-12-25 -1903-05-05 12:34:56.1287,1903-12-25 -1903-05-05 12:34:56.1288,1903-12-25 -1903-05-05 12:34:56.1289,1903-12-25 -1903-05-05 12:34:56.129,1903-12-25 -1903-05-05 12:34:56.1291,1903-12-25 -1903-05-05 12:34:56.1292,1903-12-25 -1903-05-05 12:34:56.1293,1903-12-25 -1903-05-05 12:34:56.1294,1903-12-25 -1903-05-05 12:34:56.1295,1903-12-25 -1903-05-05 12:34:56.1296,1903-12-25 -1903-05-05 12:34:56.1297,1903-12-25 -1903-05-05 12:34:56.1298,1903-12-25 -1903-05-05 12:34:56.1299,1903-12-25 -1903-05-05 12:34:56.13,1903-12-25 -1903-05-05 12:34:56.1301,1903-12-25 -1903-05-05 12:34:56.1302,1903-12-25 -1903-05-05 12:34:56.1303,1903-12-25 -1903-05-05 12:34:56.1304,1903-12-25 -1903-05-05 12:34:56.1305,1903-12-25 -1903-05-05 12:34:56.1306,1903-12-25 -1903-05-05 12:34:56.1307,1903-12-25 -1903-05-05 12:34:56.1308,1903-12-25 -1903-05-05 12:34:56.1309,1903-12-25 -1903-05-05 12:34:56.131,1903-12-25 -1903-05-05 12:34:56.1311,1903-12-25 -1903-05-05 12:34:56.1312,1903-12-25 -1903-05-05 12:34:56.1313,1903-12-25 -1903-05-05 12:34:56.1314,1903-12-25 -1903-05-05 12:34:56.1315,1903-12-25 -1903-05-05 12:34:56.1316,1903-12-25 -1903-05-05 12:34:56.1317,1903-12-25 -1903-05-05 12:34:56.1318,1903-12-25 -1903-05-05 12:34:56.1319,1903-12-25 -1903-05-05 12:34:56.132,1903-12-25 -1903-05-05 12:34:56.1321,1903-12-25 -1903-05-05 12:34:56.1322,1903-12-25 -1903-05-05 12:34:56.1323,1903-12-25 -1903-05-05 12:34:56.1324,1903-12-25 -1903-05-05 12:34:56.1325,1903-12-25 -1903-05-05 12:34:56.1326,1903-12-25 -1903-05-05 12:34:56.1327,1903-12-25 -1903-05-05 12:34:56.1328,1903-12-25 -1903-05-05 12:34:56.1329,1903-12-25 -1903-05-05 12:34:56.133,1903-12-25 -1903-05-05 12:34:56.1331,1903-12-25 -1903-05-05 12:34:56.1332,1903-12-25 -1903-05-05 12:34:56.1333,1903-12-25 -1903-05-05 12:34:56.1334,1903-12-25 -1903-05-05 12:34:56.1335,1903-12-25 -1903-05-05 12:34:56.1336,1903-12-25 -1903-05-05 12:34:56.1337,1903-12-25 -1903-05-05 12:34:56.1338,1903-12-25 -1903-05-05 12:34:56.1339,1903-12-25 -1903-05-05 12:34:56.134,1903-12-25 -1903-05-05 12:34:56.1341,1903-12-25 -1903-05-05 12:34:56.1342,1903-12-25 -1903-05-05 12:34:56.1343,1903-12-25 -1903-05-05 12:34:56.1344,1903-12-25 -1903-05-05 12:34:56.1345,1903-12-25 -1903-05-05 12:34:56.1346,1903-12-25 -1903-05-05 12:34:56.1347,1903-12-25 -1903-05-05 12:34:56.1348,1903-12-25 -1903-05-05 12:34:56.1349,1903-12-25 -1903-05-05 12:34:56.135,1903-12-25 -1903-05-05 12:34:56.1351,1903-12-25 -1903-05-05 12:34:56.1352,1903-12-25 -1903-05-05 12:34:56.1353,1903-12-25 -1903-05-05 12:34:56.1354,1903-12-25 -1903-05-05 12:34:56.1355,1903-12-25 -1903-05-05 12:34:56.1356,1903-12-25 -1903-05-05 12:34:56.1357,1903-12-25 -1903-05-05 12:34:56.1358,1903-12-25 -1903-05-05 12:34:56.1359,1903-12-25 -1903-05-05 12:34:56.136,1903-12-25 -1903-05-05 12:34:56.1361,1903-12-25 -1903-05-05 12:34:56.1362,1903-12-25 -1903-05-05 12:34:56.1363,1903-12-25 -1903-05-05 12:34:56.1364,1903-12-25 -1903-05-05 12:34:56.1365,1903-12-25 -1903-05-05 12:34:56.1366,1903-12-25 -1903-05-05 12:34:56.1367,1903-12-25 -1903-05-05 12:34:56.1368,1903-12-25 -1903-05-05 12:34:56.1369,1903-12-25 -1903-05-05 12:34:56.137,1903-12-25 -1903-05-05 12:34:56.1371,1903-12-25 -1903-05-05 12:34:56.1372,1903-12-25 -1903-05-05 12:34:56.1373,1903-12-25 -1903-05-05 12:34:56.1374,1903-12-25 -1903-05-05 12:34:56.1375,1903-12-25 -1903-05-05 12:34:56.1376,1903-12-25 -1903-05-05 12:34:56.1377,1903-12-25 -1903-05-05 12:34:56.1378,1903-12-25 -1903-05-05 12:34:56.1379,1903-12-25 -1903-05-05 12:34:56.138,1903-12-25 -1903-05-05 12:34:56.1381,1903-12-25 -1903-05-05 12:34:56.1382,1903-12-25 -1903-05-05 12:34:56.1383,1903-12-25 -1903-05-05 12:34:56.1384,1903-12-25 -1903-05-05 12:34:56.1385,1903-12-25 -1903-05-05 12:34:56.1386,1903-12-25 -1903-05-05 12:34:56.1387,1903-12-25 -1903-05-05 12:34:56.1388,1903-12-25 -1903-05-05 12:34:56.1389,1903-12-25 -1903-05-05 12:34:56.139,1903-12-25 -1903-05-05 12:34:56.1391,1903-12-25 -1903-05-05 12:34:56.1392,1903-12-25 -1903-05-05 12:34:56.1393,1903-12-25 -1903-05-05 12:34:56.1394,1903-12-25 -1903-05-05 12:34:56.1395,1903-12-25 -1903-05-05 12:34:56.1396,1903-12-25 -1903-05-05 12:34:56.1397,1903-12-25 -1903-05-05 12:34:56.1398,1903-12-25 -1903-05-05 12:34:56.1399,1903-12-25 -1903-05-05 12:34:56.14,1903-12-25 -1903-05-05 12:34:56.1401,1903-12-25 -1903-05-05 12:34:56.1402,1903-12-25 -1903-05-05 12:34:56.1403,1903-12-25 -1903-05-05 12:34:56.1404,1903-12-25 -1903-05-05 12:34:56.1405,1903-12-25 -1903-05-05 12:34:56.1406,1903-12-25 -1903-05-05 12:34:56.1407,1903-12-25 -1903-05-05 12:34:56.1408,1903-12-25 -1903-05-05 12:34:56.1409,1903-12-25 -1903-05-05 12:34:56.141,1903-12-25 -1903-05-05 12:34:56.1411,1903-12-25 -1903-05-05 12:34:56.1412,1903-12-25 -1903-05-05 12:34:56.1413,1903-12-25 -1903-05-05 12:34:56.1414,1903-12-25 -1903-05-05 12:34:56.1415,1903-12-25 -1903-05-05 12:34:56.1416,1903-12-25 -1903-05-05 12:34:56.1417,1903-12-25 -1903-05-05 12:34:56.1418,1903-12-25 -1903-05-05 12:34:56.1419,1903-12-25 -1903-05-05 12:34:56.142,1903-12-25 -1903-05-05 12:34:56.1421,1903-12-25 -1903-05-05 12:34:56.1422,1903-12-25 -1903-05-05 12:34:56.1423,1903-12-25 -1903-05-05 12:34:56.1424,1903-12-25 -1903-05-05 12:34:56.1425,1903-12-25 -1903-05-05 12:34:56.1426,1903-12-25 -1903-05-05 12:34:56.1427,1903-12-25 -1903-05-05 12:34:56.1428,1903-12-25 -1903-05-05 12:34:56.1429,1903-12-25 -1903-05-05 12:34:56.143,1903-12-25 -1903-05-05 12:34:56.1431,1903-12-25 -1903-05-05 12:34:56.1432,1903-12-25 -1903-05-05 12:34:56.1433,1903-12-25 -1903-05-05 12:34:56.1434,1903-12-25 -1903-05-05 12:34:56.1435,1903-12-25 -1903-05-05 12:34:56.1436,1903-12-25 -1903-05-05 12:34:56.1437,1903-12-25 -1903-05-05 12:34:56.1438,1903-12-25 -1903-05-05 12:34:56.1439,1903-12-25 -1903-05-05 12:34:56.144,1903-12-25 -1903-05-05 12:34:56.1441,1903-12-25 -1903-05-05 12:34:56.1442,1903-12-25 -1903-05-05 12:34:56.1443,1903-12-25 -1903-05-05 12:34:56.1444,1903-12-25 -1903-05-05 12:34:56.1445,1903-12-25 -1903-05-05 12:34:56.1446,1903-12-25 -1903-05-05 12:34:56.1447,1903-12-25 -1903-05-05 12:34:56.1448,1903-12-25 -1903-05-05 12:34:56.1449,1903-12-25 -1903-05-05 12:34:56.145,1903-12-25 -1903-05-05 12:34:56.1451,1903-12-25 -1903-05-05 12:34:56.1452,1903-12-25 -1903-05-05 12:34:56.1453,1903-12-25 -1903-05-05 12:34:56.1454,1903-12-25 -1903-05-05 12:34:56.1455,1903-12-25 -1903-05-05 12:34:56.1456,1903-12-25 -1903-05-05 12:34:56.1457,1903-12-25 -1903-05-05 12:34:56.1458,1903-12-25 -1903-05-05 12:34:56.1459,1903-12-25 -1903-05-05 12:34:56.146,1903-12-25 -1903-05-05 12:34:56.1461,1903-12-25 -1903-05-05 12:34:56.1462,1903-12-25 -1903-05-05 12:34:56.1463,1903-12-25 -1903-05-05 12:34:56.1464,1903-12-25 -1903-05-05 12:34:56.1465,1903-12-25 -1903-05-05 12:34:56.1466,1903-12-25 -1903-05-05 12:34:56.1467,1903-12-25 -1903-05-05 12:34:56.1468,1903-12-25 -1903-05-05 12:34:56.1469,1903-12-25 -1903-05-05 12:34:56.147,1903-12-25 -1903-05-05 12:34:56.1471,1903-12-25 -1903-05-05 12:34:56.1472,1903-12-25 -1903-05-05 12:34:56.1473,1903-12-25 -1903-05-05 12:34:56.1474,1903-12-25 -1903-05-05 12:34:56.1475,1903-12-25 -1903-05-05 12:34:56.1476,1903-12-25 -1903-05-05 12:34:56.1477,1903-12-25 -1903-05-05 12:34:56.1478,1903-12-25 -1903-05-05 12:34:56.1479,1903-12-25 -1903-05-05 12:34:56.148,1903-12-25 -1903-05-05 12:34:56.1481,1903-12-25 -1903-05-05 12:34:56.1482,1903-12-25 -1903-05-05 12:34:56.1483,1903-12-25 -1903-05-05 12:34:56.1484,1903-12-25 -1903-05-05 12:34:56.1485,1903-12-25 -1903-05-05 12:34:56.1486,1903-12-25 -1903-05-05 12:34:56.1487,1903-12-25 -1903-05-05 12:34:56.1488,1903-12-25 -1903-05-05 12:34:56.1489,1903-12-25 -1903-05-05 12:34:56.149,1903-12-25 -1903-05-05 12:34:56.1491,1903-12-25 -1903-05-05 12:34:56.1492,1903-12-25 -1903-05-05 12:34:56.1493,1903-12-25 -1903-05-05 12:34:56.1494,1903-12-25 -1903-05-05 12:34:56.1495,1903-12-25 -1903-05-05 12:34:56.1496,1903-12-25 -1903-05-05 12:34:56.1497,1903-12-25 -1903-05-05 12:34:56.1498,1903-12-25 -1903-05-05 12:34:56.1499,1903-12-25 -1903-05-05 12:34:56.15,1903-12-25 -1903-05-05 12:34:56.1501,1903-12-25 -1903-05-05 12:34:56.1502,1903-12-25 -1903-05-05 12:34:56.1503,1903-12-25 -1903-05-05 12:34:56.1504,1903-12-25 -1903-05-05 12:34:56.1505,1903-12-25 -1903-05-05 12:34:56.1506,1903-12-25 -1903-05-05 12:34:56.1507,1903-12-25 -1903-05-05 12:34:56.1508,1903-12-25 -1903-05-05 12:34:56.1509,1903-12-25 -1903-05-05 12:34:56.151,1903-12-25 -1903-05-05 12:34:56.1511,1903-12-25 -1903-05-05 12:34:56.1512,1903-12-25 -1903-05-05 12:34:56.1513,1903-12-25 -1903-05-05 12:34:56.1514,1903-12-25 -1903-05-05 12:34:56.1515,1903-12-25 -1903-05-05 12:34:56.1516,1903-12-25 -1903-05-05 12:34:56.1517,1903-12-25 -1903-05-05 12:34:56.1518,1903-12-25 -1903-05-05 12:34:56.1519,1903-12-25 -1903-05-05 12:34:56.152,1903-12-25 -1903-05-05 12:34:56.1521,1903-12-25 -1903-05-05 12:34:56.1522,1903-12-25 -1903-05-05 12:34:56.1523,1903-12-25 -1903-05-05 12:34:56.1524,1903-12-25 -1903-05-05 12:34:56.1525,1903-12-25 -1903-05-05 12:34:56.1526,1903-12-25 -1903-05-05 12:34:56.1527,1903-12-25 -1903-05-05 12:34:56.1528,1903-12-25 -1903-05-05 12:34:56.1529,1903-12-25 -1903-05-05 12:34:56.153,1903-12-25 -1903-05-05 12:34:56.1531,1903-12-25 -1903-05-05 12:34:56.1532,1903-12-25 -1903-05-05 12:34:56.1533,1903-12-25 -1903-05-05 12:34:56.1534,1903-12-25 -1903-05-05 12:34:56.1535,1903-12-25 -1903-05-05 12:34:56.1536,1903-12-25 -1903-05-05 12:34:56.1537,1903-12-25 -1903-05-05 12:34:56.1538,1903-12-25 -1903-05-05 12:34:56.1539,1903-12-25 -1903-05-05 12:34:56.154,1903-12-25 -1903-05-05 12:34:56.1541,1903-12-25 -1903-05-05 12:34:56.1542,1903-12-25 -1903-05-05 12:34:56.1543,1903-12-25 -1903-05-05 12:34:56.1544,1903-12-25 -1903-05-05 12:34:56.1545,1903-12-25 -1903-05-05 12:34:56.1546,1903-12-25 -1903-05-05 12:34:56.1547,1903-12-25 -1903-05-05 12:34:56.1548,1903-12-25 -1903-05-05 12:34:56.1549,1903-12-25 -1903-05-05 12:34:56.155,1903-12-25 -1903-05-05 12:34:56.1551,1903-12-25 -1903-05-05 12:34:56.1552,1903-12-25 -1903-05-05 12:34:56.1553,1903-12-25 -1903-05-05 12:34:56.1554,1903-12-25 -1903-05-05 12:34:56.1555,1903-12-25 -1903-05-05 12:34:56.1556,1903-12-25 -1903-05-05 12:34:56.1557,1903-12-25 -1903-05-05 12:34:56.1558,1903-12-25 -1903-05-05 12:34:56.1559,1903-12-25 -1903-05-05 12:34:56.156,1903-12-25 -1903-05-05 12:34:56.1561,1903-12-25 -1903-05-05 12:34:56.1562,1903-12-25 -1903-05-05 12:34:56.1563,1903-12-25 -1903-05-05 12:34:56.1564,1903-12-25 -1903-05-05 12:34:56.1565,1903-12-25 -1903-05-05 12:34:56.1566,1903-12-25 -1903-05-05 12:34:56.1567,1903-12-25 -1903-05-05 12:34:56.1568,1903-12-25 -1903-05-05 12:34:56.1569,1903-12-25 -1903-05-05 12:34:56.157,1903-12-25 -1903-05-05 12:34:56.1571,1903-12-25 -1903-05-05 12:34:56.1572,1903-12-25 -1903-05-05 12:34:56.1573,1903-12-25 -1903-05-05 12:34:56.1574,1903-12-25 -1903-05-05 12:34:56.1575,1903-12-25 -1903-05-05 12:34:56.1576,1903-12-25 -1903-05-05 12:34:56.1577,1903-12-25 -1903-05-05 12:34:56.1578,1903-12-25 -1903-05-05 12:34:56.1579,1903-12-25 -1903-05-05 12:34:56.158,1903-12-25 -1903-05-05 12:34:56.1581,1903-12-25 -1903-05-05 12:34:56.1582,1903-12-25 -1903-05-05 12:34:56.1583,1903-12-25 -1903-05-05 12:34:56.1584,1903-12-25 -1903-05-05 12:34:56.1585,1903-12-25 -1903-05-05 12:34:56.1586,1903-12-25 -1903-05-05 12:34:56.1587,1903-12-25 -1903-05-05 12:34:56.1588,1903-12-25 -1903-05-05 12:34:56.1589,1903-12-25 -1903-05-05 12:34:56.159,1903-12-25 -1903-05-05 12:34:56.1591,1903-12-25 -1903-05-05 12:34:56.1592,1903-12-25 -1903-05-05 12:34:56.1593,1903-12-25 -1903-05-05 12:34:56.1594,1903-12-25 -1903-05-05 12:34:56.1595,1903-12-25 -1903-05-05 12:34:56.1596,1903-12-25 -1903-05-05 12:34:56.1597,1903-12-25 -1903-05-05 12:34:56.1598,1903-12-25 -1903-05-05 12:34:56.1599,1903-12-25 -1903-05-05 12:34:56.16,1903-12-25 -1903-05-05 12:34:56.1601,1903-12-25 -1903-05-05 12:34:56.1602,1903-12-25 -1903-05-05 12:34:56.1603,1903-12-25 -1903-05-05 12:34:56.1604,1903-12-25 -1903-05-05 12:34:56.1605,1903-12-25 -1903-05-05 12:34:56.1606,1903-12-25 -1903-05-05 12:34:56.1607,1903-12-25 -1903-05-05 12:34:56.1608,1903-12-25 -1903-05-05 12:34:56.1609,1903-12-25 -1903-05-05 12:34:56.161,1903-12-25 -1903-05-05 12:34:56.1611,1903-12-25 -1903-05-05 12:34:56.1612,1903-12-25 -1903-05-05 12:34:56.1613,1903-12-25 -1903-05-05 12:34:56.1614,1903-12-25 -1903-05-05 12:34:56.1615,1903-12-25 -1903-05-05 12:34:56.1616,1903-12-25 -1903-05-05 12:34:56.1617,1903-12-25 -1903-05-05 12:34:56.1618,1903-12-25 -1903-05-05 12:34:56.1619,1903-12-25 -1903-05-05 12:34:56.162,1903-12-25 -1903-05-05 12:34:56.1621,1903-12-25 -1903-05-05 12:34:56.1622,1903-12-25 -1903-05-05 12:34:56.1623,1903-12-25 -1903-05-05 12:34:56.1624,1903-12-25 -1903-05-05 12:34:56.1625,1903-12-25 -1903-05-05 12:34:56.1626,1903-12-25 -1903-05-05 12:34:56.1627,1903-12-25 -1903-05-05 12:34:56.1628,1903-12-25 -1903-05-05 12:34:56.1629,1903-12-25 -1903-05-05 12:34:56.163,1903-12-25 -1903-05-05 12:34:56.1631,1903-12-25 -1903-05-05 12:34:56.1632,1903-12-25 -1903-05-05 12:34:56.1633,1903-12-25 -1903-05-05 12:34:56.1634,1903-12-25 -1903-05-05 12:34:56.1635,1903-12-25 -1903-05-05 12:34:56.1636,1903-12-25 -1903-05-05 12:34:56.1637,1903-12-25 -1903-05-05 12:34:56.1638,1903-12-25 -1903-05-05 12:34:56.1639,1903-12-25 -1903-05-05 12:34:56.164,1903-12-25 -1903-05-05 12:34:56.1641,1903-12-25 -1903-05-05 12:34:56.1642,1903-12-25 -1903-05-05 12:34:56.1643,1903-12-25 -1903-05-05 12:34:56.1644,1903-12-25 -1903-05-05 12:34:56.1645,1903-12-25 -1903-05-05 12:34:56.1646,1903-12-25 -1903-05-05 12:34:56.1647,1903-12-25 -1903-05-05 12:34:56.1648,1903-12-25 -1903-05-05 12:34:56.1649,1903-12-25 -1903-05-05 12:34:56.165,1903-12-25 -1903-05-05 12:34:56.1651,1903-12-25 -1903-05-05 12:34:56.1652,1903-12-25 -1903-05-05 12:34:56.1653,1903-12-25 -1903-05-05 12:34:56.1654,1903-12-25 -1903-05-05 12:34:56.1655,1903-12-25 -1903-05-05 12:34:56.1656,1903-12-25 -1903-05-05 12:34:56.1657,1903-12-25 -1903-05-05 12:34:56.1658,1903-12-25 -1903-05-05 12:34:56.1659,1903-12-25 -1903-05-05 12:34:56.166,1903-12-25 -1903-05-05 12:34:56.1661,1903-12-25 -1903-05-05 12:34:56.1662,1903-12-25 -1903-05-05 12:34:56.1663,1903-12-25 -1903-05-05 12:34:56.1664,1903-12-25 -1903-05-05 12:34:56.1665,1903-12-25 -1903-05-05 12:34:56.1666,1903-12-25 -1903-05-05 12:34:56.1667,1903-12-25 -1903-05-05 12:34:56.1668,1903-12-25 -1903-05-05 12:34:56.1669,1903-12-25 -1903-05-05 12:34:56.167,1903-12-25 -1903-05-05 12:34:56.1671,1903-12-25 -1903-05-05 12:34:56.1672,1903-12-25 -1903-05-05 12:34:56.1673,1903-12-25 -1903-05-05 12:34:56.1674,1903-12-25 -1903-05-05 12:34:56.1675,1903-12-25 -1903-05-05 12:34:56.1676,1903-12-25 -1903-05-05 12:34:56.1677,1903-12-25 -1903-05-05 12:34:56.1678,1903-12-25 -1903-05-05 12:34:56.1679,1903-12-25 -1903-05-05 12:34:56.168,1903-12-25 -1903-05-05 12:34:56.1681,1903-12-25 -1903-05-05 12:34:56.1682,1903-12-25 -1903-05-05 12:34:56.1683,1903-12-25 -1903-05-05 12:34:56.1684,1903-12-25 -1903-05-05 12:34:56.1685,1903-12-25 -1903-05-05 12:34:56.1686,1903-12-25 -1903-05-05 12:34:56.1687,1903-12-25 -1903-05-05 12:34:56.1688,1903-12-25 -1903-05-05 12:34:56.1689,1903-12-25 -1903-05-05 12:34:56.169,1903-12-25 -1903-05-05 12:34:56.1691,1903-12-25 -1903-05-05 12:34:56.1692,1903-12-25 -1903-05-05 12:34:56.1693,1903-12-25 -1903-05-05 12:34:56.1694,1903-12-25 -1903-05-05 12:34:56.1695,1903-12-25 -1903-05-05 12:34:56.1696,1903-12-25 -1903-05-05 12:34:56.1697,1903-12-25 -1903-05-05 12:34:56.1698,1903-12-25 -1903-05-05 12:34:56.1699,1903-12-25 -1903-05-05 12:34:56.17,1903-12-25 -1903-05-05 12:34:56.1701,1903-12-25 -1903-05-05 12:34:56.1702,1903-12-25 -1903-05-05 12:34:56.1703,1903-12-25 -1903-05-05 12:34:56.1704,1903-12-25 -1903-05-05 12:34:56.1705,1903-12-25 -1903-05-05 12:34:56.1706,1903-12-25 -1903-05-05 12:34:56.1707,1903-12-25 -1903-05-05 12:34:56.1708,1903-12-25 -1903-05-05 12:34:56.1709,1903-12-25 -1903-05-05 12:34:56.171,1903-12-25 -1903-05-05 12:34:56.1711,1903-12-25 -1903-05-05 12:34:56.1712,1903-12-25 -1903-05-05 12:34:56.1713,1903-12-25 -1903-05-05 12:34:56.1714,1903-12-25 -1903-05-05 12:34:56.1715,1903-12-25 -1903-05-05 12:34:56.1716,1903-12-25 -1903-05-05 12:34:56.1717,1903-12-25 -1903-05-05 12:34:56.1718,1903-12-25 -1903-05-05 12:34:56.1719,1903-12-25 -1903-05-05 12:34:56.172,1903-12-25 -1903-05-05 12:34:56.1721,1903-12-25 -1903-05-05 12:34:56.1722,1903-12-25 -1903-05-05 12:34:56.1723,1903-12-25 -1903-05-05 12:34:56.1724,1903-12-25 -1903-05-05 12:34:56.1725,1903-12-25 -1903-05-05 12:34:56.1726,1903-12-25 -1903-05-05 12:34:56.1727,1903-12-25 -1903-05-05 12:34:56.1728,1903-12-25 -1903-05-05 12:34:56.1729,1903-12-25 -1903-05-05 12:34:56.173,1903-12-25 -1903-05-05 12:34:56.1731,1903-12-25 -1903-05-05 12:34:56.1732,1903-12-25 -1903-05-05 12:34:56.1733,1903-12-25 -1903-05-05 12:34:56.1734,1903-12-25 -1903-05-05 12:34:56.1735,1903-12-25 -1903-05-05 12:34:56.1736,1903-12-25 -1903-05-05 12:34:56.1737,1903-12-25 -1903-05-05 12:34:56.1738,1903-12-25 -1903-05-05 12:34:56.1739,1903-12-25 -1903-05-05 12:34:56.174,1903-12-25 -1903-05-05 12:34:56.1741,1903-12-25 -1903-05-05 12:34:56.1742,1903-12-25 -1903-05-05 12:34:56.1743,1903-12-25 -1903-05-05 12:34:56.1744,1903-12-25 -1903-05-05 12:34:56.1745,1903-12-25 -1903-05-05 12:34:56.1746,1903-12-25 -1903-05-05 12:34:56.1747,1903-12-25 -1903-05-05 12:34:56.1748,1903-12-25 -1903-05-05 12:34:56.1749,1903-12-25 -1903-05-05 12:34:56.175,1903-12-25 -1903-05-05 12:34:56.1751,1903-12-25 -1903-05-05 12:34:56.1752,1903-12-25 -1903-05-05 12:34:56.1753,1903-12-25 -1903-05-05 12:34:56.1754,1903-12-25 -1903-05-05 12:34:56.1755,1903-12-25 -1903-05-05 12:34:56.1756,1903-12-25 -1903-05-05 12:34:56.1757,1903-12-25 -1903-05-05 12:34:56.1758,1903-12-25 -1903-05-05 12:34:56.1759,1903-12-25 -1903-05-05 12:34:56.176,1903-12-25 -1903-05-05 12:34:56.1761,1903-12-25 -1903-05-05 12:34:56.1762,1903-12-25 -1903-05-05 12:34:56.1763,1903-12-25 -1903-05-05 12:34:56.1764,1903-12-25 -1903-05-05 12:34:56.1765,1903-12-25 -1903-05-05 12:34:56.1766,1903-12-25 -1903-05-05 12:34:56.1767,1903-12-25 -1903-05-05 12:34:56.1768,1903-12-25 -1903-05-05 12:34:56.1769,1903-12-25 -1903-05-05 12:34:56.177,1903-12-25 -1903-05-05 12:34:56.1771,1903-12-25 -1903-05-05 12:34:56.1772,1903-12-25 -1903-05-05 12:34:56.1773,1903-12-25 -1903-05-05 12:34:56.1774,1903-12-25 -1903-05-05 12:34:56.1775,1903-12-25 -1903-05-05 12:34:56.1776,1903-12-25 -1903-05-05 12:34:56.1777,1903-12-25 -1903-05-05 12:34:56.1778,1903-12-25 -1903-05-05 12:34:56.1779,1903-12-25 -1903-05-05 12:34:56.178,1903-12-25 -1903-05-05 12:34:56.1781,1903-12-25 -1903-05-05 12:34:56.1782,1903-12-25 -1903-05-05 12:34:56.1783,1903-12-25 -1903-05-05 12:34:56.1784,1903-12-25 -1903-05-05 12:34:56.1785,1903-12-25 -1903-05-05 12:34:56.1786,1903-12-25 -1903-05-05 12:34:56.1787,1903-12-25 -1903-05-05 12:34:56.1788,1903-12-25 -1903-05-05 12:34:56.1789,1903-12-25 -1903-05-05 12:34:56.179,1903-12-25 -1903-05-05 12:34:56.1791,1903-12-25 -1903-05-05 12:34:56.1792,1903-12-25 -1903-05-05 12:34:56.1793,1903-12-25 -1903-05-05 12:34:56.1794,1903-12-25 -1903-05-05 12:34:56.1795,1903-12-25 -1903-05-05 12:34:56.1796,1903-12-25 -1903-05-05 12:34:56.1797,1903-12-25 -1903-05-05 12:34:56.1798,1903-12-25 -1903-05-05 12:34:56.1799,1903-12-25 -1903-05-05 12:34:56.18,1903-12-25 -1903-05-05 12:34:56.1801,1903-12-25 -1903-05-05 12:34:56.1802,1903-12-25 -1903-05-05 12:34:56.1803,1903-12-25 -1903-05-05 12:34:56.1804,1903-12-25 -1903-05-05 12:34:56.1805,1903-12-25 -1903-05-05 12:34:56.1806,1903-12-25 -1903-05-05 12:34:56.1807,1903-12-25 -1903-05-05 12:34:56.1808,1903-12-25 -1903-05-05 12:34:56.1809,1903-12-25 -1903-05-05 12:34:56.181,1903-12-25 -1903-05-05 12:34:56.1811,1903-12-25 -1903-05-05 12:34:56.1812,1903-12-25 -1903-05-05 12:34:56.1813,1903-12-25 -1903-05-05 12:34:56.1814,1903-12-25 -1903-05-05 12:34:56.1815,1903-12-25 -1903-05-05 12:34:56.1816,1903-12-25 -1903-05-05 12:34:56.1817,1903-12-25 -1903-05-05 12:34:56.1818,1903-12-25 -1903-05-05 12:34:56.1819,1903-12-25 -1903-05-05 12:34:56.182,1903-12-25 -1903-05-05 12:34:56.1821,1903-12-25 -1903-05-05 12:34:56.1822,1903-12-25 -1903-05-05 12:34:56.1823,1903-12-25 -1903-05-05 12:34:56.1824,1903-12-25 -1903-05-05 12:34:56.1825,1903-12-25 -1903-05-05 12:34:56.1826,1903-12-25 -1903-05-05 12:34:56.1827,1903-12-25 -1903-05-05 12:34:56.1828,1903-12-25 -1903-05-05 12:34:56.1829,1903-12-25 -1903-05-05 12:34:56.183,1903-12-25 -1903-05-05 12:34:56.1831,1903-12-25 -1903-05-05 12:34:56.1832,1903-12-25 -1903-05-05 12:34:56.1833,1903-12-25 -1903-05-05 12:34:56.1834,1903-12-25 -1903-05-05 12:34:56.1835,1903-12-25 -1903-05-05 12:34:56.1836,1903-12-25 -1903-05-05 12:34:56.1837,1903-12-25 -1903-05-05 12:34:56.1838,1903-12-25 -1903-05-05 12:34:56.1839,1903-12-25 -1903-05-05 12:34:56.184,1903-12-25 -1903-05-05 12:34:56.1841,1903-12-25 -1903-05-05 12:34:56.1842,1903-12-25 -1903-05-05 12:34:56.1843,1903-12-25 -1903-05-05 12:34:56.1844,1903-12-25 -1903-05-05 12:34:56.1845,1903-12-25 -1903-05-05 12:34:56.1846,1903-12-25 -1903-05-05 12:34:56.1847,1903-12-25 -1903-05-05 12:34:56.1848,1903-12-25 -1903-05-05 12:34:56.1849,1903-12-25 -1903-05-05 12:34:56.185,1903-12-25 -1903-05-05 12:34:56.1851,1903-12-25 -1903-05-05 12:34:56.1852,1903-12-25 -1903-05-05 12:34:56.1853,1903-12-25 -1903-05-05 12:34:56.1854,1903-12-25 -1903-05-05 12:34:56.1855,1903-12-25 -1903-05-05 12:34:56.1856,1903-12-25 -1903-05-05 12:34:56.1857,1903-12-25 -1903-05-05 12:34:56.1858,1903-12-25 -1903-05-05 12:34:56.1859,1903-12-25 -1903-05-05 12:34:56.186,1903-12-25 -1903-05-05 12:34:56.1861,1903-12-25 -1903-05-05 12:34:56.1862,1903-12-25 -1903-05-05 12:34:56.1863,1903-12-25 -1903-05-05 12:34:56.1864,1903-12-25 -1903-05-05 12:34:56.1865,1903-12-25 -1903-05-05 12:34:56.1866,1903-12-25 -1903-05-05 12:34:56.1867,1903-12-25 -1903-05-05 12:34:56.1868,1903-12-25 -1903-05-05 12:34:56.1869,1903-12-25 -1903-05-05 12:34:56.187,1903-12-25 -1903-05-05 12:34:56.1871,1903-12-25 -1903-05-05 12:34:56.1872,1903-12-25 -1903-05-05 12:34:56.1873,1903-12-25 -1903-05-05 12:34:56.1874,1903-12-25 -1903-05-05 12:34:56.1875,1903-12-25 -1903-05-05 12:34:56.1876,1903-12-25 -1903-05-05 12:34:56.1877,1903-12-25 -1903-05-05 12:34:56.1878,1903-12-25 -1903-05-05 12:34:56.1879,1903-12-25 -1903-05-05 12:34:56.188,1903-12-25 -1903-05-05 12:34:56.1881,1903-12-25 -1903-05-05 12:34:56.1882,1903-12-25 -1903-05-05 12:34:56.1883,1903-12-25 -1903-05-05 12:34:56.1884,1903-12-25 -1903-05-05 12:34:56.1885,1903-12-25 -1903-05-05 12:34:56.1886,1903-12-25 -1903-05-05 12:34:56.1887,1903-12-25 -1903-05-05 12:34:56.1888,1903-12-25 -1903-05-05 12:34:56.1889,1903-12-25 -1903-05-05 12:34:56.189,1903-12-25 -1903-05-05 12:34:56.1891,1903-12-25 -1903-05-05 12:34:56.1892,1903-12-25 -1903-05-05 12:34:56.1893,1903-12-25 -1903-05-05 12:34:56.1894,1903-12-25 -1903-05-05 12:34:56.1895,1903-12-25 -1903-05-05 12:34:56.1896,1903-12-25 -1903-05-05 12:34:56.1897,1903-12-25 -1903-05-05 12:34:56.1898,1903-12-25 -1903-05-05 12:34:56.1899,1903-12-25 -1903-05-05 12:34:56.19,1903-12-25 -1903-05-05 12:34:56.1901,1903-12-25 -1903-05-05 12:34:56.1902,1903-12-25 -1903-05-05 12:34:56.1903,1903-12-25 -1903-05-05 12:34:56.1904,1903-12-25 -1903-05-05 12:34:56.1905,1903-12-25 -1903-05-05 12:34:56.1906,1903-12-25 -1903-05-05 12:34:56.1907,1903-12-25 -1903-05-05 12:34:56.1908,1903-12-25 -1903-05-05 12:34:56.1909,1903-12-25 -1903-05-05 12:34:56.191,1903-12-25 -1903-05-05 12:34:56.1911,1903-12-25 -1903-05-05 12:34:56.1912,1903-12-25 -1903-05-05 12:34:56.1913,1903-12-25 -1903-05-05 12:34:56.1914,1903-12-25 -1903-05-05 12:34:56.1915,1903-12-25 -1903-05-05 12:34:56.1916,1903-12-25 -1903-05-05 12:34:56.1917,1903-12-25 -1903-05-05 12:34:56.1918,1903-12-25 -1903-05-05 12:34:56.1919,1903-12-25 -1903-05-05 12:34:56.192,1903-12-25 -1903-05-05 12:34:56.1921,1903-12-25 -1903-05-05 12:34:56.1922,1903-12-25 -1903-05-05 12:34:56.1923,1903-12-25 -1903-05-05 12:34:56.1924,1903-12-25 -1903-05-05 12:34:56.1925,1903-12-25 -1903-05-05 12:34:56.1926,1903-12-25 -1903-05-05 12:34:56.1927,1903-12-25 -1903-05-05 12:34:56.1928,1903-12-25 -1903-05-05 12:34:56.1929,1903-12-25 -1903-05-05 12:34:56.193,1903-12-25 -1903-05-05 12:34:56.1931,1903-12-25 -1903-05-05 12:34:56.1932,1903-12-25 -1903-05-05 12:34:56.1933,1903-12-25 -1903-05-05 12:34:56.1934,1903-12-25 -1903-05-05 12:34:56.1935,1903-12-25 -1903-05-05 12:34:56.1936,1903-12-25 -1903-05-05 12:34:56.1937,1903-12-25 -1903-05-05 12:34:56.1938,1903-12-25 -1903-05-05 12:34:56.1939,1903-12-25 -1903-05-05 12:34:56.194,1903-12-25 -1903-05-05 12:34:56.1941,1903-12-25 -1903-05-05 12:34:56.1942,1903-12-25 -1903-05-05 12:34:56.1943,1903-12-25 -1903-05-05 12:34:56.1944,1903-12-25 -1903-05-05 12:34:56.1945,1903-12-25 -1903-05-05 12:34:56.1946,1903-12-25 -1903-05-05 12:34:56.1947,1903-12-25 -1903-05-05 12:34:56.1948,1903-12-25 -1903-05-05 12:34:56.1949,1903-12-25 -1903-05-05 12:34:56.195,1903-12-25 -1903-05-05 12:34:56.1951,1903-12-25 -1903-05-05 12:34:56.1952,1903-12-25 -1903-05-05 12:34:56.1953,1903-12-25 -1903-05-05 12:34:56.1954,1903-12-25 -1903-05-05 12:34:56.1955,1903-12-25 -1903-05-05 12:34:56.1956,1903-12-25 -1903-05-05 12:34:56.1957,1903-12-25 -1903-05-05 12:34:56.1958,1903-12-25 -1903-05-05 12:34:56.1959,1903-12-25 -1903-05-05 12:34:56.196,1903-12-25 -1903-05-05 12:34:56.1961,1903-12-25 -1903-05-05 12:34:56.1962,1903-12-25 -1903-05-05 12:34:56.1963,1903-12-25 -1903-05-05 12:34:56.1964,1903-12-25 -1903-05-05 12:34:56.1965,1903-12-25 -1903-05-05 12:34:56.1966,1903-12-25 -1903-05-05 12:34:56.1967,1903-12-25 -1903-05-05 12:34:56.1968,1903-12-25 -1903-05-05 12:34:56.1969,1903-12-25 -1903-05-05 12:34:56.197,1903-12-25 -1903-05-05 12:34:56.1971,1903-12-25 -1903-05-05 12:34:56.1972,1903-12-25 -1903-05-05 12:34:56.1973,1903-12-25 -1903-05-05 12:34:56.1974,1903-12-25 -1903-05-05 12:34:56.1975,1903-12-25 -1903-05-05 12:34:56.1976,1903-12-25 -1903-05-05 12:34:56.1977,1903-12-25 -1903-05-05 12:34:56.1978,1903-12-25 -1903-05-05 12:34:56.1979,1903-12-25 -1903-05-05 12:34:56.198,1903-12-25 -1903-05-05 12:34:56.1981,1903-12-25 -1903-05-05 12:34:56.1982,1903-12-25 -1903-05-05 12:34:56.1983,1903-12-25 -1903-05-05 12:34:56.1984,1903-12-25 -1903-05-05 12:34:56.1985,1903-12-25 -1903-05-05 12:34:56.1986,1903-12-25 -1903-05-05 12:34:56.1987,1903-12-25 -1903-05-05 12:34:56.1988,1903-12-25 -1903-05-05 12:34:56.1989,1903-12-25 -1903-05-05 12:34:56.199,1903-12-25 -1903-05-05 12:34:56.1991,1903-12-25 -1903-05-05 12:34:56.1992,1903-12-25 -1903-05-05 12:34:56.1993,1903-12-25 -1903-05-05 12:34:56.1994,1903-12-25 -1903-05-05 12:34:56.1995,1903-12-25 -1903-05-05 12:34:56.1996,1903-12-25 -1903-05-05 12:34:56.1997,1903-12-25 -1903-05-05 12:34:56.1998,1903-12-25 -1903-05-05 12:34:56.1999,1903-12-25 -1904-05-05 12:34:56.1,1904-12-25 -1904-05-05 12:34:56.1001,1904-12-25 -1904-05-05 12:34:56.1002,1904-12-25 -1904-05-05 12:34:56.1003,1904-12-25 -1904-05-05 12:34:56.1004,1904-12-25 -1904-05-05 12:34:56.1005,1904-12-25 -1904-05-05 12:34:56.1006,1904-12-25 -1904-05-05 12:34:56.1007,1904-12-25 -1904-05-05 12:34:56.1008,1904-12-25 -1904-05-05 12:34:56.1009,1904-12-25 -1904-05-05 12:34:56.101,1904-12-25 -1904-05-05 12:34:56.1011,1904-12-25 -1904-05-05 12:34:56.1012,1904-12-25 -1904-05-05 12:34:56.1013,1904-12-25 -1904-05-05 12:34:56.1014,1904-12-25 -1904-05-05 12:34:56.1015,1904-12-25 -1904-05-05 12:34:56.1016,1904-12-25 -1904-05-05 12:34:56.1017,1904-12-25 -1904-05-05 12:34:56.1018,1904-12-25 -1904-05-05 12:34:56.1019,1904-12-25 -1904-05-05 12:34:56.102,1904-12-25 -1904-05-05 12:34:56.1021,1904-12-25 -1904-05-05 12:34:56.1022,1904-12-25 -1904-05-05 12:34:56.1023,1904-12-25 -1904-05-05 12:34:56.1024,1904-12-25 -1904-05-05 12:34:56.1025,1904-12-25 -1904-05-05 12:34:56.1026,1904-12-25 -1904-05-05 12:34:56.1027,1904-12-25 -1904-05-05 12:34:56.1028,1904-12-25 -1904-05-05 12:34:56.1029,1904-12-25 -1904-05-05 12:34:56.103,1904-12-25 -1904-05-05 12:34:56.1031,1904-12-25 -1904-05-05 12:34:56.1032,1904-12-25 -1904-05-05 12:34:56.1033,1904-12-25 -1904-05-05 12:34:56.1034,1904-12-25 -1904-05-05 12:34:56.1035,1904-12-25 -1904-05-05 12:34:56.1036,1904-12-25 -1904-05-05 12:34:56.1037,1904-12-25 -1904-05-05 12:34:56.1038,1904-12-25 -1904-05-05 12:34:56.1039,1904-12-25 -1904-05-05 12:34:56.104,1904-12-25 -1904-05-05 12:34:56.1041,1904-12-25 -1904-05-05 12:34:56.1042,1904-12-25 -1904-05-05 12:34:56.1043,1904-12-25 -1904-05-05 12:34:56.1044,1904-12-25 -1904-05-05 12:34:56.1045,1904-12-25 -1904-05-05 12:34:56.1046,1904-12-25 -1904-05-05 12:34:56.1047,1904-12-25 -1904-05-05 12:34:56.1048,1904-12-25 -1904-05-05 12:34:56.1049,1904-12-25 -1904-05-05 12:34:56.105,1904-12-25 -1904-05-05 12:34:56.1051,1904-12-25 -1904-05-05 12:34:56.1052,1904-12-25 -1904-05-05 12:34:56.1053,1904-12-25 -1904-05-05 12:34:56.1054,1904-12-25 -1904-05-05 12:34:56.1055,1904-12-25 -1904-05-05 12:34:56.1056,1904-12-25 -1904-05-05 12:34:56.1057,1904-12-25 -1904-05-05 12:34:56.1058,1904-12-25 -1904-05-05 12:34:56.1059,1904-12-25 -1904-05-05 12:34:56.106,1904-12-25 -1904-05-05 12:34:56.1061,1904-12-25 -1904-05-05 12:34:56.1062,1904-12-25 -1904-05-05 12:34:56.1063,1904-12-25 -1904-05-05 12:34:56.1064,1904-12-25 -1904-05-05 12:34:56.1065,1904-12-25 -1904-05-05 12:34:56.1066,1904-12-25 -1904-05-05 12:34:56.1067,1904-12-25 -1904-05-05 12:34:56.1068,1904-12-25 -1904-05-05 12:34:56.1069,1904-12-25 -1904-05-05 12:34:56.107,1904-12-25 -1904-05-05 12:34:56.1071,1904-12-25 -1904-05-05 12:34:56.1072,1904-12-25 -1904-05-05 12:34:56.1073,1904-12-25 -1904-05-05 12:34:56.1074,1904-12-25 -1904-05-05 12:34:56.1075,1904-12-25 -1904-05-05 12:34:56.1076,1904-12-25 -1904-05-05 12:34:56.1077,1904-12-25 -1904-05-05 12:34:56.1078,1904-12-25 -1904-05-05 12:34:56.1079,1904-12-25 -1904-05-05 12:34:56.108,1904-12-25 -1904-05-05 12:34:56.1081,1904-12-25 -1904-05-05 12:34:56.1082,1904-12-25 -1904-05-05 12:34:56.1083,1904-12-25 -1904-05-05 12:34:56.1084,1904-12-25 -1904-05-05 12:34:56.1085,1904-12-25 -1904-05-05 12:34:56.1086,1904-12-25 -1904-05-05 12:34:56.1087,1904-12-25 -1904-05-05 12:34:56.1088,1904-12-25 -1904-05-05 12:34:56.1089,1904-12-25 -1904-05-05 12:34:56.109,1904-12-25 -1904-05-05 12:34:56.1091,1904-12-25 -1904-05-05 12:34:56.1092,1904-12-25 -1904-05-05 12:34:56.1093,1904-12-25 -1904-05-05 12:34:56.1094,1904-12-25 -1904-05-05 12:34:56.1095,1904-12-25 -1904-05-05 12:34:56.1096,1904-12-25 -1904-05-05 12:34:56.1097,1904-12-25 -1904-05-05 12:34:56.1098,1904-12-25 -1904-05-05 12:34:56.1099,1904-12-25 -1904-05-05 12:34:56.11,1904-12-25 -1904-05-05 12:34:56.1101,1904-12-25 -1904-05-05 12:34:56.1102,1904-12-25 -1904-05-05 12:34:56.1103,1904-12-25 -1904-05-05 12:34:56.1104,1904-12-25 -1904-05-05 12:34:56.1105,1904-12-25 -1904-05-05 12:34:56.1106,1904-12-25 -1904-05-05 12:34:56.1107,1904-12-25 -1904-05-05 12:34:56.1108,1904-12-25 -1904-05-05 12:34:56.1109,1904-12-25 -1904-05-05 12:34:56.111,1904-12-25 -1904-05-05 12:34:56.1111,1904-12-25 -1904-05-05 12:34:56.1112,1904-12-25 -1904-05-05 12:34:56.1113,1904-12-25 -1904-05-05 12:34:56.1114,1904-12-25 -1904-05-05 12:34:56.1115,1904-12-25 -1904-05-05 12:34:56.1116,1904-12-25 -1904-05-05 12:34:56.1117,1904-12-25 -1904-05-05 12:34:56.1118,1904-12-25 -1904-05-05 12:34:56.1119,1904-12-25 -1904-05-05 12:34:56.112,1904-12-25 -1904-05-05 12:34:56.1121,1904-12-25 -1904-05-05 12:34:56.1122,1904-12-25 -1904-05-05 12:34:56.1123,1904-12-25 -1904-05-05 12:34:56.1124,1904-12-25 -1904-05-05 12:34:56.1125,1904-12-25 -1904-05-05 12:34:56.1126,1904-12-25 -1904-05-05 12:34:56.1127,1904-12-25 -1904-05-05 12:34:56.1128,1904-12-25 -1904-05-05 12:34:56.1129,1904-12-25 -1904-05-05 12:34:56.113,1904-12-25 -1904-05-05 12:34:56.1131,1904-12-25 -1904-05-05 12:34:56.1132,1904-12-25 -1904-05-05 12:34:56.1133,1904-12-25 -1904-05-05 12:34:56.1134,1904-12-25 -1904-05-05 12:34:56.1135,1904-12-25 -1904-05-05 12:34:56.1136,1904-12-25 -1904-05-05 12:34:56.1137,1904-12-25 -1904-05-05 12:34:56.1138,1904-12-25 -1904-05-05 12:34:56.1139,1904-12-25 -1904-05-05 12:34:56.114,1904-12-25 -1904-05-05 12:34:56.1141,1904-12-25 -1904-05-05 12:34:56.1142,1904-12-25 -1904-05-05 12:34:56.1143,1904-12-25 -1904-05-05 12:34:56.1144,1904-12-25 -1904-05-05 12:34:56.1145,1904-12-25 -1904-05-05 12:34:56.1146,1904-12-25 -1904-05-05 12:34:56.1147,1904-12-25 -1904-05-05 12:34:56.1148,1904-12-25 -1904-05-05 12:34:56.1149,1904-12-25 -1904-05-05 12:34:56.115,1904-12-25 -1904-05-05 12:34:56.1151,1904-12-25 -1904-05-05 12:34:56.1152,1904-12-25 -1904-05-05 12:34:56.1153,1904-12-25 -1904-05-05 12:34:56.1154,1904-12-25 -1904-05-05 12:34:56.1155,1904-12-25 -1904-05-05 12:34:56.1156,1904-12-25 -1904-05-05 12:34:56.1157,1904-12-25 -1904-05-05 12:34:56.1158,1904-12-25 -1904-05-05 12:34:56.1159,1904-12-25 -1904-05-05 12:34:56.116,1904-12-25 -1904-05-05 12:34:56.1161,1904-12-25 -1904-05-05 12:34:56.1162,1904-12-25 -1904-05-05 12:34:56.1163,1904-12-25 -1904-05-05 12:34:56.1164,1904-12-25 -1904-05-05 12:34:56.1165,1904-12-25 -1904-05-05 12:34:56.1166,1904-12-25 -1904-05-05 12:34:56.1167,1904-12-25 -1904-05-05 12:34:56.1168,1904-12-25 -1904-05-05 12:34:56.1169,1904-12-25 -1904-05-05 12:34:56.117,1904-12-25 -1904-05-05 12:34:56.1171,1904-12-25 -1904-05-05 12:34:56.1172,1904-12-25 -1904-05-05 12:34:56.1173,1904-12-25 -1904-05-05 12:34:56.1174,1904-12-25 -1904-05-05 12:34:56.1175,1904-12-25 -1904-05-05 12:34:56.1176,1904-12-25 -1904-05-05 12:34:56.1177,1904-12-25 -1904-05-05 12:34:56.1178,1904-12-25 -1904-05-05 12:34:56.1179,1904-12-25 -1904-05-05 12:34:56.118,1904-12-25 -1904-05-05 12:34:56.1181,1904-12-25 -1904-05-05 12:34:56.1182,1904-12-25 -1904-05-05 12:34:56.1183,1904-12-25 -1904-05-05 12:34:56.1184,1904-12-25 -1904-05-05 12:34:56.1185,1904-12-25 -1904-05-05 12:34:56.1186,1904-12-25 -1904-05-05 12:34:56.1187,1904-12-25 -1904-05-05 12:34:56.1188,1904-12-25 -1904-05-05 12:34:56.1189,1904-12-25 -1904-05-05 12:34:56.119,1904-12-25 -1904-05-05 12:34:56.1191,1904-12-25 -1904-05-05 12:34:56.1192,1904-12-25 -1904-05-05 12:34:56.1193,1904-12-25 -1904-05-05 12:34:56.1194,1904-12-25 -1904-05-05 12:34:56.1195,1904-12-25 -1904-05-05 12:34:56.1196,1904-12-25 -1904-05-05 12:34:56.1197,1904-12-25 -1904-05-05 12:34:56.1198,1904-12-25 -1904-05-05 12:34:56.1199,1904-12-25 -1904-05-05 12:34:56.12,1904-12-25 -1904-05-05 12:34:56.1201,1904-12-25 -1904-05-05 12:34:56.1202,1904-12-25 -1904-05-05 12:34:56.1203,1904-12-25 -1904-05-05 12:34:56.1204,1904-12-25 -1904-05-05 12:34:56.1205,1904-12-25 -1904-05-05 12:34:56.1206,1904-12-25 -1904-05-05 12:34:56.1207,1904-12-25 -1904-05-05 12:34:56.1208,1904-12-25 -1904-05-05 12:34:56.1209,1904-12-25 -1904-05-05 12:34:56.121,1904-12-25 -1904-05-05 12:34:56.1211,1904-12-25 -1904-05-05 12:34:56.1212,1904-12-25 -1904-05-05 12:34:56.1213,1904-12-25 -1904-05-05 12:34:56.1214,1904-12-25 -1904-05-05 12:34:56.1215,1904-12-25 -1904-05-05 12:34:56.1216,1904-12-25 -1904-05-05 12:34:56.1217,1904-12-25 -1904-05-05 12:34:56.1218,1904-12-25 -1904-05-05 12:34:56.1219,1904-12-25 -1904-05-05 12:34:56.122,1904-12-25 -1904-05-05 12:34:56.1221,1904-12-25 -1904-05-05 12:34:56.1222,1904-12-25 -1904-05-05 12:34:56.1223,1904-12-25 -1904-05-05 12:34:56.1224,1904-12-25 -1904-05-05 12:34:56.1225,1904-12-25 -1904-05-05 12:34:56.1226,1904-12-25 -1904-05-05 12:34:56.1227,1904-12-25 -1904-05-05 12:34:56.1228,1904-12-25 -1904-05-05 12:34:56.1229,1904-12-25 -1904-05-05 12:34:56.123,1904-12-25 -1904-05-05 12:34:56.1231,1904-12-25 -1904-05-05 12:34:56.1232,1904-12-25 -1904-05-05 12:34:56.1233,1904-12-25 -1904-05-05 12:34:56.1234,1904-12-25 -1904-05-05 12:34:56.1235,1904-12-25 -1904-05-05 12:34:56.1236,1904-12-25 -1904-05-05 12:34:56.1237,1904-12-25 -1904-05-05 12:34:56.1238,1904-12-25 -1904-05-05 12:34:56.1239,1904-12-25 -1904-05-05 12:34:56.124,1904-12-25 -1904-05-05 12:34:56.1241,1904-12-25 -1904-05-05 12:34:56.1242,1904-12-25 -1904-05-05 12:34:56.1243,1904-12-25 -1904-05-05 12:34:56.1244,1904-12-25 -1904-05-05 12:34:56.1245,1904-12-25 -1904-05-05 12:34:56.1246,1904-12-25 -1904-05-05 12:34:56.1247,1904-12-25 -1904-05-05 12:34:56.1248,1904-12-25 -1904-05-05 12:34:56.1249,1904-12-25 -1904-05-05 12:34:56.125,1904-12-25 -1904-05-05 12:34:56.1251,1904-12-25 -1904-05-05 12:34:56.1252,1904-12-25 -1904-05-05 12:34:56.1253,1904-12-25 -1904-05-05 12:34:56.1254,1904-12-25 -1904-05-05 12:34:56.1255,1904-12-25 -1904-05-05 12:34:56.1256,1904-12-25 -1904-05-05 12:34:56.1257,1904-12-25 -1904-05-05 12:34:56.1258,1904-12-25 -1904-05-05 12:34:56.1259,1904-12-25 -1904-05-05 12:34:56.126,1904-12-25 -1904-05-05 12:34:56.1261,1904-12-25 -1904-05-05 12:34:56.1262,1904-12-25 -1904-05-05 12:34:56.1263,1904-12-25 -1904-05-05 12:34:56.1264,1904-12-25 -1904-05-05 12:34:56.1265,1904-12-25 -1904-05-05 12:34:56.1266,1904-12-25 -1904-05-05 12:34:56.1267,1904-12-25 -1904-05-05 12:34:56.1268,1904-12-25 -1904-05-05 12:34:56.1269,1904-12-25 -1904-05-05 12:34:56.127,1904-12-25 -1904-05-05 12:34:56.1271,1904-12-25 -1904-05-05 12:34:56.1272,1904-12-25 -1904-05-05 12:34:56.1273,1904-12-25 -1904-05-05 12:34:56.1274,1904-12-25 -1904-05-05 12:34:56.1275,1904-12-25 -1904-05-05 12:34:56.1276,1904-12-25 -1904-05-05 12:34:56.1277,1904-12-25 -1904-05-05 12:34:56.1278,1904-12-25 -1904-05-05 12:34:56.1279,1904-12-25 -1904-05-05 12:34:56.128,1904-12-25 -1904-05-05 12:34:56.1281,1904-12-25 -1904-05-05 12:34:56.1282,1904-12-25 -1904-05-05 12:34:56.1283,1904-12-25 -1904-05-05 12:34:56.1284,1904-12-25 -1904-05-05 12:34:56.1285,1904-12-25 -1904-05-05 12:34:56.1286,1904-12-25 -1904-05-05 12:34:56.1287,1904-12-25 -1904-05-05 12:34:56.1288,1904-12-25 -1904-05-05 12:34:56.1289,1904-12-25 -1904-05-05 12:34:56.129,1904-12-25 -1904-05-05 12:34:56.1291,1904-12-25 -1904-05-05 12:34:56.1292,1904-12-25 -1904-05-05 12:34:56.1293,1904-12-25 -1904-05-05 12:34:56.1294,1904-12-25 -1904-05-05 12:34:56.1295,1904-12-25 -1904-05-05 12:34:56.1296,1904-12-25 -1904-05-05 12:34:56.1297,1904-12-25 -1904-05-05 12:34:56.1298,1904-12-25 -1904-05-05 12:34:56.1299,1904-12-25 -1904-05-05 12:34:56.13,1904-12-25 -1904-05-05 12:34:56.1301,1904-12-25 -1904-05-05 12:34:56.1302,1904-12-25 -1904-05-05 12:34:56.1303,1904-12-25 -1904-05-05 12:34:56.1304,1904-12-25 -1904-05-05 12:34:56.1305,1904-12-25 -1904-05-05 12:34:56.1306,1904-12-25 -1904-05-05 12:34:56.1307,1904-12-25 -1904-05-05 12:34:56.1308,1904-12-25 -1904-05-05 12:34:56.1309,1904-12-25 -1904-05-05 12:34:56.131,1904-12-25 -1904-05-05 12:34:56.1311,1904-12-25 -1904-05-05 12:34:56.1312,1904-12-25 -1904-05-05 12:34:56.1313,1904-12-25 -1904-05-05 12:34:56.1314,1904-12-25 -1904-05-05 12:34:56.1315,1904-12-25 -1904-05-05 12:34:56.1316,1904-12-25 -1904-05-05 12:34:56.1317,1904-12-25 -1904-05-05 12:34:56.1318,1904-12-25 -1904-05-05 12:34:56.1319,1904-12-25 -1904-05-05 12:34:56.132,1904-12-25 -1904-05-05 12:34:56.1321,1904-12-25 -1904-05-05 12:34:56.1322,1904-12-25 -1904-05-05 12:34:56.1323,1904-12-25 -1904-05-05 12:34:56.1324,1904-12-25 -1904-05-05 12:34:56.1325,1904-12-25 -1904-05-05 12:34:56.1326,1904-12-25 -1904-05-05 12:34:56.1327,1904-12-25 -1904-05-05 12:34:56.1328,1904-12-25 -1904-05-05 12:34:56.1329,1904-12-25 -1904-05-05 12:34:56.133,1904-12-25 -1904-05-05 12:34:56.1331,1904-12-25 -1904-05-05 12:34:56.1332,1904-12-25 -1904-05-05 12:34:56.1333,1904-12-25 -1904-05-05 12:34:56.1334,1904-12-25 -1904-05-05 12:34:56.1335,1904-12-25 -1904-05-05 12:34:56.1336,1904-12-25 -1904-05-05 12:34:56.1337,1904-12-25 -1904-05-05 12:34:56.1338,1904-12-25 -1904-05-05 12:34:56.1339,1904-12-25 -1904-05-05 12:34:56.134,1904-12-25 -1904-05-05 12:34:56.1341,1904-12-25 -1904-05-05 12:34:56.1342,1904-12-25 -1904-05-05 12:34:56.1343,1904-12-25 -1904-05-05 12:34:56.1344,1904-12-25 -1904-05-05 12:34:56.1345,1904-12-25 -1904-05-05 12:34:56.1346,1904-12-25 -1904-05-05 12:34:56.1347,1904-12-25 -1904-05-05 12:34:56.1348,1904-12-25 -1904-05-05 12:34:56.1349,1904-12-25 -1904-05-05 12:34:56.135,1904-12-25 -1904-05-05 12:34:56.1351,1904-12-25 -1904-05-05 12:34:56.1352,1904-12-25 -1904-05-05 12:34:56.1353,1904-12-25 -1904-05-05 12:34:56.1354,1904-12-25 -1904-05-05 12:34:56.1355,1904-12-25 -1904-05-05 12:34:56.1356,1904-12-25 -1904-05-05 12:34:56.1357,1904-12-25 -1904-05-05 12:34:56.1358,1904-12-25 -1904-05-05 12:34:56.1359,1904-12-25 -1904-05-05 12:34:56.136,1904-12-25 -1904-05-05 12:34:56.1361,1904-12-25 -1904-05-05 12:34:56.1362,1904-12-25 -1904-05-05 12:34:56.1363,1904-12-25 -1904-05-05 12:34:56.1364,1904-12-25 -1904-05-05 12:34:56.1365,1904-12-25 -1904-05-05 12:34:56.1366,1904-12-25 -1904-05-05 12:34:56.1367,1904-12-25 -1904-05-05 12:34:56.1368,1904-12-25 -1904-05-05 12:34:56.1369,1904-12-25 -1904-05-05 12:34:56.137,1904-12-25 -1904-05-05 12:34:56.1371,1904-12-25 -1904-05-05 12:34:56.1372,1904-12-25 -1904-05-05 12:34:56.1373,1904-12-25 -1904-05-05 12:34:56.1374,1904-12-25 -1904-05-05 12:34:56.1375,1904-12-25 -1904-05-05 12:34:56.1376,1904-12-25 -1904-05-05 12:34:56.1377,1904-12-25 -1904-05-05 12:34:56.1378,1904-12-25 -1904-05-05 12:34:56.1379,1904-12-25 -1904-05-05 12:34:56.138,1904-12-25 -1904-05-05 12:34:56.1381,1904-12-25 -1904-05-05 12:34:56.1382,1904-12-25 -1904-05-05 12:34:56.1383,1904-12-25 -1904-05-05 12:34:56.1384,1904-12-25 -1904-05-05 12:34:56.1385,1904-12-25 -1904-05-05 12:34:56.1386,1904-12-25 -1904-05-05 12:34:56.1387,1904-12-25 -1904-05-05 12:34:56.1388,1904-12-25 -1904-05-05 12:34:56.1389,1904-12-25 -1904-05-05 12:34:56.139,1904-12-25 -1904-05-05 12:34:56.1391,1904-12-25 -1904-05-05 12:34:56.1392,1904-12-25 -1904-05-05 12:34:56.1393,1904-12-25 -1904-05-05 12:34:56.1394,1904-12-25 -1904-05-05 12:34:56.1395,1904-12-25 -1904-05-05 12:34:56.1396,1904-12-25 -1904-05-05 12:34:56.1397,1904-12-25 -1904-05-05 12:34:56.1398,1904-12-25 -1904-05-05 12:34:56.1399,1904-12-25 -1904-05-05 12:34:56.14,1904-12-25 -1904-05-05 12:34:56.1401,1904-12-25 -1904-05-05 12:34:56.1402,1904-12-25 -1904-05-05 12:34:56.1403,1904-12-25 -1904-05-05 12:34:56.1404,1904-12-25 -1904-05-05 12:34:56.1405,1904-12-25 -1904-05-05 12:34:56.1406,1904-12-25 -1904-05-05 12:34:56.1407,1904-12-25 -1904-05-05 12:34:56.1408,1904-12-25 -1904-05-05 12:34:56.1409,1904-12-25 -1904-05-05 12:34:56.141,1904-12-25 -1904-05-05 12:34:56.1411,1904-12-25 -1904-05-05 12:34:56.1412,1904-12-25 -1904-05-05 12:34:56.1413,1904-12-25 -1904-05-05 12:34:56.1414,1904-12-25 -1904-05-05 12:34:56.1415,1904-12-25 -1904-05-05 12:34:56.1416,1904-12-25 -1904-05-05 12:34:56.1417,1904-12-25 -1904-05-05 12:34:56.1418,1904-12-25 -1904-05-05 12:34:56.1419,1904-12-25 -1904-05-05 12:34:56.142,1904-12-25 -1904-05-05 12:34:56.1421,1904-12-25 -1904-05-05 12:34:56.1422,1904-12-25 -1904-05-05 12:34:56.1423,1904-12-25 -1904-05-05 12:34:56.1424,1904-12-25 -1904-05-05 12:34:56.1425,1904-12-25 -1904-05-05 12:34:56.1426,1904-12-25 -1904-05-05 12:34:56.1427,1904-12-25 -1904-05-05 12:34:56.1428,1904-12-25 -1904-05-05 12:34:56.1429,1904-12-25 -1904-05-05 12:34:56.143,1904-12-25 -1904-05-05 12:34:56.1431,1904-12-25 -1904-05-05 12:34:56.1432,1904-12-25 -1904-05-05 12:34:56.1433,1904-12-25 -1904-05-05 12:34:56.1434,1904-12-25 -1904-05-05 12:34:56.1435,1904-12-25 -1904-05-05 12:34:56.1436,1904-12-25 -1904-05-05 12:34:56.1437,1904-12-25 -1904-05-05 12:34:56.1438,1904-12-25 -1904-05-05 12:34:56.1439,1904-12-25 -1904-05-05 12:34:56.144,1904-12-25 -1904-05-05 12:34:56.1441,1904-12-25 -1904-05-05 12:34:56.1442,1904-12-25 -1904-05-05 12:34:56.1443,1904-12-25 -1904-05-05 12:34:56.1444,1904-12-25 -1904-05-05 12:34:56.1445,1904-12-25 -1904-05-05 12:34:56.1446,1904-12-25 -1904-05-05 12:34:56.1447,1904-12-25 -1904-05-05 12:34:56.1448,1904-12-25 -1904-05-05 12:34:56.1449,1904-12-25 -1904-05-05 12:34:56.145,1904-12-25 -1904-05-05 12:34:56.1451,1904-12-25 -1904-05-05 12:34:56.1452,1904-12-25 -1904-05-05 12:34:56.1453,1904-12-25 -1904-05-05 12:34:56.1454,1904-12-25 -1904-05-05 12:34:56.1455,1904-12-25 -1904-05-05 12:34:56.1456,1904-12-25 -1904-05-05 12:34:56.1457,1904-12-25 -1904-05-05 12:34:56.1458,1904-12-25 -1904-05-05 12:34:56.1459,1904-12-25 -1904-05-05 12:34:56.146,1904-12-25 -1904-05-05 12:34:56.1461,1904-12-25 -1904-05-05 12:34:56.1462,1904-12-25 -1904-05-05 12:34:56.1463,1904-12-25 -1904-05-05 12:34:56.1464,1904-12-25 -1904-05-05 12:34:56.1465,1904-12-25 -1904-05-05 12:34:56.1466,1904-12-25 -1904-05-05 12:34:56.1467,1904-12-25 -1904-05-05 12:34:56.1468,1904-12-25 -1904-05-05 12:34:56.1469,1904-12-25 -1904-05-05 12:34:56.147,1904-12-25 -1904-05-05 12:34:56.1471,1904-12-25 -1904-05-05 12:34:56.1472,1904-12-25 -1904-05-05 12:34:56.1473,1904-12-25 -1904-05-05 12:34:56.1474,1904-12-25 -1904-05-05 12:34:56.1475,1904-12-25 -1904-05-05 12:34:56.1476,1904-12-25 -1904-05-05 12:34:56.1477,1904-12-25 -1904-05-05 12:34:56.1478,1904-12-25 -1904-05-05 12:34:56.1479,1904-12-25 -1904-05-05 12:34:56.148,1904-12-25 -1904-05-05 12:34:56.1481,1904-12-25 -1904-05-05 12:34:56.1482,1904-12-25 -1904-05-05 12:34:56.1483,1904-12-25 -1904-05-05 12:34:56.1484,1904-12-25 -1904-05-05 12:34:56.1485,1904-12-25 -1904-05-05 12:34:56.1486,1904-12-25 -1904-05-05 12:34:56.1487,1904-12-25 -1904-05-05 12:34:56.1488,1904-12-25 -1904-05-05 12:34:56.1489,1904-12-25 -1904-05-05 12:34:56.149,1904-12-25 -1904-05-05 12:34:56.1491,1904-12-25 -1904-05-05 12:34:56.1492,1904-12-25 -1904-05-05 12:34:56.1493,1904-12-25 -1904-05-05 12:34:56.1494,1904-12-25 -1904-05-05 12:34:56.1495,1904-12-25 -1904-05-05 12:34:56.1496,1904-12-25 -1904-05-05 12:34:56.1497,1904-12-25 -1904-05-05 12:34:56.1498,1904-12-25 -1904-05-05 12:34:56.1499,1904-12-25 -1904-05-05 12:34:56.15,1904-12-25 -1904-05-05 12:34:56.1501,1904-12-25 -1904-05-05 12:34:56.1502,1904-12-25 -1904-05-05 12:34:56.1503,1904-12-25 -1904-05-05 12:34:56.1504,1904-12-25 -1904-05-05 12:34:56.1505,1904-12-25 -1904-05-05 12:34:56.1506,1904-12-25 -1904-05-05 12:34:56.1507,1904-12-25 -1904-05-05 12:34:56.1508,1904-12-25 -1904-05-05 12:34:56.1509,1904-12-25 -1904-05-05 12:34:56.151,1904-12-25 -1904-05-05 12:34:56.1511,1904-12-25 -1904-05-05 12:34:56.1512,1904-12-25 -1904-05-05 12:34:56.1513,1904-12-25 -1904-05-05 12:34:56.1514,1904-12-25 -1904-05-05 12:34:56.1515,1904-12-25 -1904-05-05 12:34:56.1516,1904-12-25 -1904-05-05 12:34:56.1517,1904-12-25 -1904-05-05 12:34:56.1518,1904-12-25 -1904-05-05 12:34:56.1519,1904-12-25 -1904-05-05 12:34:56.152,1904-12-25 -1904-05-05 12:34:56.1521,1904-12-25 -1904-05-05 12:34:56.1522,1904-12-25 -1904-05-05 12:34:56.1523,1904-12-25 -1904-05-05 12:34:56.1524,1904-12-25 -1904-05-05 12:34:56.1525,1904-12-25 -1904-05-05 12:34:56.1526,1904-12-25 -1904-05-05 12:34:56.1527,1904-12-25 -1904-05-05 12:34:56.1528,1904-12-25 -1904-05-05 12:34:56.1529,1904-12-25 -1904-05-05 12:34:56.153,1904-12-25 -1904-05-05 12:34:56.1531,1904-12-25 -1904-05-05 12:34:56.1532,1904-12-25 -1904-05-05 12:34:56.1533,1904-12-25 -1904-05-05 12:34:56.1534,1904-12-25 -1904-05-05 12:34:56.1535,1904-12-25 -1904-05-05 12:34:56.1536,1904-12-25 -1904-05-05 12:34:56.1537,1904-12-25 -1904-05-05 12:34:56.1538,1904-12-25 -1904-05-05 12:34:56.1539,1904-12-25 -1904-05-05 12:34:56.154,1904-12-25 -1904-05-05 12:34:56.1541,1904-12-25 -1904-05-05 12:34:56.1542,1904-12-25 -1904-05-05 12:34:56.1543,1904-12-25 -1904-05-05 12:34:56.1544,1904-12-25 -1904-05-05 12:34:56.1545,1904-12-25 -1904-05-05 12:34:56.1546,1904-12-25 -1904-05-05 12:34:56.1547,1904-12-25 -1904-05-05 12:34:56.1548,1904-12-25 -1904-05-05 12:34:56.1549,1904-12-25 -1904-05-05 12:34:56.155,1904-12-25 -1904-05-05 12:34:56.1551,1904-12-25 -1904-05-05 12:34:56.1552,1904-12-25 -1904-05-05 12:34:56.1553,1904-12-25 -1904-05-05 12:34:56.1554,1904-12-25 -1904-05-05 12:34:56.1555,1904-12-25 -1904-05-05 12:34:56.1556,1904-12-25 -1904-05-05 12:34:56.1557,1904-12-25 -1904-05-05 12:34:56.1558,1904-12-25 -1904-05-05 12:34:56.1559,1904-12-25 -1904-05-05 12:34:56.156,1904-12-25 -1904-05-05 12:34:56.1561,1904-12-25 -1904-05-05 12:34:56.1562,1904-12-25 -1904-05-05 12:34:56.1563,1904-12-25 -1904-05-05 12:34:56.1564,1904-12-25 -1904-05-05 12:34:56.1565,1904-12-25 -1904-05-05 12:34:56.1566,1904-12-25 -1904-05-05 12:34:56.1567,1904-12-25 -1904-05-05 12:34:56.1568,1904-12-25 -1904-05-05 12:34:56.1569,1904-12-25 -1904-05-05 12:34:56.157,1904-12-25 -1904-05-05 12:34:56.1571,1904-12-25 -1904-05-05 12:34:56.1572,1904-12-25 -1904-05-05 12:34:56.1573,1904-12-25 -1904-05-05 12:34:56.1574,1904-12-25 -1904-05-05 12:34:56.1575,1904-12-25 -1904-05-05 12:34:56.1576,1904-12-25 -1904-05-05 12:34:56.1577,1904-12-25 -1904-05-05 12:34:56.1578,1904-12-25 -1904-05-05 12:34:56.1579,1904-12-25 -1904-05-05 12:34:56.158,1904-12-25 -1904-05-05 12:34:56.1581,1904-12-25 -1904-05-05 12:34:56.1582,1904-12-25 -1904-05-05 12:34:56.1583,1904-12-25 -1904-05-05 12:34:56.1584,1904-12-25 -1904-05-05 12:34:56.1585,1904-12-25 -1904-05-05 12:34:56.1586,1904-12-25 -1904-05-05 12:34:56.1587,1904-12-25 -1904-05-05 12:34:56.1588,1904-12-25 -1904-05-05 12:34:56.1589,1904-12-25 -1904-05-05 12:34:56.159,1904-12-25 -1904-05-05 12:34:56.1591,1904-12-25 -1904-05-05 12:34:56.1592,1904-12-25 -1904-05-05 12:34:56.1593,1904-12-25 -1904-05-05 12:34:56.1594,1904-12-25 -1904-05-05 12:34:56.1595,1904-12-25 -1904-05-05 12:34:56.1596,1904-12-25 -1904-05-05 12:34:56.1597,1904-12-25 -1904-05-05 12:34:56.1598,1904-12-25 -1904-05-05 12:34:56.1599,1904-12-25 -1904-05-05 12:34:56.16,1904-12-25 -1904-05-05 12:34:56.1601,1904-12-25 -1904-05-05 12:34:56.1602,1904-12-25 -1904-05-05 12:34:56.1603,1904-12-25 -1904-05-05 12:34:56.1604,1904-12-25 -1904-05-05 12:34:56.1605,1904-12-25 -1904-05-05 12:34:56.1606,1904-12-25 -1904-05-05 12:34:56.1607,1904-12-25 -1904-05-05 12:34:56.1608,1904-12-25 -1904-05-05 12:34:56.1609,1904-12-25 -1904-05-05 12:34:56.161,1904-12-25 -1904-05-05 12:34:56.1611,1904-12-25 -1904-05-05 12:34:56.1612,1904-12-25 -1904-05-05 12:34:56.1613,1904-12-25 -1904-05-05 12:34:56.1614,1904-12-25 -1904-05-05 12:34:56.1615,1904-12-25 -1904-05-05 12:34:56.1616,1904-12-25 -1904-05-05 12:34:56.1617,1904-12-25 -1904-05-05 12:34:56.1618,1904-12-25 -1904-05-05 12:34:56.1619,1904-12-25 -1904-05-05 12:34:56.162,1904-12-25 -1904-05-05 12:34:56.1621,1904-12-25 -1904-05-05 12:34:56.1622,1904-12-25 -1904-05-05 12:34:56.1623,1904-12-25 -1904-05-05 12:34:56.1624,1904-12-25 -1904-05-05 12:34:56.1625,1904-12-25 -1904-05-05 12:34:56.1626,1904-12-25 -1904-05-05 12:34:56.1627,1904-12-25 -1904-05-05 12:34:56.1628,1904-12-25 -1904-05-05 12:34:56.1629,1904-12-25 -1904-05-05 12:34:56.163,1904-12-25 -1904-05-05 12:34:56.1631,1904-12-25 -1904-05-05 12:34:56.1632,1904-12-25 -1904-05-05 12:34:56.1633,1904-12-25 -1904-05-05 12:34:56.1634,1904-12-25 -1904-05-05 12:34:56.1635,1904-12-25 -1904-05-05 12:34:56.1636,1904-12-25 -1904-05-05 12:34:56.1637,1904-12-25 -1904-05-05 12:34:56.1638,1904-12-25 -1904-05-05 12:34:56.1639,1904-12-25 -1904-05-05 12:34:56.164,1904-12-25 -1904-05-05 12:34:56.1641,1904-12-25 -1904-05-05 12:34:56.1642,1904-12-25 -1904-05-05 12:34:56.1643,1904-12-25 -1904-05-05 12:34:56.1644,1904-12-25 -1904-05-05 12:34:56.1645,1904-12-25 -1904-05-05 12:34:56.1646,1904-12-25 -1904-05-05 12:34:56.1647,1904-12-25 -1904-05-05 12:34:56.1648,1904-12-25 -1904-05-05 12:34:56.1649,1904-12-25 -1904-05-05 12:34:56.165,1904-12-25 -1904-05-05 12:34:56.1651,1904-12-25 -1904-05-05 12:34:56.1652,1904-12-25 -1904-05-05 12:34:56.1653,1904-12-25 -1904-05-05 12:34:56.1654,1904-12-25 -1904-05-05 12:34:56.1655,1904-12-25 -1904-05-05 12:34:56.1656,1904-12-25 -1904-05-05 12:34:56.1657,1904-12-25 -1904-05-05 12:34:56.1658,1904-12-25 -1904-05-05 12:34:56.1659,1904-12-25 -1904-05-05 12:34:56.166,1904-12-25 -1904-05-05 12:34:56.1661,1904-12-25 -1904-05-05 12:34:56.1662,1904-12-25 -1904-05-05 12:34:56.1663,1904-12-25 -1904-05-05 12:34:56.1664,1904-12-25 -1904-05-05 12:34:56.1665,1904-12-25 -1904-05-05 12:34:56.1666,1904-12-25 -1904-05-05 12:34:56.1667,1904-12-25 -1904-05-05 12:34:56.1668,1904-12-25 -1904-05-05 12:34:56.1669,1904-12-25 -1904-05-05 12:34:56.167,1904-12-25 -1904-05-05 12:34:56.1671,1904-12-25 -1904-05-05 12:34:56.1672,1904-12-25 -1904-05-05 12:34:56.1673,1904-12-25 -1904-05-05 12:34:56.1674,1904-12-25 -1904-05-05 12:34:56.1675,1904-12-25 -1904-05-05 12:34:56.1676,1904-12-25 -1904-05-05 12:34:56.1677,1904-12-25 -1904-05-05 12:34:56.1678,1904-12-25 -1904-05-05 12:34:56.1679,1904-12-25 -1904-05-05 12:34:56.168,1904-12-25 -1904-05-05 12:34:56.1681,1904-12-25 -1904-05-05 12:34:56.1682,1904-12-25 -1904-05-05 12:34:56.1683,1904-12-25 -1904-05-05 12:34:56.1684,1904-12-25 -1904-05-05 12:34:56.1685,1904-12-25 -1904-05-05 12:34:56.1686,1904-12-25 -1904-05-05 12:34:56.1687,1904-12-25 -1904-05-05 12:34:56.1688,1904-12-25 -1904-05-05 12:34:56.1689,1904-12-25 -1904-05-05 12:34:56.169,1904-12-25 -1904-05-05 12:34:56.1691,1904-12-25 -1904-05-05 12:34:56.1692,1904-12-25 -1904-05-05 12:34:56.1693,1904-12-25 -1904-05-05 12:34:56.1694,1904-12-25 -1904-05-05 12:34:56.1695,1904-12-25 -1904-05-05 12:34:56.1696,1904-12-25 -1904-05-05 12:34:56.1697,1904-12-25 -1904-05-05 12:34:56.1698,1904-12-25 -1904-05-05 12:34:56.1699,1904-12-25 -1904-05-05 12:34:56.17,1904-12-25 -1904-05-05 12:34:56.1701,1904-12-25 -1904-05-05 12:34:56.1702,1904-12-25 -1904-05-05 12:34:56.1703,1904-12-25 -1904-05-05 12:34:56.1704,1904-12-25 -1904-05-05 12:34:56.1705,1904-12-25 -1904-05-05 12:34:56.1706,1904-12-25 -1904-05-05 12:34:56.1707,1904-12-25 -1904-05-05 12:34:56.1708,1904-12-25 -1904-05-05 12:34:56.1709,1904-12-25 -1904-05-05 12:34:56.171,1904-12-25 -1904-05-05 12:34:56.1711,1904-12-25 -1904-05-05 12:34:56.1712,1904-12-25 -1904-05-05 12:34:56.1713,1904-12-25 -1904-05-05 12:34:56.1714,1904-12-25 -1904-05-05 12:34:56.1715,1904-12-25 -1904-05-05 12:34:56.1716,1904-12-25 -1904-05-05 12:34:56.1717,1904-12-25 -1904-05-05 12:34:56.1718,1904-12-25 -1904-05-05 12:34:56.1719,1904-12-25 -1904-05-05 12:34:56.172,1904-12-25 -1904-05-05 12:34:56.1721,1904-12-25 -1904-05-05 12:34:56.1722,1904-12-25 -1904-05-05 12:34:56.1723,1904-12-25 -1904-05-05 12:34:56.1724,1904-12-25 -1904-05-05 12:34:56.1725,1904-12-25 -1904-05-05 12:34:56.1726,1904-12-25 -1904-05-05 12:34:56.1727,1904-12-25 -1904-05-05 12:34:56.1728,1904-12-25 -1904-05-05 12:34:56.1729,1904-12-25 -1904-05-05 12:34:56.173,1904-12-25 -1904-05-05 12:34:56.1731,1904-12-25 -1904-05-05 12:34:56.1732,1904-12-25 -1904-05-05 12:34:56.1733,1904-12-25 -1904-05-05 12:34:56.1734,1904-12-25 -1904-05-05 12:34:56.1735,1904-12-25 -1904-05-05 12:34:56.1736,1904-12-25 -1904-05-05 12:34:56.1737,1904-12-25 -1904-05-05 12:34:56.1738,1904-12-25 -1904-05-05 12:34:56.1739,1904-12-25 -1904-05-05 12:34:56.174,1904-12-25 -1904-05-05 12:34:56.1741,1904-12-25 -1904-05-05 12:34:56.1742,1904-12-25 -1904-05-05 12:34:56.1743,1904-12-25 -1904-05-05 12:34:56.1744,1904-12-25 -1904-05-05 12:34:56.1745,1904-12-25 -1904-05-05 12:34:56.1746,1904-12-25 -1904-05-05 12:34:56.1747,1904-12-25 -1904-05-05 12:34:56.1748,1904-12-25 -1904-05-05 12:34:56.1749,1904-12-25 -1904-05-05 12:34:56.175,1904-12-25 -1904-05-05 12:34:56.1751,1904-12-25 -1904-05-05 12:34:56.1752,1904-12-25 -1904-05-05 12:34:56.1753,1904-12-25 -1904-05-05 12:34:56.1754,1904-12-25 -1904-05-05 12:34:56.1755,1904-12-25 -1904-05-05 12:34:56.1756,1904-12-25 -1904-05-05 12:34:56.1757,1904-12-25 -1904-05-05 12:34:56.1758,1904-12-25 -1904-05-05 12:34:56.1759,1904-12-25 -1904-05-05 12:34:56.176,1904-12-25 -1904-05-05 12:34:56.1761,1904-12-25 -1904-05-05 12:34:56.1762,1904-12-25 -1904-05-05 12:34:56.1763,1904-12-25 -1904-05-05 12:34:56.1764,1904-12-25 -1904-05-05 12:34:56.1765,1904-12-25 -1904-05-05 12:34:56.1766,1904-12-25 -1904-05-05 12:34:56.1767,1904-12-25 -1904-05-05 12:34:56.1768,1904-12-25 -1904-05-05 12:34:56.1769,1904-12-25 -1904-05-05 12:34:56.177,1904-12-25 -1904-05-05 12:34:56.1771,1904-12-25 -1904-05-05 12:34:56.1772,1904-12-25 -1904-05-05 12:34:56.1773,1904-12-25 -1904-05-05 12:34:56.1774,1904-12-25 -1904-05-05 12:34:56.1775,1904-12-25 -1904-05-05 12:34:56.1776,1904-12-25 -1904-05-05 12:34:56.1777,1904-12-25 -1904-05-05 12:34:56.1778,1904-12-25 -1904-05-05 12:34:56.1779,1904-12-25 -1904-05-05 12:34:56.178,1904-12-25 -1904-05-05 12:34:56.1781,1904-12-25 -1904-05-05 12:34:56.1782,1904-12-25 -1904-05-05 12:34:56.1783,1904-12-25 -1904-05-05 12:34:56.1784,1904-12-25 -1904-05-05 12:34:56.1785,1904-12-25 -1904-05-05 12:34:56.1786,1904-12-25 -1904-05-05 12:34:56.1787,1904-12-25 -1904-05-05 12:34:56.1788,1904-12-25 -1904-05-05 12:34:56.1789,1904-12-25 -1904-05-05 12:34:56.179,1904-12-25 -1904-05-05 12:34:56.1791,1904-12-25 -1904-05-05 12:34:56.1792,1904-12-25 -1904-05-05 12:34:56.1793,1904-12-25 -1904-05-05 12:34:56.1794,1904-12-25 -1904-05-05 12:34:56.1795,1904-12-25 -1904-05-05 12:34:56.1796,1904-12-25 -1904-05-05 12:34:56.1797,1904-12-25 -1904-05-05 12:34:56.1798,1904-12-25 -1904-05-05 12:34:56.1799,1904-12-25 -1904-05-05 12:34:56.18,1904-12-25 -1904-05-05 12:34:56.1801,1904-12-25 -1904-05-05 12:34:56.1802,1904-12-25 -1904-05-05 12:34:56.1803,1904-12-25 -1904-05-05 12:34:56.1804,1904-12-25 -1904-05-05 12:34:56.1805,1904-12-25 -1904-05-05 12:34:56.1806,1904-12-25 -1904-05-05 12:34:56.1807,1904-12-25 -1904-05-05 12:34:56.1808,1904-12-25 -1904-05-05 12:34:56.1809,1904-12-25 -1904-05-05 12:34:56.181,1904-12-25 -1904-05-05 12:34:56.1811,1904-12-25 -1904-05-05 12:34:56.1812,1904-12-25 -1904-05-05 12:34:56.1813,1904-12-25 -1904-05-05 12:34:56.1814,1904-12-25 -1904-05-05 12:34:56.1815,1904-12-25 -1904-05-05 12:34:56.1816,1904-12-25 -1904-05-05 12:34:56.1817,1904-12-25 -1904-05-05 12:34:56.1818,1904-12-25 -1904-05-05 12:34:56.1819,1904-12-25 -1904-05-05 12:34:56.182,1904-12-25 -1904-05-05 12:34:56.1821,1904-12-25 -1904-05-05 12:34:56.1822,1904-12-25 -1904-05-05 12:34:56.1823,1904-12-25 -1904-05-05 12:34:56.1824,1904-12-25 -1904-05-05 12:34:56.1825,1904-12-25 -1904-05-05 12:34:56.1826,1904-12-25 -1904-05-05 12:34:56.1827,1904-12-25 -1904-05-05 12:34:56.1828,1904-12-25 -1904-05-05 12:34:56.1829,1904-12-25 -1904-05-05 12:34:56.183,1904-12-25 -1904-05-05 12:34:56.1831,1904-12-25 -1904-05-05 12:34:56.1832,1904-12-25 -1904-05-05 12:34:56.1833,1904-12-25 -1904-05-05 12:34:56.1834,1904-12-25 -1904-05-05 12:34:56.1835,1904-12-25 -1904-05-05 12:34:56.1836,1904-12-25 -1904-05-05 12:34:56.1837,1904-12-25 -1904-05-05 12:34:56.1838,1904-12-25 -1904-05-05 12:34:56.1839,1904-12-25 -1904-05-05 12:34:56.184,1904-12-25 -1904-05-05 12:34:56.1841,1904-12-25 -1904-05-05 12:34:56.1842,1904-12-25 -1904-05-05 12:34:56.1843,1904-12-25 -1904-05-05 12:34:56.1844,1904-12-25 -1904-05-05 12:34:56.1845,1904-12-25 -1904-05-05 12:34:56.1846,1904-12-25 -1904-05-05 12:34:56.1847,1904-12-25 -1904-05-05 12:34:56.1848,1904-12-25 -1904-05-05 12:34:56.1849,1904-12-25 -1904-05-05 12:34:56.185,1904-12-25 -1904-05-05 12:34:56.1851,1904-12-25 -1904-05-05 12:34:56.1852,1904-12-25 -1904-05-05 12:34:56.1853,1904-12-25 -1904-05-05 12:34:56.1854,1904-12-25 -1904-05-05 12:34:56.1855,1904-12-25 -1904-05-05 12:34:56.1856,1904-12-25 -1904-05-05 12:34:56.1857,1904-12-25 -1904-05-05 12:34:56.1858,1904-12-25 -1904-05-05 12:34:56.1859,1904-12-25 -1904-05-05 12:34:56.186,1904-12-25 -1904-05-05 12:34:56.1861,1904-12-25 -1904-05-05 12:34:56.1862,1904-12-25 -1904-05-05 12:34:56.1863,1904-12-25 -1904-05-05 12:34:56.1864,1904-12-25 -1904-05-05 12:34:56.1865,1904-12-25 -1904-05-05 12:34:56.1866,1904-12-25 -1904-05-05 12:34:56.1867,1904-12-25 -1904-05-05 12:34:56.1868,1904-12-25 -1904-05-05 12:34:56.1869,1904-12-25 -1904-05-05 12:34:56.187,1904-12-25 -1904-05-05 12:34:56.1871,1904-12-25 -1904-05-05 12:34:56.1872,1904-12-25 -1904-05-05 12:34:56.1873,1904-12-25 -1904-05-05 12:34:56.1874,1904-12-25 -1904-05-05 12:34:56.1875,1904-12-25 -1904-05-05 12:34:56.1876,1904-12-25 -1904-05-05 12:34:56.1877,1904-12-25 -1904-05-05 12:34:56.1878,1904-12-25 -1904-05-05 12:34:56.1879,1904-12-25 -1904-05-05 12:34:56.188,1904-12-25 -1904-05-05 12:34:56.1881,1904-12-25 -1904-05-05 12:34:56.1882,1904-12-25 -1904-05-05 12:34:56.1883,1904-12-25 -1904-05-05 12:34:56.1884,1904-12-25 -1904-05-05 12:34:56.1885,1904-12-25 -1904-05-05 12:34:56.1886,1904-12-25 -1904-05-05 12:34:56.1887,1904-12-25 -1904-05-05 12:34:56.1888,1904-12-25 -1904-05-05 12:34:56.1889,1904-12-25 -1904-05-05 12:34:56.189,1904-12-25 -1904-05-05 12:34:56.1891,1904-12-25 -1904-05-05 12:34:56.1892,1904-12-25 -1904-05-05 12:34:56.1893,1904-12-25 -1904-05-05 12:34:56.1894,1904-12-25 -1904-05-05 12:34:56.1895,1904-12-25 -1904-05-05 12:34:56.1896,1904-12-25 -1904-05-05 12:34:56.1897,1904-12-25 -1904-05-05 12:34:56.1898,1904-12-25 -1904-05-05 12:34:56.1899,1904-12-25 -1904-05-05 12:34:56.19,1904-12-25 -1904-05-05 12:34:56.1901,1904-12-25 -1904-05-05 12:34:56.1902,1904-12-25 -1904-05-05 12:34:56.1903,1904-12-25 -1904-05-05 12:34:56.1904,1904-12-25 -1904-05-05 12:34:56.1905,1904-12-25 -1904-05-05 12:34:56.1906,1904-12-25 -1904-05-05 12:34:56.1907,1904-12-25 -1904-05-05 12:34:56.1908,1904-12-25 -1904-05-05 12:34:56.1909,1904-12-25 -1904-05-05 12:34:56.191,1904-12-25 -1904-05-05 12:34:56.1911,1904-12-25 -1904-05-05 12:34:56.1912,1904-12-25 -1904-05-05 12:34:56.1913,1904-12-25 -1904-05-05 12:34:56.1914,1904-12-25 -1904-05-05 12:34:56.1915,1904-12-25 -1904-05-05 12:34:56.1916,1904-12-25 -1904-05-05 12:34:56.1917,1904-12-25 -1904-05-05 12:34:56.1918,1904-12-25 -1904-05-05 12:34:56.1919,1904-12-25 -1904-05-05 12:34:56.192,1904-12-25 -1904-05-05 12:34:56.1921,1904-12-25 -1904-05-05 12:34:56.1922,1904-12-25 -1904-05-05 12:34:56.1923,1904-12-25 -1904-05-05 12:34:56.1924,1904-12-25 -1904-05-05 12:34:56.1925,1904-12-25 -1904-05-05 12:34:56.1926,1904-12-25 -1904-05-05 12:34:56.1927,1904-12-25 -1904-05-05 12:34:56.1928,1904-12-25 -1904-05-05 12:34:56.1929,1904-12-25 -1904-05-05 12:34:56.193,1904-12-25 -1904-05-05 12:34:56.1931,1904-12-25 -1904-05-05 12:34:56.1932,1904-12-25 -1904-05-05 12:34:56.1933,1904-12-25 -1904-05-05 12:34:56.1934,1904-12-25 -1904-05-05 12:34:56.1935,1904-12-25 -1904-05-05 12:34:56.1936,1904-12-25 -1904-05-05 12:34:56.1937,1904-12-25 -1904-05-05 12:34:56.1938,1904-12-25 -1904-05-05 12:34:56.1939,1904-12-25 -1904-05-05 12:34:56.194,1904-12-25 -1904-05-05 12:34:56.1941,1904-12-25 -1904-05-05 12:34:56.1942,1904-12-25 -1904-05-05 12:34:56.1943,1904-12-25 -1904-05-05 12:34:56.1944,1904-12-25 -1904-05-05 12:34:56.1945,1904-12-25 -1904-05-05 12:34:56.1946,1904-12-25 -1904-05-05 12:34:56.1947,1904-12-25 -1904-05-05 12:34:56.1948,1904-12-25 -1904-05-05 12:34:56.1949,1904-12-25 -1904-05-05 12:34:56.195,1904-12-25 -1904-05-05 12:34:56.1951,1904-12-25 -1904-05-05 12:34:56.1952,1904-12-25 -1904-05-05 12:34:56.1953,1904-12-25 -1904-05-05 12:34:56.1954,1904-12-25 -1904-05-05 12:34:56.1955,1904-12-25 -1904-05-05 12:34:56.1956,1904-12-25 -1904-05-05 12:34:56.1957,1904-12-25 -1904-05-05 12:34:56.1958,1904-12-25 -1904-05-05 12:34:56.1959,1904-12-25 -1904-05-05 12:34:56.196,1904-12-25 -1904-05-05 12:34:56.1961,1904-12-25 -1904-05-05 12:34:56.1962,1904-12-25 -1904-05-05 12:34:56.1963,1904-12-25 -1904-05-05 12:34:56.1964,1904-12-25 -1904-05-05 12:34:56.1965,1904-12-25 -1904-05-05 12:34:56.1966,1904-12-25 -1904-05-05 12:34:56.1967,1904-12-25 -1904-05-05 12:34:56.1968,1904-12-25 -1904-05-05 12:34:56.1969,1904-12-25 -1904-05-05 12:34:56.197,1904-12-25 -1904-05-05 12:34:56.1971,1904-12-25 -1904-05-05 12:34:56.1972,1904-12-25 -1904-05-05 12:34:56.1973,1904-12-25 -1904-05-05 12:34:56.1974,1904-12-25 -1904-05-05 12:34:56.1975,1904-12-25 -1904-05-05 12:34:56.1976,1904-12-25 -1904-05-05 12:34:56.1977,1904-12-25 -1904-05-05 12:34:56.1978,1904-12-25 -1904-05-05 12:34:56.1979,1904-12-25 -1904-05-05 12:34:56.198,1904-12-25 -1904-05-05 12:34:56.1981,1904-12-25 -1904-05-05 12:34:56.1982,1904-12-25 -1904-05-05 12:34:56.1983,1904-12-25 -1904-05-05 12:34:56.1984,1904-12-25 -1904-05-05 12:34:56.1985,1904-12-25 -1904-05-05 12:34:56.1986,1904-12-25 -1904-05-05 12:34:56.1987,1904-12-25 -1904-05-05 12:34:56.1988,1904-12-25 -1904-05-05 12:34:56.1989,1904-12-25 -1904-05-05 12:34:56.199,1904-12-25 -1904-05-05 12:34:56.1991,1904-12-25 -1904-05-05 12:34:56.1992,1904-12-25 -1904-05-05 12:34:56.1993,1904-12-25 -1904-05-05 12:34:56.1994,1904-12-25 -1904-05-05 12:34:56.1995,1904-12-25 -1904-05-05 12:34:56.1996,1904-12-25 -1904-05-05 12:34:56.1997,1904-12-25 -1904-05-05 12:34:56.1998,1904-12-25 -1904-05-05 12:34:56.1999,1904-12-25 -1905-05-05 12:34:56.1,1905-12-25 -1905-05-05 12:34:56.1001,1905-12-25 -1905-05-05 12:34:56.1002,1905-12-25 -1905-05-05 12:34:56.1003,1905-12-25 -1905-05-05 12:34:56.1004,1905-12-25 -1905-05-05 12:34:56.1005,1905-12-25 -1905-05-05 12:34:56.1006,1905-12-25 -1905-05-05 12:34:56.1007,1905-12-25 -1905-05-05 12:34:56.1008,1905-12-25 -1905-05-05 12:34:56.1009,1905-12-25 -1905-05-05 12:34:56.101,1905-12-25 -1905-05-05 12:34:56.1011,1905-12-25 -1905-05-05 12:34:56.1012,1905-12-25 -1905-05-05 12:34:56.1013,1905-12-25 -1905-05-05 12:34:56.1014,1905-12-25 -1905-05-05 12:34:56.1015,1905-12-25 -1905-05-05 12:34:56.1016,1905-12-25 -1905-05-05 12:34:56.1017,1905-12-25 -1905-05-05 12:34:56.1018,1905-12-25 -1905-05-05 12:34:56.1019,1905-12-25 -1905-05-05 12:34:56.102,1905-12-25 -1905-05-05 12:34:56.1021,1905-12-25 -1905-05-05 12:34:56.1022,1905-12-25 -1905-05-05 12:34:56.1023,1905-12-25 -1905-05-05 12:34:56.1024,1905-12-25 -1905-05-05 12:34:56.1025,1905-12-25 -1905-05-05 12:34:56.1026,1905-12-25 -1905-05-05 12:34:56.1027,1905-12-25 -1905-05-05 12:34:56.1028,1905-12-25 -1905-05-05 12:34:56.1029,1905-12-25 -1905-05-05 12:34:56.103,1905-12-25 -1905-05-05 12:34:56.1031,1905-12-25 -1905-05-05 12:34:56.1032,1905-12-25 -1905-05-05 12:34:56.1033,1905-12-25 -1905-05-05 12:34:56.1034,1905-12-25 -1905-05-05 12:34:56.1035,1905-12-25 -1905-05-05 12:34:56.1036,1905-12-25 -1905-05-05 12:34:56.1037,1905-12-25 -1905-05-05 12:34:56.1038,1905-12-25 -1905-05-05 12:34:56.1039,1905-12-25 -1905-05-05 12:34:56.104,1905-12-25 -1905-05-05 12:34:56.1041,1905-12-25 -1905-05-05 12:34:56.1042,1905-12-25 -1905-05-05 12:34:56.1043,1905-12-25 -1905-05-05 12:34:56.1044,1905-12-25 -1905-05-05 12:34:56.1045,1905-12-25 -1905-05-05 12:34:56.1046,1905-12-25 -1905-05-05 12:34:56.1047,1905-12-25 -1905-05-05 12:34:56.1048,1905-12-25 -1905-05-05 12:34:56.1049,1905-12-25 -1905-05-05 12:34:56.105,1905-12-25 -1905-05-05 12:34:56.1051,1905-12-25 -1905-05-05 12:34:56.1052,1905-12-25 -1905-05-05 12:34:56.1053,1905-12-25 -1905-05-05 12:34:56.1054,1905-12-25 -1905-05-05 12:34:56.1055,1905-12-25 -1905-05-05 12:34:56.1056,1905-12-25 -1905-05-05 12:34:56.1057,1905-12-25 -1905-05-05 12:34:56.1058,1905-12-25 -1905-05-05 12:34:56.1059,1905-12-25 -1905-05-05 12:34:56.106,1905-12-25 -1905-05-05 12:34:56.1061,1905-12-25 -1905-05-05 12:34:56.1062,1905-12-25 -1905-05-05 12:34:56.1063,1905-12-25 -1905-05-05 12:34:56.1064,1905-12-25 -1905-05-05 12:34:56.1065,1905-12-25 -1905-05-05 12:34:56.1066,1905-12-25 -1905-05-05 12:34:56.1067,1905-12-25 -1905-05-05 12:34:56.1068,1905-12-25 -1905-05-05 12:34:56.1069,1905-12-25 -1905-05-05 12:34:56.107,1905-12-25 -1905-05-05 12:34:56.1071,1905-12-25 -1905-05-05 12:34:56.1072,1905-12-25 -1905-05-05 12:34:56.1073,1905-12-25 -1905-05-05 12:34:56.1074,1905-12-25 -1905-05-05 12:34:56.1075,1905-12-25 -1905-05-05 12:34:56.1076,1905-12-25 -1905-05-05 12:34:56.1077,1905-12-25 -1905-05-05 12:34:56.1078,1905-12-25 -1905-05-05 12:34:56.1079,1905-12-25 -1905-05-05 12:34:56.108,1905-12-25 -1905-05-05 12:34:56.1081,1905-12-25 -1905-05-05 12:34:56.1082,1905-12-25 -1905-05-05 12:34:56.1083,1905-12-25 -1905-05-05 12:34:56.1084,1905-12-25 -1905-05-05 12:34:56.1085,1905-12-25 -1905-05-05 12:34:56.1086,1905-12-25 -1905-05-05 12:34:56.1087,1905-12-25 -1905-05-05 12:34:56.1088,1905-12-25 -1905-05-05 12:34:56.1089,1905-12-25 -1905-05-05 12:34:56.109,1905-12-25 -1905-05-05 12:34:56.1091,1905-12-25 -1905-05-05 12:34:56.1092,1905-12-25 -1905-05-05 12:34:56.1093,1905-12-25 -1905-05-05 12:34:56.1094,1905-12-25 -1905-05-05 12:34:56.1095,1905-12-25 -1905-05-05 12:34:56.1096,1905-12-25 -1905-05-05 12:34:56.1097,1905-12-25 -1905-05-05 12:34:56.1098,1905-12-25 -1905-05-05 12:34:56.1099,1905-12-25 -1905-05-05 12:34:56.11,1905-12-25 -1905-05-05 12:34:56.1101,1905-12-25 -1905-05-05 12:34:56.1102,1905-12-25 -1905-05-05 12:34:56.1103,1905-12-25 -1905-05-05 12:34:56.1104,1905-12-25 -1905-05-05 12:34:56.1105,1905-12-25 -1905-05-05 12:34:56.1106,1905-12-25 -1905-05-05 12:34:56.1107,1905-12-25 -1905-05-05 12:34:56.1108,1905-12-25 -1905-05-05 12:34:56.1109,1905-12-25 -1905-05-05 12:34:56.111,1905-12-25 -1905-05-05 12:34:56.1111,1905-12-25 -1905-05-05 12:34:56.1112,1905-12-25 -1905-05-05 12:34:56.1113,1905-12-25 -1905-05-05 12:34:56.1114,1905-12-25 -1905-05-05 12:34:56.1115,1905-12-25 -1905-05-05 12:34:56.1116,1905-12-25 -1905-05-05 12:34:56.1117,1905-12-25 -1905-05-05 12:34:56.1118,1905-12-25 -1905-05-05 12:34:56.1119,1905-12-25 -1905-05-05 12:34:56.112,1905-12-25 -1905-05-05 12:34:56.1121,1905-12-25 -1905-05-05 12:34:56.1122,1905-12-25 -1905-05-05 12:34:56.1123,1905-12-25 -1905-05-05 12:34:56.1124,1905-12-25 -1905-05-05 12:34:56.1125,1905-12-25 -1905-05-05 12:34:56.1126,1905-12-25 -1905-05-05 12:34:56.1127,1905-12-25 -1905-05-05 12:34:56.1128,1905-12-25 -1905-05-05 12:34:56.1129,1905-12-25 -1905-05-05 12:34:56.113,1905-12-25 -1905-05-05 12:34:56.1131,1905-12-25 -1905-05-05 12:34:56.1132,1905-12-25 -1905-05-05 12:34:56.1133,1905-12-25 -1905-05-05 12:34:56.1134,1905-12-25 -1905-05-05 12:34:56.1135,1905-12-25 -1905-05-05 12:34:56.1136,1905-12-25 -1905-05-05 12:34:56.1137,1905-12-25 -1905-05-05 12:34:56.1138,1905-12-25 -1905-05-05 12:34:56.1139,1905-12-25 -1905-05-05 12:34:56.114,1905-12-25 -1905-05-05 12:34:56.1141,1905-12-25 -1905-05-05 12:34:56.1142,1905-12-25 -1905-05-05 12:34:56.1143,1905-12-25 -1905-05-05 12:34:56.1144,1905-12-25 -1905-05-05 12:34:56.1145,1905-12-25 -1905-05-05 12:34:56.1146,1905-12-25 -1905-05-05 12:34:56.1147,1905-12-25 -1905-05-05 12:34:56.1148,1905-12-25 -1905-05-05 12:34:56.1149,1905-12-25 -1905-05-05 12:34:56.115,1905-12-25 -1905-05-05 12:34:56.1151,1905-12-25 -1905-05-05 12:34:56.1152,1905-12-25 -1905-05-05 12:34:56.1153,1905-12-25 -1905-05-05 12:34:56.1154,1905-12-25 -1905-05-05 12:34:56.1155,1905-12-25 -1905-05-05 12:34:56.1156,1905-12-25 -1905-05-05 12:34:56.1157,1905-12-25 -1905-05-05 12:34:56.1158,1905-12-25 -1905-05-05 12:34:56.1159,1905-12-25 -1905-05-05 12:34:56.116,1905-12-25 -1905-05-05 12:34:56.1161,1905-12-25 -1905-05-05 12:34:56.1162,1905-12-25 -1905-05-05 12:34:56.1163,1905-12-25 -1905-05-05 12:34:56.1164,1905-12-25 -1905-05-05 12:34:56.1165,1905-12-25 -1905-05-05 12:34:56.1166,1905-12-25 -1905-05-05 12:34:56.1167,1905-12-25 -1905-05-05 12:34:56.1168,1905-12-25 -1905-05-05 12:34:56.1169,1905-12-25 -1905-05-05 12:34:56.117,1905-12-25 -1905-05-05 12:34:56.1171,1905-12-25 -1905-05-05 12:34:56.1172,1905-12-25 -1905-05-05 12:34:56.1173,1905-12-25 -1905-05-05 12:34:56.1174,1905-12-25 -1905-05-05 12:34:56.1175,1905-12-25 -1905-05-05 12:34:56.1176,1905-12-25 -1905-05-05 12:34:56.1177,1905-12-25 -1905-05-05 12:34:56.1178,1905-12-25 -1905-05-05 12:34:56.1179,1905-12-25 -1905-05-05 12:34:56.118,1905-12-25 -1905-05-05 12:34:56.1181,1905-12-25 -1905-05-05 12:34:56.1182,1905-12-25 -1905-05-05 12:34:56.1183,1905-12-25 -1905-05-05 12:34:56.1184,1905-12-25 -1905-05-05 12:34:56.1185,1905-12-25 -1905-05-05 12:34:56.1186,1905-12-25 -1905-05-05 12:34:56.1187,1905-12-25 -1905-05-05 12:34:56.1188,1905-12-25 -1905-05-05 12:34:56.1189,1905-12-25 -1905-05-05 12:34:56.119,1905-12-25 -1905-05-05 12:34:56.1191,1905-12-25 -1905-05-05 12:34:56.1192,1905-12-25 -1905-05-05 12:34:56.1193,1905-12-25 -1905-05-05 12:34:56.1194,1905-12-25 -1905-05-05 12:34:56.1195,1905-12-25 -1905-05-05 12:34:56.1196,1905-12-25 -1905-05-05 12:34:56.1197,1905-12-25 -1905-05-05 12:34:56.1198,1905-12-25 -1905-05-05 12:34:56.1199,1905-12-25 -1905-05-05 12:34:56.12,1905-12-25 -1905-05-05 12:34:56.1201,1905-12-25 -1905-05-05 12:34:56.1202,1905-12-25 -1905-05-05 12:34:56.1203,1905-12-25 -1905-05-05 12:34:56.1204,1905-12-25 -1905-05-05 12:34:56.1205,1905-12-25 -1905-05-05 12:34:56.1206,1905-12-25 -1905-05-05 12:34:56.1207,1905-12-25 -1905-05-05 12:34:56.1208,1905-12-25 -1905-05-05 12:34:56.1209,1905-12-25 -1905-05-05 12:34:56.121,1905-12-25 -1905-05-05 12:34:56.1211,1905-12-25 -1905-05-05 12:34:56.1212,1905-12-25 -1905-05-05 12:34:56.1213,1905-12-25 -1905-05-05 12:34:56.1214,1905-12-25 -1905-05-05 12:34:56.1215,1905-12-25 -1905-05-05 12:34:56.1216,1905-12-25 -1905-05-05 12:34:56.1217,1905-12-25 -1905-05-05 12:34:56.1218,1905-12-25 -1905-05-05 12:34:56.1219,1905-12-25 -1905-05-05 12:34:56.122,1905-12-25 -1905-05-05 12:34:56.1221,1905-12-25 -1905-05-05 12:34:56.1222,1905-12-25 -1905-05-05 12:34:56.1223,1905-12-25 -1905-05-05 12:34:56.1224,1905-12-25 -1905-05-05 12:34:56.1225,1905-12-25 -1905-05-05 12:34:56.1226,1905-12-25 -1905-05-05 12:34:56.1227,1905-12-25 -1905-05-05 12:34:56.1228,1905-12-25 -1905-05-05 12:34:56.1229,1905-12-25 -1905-05-05 12:34:56.123,1905-12-25 -1905-05-05 12:34:56.1231,1905-12-25 -1905-05-05 12:34:56.1232,1905-12-25 -1905-05-05 12:34:56.1233,1905-12-25 -1905-05-05 12:34:56.1234,1905-12-25 -1905-05-05 12:34:56.1235,1905-12-25 -1905-05-05 12:34:56.1236,1905-12-25 -1905-05-05 12:34:56.1237,1905-12-25 -1905-05-05 12:34:56.1238,1905-12-25 -1905-05-05 12:34:56.1239,1905-12-25 -1905-05-05 12:34:56.124,1905-12-25 -1905-05-05 12:34:56.1241,1905-12-25 -1905-05-05 12:34:56.1242,1905-12-25 -1905-05-05 12:34:56.1243,1905-12-25 -1905-05-05 12:34:56.1244,1905-12-25 -1905-05-05 12:34:56.1245,1905-12-25 -1905-05-05 12:34:56.1246,1905-12-25 -1905-05-05 12:34:56.1247,1905-12-25 -1905-05-05 12:34:56.1248,1905-12-25 -1905-05-05 12:34:56.1249,1905-12-25 -1905-05-05 12:34:56.125,1905-12-25 -1905-05-05 12:34:56.1251,1905-12-25 -1905-05-05 12:34:56.1252,1905-12-25 -1905-05-05 12:34:56.1253,1905-12-25 -1905-05-05 12:34:56.1254,1905-12-25 -1905-05-05 12:34:56.1255,1905-12-25 -1905-05-05 12:34:56.1256,1905-12-25 -1905-05-05 12:34:56.1257,1905-12-25 -1905-05-05 12:34:56.1258,1905-12-25 -1905-05-05 12:34:56.1259,1905-12-25 -1905-05-05 12:34:56.126,1905-12-25 -1905-05-05 12:34:56.1261,1905-12-25 -1905-05-05 12:34:56.1262,1905-12-25 -1905-05-05 12:34:56.1263,1905-12-25 -1905-05-05 12:34:56.1264,1905-12-25 -1905-05-05 12:34:56.1265,1905-12-25 -1905-05-05 12:34:56.1266,1905-12-25 -1905-05-05 12:34:56.1267,1905-12-25 -1905-05-05 12:34:56.1268,1905-12-25 -1905-05-05 12:34:56.1269,1905-12-25 -1905-05-05 12:34:56.127,1905-12-25 -1905-05-05 12:34:56.1271,1905-12-25 -1905-05-05 12:34:56.1272,1905-12-25 -1905-05-05 12:34:56.1273,1905-12-25 -1905-05-05 12:34:56.1274,1905-12-25 -1905-05-05 12:34:56.1275,1905-12-25 -1905-05-05 12:34:56.1276,1905-12-25 -1905-05-05 12:34:56.1277,1905-12-25 -1905-05-05 12:34:56.1278,1905-12-25 -1905-05-05 12:34:56.1279,1905-12-25 -1905-05-05 12:34:56.128,1905-12-25 -1905-05-05 12:34:56.1281,1905-12-25 -1905-05-05 12:34:56.1282,1905-12-25 -1905-05-05 12:34:56.1283,1905-12-25 -1905-05-05 12:34:56.1284,1905-12-25 -1905-05-05 12:34:56.1285,1905-12-25 -1905-05-05 12:34:56.1286,1905-12-25 -1905-05-05 12:34:56.1287,1905-12-25 -1905-05-05 12:34:56.1288,1905-12-25 -1905-05-05 12:34:56.1289,1905-12-25 -1905-05-05 12:34:56.129,1905-12-25 -1905-05-05 12:34:56.1291,1905-12-25 -1905-05-05 12:34:56.1292,1905-12-25 -1905-05-05 12:34:56.1293,1905-12-25 -1905-05-05 12:34:56.1294,1905-12-25 -1905-05-05 12:34:56.1295,1905-12-25 -1905-05-05 12:34:56.1296,1905-12-25 -1905-05-05 12:34:56.1297,1905-12-25 -1905-05-05 12:34:56.1298,1905-12-25 -1905-05-05 12:34:56.1299,1905-12-25 -1905-05-05 12:34:56.13,1905-12-25 -1905-05-05 12:34:56.1301,1905-12-25 -1905-05-05 12:34:56.1302,1905-12-25 -1905-05-05 12:34:56.1303,1905-12-25 -1905-05-05 12:34:56.1304,1905-12-25 -1905-05-05 12:34:56.1305,1905-12-25 -1905-05-05 12:34:56.1306,1905-12-25 -1905-05-05 12:34:56.1307,1905-12-25 -1905-05-05 12:34:56.1308,1905-12-25 -1905-05-05 12:34:56.1309,1905-12-25 -1905-05-05 12:34:56.131,1905-12-25 -1905-05-05 12:34:56.1311,1905-12-25 -1905-05-05 12:34:56.1312,1905-12-25 -1905-05-05 12:34:56.1313,1905-12-25 -1905-05-05 12:34:56.1314,1905-12-25 -1905-05-05 12:34:56.1315,1905-12-25 -1905-05-05 12:34:56.1316,1905-12-25 -1905-05-05 12:34:56.1317,1905-12-25 -1905-05-05 12:34:56.1318,1905-12-25 -1905-05-05 12:34:56.1319,1905-12-25 -1905-05-05 12:34:56.132,1905-12-25 -1905-05-05 12:34:56.1321,1905-12-25 -1905-05-05 12:34:56.1322,1905-12-25 -1905-05-05 12:34:56.1323,1905-12-25 -1905-05-05 12:34:56.1324,1905-12-25 -1905-05-05 12:34:56.1325,1905-12-25 -1905-05-05 12:34:56.1326,1905-12-25 -1905-05-05 12:34:56.1327,1905-12-25 -1905-05-05 12:34:56.1328,1905-12-25 -1905-05-05 12:34:56.1329,1905-12-25 -1905-05-05 12:34:56.133,1905-12-25 -1905-05-05 12:34:56.1331,1905-12-25 -1905-05-05 12:34:56.1332,1905-12-25 -1905-05-05 12:34:56.1333,1905-12-25 -1905-05-05 12:34:56.1334,1905-12-25 -1905-05-05 12:34:56.1335,1905-12-25 -1905-05-05 12:34:56.1336,1905-12-25 -1905-05-05 12:34:56.1337,1905-12-25 -1905-05-05 12:34:56.1338,1905-12-25 -1905-05-05 12:34:56.1339,1905-12-25 -1905-05-05 12:34:56.134,1905-12-25 -1905-05-05 12:34:56.1341,1905-12-25 -1905-05-05 12:34:56.1342,1905-12-25 -1905-05-05 12:34:56.1343,1905-12-25 -1905-05-05 12:34:56.1344,1905-12-25 -1905-05-05 12:34:56.1345,1905-12-25 -1905-05-05 12:34:56.1346,1905-12-25 -1905-05-05 12:34:56.1347,1905-12-25 -1905-05-05 12:34:56.1348,1905-12-25 -1905-05-05 12:34:56.1349,1905-12-25 -1905-05-05 12:34:56.135,1905-12-25 -1905-05-05 12:34:56.1351,1905-12-25 -1905-05-05 12:34:56.1352,1905-12-25 -1905-05-05 12:34:56.1353,1905-12-25 -1905-05-05 12:34:56.1354,1905-12-25 -1905-05-05 12:34:56.1355,1905-12-25 -1905-05-05 12:34:56.1356,1905-12-25 -1905-05-05 12:34:56.1357,1905-12-25 -1905-05-05 12:34:56.1358,1905-12-25 -1905-05-05 12:34:56.1359,1905-12-25 -1905-05-05 12:34:56.136,1905-12-25 -1905-05-05 12:34:56.1361,1905-12-25 -1905-05-05 12:34:56.1362,1905-12-25 -1905-05-05 12:34:56.1363,1905-12-25 -1905-05-05 12:34:56.1364,1905-12-25 -1905-05-05 12:34:56.1365,1905-12-25 -1905-05-05 12:34:56.1366,1905-12-25 -1905-05-05 12:34:56.1367,1905-12-25 -1905-05-05 12:34:56.1368,1905-12-25 -1905-05-05 12:34:56.1369,1905-12-25 -1905-05-05 12:34:56.137,1905-12-25 -1905-05-05 12:34:56.1371,1905-12-25 -1905-05-05 12:34:56.1372,1905-12-25 -1905-05-05 12:34:56.1373,1905-12-25 -1905-05-05 12:34:56.1374,1905-12-25 -1905-05-05 12:34:56.1375,1905-12-25 -1905-05-05 12:34:56.1376,1905-12-25 -1905-05-05 12:34:56.1377,1905-12-25 -1905-05-05 12:34:56.1378,1905-12-25 -1905-05-05 12:34:56.1379,1905-12-25 -1905-05-05 12:34:56.138,1905-12-25 -1905-05-05 12:34:56.1381,1905-12-25 -1905-05-05 12:34:56.1382,1905-12-25 -1905-05-05 12:34:56.1383,1905-12-25 -1905-05-05 12:34:56.1384,1905-12-25 -1905-05-05 12:34:56.1385,1905-12-25 -1905-05-05 12:34:56.1386,1905-12-25 -1905-05-05 12:34:56.1387,1905-12-25 -1905-05-05 12:34:56.1388,1905-12-25 -1905-05-05 12:34:56.1389,1905-12-25 -1905-05-05 12:34:56.139,1905-12-25 -1905-05-05 12:34:56.1391,1905-12-25 -1905-05-05 12:34:56.1392,1905-12-25 -1905-05-05 12:34:56.1393,1905-12-25 -1905-05-05 12:34:56.1394,1905-12-25 -1905-05-05 12:34:56.1395,1905-12-25 -1905-05-05 12:34:56.1396,1905-12-25 -1905-05-05 12:34:56.1397,1905-12-25 -1905-05-05 12:34:56.1398,1905-12-25 -1905-05-05 12:34:56.1399,1905-12-25 -1905-05-05 12:34:56.14,1905-12-25 -1905-05-05 12:34:56.1401,1905-12-25 -1905-05-05 12:34:56.1402,1905-12-25 -1905-05-05 12:34:56.1403,1905-12-25 -1905-05-05 12:34:56.1404,1905-12-25 -1905-05-05 12:34:56.1405,1905-12-25 -1905-05-05 12:34:56.1406,1905-12-25 -1905-05-05 12:34:56.1407,1905-12-25 -1905-05-05 12:34:56.1408,1905-12-25 -1905-05-05 12:34:56.1409,1905-12-25 -1905-05-05 12:34:56.141,1905-12-25 -1905-05-05 12:34:56.1411,1905-12-25 -1905-05-05 12:34:56.1412,1905-12-25 -1905-05-05 12:34:56.1413,1905-12-25 -1905-05-05 12:34:56.1414,1905-12-25 -1905-05-05 12:34:56.1415,1905-12-25 -1905-05-05 12:34:56.1416,1905-12-25 -1905-05-05 12:34:56.1417,1905-12-25 -1905-05-05 12:34:56.1418,1905-12-25 -1905-05-05 12:34:56.1419,1905-12-25 -1905-05-05 12:34:56.142,1905-12-25 -1905-05-05 12:34:56.1421,1905-12-25 -1905-05-05 12:34:56.1422,1905-12-25 -1905-05-05 12:34:56.1423,1905-12-25 -1905-05-05 12:34:56.1424,1905-12-25 -1905-05-05 12:34:56.1425,1905-12-25 -1905-05-05 12:34:56.1426,1905-12-25 -1905-05-05 12:34:56.1427,1905-12-25 -1905-05-05 12:34:56.1428,1905-12-25 -1905-05-05 12:34:56.1429,1905-12-25 -1905-05-05 12:34:56.143,1905-12-25 -1905-05-05 12:34:56.1431,1905-12-25 -1905-05-05 12:34:56.1432,1905-12-25 -1905-05-05 12:34:56.1433,1905-12-25 -1905-05-05 12:34:56.1434,1905-12-25 -1905-05-05 12:34:56.1435,1905-12-25 -1905-05-05 12:34:56.1436,1905-12-25 -1905-05-05 12:34:56.1437,1905-12-25 -1905-05-05 12:34:56.1438,1905-12-25 -1905-05-05 12:34:56.1439,1905-12-25 -1905-05-05 12:34:56.144,1905-12-25 -1905-05-05 12:34:56.1441,1905-12-25 -1905-05-05 12:34:56.1442,1905-12-25 -1905-05-05 12:34:56.1443,1905-12-25 -1905-05-05 12:34:56.1444,1905-12-25 -1905-05-05 12:34:56.1445,1905-12-25 -1905-05-05 12:34:56.1446,1905-12-25 -1905-05-05 12:34:56.1447,1905-12-25 -1905-05-05 12:34:56.1448,1905-12-25 -1905-05-05 12:34:56.1449,1905-12-25 -1905-05-05 12:34:56.145,1905-12-25 -1905-05-05 12:34:56.1451,1905-12-25 -1905-05-05 12:34:56.1452,1905-12-25 -1905-05-05 12:34:56.1453,1905-12-25 -1905-05-05 12:34:56.1454,1905-12-25 -1905-05-05 12:34:56.1455,1905-12-25 -1905-05-05 12:34:56.1456,1905-12-25 -1905-05-05 12:34:56.1457,1905-12-25 -1905-05-05 12:34:56.1458,1905-12-25 -1905-05-05 12:34:56.1459,1905-12-25 -1905-05-05 12:34:56.146,1905-12-25 -1905-05-05 12:34:56.1461,1905-12-25 -1905-05-05 12:34:56.1462,1905-12-25 -1905-05-05 12:34:56.1463,1905-12-25 -1905-05-05 12:34:56.1464,1905-12-25 -1905-05-05 12:34:56.1465,1905-12-25 -1905-05-05 12:34:56.1466,1905-12-25 -1905-05-05 12:34:56.1467,1905-12-25 -1905-05-05 12:34:56.1468,1905-12-25 -1905-05-05 12:34:56.1469,1905-12-25 -1905-05-05 12:34:56.147,1905-12-25 -1905-05-05 12:34:56.1471,1905-12-25 -1905-05-05 12:34:56.1472,1905-12-25 -1905-05-05 12:34:56.1473,1905-12-25 -1905-05-05 12:34:56.1474,1905-12-25 -1905-05-05 12:34:56.1475,1905-12-25 -1905-05-05 12:34:56.1476,1905-12-25 -1905-05-05 12:34:56.1477,1905-12-25 -1905-05-05 12:34:56.1478,1905-12-25 -1905-05-05 12:34:56.1479,1905-12-25 -1905-05-05 12:34:56.148,1905-12-25 -1905-05-05 12:34:56.1481,1905-12-25 -1905-05-05 12:34:56.1482,1905-12-25 -1905-05-05 12:34:56.1483,1905-12-25 -1905-05-05 12:34:56.1484,1905-12-25 -1905-05-05 12:34:56.1485,1905-12-25 -1905-05-05 12:34:56.1486,1905-12-25 -1905-05-05 12:34:56.1487,1905-12-25 -1905-05-05 12:34:56.1488,1905-12-25 -1905-05-05 12:34:56.1489,1905-12-25 -1905-05-05 12:34:56.149,1905-12-25 -1905-05-05 12:34:56.1491,1905-12-25 -1905-05-05 12:34:56.1492,1905-12-25 -1905-05-05 12:34:56.1493,1905-12-25 -1905-05-05 12:34:56.1494,1905-12-25 -1905-05-05 12:34:56.1495,1905-12-25 -1905-05-05 12:34:56.1496,1905-12-25 -1905-05-05 12:34:56.1497,1905-12-25 -1905-05-05 12:34:56.1498,1905-12-25 -1905-05-05 12:34:56.1499,1905-12-25 -1905-05-05 12:34:56.15,1905-12-25 -1905-05-05 12:34:56.1501,1905-12-25 -1905-05-05 12:34:56.1502,1905-12-25 -1905-05-05 12:34:56.1503,1905-12-25 -1905-05-05 12:34:56.1504,1905-12-25 -1905-05-05 12:34:56.1505,1905-12-25 -1905-05-05 12:34:56.1506,1905-12-25 -1905-05-05 12:34:56.1507,1905-12-25 -1905-05-05 12:34:56.1508,1905-12-25 -1905-05-05 12:34:56.1509,1905-12-25 -1905-05-05 12:34:56.151,1905-12-25 -1905-05-05 12:34:56.1511,1905-12-25 -1905-05-05 12:34:56.1512,1905-12-25 -1905-05-05 12:34:56.1513,1905-12-25 -1905-05-05 12:34:56.1514,1905-12-25 -1905-05-05 12:34:56.1515,1905-12-25 -1905-05-05 12:34:56.1516,1905-12-25 -1905-05-05 12:34:56.1517,1905-12-25 -1905-05-05 12:34:56.1518,1905-12-25 -1905-05-05 12:34:56.1519,1905-12-25 -1905-05-05 12:34:56.152,1905-12-25 -1905-05-05 12:34:56.1521,1905-12-25 -1905-05-05 12:34:56.1522,1905-12-25 -1905-05-05 12:34:56.1523,1905-12-25 -1905-05-05 12:34:56.1524,1905-12-25 -1905-05-05 12:34:56.1525,1905-12-25 -1905-05-05 12:34:56.1526,1905-12-25 -1905-05-05 12:34:56.1527,1905-12-25 -1905-05-05 12:34:56.1528,1905-12-25 -1905-05-05 12:34:56.1529,1905-12-25 -1905-05-05 12:34:56.153,1905-12-25 -1905-05-05 12:34:56.1531,1905-12-25 -1905-05-05 12:34:56.1532,1905-12-25 -1905-05-05 12:34:56.1533,1905-12-25 -1905-05-05 12:34:56.1534,1905-12-25 -1905-05-05 12:34:56.1535,1905-12-25 -1905-05-05 12:34:56.1536,1905-12-25 -1905-05-05 12:34:56.1537,1905-12-25 -1905-05-05 12:34:56.1538,1905-12-25 -1905-05-05 12:34:56.1539,1905-12-25 -1905-05-05 12:34:56.154,1905-12-25 -1905-05-05 12:34:56.1541,1905-12-25 -1905-05-05 12:34:56.1542,1905-12-25 -1905-05-05 12:34:56.1543,1905-12-25 -1905-05-05 12:34:56.1544,1905-12-25 -1905-05-05 12:34:56.1545,1905-12-25 -1905-05-05 12:34:56.1546,1905-12-25 -1905-05-05 12:34:56.1547,1905-12-25 -1905-05-05 12:34:56.1548,1905-12-25 -1905-05-05 12:34:56.1549,1905-12-25 -1905-05-05 12:34:56.155,1905-12-25 -1905-05-05 12:34:56.1551,1905-12-25 -1905-05-05 12:34:56.1552,1905-12-25 -1905-05-05 12:34:56.1553,1905-12-25 -1905-05-05 12:34:56.1554,1905-12-25 -1905-05-05 12:34:56.1555,1905-12-25 -1905-05-05 12:34:56.1556,1905-12-25 -1905-05-05 12:34:56.1557,1905-12-25 -1905-05-05 12:34:56.1558,1905-12-25 -1905-05-05 12:34:56.1559,1905-12-25 -1905-05-05 12:34:56.156,1905-12-25 -1905-05-05 12:34:56.1561,1905-12-25 -1905-05-05 12:34:56.1562,1905-12-25 -1905-05-05 12:34:56.1563,1905-12-25 -1905-05-05 12:34:56.1564,1905-12-25 -1905-05-05 12:34:56.1565,1905-12-25 -1905-05-05 12:34:56.1566,1905-12-25 -1905-05-05 12:34:56.1567,1905-12-25 -1905-05-05 12:34:56.1568,1905-12-25 -1905-05-05 12:34:56.1569,1905-12-25 -1905-05-05 12:34:56.157,1905-12-25 -1905-05-05 12:34:56.1571,1905-12-25 -1905-05-05 12:34:56.1572,1905-12-25 -1905-05-05 12:34:56.1573,1905-12-25 -1905-05-05 12:34:56.1574,1905-12-25 -1905-05-05 12:34:56.1575,1905-12-25 -1905-05-05 12:34:56.1576,1905-12-25 -1905-05-05 12:34:56.1577,1905-12-25 -1905-05-05 12:34:56.1578,1905-12-25 -1905-05-05 12:34:56.1579,1905-12-25 -1905-05-05 12:34:56.158,1905-12-25 -1905-05-05 12:34:56.1581,1905-12-25 -1905-05-05 12:34:56.1582,1905-12-25 -1905-05-05 12:34:56.1583,1905-12-25 -1905-05-05 12:34:56.1584,1905-12-25 -1905-05-05 12:34:56.1585,1905-12-25 -1905-05-05 12:34:56.1586,1905-12-25 -1905-05-05 12:34:56.1587,1905-12-25 -1905-05-05 12:34:56.1588,1905-12-25 -1905-05-05 12:34:56.1589,1905-12-25 -1905-05-05 12:34:56.159,1905-12-25 -1905-05-05 12:34:56.1591,1905-12-25 -1905-05-05 12:34:56.1592,1905-12-25 -1905-05-05 12:34:56.1593,1905-12-25 -1905-05-05 12:34:56.1594,1905-12-25 -1905-05-05 12:34:56.1595,1905-12-25 -1905-05-05 12:34:56.1596,1905-12-25 -1905-05-05 12:34:56.1597,1905-12-25 -1905-05-05 12:34:56.1598,1905-12-25 -1905-05-05 12:34:56.1599,1905-12-25 -1905-05-05 12:34:56.16,1905-12-25 -1905-05-05 12:34:56.1601,1905-12-25 -1905-05-05 12:34:56.1602,1905-12-25 -1905-05-05 12:34:56.1603,1905-12-25 -1905-05-05 12:34:56.1604,1905-12-25 -1905-05-05 12:34:56.1605,1905-12-25 -1905-05-05 12:34:56.1606,1905-12-25 -1905-05-05 12:34:56.1607,1905-12-25 -1905-05-05 12:34:56.1608,1905-12-25 -1905-05-05 12:34:56.1609,1905-12-25 -1905-05-05 12:34:56.161,1905-12-25 -1905-05-05 12:34:56.1611,1905-12-25 -1905-05-05 12:34:56.1612,1905-12-25 -1905-05-05 12:34:56.1613,1905-12-25 -1905-05-05 12:34:56.1614,1905-12-25 -1905-05-05 12:34:56.1615,1905-12-25 -1905-05-05 12:34:56.1616,1905-12-25 -1905-05-05 12:34:56.1617,1905-12-25 -1905-05-05 12:34:56.1618,1905-12-25 -1905-05-05 12:34:56.1619,1905-12-25 -1905-05-05 12:34:56.162,1905-12-25 -1905-05-05 12:34:56.1621,1905-12-25 -1905-05-05 12:34:56.1622,1905-12-25 -1905-05-05 12:34:56.1623,1905-12-25 -1905-05-05 12:34:56.1624,1905-12-25 -1905-05-05 12:34:56.1625,1905-12-25 -1905-05-05 12:34:56.1626,1905-12-25 -1905-05-05 12:34:56.1627,1905-12-25 -1905-05-05 12:34:56.1628,1905-12-25 -1905-05-05 12:34:56.1629,1905-12-25 -1905-05-05 12:34:56.163,1905-12-25 -1905-05-05 12:34:56.1631,1905-12-25 -1905-05-05 12:34:56.1632,1905-12-25 -1905-05-05 12:34:56.1633,1905-12-25 -1905-05-05 12:34:56.1634,1905-12-25 -1905-05-05 12:34:56.1635,1905-12-25 -1905-05-05 12:34:56.1636,1905-12-25 -1905-05-05 12:34:56.1637,1905-12-25 -1905-05-05 12:34:56.1638,1905-12-25 -1905-05-05 12:34:56.1639,1905-12-25 -1905-05-05 12:34:56.164,1905-12-25 -1905-05-05 12:34:56.1641,1905-12-25 -1905-05-05 12:34:56.1642,1905-12-25 -1905-05-05 12:34:56.1643,1905-12-25 -1905-05-05 12:34:56.1644,1905-12-25 -1905-05-05 12:34:56.1645,1905-12-25 -1905-05-05 12:34:56.1646,1905-12-25 -1905-05-05 12:34:56.1647,1905-12-25 -1905-05-05 12:34:56.1648,1905-12-25 -1905-05-05 12:34:56.1649,1905-12-25 -1905-05-05 12:34:56.165,1905-12-25 -1905-05-05 12:34:56.1651,1905-12-25 -1905-05-05 12:34:56.1652,1905-12-25 -1905-05-05 12:34:56.1653,1905-12-25 -1905-05-05 12:34:56.1654,1905-12-25 -1905-05-05 12:34:56.1655,1905-12-25 -1905-05-05 12:34:56.1656,1905-12-25 -1905-05-05 12:34:56.1657,1905-12-25 -1905-05-05 12:34:56.1658,1905-12-25 -1905-05-05 12:34:56.1659,1905-12-25 -1905-05-05 12:34:56.166,1905-12-25 -1905-05-05 12:34:56.1661,1905-12-25 -1905-05-05 12:34:56.1662,1905-12-25 -1905-05-05 12:34:56.1663,1905-12-25 -1905-05-05 12:34:56.1664,1905-12-25 -1905-05-05 12:34:56.1665,1905-12-25 -1905-05-05 12:34:56.1666,1905-12-25 -1905-05-05 12:34:56.1667,1905-12-25 -1905-05-05 12:34:56.1668,1905-12-25 -1905-05-05 12:34:56.1669,1905-12-25 -1905-05-05 12:34:56.167,1905-12-25 -1905-05-05 12:34:56.1671,1905-12-25 -1905-05-05 12:34:56.1672,1905-12-25 -1905-05-05 12:34:56.1673,1905-12-25 -1905-05-05 12:34:56.1674,1905-12-25 -1905-05-05 12:34:56.1675,1905-12-25 -1905-05-05 12:34:56.1676,1905-12-25 -1905-05-05 12:34:56.1677,1905-12-25 -1905-05-05 12:34:56.1678,1905-12-25 -1905-05-05 12:34:56.1679,1905-12-25 -1905-05-05 12:34:56.168,1905-12-25 -1905-05-05 12:34:56.1681,1905-12-25 -1905-05-05 12:34:56.1682,1905-12-25 -1905-05-05 12:34:56.1683,1905-12-25 -1905-05-05 12:34:56.1684,1905-12-25 -1905-05-05 12:34:56.1685,1905-12-25 -1905-05-05 12:34:56.1686,1905-12-25 -1905-05-05 12:34:56.1687,1905-12-25 -1905-05-05 12:34:56.1688,1905-12-25 -1905-05-05 12:34:56.1689,1905-12-25 -1905-05-05 12:34:56.169,1905-12-25 -1905-05-05 12:34:56.1691,1905-12-25 -1905-05-05 12:34:56.1692,1905-12-25 -1905-05-05 12:34:56.1693,1905-12-25 -1905-05-05 12:34:56.1694,1905-12-25 -1905-05-05 12:34:56.1695,1905-12-25 -1905-05-05 12:34:56.1696,1905-12-25 -1905-05-05 12:34:56.1697,1905-12-25 -1905-05-05 12:34:56.1698,1905-12-25 -1905-05-05 12:34:56.1699,1905-12-25 -1905-05-05 12:34:56.17,1905-12-25 -1905-05-05 12:34:56.1701,1905-12-25 -1905-05-05 12:34:56.1702,1905-12-25 -1905-05-05 12:34:56.1703,1905-12-25 -1905-05-05 12:34:56.1704,1905-12-25 -1905-05-05 12:34:56.1705,1905-12-25 -1905-05-05 12:34:56.1706,1905-12-25 -1905-05-05 12:34:56.1707,1905-12-25 -1905-05-05 12:34:56.1708,1905-12-25 -1905-05-05 12:34:56.1709,1905-12-25 -1905-05-05 12:34:56.171,1905-12-25 -1905-05-05 12:34:56.1711,1905-12-25 -1905-05-05 12:34:56.1712,1905-12-25 -1905-05-05 12:34:56.1713,1905-12-25 -1905-05-05 12:34:56.1714,1905-12-25 -1905-05-05 12:34:56.1715,1905-12-25 -1905-05-05 12:34:56.1716,1905-12-25 -1905-05-05 12:34:56.1717,1905-12-25 -1905-05-05 12:34:56.1718,1905-12-25 -1905-05-05 12:34:56.1719,1905-12-25 -1905-05-05 12:34:56.172,1905-12-25 -1905-05-05 12:34:56.1721,1905-12-25 -1905-05-05 12:34:56.1722,1905-12-25 -1905-05-05 12:34:56.1723,1905-12-25 -1905-05-05 12:34:56.1724,1905-12-25 -1905-05-05 12:34:56.1725,1905-12-25 -1905-05-05 12:34:56.1726,1905-12-25 -1905-05-05 12:34:56.1727,1905-12-25 -1905-05-05 12:34:56.1728,1905-12-25 -1905-05-05 12:34:56.1729,1905-12-25 -1905-05-05 12:34:56.173,1905-12-25 -1905-05-05 12:34:56.1731,1905-12-25 -1905-05-05 12:34:56.1732,1905-12-25 -1905-05-05 12:34:56.1733,1905-12-25 -1905-05-05 12:34:56.1734,1905-12-25 -1905-05-05 12:34:56.1735,1905-12-25 -1905-05-05 12:34:56.1736,1905-12-25 -1905-05-05 12:34:56.1737,1905-12-25 -1905-05-05 12:34:56.1738,1905-12-25 -1905-05-05 12:34:56.1739,1905-12-25 -1905-05-05 12:34:56.174,1905-12-25 -1905-05-05 12:34:56.1741,1905-12-25 -1905-05-05 12:34:56.1742,1905-12-25 -1905-05-05 12:34:56.1743,1905-12-25 -1905-05-05 12:34:56.1744,1905-12-25 -1905-05-05 12:34:56.1745,1905-12-25 -1905-05-05 12:34:56.1746,1905-12-25 -1905-05-05 12:34:56.1747,1905-12-25 -1905-05-05 12:34:56.1748,1905-12-25 -1905-05-05 12:34:56.1749,1905-12-25 -1905-05-05 12:34:56.175,1905-12-25 -1905-05-05 12:34:56.1751,1905-12-25 -1905-05-05 12:34:56.1752,1905-12-25 -1905-05-05 12:34:56.1753,1905-12-25 -1905-05-05 12:34:56.1754,1905-12-25 -1905-05-05 12:34:56.1755,1905-12-25 -1905-05-05 12:34:56.1756,1905-12-25 -1905-05-05 12:34:56.1757,1905-12-25 -1905-05-05 12:34:56.1758,1905-12-25 -1905-05-05 12:34:56.1759,1905-12-25 -1905-05-05 12:34:56.176,1905-12-25 -1905-05-05 12:34:56.1761,1905-12-25 -1905-05-05 12:34:56.1762,1905-12-25 -1905-05-05 12:34:56.1763,1905-12-25 -1905-05-05 12:34:56.1764,1905-12-25 -1905-05-05 12:34:56.1765,1905-12-25 -1905-05-05 12:34:56.1766,1905-12-25 -1905-05-05 12:34:56.1767,1905-12-25 -1905-05-05 12:34:56.1768,1905-12-25 -1905-05-05 12:34:56.1769,1905-12-25 -1905-05-05 12:34:56.177,1905-12-25 -1905-05-05 12:34:56.1771,1905-12-25 -1905-05-05 12:34:56.1772,1905-12-25 -1905-05-05 12:34:56.1773,1905-12-25 -1905-05-05 12:34:56.1774,1905-12-25 -1905-05-05 12:34:56.1775,1905-12-25 -1905-05-05 12:34:56.1776,1905-12-25 -1905-05-05 12:34:56.1777,1905-12-25 -1905-05-05 12:34:56.1778,1905-12-25 -1905-05-05 12:34:56.1779,1905-12-25 -1905-05-05 12:34:56.178,1905-12-25 -1905-05-05 12:34:56.1781,1905-12-25 -1905-05-05 12:34:56.1782,1905-12-25 -1905-05-05 12:34:56.1783,1905-12-25 -1905-05-05 12:34:56.1784,1905-12-25 -1905-05-05 12:34:56.1785,1905-12-25 -1905-05-05 12:34:56.1786,1905-12-25 -1905-05-05 12:34:56.1787,1905-12-25 -1905-05-05 12:34:56.1788,1905-12-25 -1905-05-05 12:34:56.1789,1905-12-25 -1905-05-05 12:34:56.179,1905-12-25 -1905-05-05 12:34:56.1791,1905-12-25 -1905-05-05 12:34:56.1792,1905-12-25 -1905-05-05 12:34:56.1793,1905-12-25 -1905-05-05 12:34:56.1794,1905-12-25 -1905-05-05 12:34:56.1795,1905-12-25 -1905-05-05 12:34:56.1796,1905-12-25 -1905-05-05 12:34:56.1797,1905-12-25 -1905-05-05 12:34:56.1798,1905-12-25 -1905-05-05 12:34:56.1799,1905-12-25 -1905-05-05 12:34:56.18,1905-12-25 -1905-05-05 12:34:56.1801,1905-12-25 -1905-05-05 12:34:56.1802,1905-12-25 -1905-05-05 12:34:56.1803,1905-12-25 -1905-05-05 12:34:56.1804,1905-12-25 -1905-05-05 12:34:56.1805,1905-12-25 -1905-05-05 12:34:56.1806,1905-12-25 -1905-05-05 12:34:56.1807,1905-12-25 -1905-05-05 12:34:56.1808,1905-12-25 -1905-05-05 12:34:56.1809,1905-12-25 -1905-05-05 12:34:56.181,1905-12-25 -1905-05-05 12:34:56.1811,1905-12-25 -1905-05-05 12:34:56.1812,1905-12-25 -1905-05-05 12:34:56.1813,1905-12-25 -1905-05-05 12:34:56.1814,1905-12-25 -1905-05-05 12:34:56.1815,1905-12-25 -1905-05-05 12:34:56.1816,1905-12-25 -1905-05-05 12:34:56.1817,1905-12-25 -1905-05-05 12:34:56.1818,1905-12-25 -1905-05-05 12:34:56.1819,1905-12-25 -1905-05-05 12:34:56.182,1905-12-25 -1905-05-05 12:34:56.1821,1905-12-25 -1905-05-05 12:34:56.1822,1905-12-25 -1905-05-05 12:34:56.1823,1905-12-25 -1905-05-05 12:34:56.1824,1905-12-25 -1905-05-05 12:34:56.1825,1905-12-25 -1905-05-05 12:34:56.1826,1905-12-25 -1905-05-05 12:34:56.1827,1905-12-25 -1905-05-05 12:34:56.1828,1905-12-25 -1905-05-05 12:34:56.1829,1905-12-25 -1905-05-05 12:34:56.183,1905-12-25 -1905-05-05 12:34:56.1831,1905-12-25 -1905-05-05 12:34:56.1832,1905-12-25 -1905-05-05 12:34:56.1833,1905-12-25 -1905-05-05 12:34:56.1834,1905-12-25 -1905-05-05 12:34:56.1835,1905-12-25 -1905-05-05 12:34:56.1836,1905-12-25 -1905-05-05 12:34:56.1837,1905-12-25 -1905-05-05 12:34:56.1838,1905-12-25 -1905-05-05 12:34:56.1839,1905-12-25 -1905-05-05 12:34:56.184,1905-12-25 -1905-05-05 12:34:56.1841,1905-12-25 -1905-05-05 12:34:56.1842,1905-12-25 -1905-05-05 12:34:56.1843,1905-12-25 -1905-05-05 12:34:56.1844,1905-12-25 -1905-05-05 12:34:56.1845,1905-12-25 -1905-05-05 12:34:56.1846,1905-12-25 -1905-05-05 12:34:56.1847,1905-12-25 -1905-05-05 12:34:56.1848,1905-12-25 -1905-05-05 12:34:56.1849,1905-12-25 -1905-05-05 12:34:56.185,1905-12-25 -1905-05-05 12:34:56.1851,1905-12-25 -1905-05-05 12:34:56.1852,1905-12-25 -1905-05-05 12:34:56.1853,1905-12-25 -1905-05-05 12:34:56.1854,1905-12-25 -1905-05-05 12:34:56.1855,1905-12-25 -1905-05-05 12:34:56.1856,1905-12-25 -1905-05-05 12:34:56.1857,1905-12-25 -1905-05-05 12:34:56.1858,1905-12-25 -1905-05-05 12:34:56.1859,1905-12-25 -1905-05-05 12:34:56.186,1905-12-25 -1905-05-05 12:34:56.1861,1905-12-25 -1905-05-05 12:34:56.1862,1905-12-25 -1905-05-05 12:34:56.1863,1905-12-25 -1905-05-05 12:34:56.1864,1905-12-25 -1905-05-05 12:34:56.1865,1905-12-25 -1905-05-05 12:34:56.1866,1905-12-25 -1905-05-05 12:34:56.1867,1905-12-25 -1905-05-05 12:34:56.1868,1905-12-25 -1905-05-05 12:34:56.1869,1905-12-25 -1905-05-05 12:34:56.187,1905-12-25 -1905-05-05 12:34:56.1871,1905-12-25 -1905-05-05 12:34:56.1872,1905-12-25 -1905-05-05 12:34:56.1873,1905-12-25 -1905-05-05 12:34:56.1874,1905-12-25 -1905-05-05 12:34:56.1875,1905-12-25 -1905-05-05 12:34:56.1876,1905-12-25 -1905-05-05 12:34:56.1877,1905-12-25 -1905-05-05 12:34:56.1878,1905-12-25 -1905-05-05 12:34:56.1879,1905-12-25 -1905-05-05 12:34:56.188,1905-12-25 -1905-05-05 12:34:56.1881,1905-12-25 -1905-05-05 12:34:56.1882,1905-12-25 -1905-05-05 12:34:56.1883,1905-12-25 -1905-05-05 12:34:56.1884,1905-12-25 -1905-05-05 12:34:56.1885,1905-12-25 -1905-05-05 12:34:56.1886,1905-12-25 -1905-05-05 12:34:56.1887,1905-12-25 -1905-05-05 12:34:56.1888,1905-12-25 -1905-05-05 12:34:56.1889,1905-12-25 -1905-05-05 12:34:56.189,1905-12-25 -1905-05-05 12:34:56.1891,1905-12-25 -1905-05-05 12:34:56.1892,1905-12-25 -1905-05-05 12:34:56.1893,1905-12-25 -1905-05-05 12:34:56.1894,1905-12-25 -1905-05-05 12:34:56.1895,1905-12-25 -1905-05-05 12:34:56.1896,1905-12-25 -1905-05-05 12:34:56.1897,1905-12-25 -1905-05-05 12:34:56.1898,1905-12-25 -1905-05-05 12:34:56.1899,1905-12-25 -1905-05-05 12:34:56.19,1905-12-25 -1905-05-05 12:34:56.1901,1905-12-25 -1905-05-05 12:34:56.1902,1905-12-25 -1905-05-05 12:34:56.1903,1905-12-25 -1905-05-05 12:34:56.1904,1905-12-25 -1905-05-05 12:34:56.1905,1905-12-25 -1905-05-05 12:34:56.1906,1905-12-25 -1905-05-05 12:34:56.1907,1905-12-25 -1905-05-05 12:34:56.1908,1905-12-25 -1905-05-05 12:34:56.1909,1905-12-25 -1905-05-05 12:34:56.191,1905-12-25 -1905-05-05 12:34:56.1911,1905-12-25 -1905-05-05 12:34:56.1912,1905-12-25 -1905-05-05 12:34:56.1913,1905-12-25 -1905-05-05 12:34:56.1914,1905-12-25 -1905-05-05 12:34:56.1915,1905-12-25 -1905-05-05 12:34:56.1916,1905-12-25 -1905-05-05 12:34:56.1917,1905-12-25 -1905-05-05 12:34:56.1918,1905-12-25 -1905-05-05 12:34:56.1919,1905-12-25 -1905-05-05 12:34:56.192,1905-12-25 -1905-05-05 12:34:56.1921,1905-12-25 -1905-05-05 12:34:56.1922,1905-12-25 -1905-05-05 12:34:56.1923,1905-12-25 -1905-05-05 12:34:56.1924,1905-12-25 -1905-05-05 12:34:56.1925,1905-12-25 -1905-05-05 12:34:56.1926,1905-12-25 -1905-05-05 12:34:56.1927,1905-12-25 -1905-05-05 12:34:56.1928,1905-12-25 -1905-05-05 12:34:56.1929,1905-12-25 -1905-05-05 12:34:56.193,1905-12-25 -1905-05-05 12:34:56.1931,1905-12-25 -1905-05-05 12:34:56.1932,1905-12-25 -1905-05-05 12:34:56.1933,1905-12-25 -1905-05-05 12:34:56.1934,1905-12-25 -1905-05-05 12:34:56.1935,1905-12-25 -1905-05-05 12:34:56.1936,1905-12-25 -1905-05-05 12:34:56.1937,1905-12-25 -1905-05-05 12:34:56.1938,1905-12-25 -1905-05-05 12:34:56.1939,1905-12-25 -1905-05-05 12:34:56.194,1905-12-25 -1905-05-05 12:34:56.1941,1905-12-25 -1905-05-05 12:34:56.1942,1905-12-25 -1905-05-05 12:34:56.1943,1905-12-25 -1905-05-05 12:34:56.1944,1905-12-25 -1905-05-05 12:34:56.1945,1905-12-25 -1905-05-05 12:34:56.1946,1905-12-25 -1905-05-05 12:34:56.1947,1905-12-25 -1905-05-05 12:34:56.1948,1905-12-25 -1905-05-05 12:34:56.1949,1905-12-25 -1905-05-05 12:34:56.195,1905-12-25 -1905-05-05 12:34:56.1951,1905-12-25 -1905-05-05 12:34:56.1952,1905-12-25 -1905-05-05 12:34:56.1953,1905-12-25 -1905-05-05 12:34:56.1954,1905-12-25 -1905-05-05 12:34:56.1955,1905-12-25 -1905-05-05 12:34:56.1956,1905-12-25 -1905-05-05 12:34:56.1957,1905-12-25 -1905-05-05 12:34:56.1958,1905-12-25 -1905-05-05 12:34:56.1959,1905-12-25 -1905-05-05 12:34:56.196,1905-12-25 -1905-05-05 12:34:56.1961,1905-12-25 -1905-05-05 12:34:56.1962,1905-12-25 -1905-05-05 12:34:56.1963,1905-12-25 -1905-05-05 12:34:56.1964,1905-12-25 -1905-05-05 12:34:56.1965,1905-12-25 -1905-05-05 12:34:56.1966,1905-12-25 -1905-05-05 12:34:56.1967,1905-12-25 -1905-05-05 12:34:56.1968,1905-12-25 -1905-05-05 12:34:56.1969,1905-12-25 -1905-05-05 12:34:56.197,1905-12-25 -1905-05-05 12:34:56.1971,1905-12-25 -1905-05-05 12:34:56.1972,1905-12-25 -1905-05-05 12:34:56.1973,1905-12-25 -1905-05-05 12:34:56.1974,1905-12-25 -1905-05-05 12:34:56.1975,1905-12-25 -1905-05-05 12:34:56.1976,1905-12-25 -1905-05-05 12:34:56.1977,1905-12-25 -1905-05-05 12:34:56.1978,1905-12-25 -1905-05-05 12:34:56.1979,1905-12-25 -1905-05-05 12:34:56.198,1905-12-25 -1905-05-05 12:34:56.1981,1905-12-25 -1905-05-05 12:34:56.1982,1905-12-25 -1905-05-05 12:34:56.1983,1905-12-25 -1905-05-05 12:34:56.1984,1905-12-25 -1905-05-05 12:34:56.1985,1905-12-25 -1905-05-05 12:34:56.1986,1905-12-25 -1905-05-05 12:34:56.1987,1905-12-25 -1905-05-05 12:34:56.1988,1905-12-25 -1905-05-05 12:34:56.1989,1905-12-25 -1905-05-05 12:34:56.199,1905-12-25 -1905-05-05 12:34:56.1991,1905-12-25 -1905-05-05 12:34:56.1992,1905-12-25 -1905-05-05 12:34:56.1993,1905-12-25 -1905-05-05 12:34:56.1994,1905-12-25 -1905-05-05 12:34:56.1995,1905-12-25 -1905-05-05 12:34:56.1996,1905-12-25 -1905-05-05 12:34:56.1997,1905-12-25 -1905-05-05 12:34:56.1998,1905-12-25 -1905-05-05 12:34:56.1999,1905-12-25 -1906-05-05 12:34:56.1,1906-12-25 -1906-05-05 12:34:56.1001,1906-12-25 -1906-05-05 12:34:56.1002,1906-12-25 -1906-05-05 12:34:56.1003,1906-12-25 -1906-05-05 12:34:56.1004,1906-12-25 -1906-05-05 12:34:56.1005,1906-12-25 -1906-05-05 12:34:56.1006,1906-12-25 -1906-05-05 12:34:56.1007,1906-12-25 -1906-05-05 12:34:56.1008,1906-12-25 -1906-05-05 12:34:56.1009,1906-12-25 -1906-05-05 12:34:56.101,1906-12-25 -1906-05-05 12:34:56.1011,1906-12-25 -1906-05-05 12:34:56.1012,1906-12-25 -1906-05-05 12:34:56.1013,1906-12-25 -1906-05-05 12:34:56.1014,1906-12-25 -1906-05-05 12:34:56.1015,1906-12-25 -1906-05-05 12:34:56.1016,1906-12-25 -1906-05-05 12:34:56.1017,1906-12-25 -1906-05-05 12:34:56.1018,1906-12-25 -1906-05-05 12:34:56.1019,1906-12-25 -1906-05-05 12:34:56.102,1906-12-25 -1906-05-05 12:34:56.1021,1906-12-25 -1906-05-05 12:34:56.1022,1906-12-25 -1906-05-05 12:34:56.1023,1906-12-25 -1906-05-05 12:34:56.1024,1906-12-25 -1906-05-05 12:34:56.1025,1906-12-25 -1906-05-05 12:34:56.1026,1906-12-25 -1906-05-05 12:34:56.1027,1906-12-25 -1906-05-05 12:34:56.1028,1906-12-25 -1906-05-05 12:34:56.1029,1906-12-25 -1906-05-05 12:34:56.103,1906-12-25 -1906-05-05 12:34:56.1031,1906-12-25 -1906-05-05 12:34:56.1032,1906-12-25 -1906-05-05 12:34:56.1033,1906-12-25 -1906-05-05 12:34:56.1034,1906-12-25 -1906-05-05 12:34:56.1035,1906-12-25 -1906-05-05 12:34:56.1036,1906-12-25 -1906-05-05 12:34:56.1037,1906-12-25 -1906-05-05 12:34:56.1038,1906-12-25 -1906-05-05 12:34:56.1039,1906-12-25 -1906-05-05 12:34:56.104,1906-12-25 -1906-05-05 12:34:56.1041,1906-12-25 -1906-05-05 12:34:56.1042,1906-12-25 -1906-05-05 12:34:56.1043,1906-12-25 -1906-05-05 12:34:56.1044,1906-12-25 -1906-05-05 12:34:56.1045,1906-12-25 -1906-05-05 12:34:56.1046,1906-12-25 -1906-05-05 12:34:56.1047,1906-12-25 -1906-05-05 12:34:56.1048,1906-12-25 -1906-05-05 12:34:56.1049,1906-12-25 -1906-05-05 12:34:56.105,1906-12-25 -1906-05-05 12:34:56.1051,1906-12-25 -1906-05-05 12:34:56.1052,1906-12-25 -1906-05-05 12:34:56.1053,1906-12-25 -1906-05-05 12:34:56.1054,1906-12-25 -1906-05-05 12:34:56.1055,1906-12-25 -1906-05-05 12:34:56.1056,1906-12-25 -1906-05-05 12:34:56.1057,1906-12-25 -1906-05-05 12:34:56.1058,1906-12-25 -1906-05-05 12:34:56.1059,1906-12-25 -1906-05-05 12:34:56.106,1906-12-25 -1906-05-05 12:34:56.1061,1906-12-25 -1906-05-05 12:34:56.1062,1906-12-25 -1906-05-05 12:34:56.1063,1906-12-25 -1906-05-05 12:34:56.1064,1906-12-25 -1906-05-05 12:34:56.1065,1906-12-25 -1906-05-05 12:34:56.1066,1906-12-25 -1906-05-05 12:34:56.1067,1906-12-25 -1906-05-05 12:34:56.1068,1906-12-25 -1906-05-05 12:34:56.1069,1906-12-25 -1906-05-05 12:34:56.107,1906-12-25 -1906-05-05 12:34:56.1071,1906-12-25 -1906-05-05 12:34:56.1072,1906-12-25 -1906-05-05 12:34:56.1073,1906-12-25 -1906-05-05 12:34:56.1074,1906-12-25 -1906-05-05 12:34:56.1075,1906-12-25 -1906-05-05 12:34:56.1076,1906-12-25 -1906-05-05 12:34:56.1077,1906-12-25 -1906-05-05 12:34:56.1078,1906-12-25 -1906-05-05 12:34:56.1079,1906-12-25 -1906-05-05 12:34:56.108,1906-12-25 -1906-05-05 12:34:56.1081,1906-12-25 -1906-05-05 12:34:56.1082,1906-12-25 -1906-05-05 12:34:56.1083,1906-12-25 -1906-05-05 12:34:56.1084,1906-12-25 -1906-05-05 12:34:56.1085,1906-12-25 -1906-05-05 12:34:56.1086,1906-12-25 -1906-05-05 12:34:56.1087,1906-12-25 -1906-05-05 12:34:56.1088,1906-12-25 -1906-05-05 12:34:56.1089,1906-12-25 -1906-05-05 12:34:56.109,1906-12-25 -1906-05-05 12:34:56.1091,1906-12-25 -1906-05-05 12:34:56.1092,1906-12-25 -1906-05-05 12:34:56.1093,1906-12-25 -1906-05-05 12:34:56.1094,1906-12-25 -1906-05-05 12:34:56.1095,1906-12-25 -1906-05-05 12:34:56.1096,1906-12-25 -1906-05-05 12:34:56.1097,1906-12-25 -1906-05-05 12:34:56.1098,1906-12-25 -1906-05-05 12:34:56.1099,1906-12-25 -1906-05-05 12:34:56.11,1906-12-25 -1906-05-05 12:34:56.1101,1906-12-25 -1906-05-05 12:34:56.1102,1906-12-25 -1906-05-05 12:34:56.1103,1906-12-25 -1906-05-05 12:34:56.1104,1906-12-25 -1906-05-05 12:34:56.1105,1906-12-25 -1906-05-05 12:34:56.1106,1906-12-25 -1906-05-05 12:34:56.1107,1906-12-25 -1906-05-05 12:34:56.1108,1906-12-25 -1906-05-05 12:34:56.1109,1906-12-25 -1906-05-05 12:34:56.111,1906-12-25 -1906-05-05 12:34:56.1111,1906-12-25 -1906-05-05 12:34:56.1112,1906-12-25 -1906-05-05 12:34:56.1113,1906-12-25 -1906-05-05 12:34:56.1114,1906-12-25 -1906-05-05 12:34:56.1115,1906-12-25 -1906-05-05 12:34:56.1116,1906-12-25 -1906-05-05 12:34:56.1117,1906-12-25 -1906-05-05 12:34:56.1118,1906-12-25 -1906-05-05 12:34:56.1119,1906-12-25 -1906-05-05 12:34:56.112,1906-12-25 -1906-05-05 12:34:56.1121,1906-12-25 -1906-05-05 12:34:56.1122,1906-12-25 -1906-05-05 12:34:56.1123,1906-12-25 -1906-05-05 12:34:56.1124,1906-12-25 -1906-05-05 12:34:56.1125,1906-12-25 -1906-05-05 12:34:56.1126,1906-12-25 -1906-05-05 12:34:56.1127,1906-12-25 -1906-05-05 12:34:56.1128,1906-12-25 -1906-05-05 12:34:56.1129,1906-12-25 -1906-05-05 12:34:56.113,1906-12-25 -1906-05-05 12:34:56.1131,1906-12-25 -1906-05-05 12:34:56.1132,1906-12-25 -1906-05-05 12:34:56.1133,1906-12-25 -1906-05-05 12:34:56.1134,1906-12-25 -1906-05-05 12:34:56.1135,1906-12-25 -1906-05-05 12:34:56.1136,1906-12-25 -1906-05-05 12:34:56.1137,1906-12-25 -1906-05-05 12:34:56.1138,1906-12-25 -1906-05-05 12:34:56.1139,1906-12-25 -1906-05-05 12:34:56.114,1906-12-25 -1906-05-05 12:34:56.1141,1906-12-25 -1906-05-05 12:34:56.1142,1906-12-25 -1906-05-05 12:34:56.1143,1906-12-25 -1906-05-05 12:34:56.1144,1906-12-25 -1906-05-05 12:34:56.1145,1906-12-25 -1906-05-05 12:34:56.1146,1906-12-25 -1906-05-05 12:34:56.1147,1906-12-25 -1906-05-05 12:34:56.1148,1906-12-25 -1906-05-05 12:34:56.1149,1906-12-25 -1906-05-05 12:34:56.115,1906-12-25 -1906-05-05 12:34:56.1151,1906-12-25 -1906-05-05 12:34:56.1152,1906-12-25 -1906-05-05 12:34:56.1153,1906-12-25 -1906-05-05 12:34:56.1154,1906-12-25 -1906-05-05 12:34:56.1155,1906-12-25 -1906-05-05 12:34:56.1156,1906-12-25 -1906-05-05 12:34:56.1157,1906-12-25 -1906-05-05 12:34:56.1158,1906-12-25 -1906-05-05 12:34:56.1159,1906-12-25 -1906-05-05 12:34:56.116,1906-12-25 -1906-05-05 12:34:56.1161,1906-12-25 -1906-05-05 12:34:56.1162,1906-12-25 -1906-05-05 12:34:56.1163,1906-12-25 -1906-05-05 12:34:56.1164,1906-12-25 -1906-05-05 12:34:56.1165,1906-12-25 -1906-05-05 12:34:56.1166,1906-12-25 -1906-05-05 12:34:56.1167,1906-12-25 -1906-05-05 12:34:56.1168,1906-12-25 -1906-05-05 12:34:56.1169,1906-12-25 -1906-05-05 12:34:56.117,1906-12-25 -1906-05-05 12:34:56.1171,1906-12-25 -1906-05-05 12:34:56.1172,1906-12-25 -1906-05-05 12:34:56.1173,1906-12-25 -1906-05-05 12:34:56.1174,1906-12-25 -1906-05-05 12:34:56.1175,1906-12-25 -1906-05-05 12:34:56.1176,1906-12-25 -1906-05-05 12:34:56.1177,1906-12-25 -1906-05-05 12:34:56.1178,1906-12-25 -1906-05-05 12:34:56.1179,1906-12-25 -1906-05-05 12:34:56.118,1906-12-25 -1906-05-05 12:34:56.1181,1906-12-25 -1906-05-05 12:34:56.1182,1906-12-25 -1906-05-05 12:34:56.1183,1906-12-25 -1906-05-05 12:34:56.1184,1906-12-25 -1906-05-05 12:34:56.1185,1906-12-25 -1906-05-05 12:34:56.1186,1906-12-25 -1906-05-05 12:34:56.1187,1906-12-25 -1906-05-05 12:34:56.1188,1906-12-25 -1906-05-05 12:34:56.1189,1906-12-25 -1906-05-05 12:34:56.119,1906-12-25 -1906-05-05 12:34:56.1191,1906-12-25 -1906-05-05 12:34:56.1192,1906-12-25 -1906-05-05 12:34:56.1193,1906-12-25 -1906-05-05 12:34:56.1194,1906-12-25 -1906-05-05 12:34:56.1195,1906-12-25 -1906-05-05 12:34:56.1196,1906-12-25 -1906-05-05 12:34:56.1197,1906-12-25 -1906-05-05 12:34:56.1198,1906-12-25 -1906-05-05 12:34:56.1199,1906-12-25 -1906-05-05 12:34:56.12,1906-12-25 -1906-05-05 12:34:56.1201,1906-12-25 -1906-05-05 12:34:56.1202,1906-12-25 -1906-05-05 12:34:56.1203,1906-12-25 -1906-05-05 12:34:56.1204,1906-12-25 -1906-05-05 12:34:56.1205,1906-12-25 -1906-05-05 12:34:56.1206,1906-12-25 -1906-05-05 12:34:56.1207,1906-12-25 -1906-05-05 12:34:56.1208,1906-12-25 -1906-05-05 12:34:56.1209,1906-12-25 -1906-05-05 12:34:56.121,1906-12-25 -1906-05-05 12:34:56.1211,1906-12-25 -1906-05-05 12:34:56.1212,1906-12-25 -1906-05-05 12:34:56.1213,1906-12-25 -1906-05-05 12:34:56.1214,1906-12-25 -1906-05-05 12:34:56.1215,1906-12-25 -1906-05-05 12:34:56.1216,1906-12-25 -1906-05-05 12:34:56.1217,1906-12-25 -1906-05-05 12:34:56.1218,1906-12-25 -1906-05-05 12:34:56.1219,1906-12-25 -1906-05-05 12:34:56.122,1906-12-25 -1906-05-05 12:34:56.1221,1906-12-25 -1906-05-05 12:34:56.1222,1906-12-25 -1906-05-05 12:34:56.1223,1906-12-25 -1906-05-05 12:34:56.1224,1906-12-25 -1906-05-05 12:34:56.1225,1906-12-25 -1906-05-05 12:34:56.1226,1906-12-25 -1906-05-05 12:34:56.1227,1906-12-25 -1906-05-05 12:34:56.1228,1906-12-25 -1906-05-05 12:34:56.1229,1906-12-25 -1906-05-05 12:34:56.123,1906-12-25 -1906-05-05 12:34:56.1231,1906-12-25 -1906-05-05 12:34:56.1232,1906-12-25 -1906-05-05 12:34:56.1233,1906-12-25 -1906-05-05 12:34:56.1234,1906-12-25 -1906-05-05 12:34:56.1235,1906-12-25 -1906-05-05 12:34:56.1236,1906-12-25 -1906-05-05 12:34:56.1237,1906-12-25 -1906-05-05 12:34:56.1238,1906-12-25 -1906-05-05 12:34:56.1239,1906-12-25 -1906-05-05 12:34:56.124,1906-12-25 -1906-05-05 12:34:56.1241,1906-12-25 -1906-05-05 12:34:56.1242,1906-12-25 -1906-05-05 12:34:56.1243,1906-12-25 -1906-05-05 12:34:56.1244,1906-12-25 -1906-05-05 12:34:56.1245,1906-12-25 -1906-05-05 12:34:56.1246,1906-12-25 -1906-05-05 12:34:56.1247,1906-12-25 -1906-05-05 12:34:56.1248,1906-12-25 -1906-05-05 12:34:56.1249,1906-12-25 -1906-05-05 12:34:56.125,1906-12-25 -1906-05-05 12:34:56.1251,1906-12-25 -1906-05-05 12:34:56.1252,1906-12-25 -1906-05-05 12:34:56.1253,1906-12-25 -1906-05-05 12:34:56.1254,1906-12-25 -1906-05-05 12:34:56.1255,1906-12-25 -1906-05-05 12:34:56.1256,1906-12-25 -1906-05-05 12:34:56.1257,1906-12-25 -1906-05-05 12:34:56.1258,1906-12-25 -1906-05-05 12:34:56.1259,1906-12-25 -1906-05-05 12:34:56.126,1906-12-25 -1906-05-05 12:34:56.1261,1906-12-25 -1906-05-05 12:34:56.1262,1906-12-25 -1906-05-05 12:34:56.1263,1906-12-25 -1906-05-05 12:34:56.1264,1906-12-25 -1906-05-05 12:34:56.1265,1906-12-25 -1906-05-05 12:34:56.1266,1906-12-25 -1906-05-05 12:34:56.1267,1906-12-25 -1906-05-05 12:34:56.1268,1906-12-25 -1906-05-05 12:34:56.1269,1906-12-25 -1906-05-05 12:34:56.127,1906-12-25 -1906-05-05 12:34:56.1271,1906-12-25 -1906-05-05 12:34:56.1272,1906-12-25 -1906-05-05 12:34:56.1273,1906-12-25 -1906-05-05 12:34:56.1274,1906-12-25 -1906-05-05 12:34:56.1275,1906-12-25 -1906-05-05 12:34:56.1276,1906-12-25 -1906-05-05 12:34:56.1277,1906-12-25 -1906-05-05 12:34:56.1278,1906-12-25 -1906-05-05 12:34:56.1279,1906-12-25 -1906-05-05 12:34:56.128,1906-12-25 -1906-05-05 12:34:56.1281,1906-12-25 -1906-05-05 12:34:56.1282,1906-12-25 -1906-05-05 12:34:56.1283,1906-12-25 -1906-05-05 12:34:56.1284,1906-12-25 -1906-05-05 12:34:56.1285,1906-12-25 -1906-05-05 12:34:56.1286,1906-12-25 -1906-05-05 12:34:56.1287,1906-12-25 -1906-05-05 12:34:56.1288,1906-12-25 -1906-05-05 12:34:56.1289,1906-12-25 -1906-05-05 12:34:56.129,1906-12-25 -1906-05-05 12:34:56.1291,1906-12-25 -1906-05-05 12:34:56.1292,1906-12-25 -1906-05-05 12:34:56.1293,1906-12-25 -1906-05-05 12:34:56.1294,1906-12-25 -1906-05-05 12:34:56.1295,1906-12-25 -1906-05-05 12:34:56.1296,1906-12-25 -1906-05-05 12:34:56.1297,1906-12-25 -1906-05-05 12:34:56.1298,1906-12-25 -1906-05-05 12:34:56.1299,1906-12-25 -1906-05-05 12:34:56.13,1906-12-25 -1906-05-05 12:34:56.1301,1906-12-25 -1906-05-05 12:34:56.1302,1906-12-25 -1906-05-05 12:34:56.1303,1906-12-25 -1906-05-05 12:34:56.1304,1906-12-25 -1906-05-05 12:34:56.1305,1906-12-25 -1906-05-05 12:34:56.1306,1906-12-25 -1906-05-05 12:34:56.1307,1906-12-25 -1906-05-05 12:34:56.1308,1906-12-25 -1906-05-05 12:34:56.1309,1906-12-25 -1906-05-05 12:34:56.131,1906-12-25 -1906-05-05 12:34:56.1311,1906-12-25 -1906-05-05 12:34:56.1312,1906-12-25 -1906-05-05 12:34:56.1313,1906-12-25 -1906-05-05 12:34:56.1314,1906-12-25 -1906-05-05 12:34:56.1315,1906-12-25 -1906-05-05 12:34:56.1316,1906-12-25 -1906-05-05 12:34:56.1317,1906-12-25 -1906-05-05 12:34:56.1318,1906-12-25 -1906-05-05 12:34:56.1319,1906-12-25 -1906-05-05 12:34:56.132,1906-12-25 -1906-05-05 12:34:56.1321,1906-12-25 -1906-05-05 12:34:56.1322,1906-12-25 -1906-05-05 12:34:56.1323,1906-12-25 -1906-05-05 12:34:56.1324,1906-12-25 -1906-05-05 12:34:56.1325,1906-12-25 -1906-05-05 12:34:56.1326,1906-12-25 -1906-05-05 12:34:56.1327,1906-12-25 -1906-05-05 12:34:56.1328,1906-12-25 -1906-05-05 12:34:56.1329,1906-12-25 -1906-05-05 12:34:56.133,1906-12-25 -1906-05-05 12:34:56.1331,1906-12-25 -1906-05-05 12:34:56.1332,1906-12-25 -1906-05-05 12:34:56.1333,1906-12-25 -1906-05-05 12:34:56.1334,1906-12-25 -1906-05-05 12:34:56.1335,1906-12-25 -1906-05-05 12:34:56.1336,1906-12-25 -1906-05-05 12:34:56.1337,1906-12-25 -1906-05-05 12:34:56.1338,1906-12-25 -1906-05-05 12:34:56.1339,1906-12-25 -1906-05-05 12:34:56.134,1906-12-25 -1906-05-05 12:34:56.1341,1906-12-25 -1906-05-05 12:34:56.1342,1906-12-25 -1906-05-05 12:34:56.1343,1906-12-25 -1906-05-05 12:34:56.1344,1906-12-25 -1906-05-05 12:34:56.1345,1906-12-25 -1906-05-05 12:34:56.1346,1906-12-25 -1906-05-05 12:34:56.1347,1906-12-25 -1906-05-05 12:34:56.1348,1906-12-25 -1906-05-05 12:34:56.1349,1906-12-25 -1906-05-05 12:34:56.135,1906-12-25 -1906-05-05 12:34:56.1351,1906-12-25 -1906-05-05 12:34:56.1352,1906-12-25 -1906-05-05 12:34:56.1353,1906-12-25 -1906-05-05 12:34:56.1354,1906-12-25 -1906-05-05 12:34:56.1355,1906-12-25 -1906-05-05 12:34:56.1356,1906-12-25 -1906-05-05 12:34:56.1357,1906-12-25 -1906-05-05 12:34:56.1358,1906-12-25 -1906-05-05 12:34:56.1359,1906-12-25 -1906-05-05 12:34:56.136,1906-12-25 -1906-05-05 12:34:56.1361,1906-12-25 -1906-05-05 12:34:56.1362,1906-12-25 -1906-05-05 12:34:56.1363,1906-12-25 -1906-05-05 12:34:56.1364,1906-12-25 -1906-05-05 12:34:56.1365,1906-12-25 -1906-05-05 12:34:56.1366,1906-12-25 -1906-05-05 12:34:56.1367,1906-12-25 -1906-05-05 12:34:56.1368,1906-12-25 -1906-05-05 12:34:56.1369,1906-12-25 -1906-05-05 12:34:56.137,1906-12-25 -1906-05-05 12:34:56.1371,1906-12-25 -1906-05-05 12:34:56.1372,1906-12-25 -1906-05-05 12:34:56.1373,1906-12-25 -1906-05-05 12:34:56.1374,1906-12-25 -1906-05-05 12:34:56.1375,1906-12-25 -1906-05-05 12:34:56.1376,1906-12-25 -1906-05-05 12:34:56.1377,1906-12-25 -1906-05-05 12:34:56.1378,1906-12-25 -1906-05-05 12:34:56.1379,1906-12-25 -1906-05-05 12:34:56.138,1906-12-25 -1906-05-05 12:34:56.1381,1906-12-25 -1906-05-05 12:34:56.1382,1906-12-25 -1906-05-05 12:34:56.1383,1906-12-25 -1906-05-05 12:34:56.1384,1906-12-25 -1906-05-05 12:34:56.1385,1906-12-25 -1906-05-05 12:34:56.1386,1906-12-25 -1906-05-05 12:34:56.1387,1906-12-25 -1906-05-05 12:34:56.1388,1906-12-25 -1906-05-05 12:34:56.1389,1906-12-25 -1906-05-05 12:34:56.139,1906-12-25 -1906-05-05 12:34:56.1391,1906-12-25 -1906-05-05 12:34:56.1392,1906-12-25 -1906-05-05 12:34:56.1393,1906-12-25 -1906-05-05 12:34:56.1394,1906-12-25 -1906-05-05 12:34:56.1395,1906-12-25 -1906-05-05 12:34:56.1396,1906-12-25 -1906-05-05 12:34:56.1397,1906-12-25 -1906-05-05 12:34:56.1398,1906-12-25 -1906-05-05 12:34:56.1399,1906-12-25 -1906-05-05 12:34:56.14,1906-12-25 -1906-05-05 12:34:56.1401,1906-12-25 -1906-05-05 12:34:56.1402,1906-12-25 -1906-05-05 12:34:56.1403,1906-12-25 -1906-05-05 12:34:56.1404,1906-12-25 -1906-05-05 12:34:56.1405,1906-12-25 -1906-05-05 12:34:56.1406,1906-12-25 -1906-05-05 12:34:56.1407,1906-12-25 -1906-05-05 12:34:56.1408,1906-12-25 -1906-05-05 12:34:56.1409,1906-12-25 -1906-05-05 12:34:56.141,1906-12-25 -1906-05-05 12:34:56.1411,1906-12-25 -1906-05-05 12:34:56.1412,1906-12-25 -1906-05-05 12:34:56.1413,1906-12-25 -1906-05-05 12:34:56.1414,1906-12-25 -1906-05-05 12:34:56.1415,1906-12-25 -1906-05-05 12:34:56.1416,1906-12-25 -1906-05-05 12:34:56.1417,1906-12-25 -1906-05-05 12:34:56.1418,1906-12-25 -1906-05-05 12:34:56.1419,1906-12-25 -1906-05-05 12:34:56.142,1906-12-25 -1906-05-05 12:34:56.1421,1906-12-25 -1906-05-05 12:34:56.1422,1906-12-25 -1906-05-05 12:34:56.1423,1906-12-25 -1906-05-05 12:34:56.1424,1906-12-25 -1906-05-05 12:34:56.1425,1906-12-25 -1906-05-05 12:34:56.1426,1906-12-25 -1906-05-05 12:34:56.1427,1906-12-25 -1906-05-05 12:34:56.1428,1906-12-25 -1906-05-05 12:34:56.1429,1906-12-25 -1906-05-05 12:34:56.143,1906-12-25 -1906-05-05 12:34:56.1431,1906-12-25 -1906-05-05 12:34:56.1432,1906-12-25 -1906-05-05 12:34:56.1433,1906-12-25 -1906-05-05 12:34:56.1434,1906-12-25 -1906-05-05 12:34:56.1435,1906-12-25 -1906-05-05 12:34:56.1436,1906-12-25 -1906-05-05 12:34:56.1437,1906-12-25 -1906-05-05 12:34:56.1438,1906-12-25 -1906-05-05 12:34:56.1439,1906-12-25 -1906-05-05 12:34:56.144,1906-12-25 -1906-05-05 12:34:56.1441,1906-12-25 -1906-05-05 12:34:56.1442,1906-12-25 -1906-05-05 12:34:56.1443,1906-12-25 -1906-05-05 12:34:56.1444,1906-12-25 -1906-05-05 12:34:56.1445,1906-12-25 -1906-05-05 12:34:56.1446,1906-12-25 -1906-05-05 12:34:56.1447,1906-12-25 -1906-05-05 12:34:56.1448,1906-12-25 -1906-05-05 12:34:56.1449,1906-12-25 -1906-05-05 12:34:56.145,1906-12-25 -1906-05-05 12:34:56.1451,1906-12-25 -1906-05-05 12:34:56.1452,1906-12-25 -1906-05-05 12:34:56.1453,1906-12-25 -1906-05-05 12:34:56.1454,1906-12-25 -1906-05-05 12:34:56.1455,1906-12-25 -1906-05-05 12:34:56.1456,1906-12-25 -1906-05-05 12:34:56.1457,1906-12-25 -1906-05-05 12:34:56.1458,1906-12-25 -1906-05-05 12:34:56.1459,1906-12-25 -1906-05-05 12:34:56.146,1906-12-25 -1906-05-05 12:34:56.1461,1906-12-25 -1906-05-05 12:34:56.1462,1906-12-25 -1906-05-05 12:34:56.1463,1906-12-25 -1906-05-05 12:34:56.1464,1906-12-25 -1906-05-05 12:34:56.1465,1906-12-25 -1906-05-05 12:34:56.1466,1906-12-25 -1906-05-05 12:34:56.1467,1906-12-25 -1906-05-05 12:34:56.1468,1906-12-25 -1906-05-05 12:34:56.1469,1906-12-25 -1906-05-05 12:34:56.147,1906-12-25 -1906-05-05 12:34:56.1471,1906-12-25 -1906-05-05 12:34:56.1472,1906-12-25 -1906-05-05 12:34:56.1473,1906-12-25 -1906-05-05 12:34:56.1474,1906-12-25 -1906-05-05 12:34:56.1475,1906-12-25 -1906-05-05 12:34:56.1476,1906-12-25 -1906-05-05 12:34:56.1477,1906-12-25 -1906-05-05 12:34:56.1478,1906-12-25 -1906-05-05 12:34:56.1479,1906-12-25 -1906-05-05 12:34:56.148,1906-12-25 -1906-05-05 12:34:56.1481,1906-12-25 -1906-05-05 12:34:56.1482,1906-12-25 -1906-05-05 12:34:56.1483,1906-12-25 -1906-05-05 12:34:56.1484,1906-12-25 -1906-05-05 12:34:56.1485,1906-12-25 -1906-05-05 12:34:56.1486,1906-12-25 -1906-05-05 12:34:56.1487,1906-12-25 -1906-05-05 12:34:56.1488,1906-12-25 -1906-05-05 12:34:56.1489,1906-12-25 -1906-05-05 12:34:56.149,1906-12-25 -1906-05-05 12:34:56.1491,1906-12-25 -1906-05-05 12:34:56.1492,1906-12-25 -1906-05-05 12:34:56.1493,1906-12-25 -1906-05-05 12:34:56.1494,1906-12-25 -1906-05-05 12:34:56.1495,1906-12-25 -1906-05-05 12:34:56.1496,1906-12-25 -1906-05-05 12:34:56.1497,1906-12-25 -1906-05-05 12:34:56.1498,1906-12-25 -1906-05-05 12:34:56.1499,1906-12-25 -1906-05-05 12:34:56.15,1906-12-25 -1906-05-05 12:34:56.1501,1906-12-25 -1906-05-05 12:34:56.1502,1906-12-25 -1906-05-05 12:34:56.1503,1906-12-25 -1906-05-05 12:34:56.1504,1906-12-25 -1906-05-05 12:34:56.1505,1906-12-25 -1906-05-05 12:34:56.1506,1906-12-25 -1906-05-05 12:34:56.1507,1906-12-25 -1906-05-05 12:34:56.1508,1906-12-25 -1906-05-05 12:34:56.1509,1906-12-25 -1906-05-05 12:34:56.151,1906-12-25 -1906-05-05 12:34:56.1511,1906-12-25 -1906-05-05 12:34:56.1512,1906-12-25 -1906-05-05 12:34:56.1513,1906-12-25 -1906-05-05 12:34:56.1514,1906-12-25 -1906-05-05 12:34:56.1515,1906-12-25 -1906-05-05 12:34:56.1516,1906-12-25 -1906-05-05 12:34:56.1517,1906-12-25 -1906-05-05 12:34:56.1518,1906-12-25 -1906-05-05 12:34:56.1519,1906-12-25 -1906-05-05 12:34:56.152,1906-12-25 -1906-05-05 12:34:56.1521,1906-12-25 -1906-05-05 12:34:56.1522,1906-12-25 -1906-05-05 12:34:56.1523,1906-12-25 -1906-05-05 12:34:56.1524,1906-12-25 -1906-05-05 12:34:56.1525,1906-12-25 -1906-05-05 12:34:56.1526,1906-12-25 -1906-05-05 12:34:56.1527,1906-12-25 -1906-05-05 12:34:56.1528,1906-12-25 -1906-05-05 12:34:56.1529,1906-12-25 -1906-05-05 12:34:56.153,1906-12-25 -1906-05-05 12:34:56.1531,1906-12-25 -1906-05-05 12:34:56.1532,1906-12-25 -1906-05-05 12:34:56.1533,1906-12-25 -1906-05-05 12:34:56.1534,1906-12-25 -1906-05-05 12:34:56.1535,1906-12-25 -1906-05-05 12:34:56.1536,1906-12-25 -1906-05-05 12:34:56.1537,1906-12-25 -1906-05-05 12:34:56.1538,1906-12-25 -1906-05-05 12:34:56.1539,1906-12-25 -1906-05-05 12:34:56.154,1906-12-25 -1906-05-05 12:34:56.1541,1906-12-25 -1906-05-05 12:34:56.1542,1906-12-25 -1906-05-05 12:34:56.1543,1906-12-25 -1906-05-05 12:34:56.1544,1906-12-25 -1906-05-05 12:34:56.1545,1906-12-25 -1906-05-05 12:34:56.1546,1906-12-25 -1906-05-05 12:34:56.1547,1906-12-25 -1906-05-05 12:34:56.1548,1906-12-25 -1906-05-05 12:34:56.1549,1906-12-25 -1906-05-05 12:34:56.155,1906-12-25 -1906-05-05 12:34:56.1551,1906-12-25 -1906-05-05 12:34:56.1552,1906-12-25 -1906-05-05 12:34:56.1553,1906-12-25 -1906-05-05 12:34:56.1554,1906-12-25 -1906-05-05 12:34:56.1555,1906-12-25 -1906-05-05 12:34:56.1556,1906-12-25 -1906-05-05 12:34:56.1557,1906-12-25 -1906-05-05 12:34:56.1558,1906-12-25 -1906-05-05 12:34:56.1559,1906-12-25 -1906-05-05 12:34:56.156,1906-12-25 -1906-05-05 12:34:56.1561,1906-12-25 -1906-05-05 12:34:56.1562,1906-12-25 -1906-05-05 12:34:56.1563,1906-12-25 -1906-05-05 12:34:56.1564,1906-12-25 -1906-05-05 12:34:56.1565,1906-12-25 -1906-05-05 12:34:56.1566,1906-12-25 -1906-05-05 12:34:56.1567,1906-12-25 -1906-05-05 12:34:56.1568,1906-12-25 -1906-05-05 12:34:56.1569,1906-12-25 -1906-05-05 12:34:56.157,1906-12-25 -1906-05-05 12:34:56.1571,1906-12-25 -1906-05-05 12:34:56.1572,1906-12-25 -1906-05-05 12:34:56.1573,1906-12-25 -1906-05-05 12:34:56.1574,1906-12-25 -1906-05-05 12:34:56.1575,1906-12-25 -1906-05-05 12:34:56.1576,1906-12-25 -1906-05-05 12:34:56.1577,1906-12-25 -1906-05-05 12:34:56.1578,1906-12-25 -1906-05-05 12:34:56.1579,1906-12-25 -1906-05-05 12:34:56.158,1906-12-25 -1906-05-05 12:34:56.1581,1906-12-25 -1906-05-05 12:34:56.1582,1906-12-25 -1906-05-05 12:34:56.1583,1906-12-25 -1906-05-05 12:34:56.1584,1906-12-25 -1906-05-05 12:34:56.1585,1906-12-25 -1906-05-05 12:34:56.1586,1906-12-25 -1906-05-05 12:34:56.1587,1906-12-25 -1906-05-05 12:34:56.1588,1906-12-25 -1906-05-05 12:34:56.1589,1906-12-25 -1906-05-05 12:34:56.159,1906-12-25 -1906-05-05 12:34:56.1591,1906-12-25 -1906-05-05 12:34:56.1592,1906-12-25 -1906-05-05 12:34:56.1593,1906-12-25 -1906-05-05 12:34:56.1594,1906-12-25 -1906-05-05 12:34:56.1595,1906-12-25 -1906-05-05 12:34:56.1596,1906-12-25 -1906-05-05 12:34:56.1597,1906-12-25 -1906-05-05 12:34:56.1598,1906-12-25 -1906-05-05 12:34:56.1599,1906-12-25 -1906-05-05 12:34:56.16,1906-12-25 -1906-05-05 12:34:56.1601,1906-12-25 -1906-05-05 12:34:56.1602,1906-12-25 -1906-05-05 12:34:56.1603,1906-12-25 -1906-05-05 12:34:56.1604,1906-12-25 -1906-05-05 12:34:56.1605,1906-12-25 -1906-05-05 12:34:56.1606,1906-12-25 -1906-05-05 12:34:56.1607,1906-12-25 -1906-05-05 12:34:56.1608,1906-12-25 -1906-05-05 12:34:56.1609,1906-12-25 -1906-05-05 12:34:56.161,1906-12-25 -1906-05-05 12:34:56.1611,1906-12-25 -1906-05-05 12:34:56.1612,1906-12-25 -1906-05-05 12:34:56.1613,1906-12-25 -1906-05-05 12:34:56.1614,1906-12-25 -1906-05-05 12:34:56.1615,1906-12-25 -1906-05-05 12:34:56.1616,1906-12-25 -1906-05-05 12:34:56.1617,1906-12-25 -1906-05-05 12:34:56.1618,1906-12-25 -1906-05-05 12:34:56.1619,1906-12-25 -1906-05-05 12:34:56.162,1906-12-25 -1906-05-05 12:34:56.1621,1906-12-25 -1906-05-05 12:34:56.1622,1906-12-25 -1906-05-05 12:34:56.1623,1906-12-25 -1906-05-05 12:34:56.1624,1906-12-25 -1906-05-05 12:34:56.1625,1906-12-25 -1906-05-05 12:34:56.1626,1906-12-25 -1906-05-05 12:34:56.1627,1906-12-25 -1906-05-05 12:34:56.1628,1906-12-25 -1906-05-05 12:34:56.1629,1906-12-25 -1906-05-05 12:34:56.163,1906-12-25 -1906-05-05 12:34:56.1631,1906-12-25 -1906-05-05 12:34:56.1632,1906-12-25 -1906-05-05 12:34:56.1633,1906-12-25 -1906-05-05 12:34:56.1634,1906-12-25 -1906-05-05 12:34:56.1635,1906-12-25 -1906-05-05 12:34:56.1636,1906-12-25 -1906-05-05 12:34:56.1637,1906-12-25 -1906-05-05 12:34:56.1638,1906-12-25 -1906-05-05 12:34:56.1639,1906-12-25 -1906-05-05 12:34:56.164,1906-12-25 -1906-05-05 12:34:56.1641,1906-12-25 -1906-05-05 12:34:56.1642,1906-12-25 -1906-05-05 12:34:56.1643,1906-12-25 -1906-05-05 12:34:56.1644,1906-12-25 -1906-05-05 12:34:56.1645,1906-12-25 -1906-05-05 12:34:56.1646,1906-12-25 -1906-05-05 12:34:56.1647,1906-12-25 -1906-05-05 12:34:56.1648,1906-12-25 -1906-05-05 12:34:56.1649,1906-12-25 -1906-05-05 12:34:56.165,1906-12-25 -1906-05-05 12:34:56.1651,1906-12-25 -1906-05-05 12:34:56.1652,1906-12-25 -1906-05-05 12:34:56.1653,1906-12-25 -1906-05-05 12:34:56.1654,1906-12-25 -1906-05-05 12:34:56.1655,1906-12-25 -1906-05-05 12:34:56.1656,1906-12-25 -1906-05-05 12:34:56.1657,1906-12-25 -1906-05-05 12:34:56.1658,1906-12-25 -1906-05-05 12:34:56.1659,1906-12-25 -1906-05-05 12:34:56.166,1906-12-25 -1906-05-05 12:34:56.1661,1906-12-25 -1906-05-05 12:34:56.1662,1906-12-25 -1906-05-05 12:34:56.1663,1906-12-25 -1906-05-05 12:34:56.1664,1906-12-25 -1906-05-05 12:34:56.1665,1906-12-25 -1906-05-05 12:34:56.1666,1906-12-25 -1906-05-05 12:34:56.1667,1906-12-25 -1906-05-05 12:34:56.1668,1906-12-25 -1906-05-05 12:34:56.1669,1906-12-25 -1906-05-05 12:34:56.167,1906-12-25 -1906-05-05 12:34:56.1671,1906-12-25 -1906-05-05 12:34:56.1672,1906-12-25 -1906-05-05 12:34:56.1673,1906-12-25 -1906-05-05 12:34:56.1674,1906-12-25 -1906-05-05 12:34:56.1675,1906-12-25 -1906-05-05 12:34:56.1676,1906-12-25 -1906-05-05 12:34:56.1677,1906-12-25 -1906-05-05 12:34:56.1678,1906-12-25 -1906-05-05 12:34:56.1679,1906-12-25 -1906-05-05 12:34:56.168,1906-12-25 -1906-05-05 12:34:56.1681,1906-12-25 -1906-05-05 12:34:56.1682,1906-12-25 -1906-05-05 12:34:56.1683,1906-12-25 -1906-05-05 12:34:56.1684,1906-12-25 -1906-05-05 12:34:56.1685,1906-12-25 -1906-05-05 12:34:56.1686,1906-12-25 -1906-05-05 12:34:56.1687,1906-12-25 -1906-05-05 12:34:56.1688,1906-12-25 -1906-05-05 12:34:56.1689,1906-12-25 -1906-05-05 12:34:56.169,1906-12-25 -1906-05-05 12:34:56.1691,1906-12-25 -1906-05-05 12:34:56.1692,1906-12-25 -1906-05-05 12:34:56.1693,1906-12-25 -1906-05-05 12:34:56.1694,1906-12-25 -1906-05-05 12:34:56.1695,1906-12-25 -1906-05-05 12:34:56.1696,1906-12-25 -1906-05-05 12:34:56.1697,1906-12-25 -1906-05-05 12:34:56.1698,1906-12-25 -1906-05-05 12:34:56.1699,1906-12-25 -1906-05-05 12:34:56.17,1906-12-25 -1906-05-05 12:34:56.1701,1906-12-25 -1906-05-05 12:34:56.1702,1906-12-25 -1906-05-05 12:34:56.1703,1906-12-25 -1906-05-05 12:34:56.1704,1906-12-25 -1906-05-05 12:34:56.1705,1906-12-25 -1906-05-05 12:34:56.1706,1906-12-25 -1906-05-05 12:34:56.1707,1906-12-25 -1906-05-05 12:34:56.1708,1906-12-25 -1906-05-05 12:34:56.1709,1906-12-25 -1906-05-05 12:34:56.171,1906-12-25 -1906-05-05 12:34:56.1711,1906-12-25 -1906-05-05 12:34:56.1712,1906-12-25 -1906-05-05 12:34:56.1713,1906-12-25 -1906-05-05 12:34:56.1714,1906-12-25 -1906-05-05 12:34:56.1715,1906-12-25 -1906-05-05 12:34:56.1716,1906-12-25 -1906-05-05 12:34:56.1717,1906-12-25 -1906-05-05 12:34:56.1718,1906-12-25 -1906-05-05 12:34:56.1719,1906-12-25 -1906-05-05 12:34:56.172,1906-12-25 -1906-05-05 12:34:56.1721,1906-12-25 -1906-05-05 12:34:56.1722,1906-12-25 -1906-05-05 12:34:56.1723,1906-12-25 -1906-05-05 12:34:56.1724,1906-12-25 -1906-05-05 12:34:56.1725,1906-12-25 -1906-05-05 12:34:56.1726,1906-12-25 -1906-05-05 12:34:56.1727,1906-12-25 -1906-05-05 12:34:56.1728,1906-12-25 -1906-05-05 12:34:56.1729,1906-12-25 -1906-05-05 12:34:56.173,1906-12-25 -1906-05-05 12:34:56.1731,1906-12-25 -1906-05-05 12:34:56.1732,1906-12-25 -1906-05-05 12:34:56.1733,1906-12-25 -1906-05-05 12:34:56.1734,1906-12-25 -1906-05-05 12:34:56.1735,1906-12-25 -1906-05-05 12:34:56.1736,1906-12-25 -1906-05-05 12:34:56.1737,1906-12-25 -1906-05-05 12:34:56.1738,1906-12-25 -1906-05-05 12:34:56.1739,1906-12-25 -1906-05-05 12:34:56.174,1906-12-25 -1906-05-05 12:34:56.1741,1906-12-25 -1906-05-05 12:34:56.1742,1906-12-25 -1906-05-05 12:34:56.1743,1906-12-25 -1906-05-05 12:34:56.1744,1906-12-25 -1906-05-05 12:34:56.1745,1906-12-25 -1906-05-05 12:34:56.1746,1906-12-25 -1906-05-05 12:34:56.1747,1906-12-25 -1906-05-05 12:34:56.1748,1906-12-25 -1906-05-05 12:34:56.1749,1906-12-25 -1906-05-05 12:34:56.175,1906-12-25 -1906-05-05 12:34:56.1751,1906-12-25 -1906-05-05 12:34:56.1752,1906-12-25 -1906-05-05 12:34:56.1753,1906-12-25 -1906-05-05 12:34:56.1754,1906-12-25 -1906-05-05 12:34:56.1755,1906-12-25 -1906-05-05 12:34:56.1756,1906-12-25 -1906-05-05 12:34:56.1757,1906-12-25 -1906-05-05 12:34:56.1758,1906-12-25 -1906-05-05 12:34:56.1759,1906-12-25 -1906-05-05 12:34:56.176,1906-12-25 -1906-05-05 12:34:56.1761,1906-12-25 -1906-05-05 12:34:56.1762,1906-12-25 -1906-05-05 12:34:56.1763,1906-12-25 -1906-05-05 12:34:56.1764,1906-12-25 -1906-05-05 12:34:56.1765,1906-12-25 -1906-05-05 12:34:56.1766,1906-12-25 -1906-05-05 12:34:56.1767,1906-12-25 -1906-05-05 12:34:56.1768,1906-12-25 -1906-05-05 12:34:56.1769,1906-12-25 -1906-05-05 12:34:56.177,1906-12-25 -1906-05-05 12:34:56.1771,1906-12-25 -1906-05-05 12:34:56.1772,1906-12-25 -1906-05-05 12:34:56.1773,1906-12-25 -1906-05-05 12:34:56.1774,1906-12-25 -1906-05-05 12:34:56.1775,1906-12-25 -1906-05-05 12:34:56.1776,1906-12-25 -1906-05-05 12:34:56.1777,1906-12-25 -1906-05-05 12:34:56.1778,1906-12-25 -1906-05-05 12:34:56.1779,1906-12-25 -1906-05-05 12:34:56.178,1906-12-25 -1906-05-05 12:34:56.1781,1906-12-25 -1906-05-05 12:34:56.1782,1906-12-25 -1906-05-05 12:34:56.1783,1906-12-25 -1906-05-05 12:34:56.1784,1906-12-25 -1906-05-05 12:34:56.1785,1906-12-25 -1906-05-05 12:34:56.1786,1906-12-25 -1906-05-05 12:34:56.1787,1906-12-25 -1906-05-05 12:34:56.1788,1906-12-25 -1906-05-05 12:34:56.1789,1906-12-25 -1906-05-05 12:34:56.179,1906-12-25 -1906-05-05 12:34:56.1791,1906-12-25 -1906-05-05 12:34:56.1792,1906-12-25 -1906-05-05 12:34:56.1793,1906-12-25 -1906-05-05 12:34:56.1794,1906-12-25 -1906-05-05 12:34:56.1795,1906-12-25 -1906-05-05 12:34:56.1796,1906-12-25 -1906-05-05 12:34:56.1797,1906-12-25 -1906-05-05 12:34:56.1798,1906-12-25 -1906-05-05 12:34:56.1799,1906-12-25 -1906-05-05 12:34:56.18,1906-12-25 -1906-05-05 12:34:56.1801,1906-12-25 -1906-05-05 12:34:56.1802,1906-12-25 -1906-05-05 12:34:56.1803,1906-12-25 -1906-05-05 12:34:56.1804,1906-12-25 -1906-05-05 12:34:56.1805,1906-12-25 -1906-05-05 12:34:56.1806,1906-12-25 -1906-05-05 12:34:56.1807,1906-12-25 -1906-05-05 12:34:56.1808,1906-12-25 -1906-05-05 12:34:56.1809,1906-12-25 -1906-05-05 12:34:56.181,1906-12-25 -1906-05-05 12:34:56.1811,1906-12-25 -1906-05-05 12:34:56.1812,1906-12-25 -1906-05-05 12:34:56.1813,1906-12-25 -1906-05-05 12:34:56.1814,1906-12-25 -1906-05-05 12:34:56.1815,1906-12-25 -1906-05-05 12:34:56.1816,1906-12-25 -1906-05-05 12:34:56.1817,1906-12-25 -1906-05-05 12:34:56.1818,1906-12-25 -1906-05-05 12:34:56.1819,1906-12-25 -1906-05-05 12:34:56.182,1906-12-25 -1906-05-05 12:34:56.1821,1906-12-25 -1906-05-05 12:34:56.1822,1906-12-25 -1906-05-05 12:34:56.1823,1906-12-25 -1906-05-05 12:34:56.1824,1906-12-25 -1906-05-05 12:34:56.1825,1906-12-25 -1906-05-05 12:34:56.1826,1906-12-25 -1906-05-05 12:34:56.1827,1906-12-25 -1906-05-05 12:34:56.1828,1906-12-25 -1906-05-05 12:34:56.1829,1906-12-25 -1906-05-05 12:34:56.183,1906-12-25 -1906-05-05 12:34:56.1831,1906-12-25 -1906-05-05 12:34:56.1832,1906-12-25 -1906-05-05 12:34:56.1833,1906-12-25 -1906-05-05 12:34:56.1834,1906-12-25 -1906-05-05 12:34:56.1835,1906-12-25 -1906-05-05 12:34:56.1836,1906-12-25 -1906-05-05 12:34:56.1837,1906-12-25 -1906-05-05 12:34:56.1838,1906-12-25 -1906-05-05 12:34:56.1839,1906-12-25 -1906-05-05 12:34:56.184,1906-12-25 -1906-05-05 12:34:56.1841,1906-12-25 -1906-05-05 12:34:56.1842,1906-12-25 -1906-05-05 12:34:56.1843,1906-12-25 -1906-05-05 12:34:56.1844,1906-12-25 -1906-05-05 12:34:56.1845,1906-12-25 -1906-05-05 12:34:56.1846,1906-12-25 -1906-05-05 12:34:56.1847,1906-12-25 -1906-05-05 12:34:56.1848,1906-12-25 -1906-05-05 12:34:56.1849,1906-12-25 -1906-05-05 12:34:56.185,1906-12-25 -1906-05-05 12:34:56.1851,1906-12-25 -1906-05-05 12:34:56.1852,1906-12-25 -1906-05-05 12:34:56.1853,1906-12-25 -1906-05-05 12:34:56.1854,1906-12-25 -1906-05-05 12:34:56.1855,1906-12-25 -1906-05-05 12:34:56.1856,1906-12-25 -1906-05-05 12:34:56.1857,1906-12-25 -1906-05-05 12:34:56.1858,1906-12-25 -1906-05-05 12:34:56.1859,1906-12-25 -1906-05-05 12:34:56.186,1906-12-25 -1906-05-05 12:34:56.1861,1906-12-25 -1906-05-05 12:34:56.1862,1906-12-25 -1906-05-05 12:34:56.1863,1906-12-25 -1906-05-05 12:34:56.1864,1906-12-25 -1906-05-05 12:34:56.1865,1906-12-25 -1906-05-05 12:34:56.1866,1906-12-25 -1906-05-05 12:34:56.1867,1906-12-25 -1906-05-05 12:34:56.1868,1906-12-25 -1906-05-05 12:34:56.1869,1906-12-25 -1906-05-05 12:34:56.187,1906-12-25 -1906-05-05 12:34:56.1871,1906-12-25 -1906-05-05 12:34:56.1872,1906-12-25 -1906-05-05 12:34:56.1873,1906-12-25 -1906-05-05 12:34:56.1874,1906-12-25 -1906-05-05 12:34:56.1875,1906-12-25 -1906-05-05 12:34:56.1876,1906-12-25 -1906-05-05 12:34:56.1877,1906-12-25 -1906-05-05 12:34:56.1878,1906-12-25 -1906-05-05 12:34:56.1879,1906-12-25 -1906-05-05 12:34:56.188,1906-12-25 -1906-05-05 12:34:56.1881,1906-12-25 -1906-05-05 12:34:56.1882,1906-12-25 -1906-05-05 12:34:56.1883,1906-12-25 -1906-05-05 12:34:56.1884,1906-12-25 -1906-05-05 12:34:56.1885,1906-12-25 -1906-05-05 12:34:56.1886,1906-12-25 -1906-05-05 12:34:56.1887,1906-12-25 -1906-05-05 12:34:56.1888,1906-12-25 -1906-05-05 12:34:56.1889,1906-12-25 -1906-05-05 12:34:56.189,1906-12-25 -1906-05-05 12:34:56.1891,1906-12-25 -1906-05-05 12:34:56.1892,1906-12-25 -1906-05-05 12:34:56.1893,1906-12-25 -1906-05-05 12:34:56.1894,1906-12-25 -1906-05-05 12:34:56.1895,1906-12-25 -1906-05-05 12:34:56.1896,1906-12-25 -1906-05-05 12:34:56.1897,1906-12-25 -1906-05-05 12:34:56.1898,1906-12-25 -1906-05-05 12:34:56.1899,1906-12-25 -1906-05-05 12:34:56.19,1906-12-25 -1906-05-05 12:34:56.1901,1906-12-25 -1906-05-05 12:34:56.1902,1906-12-25 -1906-05-05 12:34:56.1903,1906-12-25 -1906-05-05 12:34:56.1904,1906-12-25 -1906-05-05 12:34:56.1905,1906-12-25 -1906-05-05 12:34:56.1906,1906-12-25 -1906-05-05 12:34:56.1907,1906-12-25 -1906-05-05 12:34:56.1908,1906-12-25 -1906-05-05 12:34:56.1909,1906-12-25 -1906-05-05 12:34:56.191,1906-12-25 -1906-05-05 12:34:56.1911,1906-12-25 -1906-05-05 12:34:56.1912,1906-12-25 -1906-05-05 12:34:56.1913,1906-12-25 -1906-05-05 12:34:56.1914,1906-12-25 -1906-05-05 12:34:56.1915,1906-12-25 -1906-05-05 12:34:56.1916,1906-12-25 -1906-05-05 12:34:56.1917,1906-12-25 -1906-05-05 12:34:56.1918,1906-12-25 -1906-05-05 12:34:56.1919,1906-12-25 -1906-05-05 12:34:56.192,1906-12-25 -1906-05-05 12:34:56.1921,1906-12-25 -1906-05-05 12:34:56.1922,1906-12-25 -1906-05-05 12:34:56.1923,1906-12-25 -1906-05-05 12:34:56.1924,1906-12-25 -1906-05-05 12:34:56.1925,1906-12-25 -1906-05-05 12:34:56.1926,1906-12-25 -1906-05-05 12:34:56.1927,1906-12-25 -1906-05-05 12:34:56.1928,1906-12-25 -1906-05-05 12:34:56.1929,1906-12-25 -1906-05-05 12:34:56.193,1906-12-25 -1906-05-05 12:34:56.1931,1906-12-25 -1906-05-05 12:34:56.1932,1906-12-25 -1906-05-05 12:34:56.1933,1906-12-25 -1906-05-05 12:34:56.1934,1906-12-25 -1906-05-05 12:34:56.1935,1906-12-25 -1906-05-05 12:34:56.1936,1906-12-25 -1906-05-05 12:34:56.1937,1906-12-25 -1906-05-05 12:34:56.1938,1906-12-25 -1906-05-05 12:34:56.1939,1906-12-25 -1906-05-05 12:34:56.194,1906-12-25 -1906-05-05 12:34:56.1941,1906-12-25 -1906-05-05 12:34:56.1942,1906-12-25 -1906-05-05 12:34:56.1943,1906-12-25 -1906-05-05 12:34:56.1944,1906-12-25 -1906-05-05 12:34:56.1945,1906-12-25 -1906-05-05 12:34:56.1946,1906-12-25 -1906-05-05 12:34:56.1947,1906-12-25 -1906-05-05 12:34:56.1948,1906-12-25 -1906-05-05 12:34:56.1949,1906-12-25 -1906-05-05 12:34:56.195,1906-12-25 -1906-05-05 12:34:56.1951,1906-12-25 -1906-05-05 12:34:56.1952,1906-12-25 -1906-05-05 12:34:56.1953,1906-12-25 -1906-05-05 12:34:56.1954,1906-12-25 -1906-05-05 12:34:56.1955,1906-12-25 -1906-05-05 12:34:56.1956,1906-12-25 -1906-05-05 12:34:56.1957,1906-12-25 -1906-05-05 12:34:56.1958,1906-12-25 -1906-05-05 12:34:56.1959,1906-12-25 -1906-05-05 12:34:56.196,1906-12-25 -1906-05-05 12:34:56.1961,1906-12-25 -1906-05-05 12:34:56.1962,1906-12-25 -1906-05-05 12:34:56.1963,1906-12-25 -1906-05-05 12:34:56.1964,1906-12-25 -1906-05-05 12:34:56.1965,1906-12-25 -1906-05-05 12:34:56.1966,1906-12-25 -1906-05-05 12:34:56.1967,1906-12-25 -1906-05-05 12:34:56.1968,1906-12-25 -1906-05-05 12:34:56.1969,1906-12-25 -1906-05-05 12:34:56.197,1906-12-25 -1906-05-05 12:34:56.1971,1906-12-25 -1906-05-05 12:34:56.1972,1906-12-25 -1906-05-05 12:34:56.1973,1906-12-25 -1906-05-05 12:34:56.1974,1906-12-25 -1906-05-05 12:34:56.1975,1906-12-25 -1906-05-05 12:34:56.1976,1906-12-25 -1906-05-05 12:34:56.1977,1906-12-25 -1906-05-05 12:34:56.1978,1906-12-25 -1906-05-05 12:34:56.1979,1906-12-25 -1906-05-05 12:34:56.198,1906-12-25 -1906-05-05 12:34:56.1981,1906-12-25 -1906-05-05 12:34:56.1982,1906-12-25 -1906-05-05 12:34:56.1983,1906-12-25 -1906-05-05 12:34:56.1984,1906-12-25 -1906-05-05 12:34:56.1985,1906-12-25 -1906-05-05 12:34:56.1986,1906-12-25 -1906-05-05 12:34:56.1987,1906-12-25 -1906-05-05 12:34:56.1988,1906-12-25 -1906-05-05 12:34:56.1989,1906-12-25 -1906-05-05 12:34:56.199,1906-12-25 -1906-05-05 12:34:56.1991,1906-12-25 -1906-05-05 12:34:56.1992,1906-12-25 -1906-05-05 12:34:56.1993,1906-12-25 -1906-05-05 12:34:56.1994,1906-12-25 -1906-05-05 12:34:56.1995,1906-12-25 -1906-05-05 12:34:56.1996,1906-12-25 -1906-05-05 12:34:56.1997,1906-12-25 -1906-05-05 12:34:56.1998,1906-12-25 -1906-05-05 12:34:56.1999,1906-12-25 -1907-05-05 12:34:56.1,1907-12-25 -1907-05-05 12:34:56.1001,1907-12-25 -1907-05-05 12:34:56.1002,1907-12-25 -1907-05-05 12:34:56.1003,1907-12-25 -1907-05-05 12:34:56.1004,1907-12-25 -1907-05-05 12:34:56.1005,1907-12-25 -1907-05-05 12:34:56.1006,1907-12-25 -1907-05-05 12:34:56.1007,1907-12-25 -1907-05-05 12:34:56.1008,1907-12-25 -1907-05-05 12:34:56.1009,1907-12-25 -1907-05-05 12:34:56.101,1907-12-25 -1907-05-05 12:34:56.1011,1907-12-25 -1907-05-05 12:34:56.1012,1907-12-25 -1907-05-05 12:34:56.1013,1907-12-25 -1907-05-05 12:34:56.1014,1907-12-25 -1907-05-05 12:34:56.1015,1907-12-25 -1907-05-05 12:34:56.1016,1907-12-25 -1907-05-05 12:34:56.1017,1907-12-25 -1907-05-05 12:34:56.1018,1907-12-25 -1907-05-05 12:34:56.1019,1907-12-25 -1907-05-05 12:34:56.102,1907-12-25 -1907-05-05 12:34:56.1021,1907-12-25 -1907-05-05 12:34:56.1022,1907-12-25 -1907-05-05 12:34:56.1023,1907-12-25 -1907-05-05 12:34:56.1024,1907-12-25 -1907-05-05 12:34:56.1025,1907-12-25 -1907-05-05 12:34:56.1026,1907-12-25 -1907-05-05 12:34:56.1027,1907-12-25 -1907-05-05 12:34:56.1028,1907-12-25 -1907-05-05 12:34:56.1029,1907-12-25 -1907-05-05 12:34:56.103,1907-12-25 -1907-05-05 12:34:56.1031,1907-12-25 -1907-05-05 12:34:56.1032,1907-12-25 -1907-05-05 12:34:56.1033,1907-12-25 -1907-05-05 12:34:56.1034,1907-12-25 -1907-05-05 12:34:56.1035,1907-12-25 -1907-05-05 12:34:56.1036,1907-12-25 -1907-05-05 12:34:56.1037,1907-12-25 -1907-05-05 12:34:56.1038,1907-12-25 -1907-05-05 12:34:56.1039,1907-12-25 -1907-05-05 12:34:56.104,1907-12-25 -1907-05-05 12:34:56.1041,1907-12-25 -1907-05-05 12:34:56.1042,1907-12-25 -1907-05-05 12:34:56.1043,1907-12-25 -1907-05-05 12:34:56.1044,1907-12-25 -1907-05-05 12:34:56.1045,1907-12-25 -1907-05-05 12:34:56.1046,1907-12-25 -1907-05-05 12:34:56.1047,1907-12-25 -1907-05-05 12:34:56.1048,1907-12-25 -1907-05-05 12:34:56.1049,1907-12-25 -1907-05-05 12:34:56.105,1907-12-25 -1907-05-05 12:34:56.1051,1907-12-25 -1907-05-05 12:34:56.1052,1907-12-25 -1907-05-05 12:34:56.1053,1907-12-25 -1907-05-05 12:34:56.1054,1907-12-25 -1907-05-05 12:34:56.1055,1907-12-25 -1907-05-05 12:34:56.1056,1907-12-25 -1907-05-05 12:34:56.1057,1907-12-25 -1907-05-05 12:34:56.1058,1907-12-25 -1907-05-05 12:34:56.1059,1907-12-25 -1907-05-05 12:34:56.106,1907-12-25 -1907-05-05 12:34:56.1061,1907-12-25 -1907-05-05 12:34:56.1062,1907-12-25 -1907-05-05 12:34:56.1063,1907-12-25 -1907-05-05 12:34:56.1064,1907-12-25 -1907-05-05 12:34:56.1065,1907-12-25 -1907-05-05 12:34:56.1066,1907-12-25 -1907-05-05 12:34:56.1067,1907-12-25 -1907-05-05 12:34:56.1068,1907-12-25 -1907-05-05 12:34:56.1069,1907-12-25 -1907-05-05 12:34:56.107,1907-12-25 -1907-05-05 12:34:56.1071,1907-12-25 -1907-05-05 12:34:56.1072,1907-12-25 -1907-05-05 12:34:56.1073,1907-12-25 -1907-05-05 12:34:56.1074,1907-12-25 -1907-05-05 12:34:56.1075,1907-12-25 -1907-05-05 12:34:56.1076,1907-12-25 -1907-05-05 12:34:56.1077,1907-12-25 -1907-05-05 12:34:56.1078,1907-12-25 -1907-05-05 12:34:56.1079,1907-12-25 -1907-05-05 12:34:56.108,1907-12-25 -1907-05-05 12:34:56.1081,1907-12-25 -1907-05-05 12:34:56.1082,1907-12-25 -1907-05-05 12:34:56.1083,1907-12-25 -1907-05-05 12:34:56.1084,1907-12-25 -1907-05-05 12:34:56.1085,1907-12-25 -1907-05-05 12:34:56.1086,1907-12-25 -1907-05-05 12:34:56.1087,1907-12-25 -1907-05-05 12:34:56.1088,1907-12-25 -1907-05-05 12:34:56.1089,1907-12-25 -1907-05-05 12:34:56.109,1907-12-25 -1907-05-05 12:34:56.1091,1907-12-25 -1907-05-05 12:34:56.1092,1907-12-25 -1907-05-05 12:34:56.1093,1907-12-25 -1907-05-05 12:34:56.1094,1907-12-25 -1907-05-05 12:34:56.1095,1907-12-25 -1907-05-05 12:34:56.1096,1907-12-25 -1907-05-05 12:34:56.1097,1907-12-25 -1907-05-05 12:34:56.1098,1907-12-25 -1907-05-05 12:34:56.1099,1907-12-25 -1907-05-05 12:34:56.11,1907-12-25 -1907-05-05 12:34:56.1101,1907-12-25 -1907-05-05 12:34:56.1102,1907-12-25 -1907-05-05 12:34:56.1103,1907-12-25 -1907-05-05 12:34:56.1104,1907-12-25 -1907-05-05 12:34:56.1105,1907-12-25 -1907-05-05 12:34:56.1106,1907-12-25 -1907-05-05 12:34:56.1107,1907-12-25 -1907-05-05 12:34:56.1108,1907-12-25 -1907-05-05 12:34:56.1109,1907-12-25 -1907-05-05 12:34:56.111,1907-12-25 -1907-05-05 12:34:56.1111,1907-12-25 -1907-05-05 12:34:56.1112,1907-12-25 -1907-05-05 12:34:56.1113,1907-12-25 -1907-05-05 12:34:56.1114,1907-12-25 -1907-05-05 12:34:56.1115,1907-12-25 -1907-05-05 12:34:56.1116,1907-12-25 -1907-05-05 12:34:56.1117,1907-12-25 -1907-05-05 12:34:56.1118,1907-12-25 -1907-05-05 12:34:56.1119,1907-12-25 -1907-05-05 12:34:56.112,1907-12-25 -1907-05-05 12:34:56.1121,1907-12-25 -1907-05-05 12:34:56.1122,1907-12-25 -1907-05-05 12:34:56.1123,1907-12-25 -1907-05-05 12:34:56.1124,1907-12-25 -1907-05-05 12:34:56.1125,1907-12-25 -1907-05-05 12:34:56.1126,1907-12-25 -1907-05-05 12:34:56.1127,1907-12-25 -1907-05-05 12:34:56.1128,1907-12-25 -1907-05-05 12:34:56.1129,1907-12-25 -1907-05-05 12:34:56.113,1907-12-25 -1907-05-05 12:34:56.1131,1907-12-25 -1907-05-05 12:34:56.1132,1907-12-25 -1907-05-05 12:34:56.1133,1907-12-25 -1907-05-05 12:34:56.1134,1907-12-25 -1907-05-05 12:34:56.1135,1907-12-25 -1907-05-05 12:34:56.1136,1907-12-25 -1907-05-05 12:34:56.1137,1907-12-25 -1907-05-05 12:34:56.1138,1907-12-25 -1907-05-05 12:34:56.1139,1907-12-25 -1907-05-05 12:34:56.114,1907-12-25 -1907-05-05 12:34:56.1141,1907-12-25 -1907-05-05 12:34:56.1142,1907-12-25 -1907-05-05 12:34:56.1143,1907-12-25 -1907-05-05 12:34:56.1144,1907-12-25 -1907-05-05 12:34:56.1145,1907-12-25 -1907-05-05 12:34:56.1146,1907-12-25 -1907-05-05 12:34:56.1147,1907-12-25 -1907-05-05 12:34:56.1148,1907-12-25 -1907-05-05 12:34:56.1149,1907-12-25 -1907-05-05 12:34:56.115,1907-12-25 -1907-05-05 12:34:56.1151,1907-12-25 -1907-05-05 12:34:56.1152,1907-12-25 -1907-05-05 12:34:56.1153,1907-12-25 -1907-05-05 12:34:56.1154,1907-12-25 -1907-05-05 12:34:56.1155,1907-12-25 -1907-05-05 12:34:56.1156,1907-12-25 -1907-05-05 12:34:56.1157,1907-12-25 -1907-05-05 12:34:56.1158,1907-12-25 -1907-05-05 12:34:56.1159,1907-12-25 -1907-05-05 12:34:56.116,1907-12-25 -1907-05-05 12:34:56.1161,1907-12-25 -1907-05-05 12:34:56.1162,1907-12-25 -1907-05-05 12:34:56.1163,1907-12-25 -1907-05-05 12:34:56.1164,1907-12-25 -1907-05-05 12:34:56.1165,1907-12-25 -1907-05-05 12:34:56.1166,1907-12-25 -1907-05-05 12:34:56.1167,1907-12-25 -1907-05-05 12:34:56.1168,1907-12-25 -1907-05-05 12:34:56.1169,1907-12-25 -1907-05-05 12:34:56.117,1907-12-25 -1907-05-05 12:34:56.1171,1907-12-25 -1907-05-05 12:34:56.1172,1907-12-25 -1907-05-05 12:34:56.1173,1907-12-25 -1907-05-05 12:34:56.1174,1907-12-25 -1907-05-05 12:34:56.1175,1907-12-25 -1907-05-05 12:34:56.1176,1907-12-25 -1907-05-05 12:34:56.1177,1907-12-25 -1907-05-05 12:34:56.1178,1907-12-25 -1907-05-05 12:34:56.1179,1907-12-25 -1907-05-05 12:34:56.118,1907-12-25 -1907-05-05 12:34:56.1181,1907-12-25 -1907-05-05 12:34:56.1182,1907-12-25 -1907-05-05 12:34:56.1183,1907-12-25 -1907-05-05 12:34:56.1184,1907-12-25 -1907-05-05 12:34:56.1185,1907-12-25 -1907-05-05 12:34:56.1186,1907-12-25 -1907-05-05 12:34:56.1187,1907-12-25 -1907-05-05 12:34:56.1188,1907-12-25 -1907-05-05 12:34:56.1189,1907-12-25 -1907-05-05 12:34:56.119,1907-12-25 -1907-05-05 12:34:56.1191,1907-12-25 -1907-05-05 12:34:56.1192,1907-12-25 -1907-05-05 12:34:56.1193,1907-12-25 -1907-05-05 12:34:56.1194,1907-12-25 -1907-05-05 12:34:56.1195,1907-12-25 -1907-05-05 12:34:56.1196,1907-12-25 -1907-05-05 12:34:56.1197,1907-12-25 -1907-05-05 12:34:56.1198,1907-12-25 -1907-05-05 12:34:56.1199,1907-12-25 -1907-05-05 12:34:56.12,1907-12-25 -1907-05-05 12:34:56.1201,1907-12-25 -1907-05-05 12:34:56.1202,1907-12-25 -1907-05-05 12:34:56.1203,1907-12-25 -1907-05-05 12:34:56.1204,1907-12-25 -1907-05-05 12:34:56.1205,1907-12-25 -1907-05-05 12:34:56.1206,1907-12-25 -1907-05-05 12:34:56.1207,1907-12-25 -1907-05-05 12:34:56.1208,1907-12-25 -1907-05-05 12:34:56.1209,1907-12-25 -1907-05-05 12:34:56.121,1907-12-25 -1907-05-05 12:34:56.1211,1907-12-25 -1907-05-05 12:34:56.1212,1907-12-25 -1907-05-05 12:34:56.1213,1907-12-25 -1907-05-05 12:34:56.1214,1907-12-25 -1907-05-05 12:34:56.1215,1907-12-25 -1907-05-05 12:34:56.1216,1907-12-25 -1907-05-05 12:34:56.1217,1907-12-25 -1907-05-05 12:34:56.1218,1907-12-25 -1907-05-05 12:34:56.1219,1907-12-25 -1907-05-05 12:34:56.122,1907-12-25 -1907-05-05 12:34:56.1221,1907-12-25 -1907-05-05 12:34:56.1222,1907-12-25 -1907-05-05 12:34:56.1223,1907-12-25 -1907-05-05 12:34:56.1224,1907-12-25 -1907-05-05 12:34:56.1225,1907-12-25 -1907-05-05 12:34:56.1226,1907-12-25 -1907-05-05 12:34:56.1227,1907-12-25 -1907-05-05 12:34:56.1228,1907-12-25 -1907-05-05 12:34:56.1229,1907-12-25 -1907-05-05 12:34:56.123,1907-12-25 -1907-05-05 12:34:56.1231,1907-12-25 -1907-05-05 12:34:56.1232,1907-12-25 -1907-05-05 12:34:56.1233,1907-12-25 -1907-05-05 12:34:56.1234,1907-12-25 -1907-05-05 12:34:56.1235,1907-12-25 -1907-05-05 12:34:56.1236,1907-12-25 -1907-05-05 12:34:56.1237,1907-12-25 -1907-05-05 12:34:56.1238,1907-12-25 -1907-05-05 12:34:56.1239,1907-12-25 -1907-05-05 12:34:56.124,1907-12-25 -1907-05-05 12:34:56.1241,1907-12-25 -1907-05-05 12:34:56.1242,1907-12-25 -1907-05-05 12:34:56.1243,1907-12-25 -1907-05-05 12:34:56.1244,1907-12-25 -1907-05-05 12:34:56.1245,1907-12-25 -1907-05-05 12:34:56.1246,1907-12-25 -1907-05-05 12:34:56.1247,1907-12-25 -1907-05-05 12:34:56.1248,1907-12-25 -1907-05-05 12:34:56.1249,1907-12-25 -1907-05-05 12:34:56.125,1907-12-25 -1907-05-05 12:34:56.1251,1907-12-25 -1907-05-05 12:34:56.1252,1907-12-25 -1907-05-05 12:34:56.1253,1907-12-25 -1907-05-05 12:34:56.1254,1907-12-25 -1907-05-05 12:34:56.1255,1907-12-25 -1907-05-05 12:34:56.1256,1907-12-25 -1907-05-05 12:34:56.1257,1907-12-25 -1907-05-05 12:34:56.1258,1907-12-25 -1907-05-05 12:34:56.1259,1907-12-25 -1907-05-05 12:34:56.126,1907-12-25 -1907-05-05 12:34:56.1261,1907-12-25 -1907-05-05 12:34:56.1262,1907-12-25 -1907-05-05 12:34:56.1263,1907-12-25 -1907-05-05 12:34:56.1264,1907-12-25 -1907-05-05 12:34:56.1265,1907-12-25 -1907-05-05 12:34:56.1266,1907-12-25 -1907-05-05 12:34:56.1267,1907-12-25 -1907-05-05 12:34:56.1268,1907-12-25 -1907-05-05 12:34:56.1269,1907-12-25 -1907-05-05 12:34:56.127,1907-12-25 -1907-05-05 12:34:56.1271,1907-12-25 -1907-05-05 12:34:56.1272,1907-12-25 -1907-05-05 12:34:56.1273,1907-12-25 -1907-05-05 12:34:56.1274,1907-12-25 -1907-05-05 12:34:56.1275,1907-12-25 -1907-05-05 12:34:56.1276,1907-12-25 -1907-05-05 12:34:56.1277,1907-12-25 -1907-05-05 12:34:56.1278,1907-12-25 -1907-05-05 12:34:56.1279,1907-12-25 -1907-05-05 12:34:56.128,1907-12-25 -1907-05-05 12:34:56.1281,1907-12-25 -1907-05-05 12:34:56.1282,1907-12-25 -1907-05-05 12:34:56.1283,1907-12-25 -1907-05-05 12:34:56.1284,1907-12-25 -1907-05-05 12:34:56.1285,1907-12-25 -1907-05-05 12:34:56.1286,1907-12-25 -1907-05-05 12:34:56.1287,1907-12-25 -1907-05-05 12:34:56.1288,1907-12-25 -1907-05-05 12:34:56.1289,1907-12-25 -1907-05-05 12:34:56.129,1907-12-25 -1907-05-05 12:34:56.1291,1907-12-25 -1907-05-05 12:34:56.1292,1907-12-25 -1907-05-05 12:34:56.1293,1907-12-25 -1907-05-05 12:34:56.1294,1907-12-25 -1907-05-05 12:34:56.1295,1907-12-25 -1907-05-05 12:34:56.1296,1907-12-25 -1907-05-05 12:34:56.1297,1907-12-25 -1907-05-05 12:34:56.1298,1907-12-25 -1907-05-05 12:34:56.1299,1907-12-25 -1907-05-05 12:34:56.13,1907-12-25 -1907-05-05 12:34:56.1301,1907-12-25 -1907-05-05 12:34:56.1302,1907-12-25 -1907-05-05 12:34:56.1303,1907-12-25 -1907-05-05 12:34:56.1304,1907-12-25 -1907-05-05 12:34:56.1305,1907-12-25 -1907-05-05 12:34:56.1306,1907-12-25 -1907-05-05 12:34:56.1307,1907-12-25 -1907-05-05 12:34:56.1308,1907-12-25 -1907-05-05 12:34:56.1309,1907-12-25 -1907-05-05 12:34:56.131,1907-12-25 -1907-05-05 12:34:56.1311,1907-12-25 -1907-05-05 12:34:56.1312,1907-12-25 -1907-05-05 12:34:56.1313,1907-12-25 -1907-05-05 12:34:56.1314,1907-12-25 -1907-05-05 12:34:56.1315,1907-12-25 -1907-05-05 12:34:56.1316,1907-12-25 -1907-05-05 12:34:56.1317,1907-12-25 -1907-05-05 12:34:56.1318,1907-12-25 -1907-05-05 12:34:56.1319,1907-12-25 -1907-05-05 12:34:56.132,1907-12-25 -1907-05-05 12:34:56.1321,1907-12-25 -1907-05-05 12:34:56.1322,1907-12-25 -1907-05-05 12:34:56.1323,1907-12-25 -1907-05-05 12:34:56.1324,1907-12-25 -1907-05-05 12:34:56.1325,1907-12-25 -1907-05-05 12:34:56.1326,1907-12-25 -1907-05-05 12:34:56.1327,1907-12-25 -1907-05-05 12:34:56.1328,1907-12-25 -1907-05-05 12:34:56.1329,1907-12-25 -1907-05-05 12:34:56.133,1907-12-25 -1907-05-05 12:34:56.1331,1907-12-25 -1907-05-05 12:34:56.1332,1907-12-25 -1907-05-05 12:34:56.1333,1907-12-25 -1907-05-05 12:34:56.1334,1907-12-25 -1907-05-05 12:34:56.1335,1907-12-25 -1907-05-05 12:34:56.1336,1907-12-25 -1907-05-05 12:34:56.1337,1907-12-25 -1907-05-05 12:34:56.1338,1907-12-25 -1907-05-05 12:34:56.1339,1907-12-25 -1907-05-05 12:34:56.134,1907-12-25 -1907-05-05 12:34:56.1341,1907-12-25 -1907-05-05 12:34:56.1342,1907-12-25 -1907-05-05 12:34:56.1343,1907-12-25 -1907-05-05 12:34:56.1344,1907-12-25 -1907-05-05 12:34:56.1345,1907-12-25 -1907-05-05 12:34:56.1346,1907-12-25 -1907-05-05 12:34:56.1347,1907-12-25 -1907-05-05 12:34:56.1348,1907-12-25 -1907-05-05 12:34:56.1349,1907-12-25 -1907-05-05 12:34:56.135,1907-12-25 -1907-05-05 12:34:56.1351,1907-12-25 -1907-05-05 12:34:56.1352,1907-12-25 -1907-05-05 12:34:56.1353,1907-12-25 -1907-05-05 12:34:56.1354,1907-12-25 -1907-05-05 12:34:56.1355,1907-12-25 -1907-05-05 12:34:56.1356,1907-12-25 -1907-05-05 12:34:56.1357,1907-12-25 -1907-05-05 12:34:56.1358,1907-12-25 -1907-05-05 12:34:56.1359,1907-12-25 -1907-05-05 12:34:56.136,1907-12-25 -1907-05-05 12:34:56.1361,1907-12-25 -1907-05-05 12:34:56.1362,1907-12-25 -1907-05-05 12:34:56.1363,1907-12-25 -1907-05-05 12:34:56.1364,1907-12-25 -1907-05-05 12:34:56.1365,1907-12-25 -1907-05-05 12:34:56.1366,1907-12-25 -1907-05-05 12:34:56.1367,1907-12-25 -1907-05-05 12:34:56.1368,1907-12-25 -1907-05-05 12:34:56.1369,1907-12-25 -1907-05-05 12:34:56.137,1907-12-25 -1907-05-05 12:34:56.1371,1907-12-25 -1907-05-05 12:34:56.1372,1907-12-25 -1907-05-05 12:34:56.1373,1907-12-25 -1907-05-05 12:34:56.1374,1907-12-25 -1907-05-05 12:34:56.1375,1907-12-25 -1907-05-05 12:34:56.1376,1907-12-25 -1907-05-05 12:34:56.1377,1907-12-25 -1907-05-05 12:34:56.1378,1907-12-25 -1907-05-05 12:34:56.1379,1907-12-25 -1907-05-05 12:34:56.138,1907-12-25 -1907-05-05 12:34:56.1381,1907-12-25 -1907-05-05 12:34:56.1382,1907-12-25 -1907-05-05 12:34:56.1383,1907-12-25 -1907-05-05 12:34:56.1384,1907-12-25 -1907-05-05 12:34:56.1385,1907-12-25 -1907-05-05 12:34:56.1386,1907-12-25 -1907-05-05 12:34:56.1387,1907-12-25 -1907-05-05 12:34:56.1388,1907-12-25 -1907-05-05 12:34:56.1389,1907-12-25 -1907-05-05 12:34:56.139,1907-12-25 -1907-05-05 12:34:56.1391,1907-12-25 -1907-05-05 12:34:56.1392,1907-12-25 -1907-05-05 12:34:56.1393,1907-12-25 -1907-05-05 12:34:56.1394,1907-12-25 -1907-05-05 12:34:56.1395,1907-12-25 -1907-05-05 12:34:56.1396,1907-12-25 -1907-05-05 12:34:56.1397,1907-12-25 -1907-05-05 12:34:56.1398,1907-12-25 -1907-05-05 12:34:56.1399,1907-12-25 -1907-05-05 12:34:56.14,1907-12-25 -1907-05-05 12:34:56.1401,1907-12-25 -1907-05-05 12:34:56.1402,1907-12-25 -1907-05-05 12:34:56.1403,1907-12-25 -1907-05-05 12:34:56.1404,1907-12-25 -1907-05-05 12:34:56.1405,1907-12-25 -1907-05-05 12:34:56.1406,1907-12-25 -1907-05-05 12:34:56.1407,1907-12-25 -1907-05-05 12:34:56.1408,1907-12-25 -1907-05-05 12:34:56.1409,1907-12-25 -1907-05-05 12:34:56.141,1907-12-25 -1907-05-05 12:34:56.1411,1907-12-25 -1907-05-05 12:34:56.1412,1907-12-25 -1907-05-05 12:34:56.1413,1907-12-25 -1907-05-05 12:34:56.1414,1907-12-25 -1907-05-05 12:34:56.1415,1907-12-25 -1907-05-05 12:34:56.1416,1907-12-25 -1907-05-05 12:34:56.1417,1907-12-25 -1907-05-05 12:34:56.1418,1907-12-25 -1907-05-05 12:34:56.1419,1907-12-25 -1907-05-05 12:34:56.142,1907-12-25 -1907-05-05 12:34:56.1421,1907-12-25 -1907-05-05 12:34:56.1422,1907-12-25 -1907-05-05 12:34:56.1423,1907-12-25 -1907-05-05 12:34:56.1424,1907-12-25 -1907-05-05 12:34:56.1425,1907-12-25 -1907-05-05 12:34:56.1426,1907-12-25 -1907-05-05 12:34:56.1427,1907-12-25 -1907-05-05 12:34:56.1428,1907-12-25 -1907-05-05 12:34:56.1429,1907-12-25 -1907-05-05 12:34:56.143,1907-12-25 -1907-05-05 12:34:56.1431,1907-12-25 -1907-05-05 12:34:56.1432,1907-12-25 -1907-05-05 12:34:56.1433,1907-12-25 -1907-05-05 12:34:56.1434,1907-12-25 -1907-05-05 12:34:56.1435,1907-12-25 -1907-05-05 12:34:56.1436,1907-12-25 -1907-05-05 12:34:56.1437,1907-12-25 -1907-05-05 12:34:56.1438,1907-12-25 -1907-05-05 12:34:56.1439,1907-12-25 -1907-05-05 12:34:56.144,1907-12-25 -1907-05-05 12:34:56.1441,1907-12-25 -1907-05-05 12:34:56.1442,1907-12-25 -1907-05-05 12:34:56.1443,1907-12-25 -1907-05-05 12:34:56.1444,1907-12-25 -1907-05-05 12:34:56.1445,1907-12-25 -1907-05-05 12:34:56.1446,1907-12-25 -1907-05-05 12:34:56.1447,1907-12-25 -1907-05-05 12:34:56.1448,1907-12-25 -1907-05-05 12:34:56.1449,1907-12-25 -1907-05-05 12:34:56.145,1907-12-25 -1907-05-05 12:34:56.1451,1907-12-25 -1907-05-05 12:34:56.1452,1907-12-25 -1907-05-05 12:34:56.1453,1907-12-25 -1907-05-05 12:34:56.1454,1907-12-25 -1907-05-05 12:34:56.1455,1907-12-25 -1907-05-05 12:34:56.1456,1907-12-25 -1907-05-05 12:34:56.1457,1907-12-25 -1907-05-05 12:34:56.1458,1907-12-25 -1907-05-05 12:34:56.1459,1907-12-25 -1907-05-05 12:34:56.146,1907-12-25 -1907-05-05 12:34:56.1461,1907-12-25 -1907-05-05 12:34:56.1462,1907-12-25 -1907-05-05 12:34:56.1463,1907-12-25 -1907-05-05 12:34:56.1464,1907-12-25 -1907-05-05 12:34:56.1465,1907-12-25 -1907-05-05 12:34:56.1466,1907-12-25 -1907-05-05 12:34:56.1467,1907-12-25 -1907-05-05 12:34:56.1468,1907-12-25 -1907-05-05 12:34:56.1469,1907-12-25 -1907-05-05 12:34:56.147,1907-12-25 -1907-05-05 12:34:56.1471,1907-12-25 -1907-05-05 12:34:56.1472,1907-12-25 -1907-05-05 12:34:56.1473,1907-12-25 -1907-05-05 12:34:56.1474,1907-12-25 -1907-05-05 12:34:56.1475,1907-12-25 -1907-05-05 12:34:56.1476,1907-12-25 -1907-05-05 12:34:56.1477,1907-12-25 -1907-05-05 12:34:56.1478,1907-12-25 -1907-05-05 12:34:56.1479,1907-12-25 -1907-05-05 12:34:56.148,1907-12-25 -1907-05-05 12:34:56.1481,1907-12-25 -1907-05-05 12:34:56.1482,1907-12-25 -1907-05-05 12:34:56.1483,1907-12-25 -1907-05-05 12:34:56.1484,1907-12-25 -1907-05-05 12:34:56.1485,1907-12-25 -1907-05-05 12:34:56.1486,1907-12-25 -1907-05-05 12:34:56.1487,1907-12-25 -1907-05-05 12:34:56.1488,1907-12-25 -1907-05-05 12:34:56.1489,1907-12-25 -1907-05-05 12:34:56.149,1907-12-25 -1907-05-05 12:34:56.1491,1907-12-25 -1907-05-05 12:34:56.1492,1907-12-25 -1907-05-05 12:34:56.1493,1907-12-25 -1907-05-05 12:34:56.1494,1907-12-25 -1907-05-05 12:34:56.1495,1907-12-25 -1907-05-05 12:34:56.1496,1907-12-25 -1907-05-05 12:34:56.1497,1907-12-25 -1907-05-05 12:34:56.1498,1907-12-25 -1907-05-05 12:34:56.1499,1907-12-25 -1907-05-05 12:34:56.15,1907-12-25 -1907-05-05 12:34:56.1501,1907-12-25 -1907-05-05 12:34:56.1502,1907-12-25 -1907-05-05 12:34:56.1503,1907-12-25 -1907-05-05 12:34:56.1504,1907-12-25 -1907-05-05 12:34:56.1505,1907-12-25 -1907-05-05 12:34:56.1506,1907-12-25 -1907-05-05 12:34:56.1507,1907-12-25 -1907-05-05 12:34:56.1508,1907-12-25 -1907-05-05 12:34:56.1509,1907-12-25 -1907-05-05 12:34:56.151,1907-12-25 -1907-05-05 12:34:56.1511,1907-12-25 -1907-05-05 12:34:56.1512,1907-12-25 -1907-05-05 12:34:56.1513,1907-12-25 -1907-05-05 12:34:56.1514,1907-12-25 -1907-05-05 12:34:56.1515,1907-12-25 -1907-05-05 12:34:56.1516,1907-12-25 -1907-05-05 12:34:56.1517,1907-12-25 -1907-05-05 12:34:56.1518,1907-12-25 -1907-05-05 12:34:56.1519,1907-12-25 -1907-05-05 12:34:56.152,1907-12-25 -1907-05-05 12:34:56.1521,1907-12-25 -1907-05-05 12:34:56.1522,1907-12-25 -1907-05-05 12:34:56.1523,1907-12-25 -1907-05-05 12:34:56.1524,1907-12-25 -1907-05-05 12:34:56.1525,1907-12-25 -1907-05-05 12:34:56.1526,1907-12-25 -1907-05-05 12:34:56.1527,1907-12-25 -1907-05-05 12:34:56.1528,1907-12-25 -1907-05-05 12:34:56.1529,1907-12-25 -1907-05-05 12:34:56.153,1907-12-25 -1907-05-05 12:34:56.1531,1907-12-25 -1907-05-05 12:34:56.1532,1907-12-25 -1907-05-05 12:34:56.1533,1907-12-25 -1907-05-05 12:34:56.1534,1907-12-25 -1907-05-05 12:34:56.1535,1907-12-25 -1907-05-05 12:34:56.1536,1907-12-25 -1907-05-05 12:34:56.1537,1907-12-25 -1907-05-05 12:34:56.1538,1907-12-25 -1907-05-05 12:34:56.1539,1907-12-25 -1907-05-05 12:34:56.154,1907-12-25 -1907-05-05 12:34:56.1541,1907-12-25 -1907-05-05 12:34:56.1542,1907-12-25 -1907-05-05 12:34:56.1543,1907-12-25 -1907-05-05 12:34:56.1544,1907-12-25 -1907-05-05 12:34:56.1545,1907-12-25 -1907-05-05 12:34:56.1546,1907-12-25 -1907-05-05 12:34:56.1547,1907-12-25 -1907-05-05 12:34:56.1548,1907-12-25 -1907-05-05 12:34:56.1549,1907-12-25 -1907-05-05 12:34:56.155,1907-12-25 -1907-05-05 12:34:56.1551,1907-12-25 -1907-05-05 12:34:56.1552,1907-12-25 -1907-05-05 12:34:56.1553,1907-12-25 -1907-05-05 12:34:56.1554,1907-12-25 -1907-05-05 12:34:56.1555,1907-12-25 -1907-05-05 12:34:56.1556,1907-12-25 -1907-05-05 12:34:56.1557,1907-12-25 -1907-05-05 12:34:56.1558,1907-12-25 -1907-05-05 12:34:56.1559,1907-12-25 -1907-05-05 12:34:56.156,1907-12-25 -1907-05-05 12:34:56.1561,1907-12-25 -1907-05-05 12:34:56.1562,1907-12-25 -1907-05-05 12:34:56.1563,1907-12-25 -1907-05-05 12:34:56.1564,1907-12-25 -1907-05-05 12:34:56.1565,1907-12-25 -1907-05-05 12:34:56.1566,1907-12-25 -1907-05-05 12:34:56.1567,1907-12-25 -1907-05-05 12:34:56.1568,1907-12-25 -1907-05-05 12:34:56.1569,1907-12-25 -1907-05-05 12:34:56.157,1907-12-25 -1907-05-05 12:34:56.1571,1907-12-25 -1907-05-05 12:34:56.1572,1907-12-25 -1907-05-05 12:34:56.1573,1907-12-25 -1907-05-05 12:34:56.1574,1907-12-25 -1907-05-05 12:34:56.1575,1907-12-25 -1907-05-05 12:34:56.1576,1907-12-25 -1907-05-05 12:34:56.1577,1907-12-25 -1907-05-05 12:34:56.1578,1907-12-25 -1907-05-05 12:34:56.1579,1907-12-25 -1907-05-05 12:34:56.158,1907-12-25 -1907-05-05 12:34:56.1581,1907-12-25 -1907-05-05 12:34:56.1582,1907-12-25 -1907-05-05 12:34:56.1583,1907-12-25 -1907-05-05 12:34:56.1584,1907-12-25 -1907-05-05 12:34:56.1585,1907-12-25 -1907-05-05 12:34:56.1586,1907-12-25 -1907-05-05 12:34:56.1587,1907-12-25 -1907-05-05 12:34:56.1588,1907-12-25 -1907-05-05 12:34:56.1589,1907-12-25 -1907-05-05 12:34:56.159,1907-12-25 -1907-05-05 12:34:56.1591,1907-12-25 -1907-05-05 12:34:56.1592,1907-12-25 -1907-05-05 12:34:56.1593,1907-12-25 -1907-05-05 12:34:56.1594,1907-12-25 -1907-05-05 12:34:56.1595,1907-12-25 -1907-05-05 12:34:56.1596,1907-12-25 -1907-05-05 12:34:56.1597,1907-12-25 -1907-05-05 12:34:56.1598,1907-12-25 -1907-05-05 12:34:56.1599,1907-12-25 -1907-05-05 12:34:56.16,1907-12-25 -1907-05-05 12:34:56.1601,1907-12-25 -1907-05-05 12:34:56.1602,1907-12-25 -1907-05-05 12:34:56.1603,1907-12-25 -1907-05-05 12:34:56.1604,1907-12-25 -1907-05-05 12:34:56.1605,1907-12-25 -1907-05-05 12:34:56.1606,1907-12-25 -1907-05-05 12:34:56.1607,1907-12-25 -1907-05-05 12:34:56.1608,1907-12-25 -1907-05-05 12:34:56.1609,1907-12-25 -1907-05-05 12:34:56.161,1907-12-25 -1907-05-05 12:34:56.1611,1907-12-25 -1907-05-05 12:34:56.1612,1907-12-25 -1907-05-05 12:34:56.1613,1907-12-25 -1907-05-05 12:34:56.1614,1907-12-25 -1907-05-05 12:34:56.1615,1907-12-25 -1907-05-05 12:34:56.1616,1907-12-25 -1907-05-05 12:34:56.1617,1907-12-25 -1907-05-05 12:34:56.1618,1907-12-25 -1907-05-05 12:34:56.1619,1907-12-25 -1907-05-05 12:34:56.162,1907-12-25 -1907-05-05 12:34:56.1621,1907-12-25 -1907-05-05 12:34:56.1622,1907-12-25 -1907-05-05 12:34:56.1623,1907-12-25 -1907-05-05 12:34:56.1624,1907-12-25 -1907-05-05 12:34:56.1625,1907-12-25 -1907-05-05 12:34:56.1626,1907-12-25 -1907-05-05 12:34:56.1627,1907-12-25 -1907-05-05 12:34:56.1628,1907-12-25 -1907-05-05 12:34:56.1629,1907-12-25 -1907-05-05 12:34:56.163,1907-12-25 -1907-05-05 12:34:56.1631,1907-12-25 -1907-05-05 12:34:56.1632,1907-12-25 -1907-05-05 12:34:56.1633,1907-12-25 -1907-05-05 12:34:56.1634,1907-12-25 -1907-05-05 12:34:56.1635,1907-12-25 -1907-05-05 12:34:56.1636,1907-12-25 -1907-05-05 12:34:56.1637,1907-12-25 -1907-05-05 12:34:56.1638,1907-12-25 -1907-05-05 12:34:56.1639,1907-12-25 -1907-05-05 12:34:56.164,1907-12-25 -1907-05-05 12:34:56.1641,1907-12-25 -1907-05-05 12:34:56.1642,1907-12-25 -1907-05-05 12:34:56.1643,1907-12-25 -1907-05-05 12:34:56.1644,1907-12-25 -1907-05-05 12:34:56.1645,1907-12-25 -1907-05-05 12:34:56.1646,1907-12-25 -1907-05-05 12:34:56.1647,1907-12-25 -1907-05-05 12:34:56.1648,1907-12-25 -1907-05-05 12:34:56.1649,1907-12-25 -1907-05-05 12:34:56.165,1907-12-25 -1907-05-05 12:34:56.1651,1907-12-25 -1907-05-05 12:34:56.1652,1907-12-25 -1907-05-05 12:34:56.1653,1907-12-25 -1907-05-05 12:34:56.1654,1907-12-25 -1907-05-05 12:34:56.1655,1907-12-25 -1907-05-05 12:34:56.1656,1907-12-25 -1907-05-05 12:34:56.1657,1907-12-25 -1907-05-05 12:34:56.1658,1907-12-25 -1907-05-05 12:34:56.1659,1907-12-25 -1907-05-05 12:34:56.166,1907-12-25 -1907-05-05 12:34:56.1661,1907-12-25 -1907-05-05 12:34:56.1662,1907-12-25 -1907-05-05 12:34:56.1663,1907-12-25 -1907-05-05 12:34:56.1664,1907-12-25 -1907-05-05 12:34:56.1665,1907-12-25 -1907-05-05 12:34:56.1666,1907-12-25 -1907-05-05 12:34:56.1667,1907-12-25 -1907-05-05 12:34:56.1668,1907-12-25 -1907-05-05 12:34:56.1669,1907-12-25 -1907-05-05 12:34:56.167,1907-12-25 -1907-05-05 12:34:56.1671,1907-12-25 -1907-05-05 12:34:56.1672,1907-12-25 -1907-05-05 12:34:56.1673,1907-12-25 -1907-05-05 12:34:56.1674,1907-12-25 -1907-05-05 12:34:56.1675,1907-12-25 -1907-05-05 12:34:56.1676,1907-12-25 -1907-05-05 12:34:56.1677,1907-12-25 -1907-05-05 12:34:56.1678,1907-12-25 -1907-05-05 12:34:56.1679,1907-12-25 -1907-05-05 12:34:56.168,1907-12-25 -1907-05-05 12:34:56.1681,1907-12-25 -1907-05-05 12:34:56.1682,1907-12-25 -1907-05-05 12:34:56.1683,1907-12-25 -1907-05-05 12:34:56.1684,1907-12-25 -1907-05-05 12:34:56.1685,1907-12-25 -1907-05-05 12:34:56.1686,1907-12-25 -1907-05-05 12:34:56.1687,1907-12-25 -1907-05-05 12:34:56.1688,1907-12-25 -1907-05-05 12:34:56.1689,1907-12-25 -1907-05-05 12:34:56.169,1907-12-25 -1907-05-05 12:34:56.1691,1907-12-25 -1907-05-05 12:34:56.1692,1907-12-25 -1907-05-05 12:34:56.1693,1907-12-25 -1907-05-05 12:34:56.1694,1907-12-25 -1907-05-05 12:34:56.1695,1907-12-25 -1907-05-05 12:34:56.1696,1907-12-25 -1907-05-05 12:34:56.1697,1907-12-25 -1907-05-05 12:34:56.1698,1907-12-25 -1907-05-05 12:34:56.1699,1907-12-25 -1907-05-05 12:34:56.17,1907-12-25 -1907-05-05 12:34:56.1701,1907-12-25 -1907-05-05 12:34:56.1702,1907-12-25 -1907-05-05 12:34:56.1703,1907-12-25 -1907-05-05 12:34:56.1704,1907-12-25 -1907-05-05 12:34:56.1705,1907-12-25 -1907-05-05 12:34:56.1706,1907-12-25 -1907-05-05 12:34:56.1707,1907-12-25 -1907-05-05 12:34:56.1708,1907-12-25 -1907-05-05 12:34:56.1709,1907-12-25 -1907-05-05 12:34:56.171,1907-12-25 -1907-05-05 12:34:56.1711,1907-12-25 -1907-05-05 12:34:56.1712,1907-12-25 -1907-05-05 12:34:56.1713,1907-12-25 -1907-05-05 12:34:56.1714,1907-12-25 -1907-05-05 12:34:56.1715,1907-12-25 -1907-05-05 12:34:56.1716,1907-12-25 -1907-05-05 12:34:56.1717,1907-12-25 -1907-05-05 12:34:56.1718,1907-12-25 -1907-05-05 12:34:56.1719,1907-12-25 -1907-05-05 12:34:56.172,1907-12-25 -1907-05-05 12:34:56.1721,1907-12-25 -1907-05-05 12:34:56.1722,1907-12-25 -1907-05-05 12:34:56.1723,1907-12-25 -1907-05-05 12:34:56.1724,1907-12-25 -1907-05-05 12:34:56.1725,1907-12-25 -1907-05-05 12:34:56.1726,1907-12-25 -1907-05-05 12:34:56.1727,1907-12-25 -1907-05-05 12:34:56.1728,1907-12-25 -1907-05-05 12:34:56.1729,1907-12-25 -1907-05-05 12:34:56.173,1907-12-25 -1907-05-05 12:34:56.1731,1907-12-25 -1907-05-05 12:34:56.1732,1907-12-25 -1907-05-05 12:34:56.1733,1907-12-25 -1907-05-05 12:34:56.1734,1907-12-25 -1907-05-05 12:34:56.1735,1907-12-25 -1907-05-05 12:34:56.1736,1907-12-25 -1907-05-05 12:34:56.1737,1907-12-25 -1907-05-05 12:34:56.1738,1907-12-25 -1907-05-05 12:34:56.1739,1907-12-25 -1907-05-05 12:34:56.174,1907-12-25 -1907-05-05 12:34:56.1741,1907-12-25 -1907-05-05 12:34:56.1742,1907-12-25 -1907-05-05 12:34:56.1743,1907-12-25 -1907-05-05 12:34:56.1744,1907-12-25 -1907-05-05 12:34:56.1745,1907-12-25 -1907-05-05 12:34:56.1746,1907-12-25 -1907-05-05 12:34:56.1747,1907-12-25 -1907-05-05 12:34:56.1748,1907-12-25 -1907-05-05 12:34:56.1749,1907-12-25 -1907-05-05 12:34:56.175,1907-12-25 -1907-05-05 12:34:56.1751,1907-12-25 -1907-05-05 12:34:56.1752,1907-12-25 -1907-05-05 12:34:56.1753,1907-12-25 -1907-05-05 12:34:56.1754,1907-12-25 -1907-05-05 12:34:56.1755,1907-12-25 -1907-05-05 12:34:56.1756,1907-12-25 -1907-05-05 12:34:56.1757,1907-12-25 -1907-05-05 12:34:56.1758,1907-12-25 -1907-05-05 12:34:56.1759,1907-12-25 -1907-05-05 12:34:56.176,1907-12-25 -1907-05-05 12:34:56.1761,1907-12-25 -1907-05-05 12:34:56.1762,1907-12-25 -1907-05-05 12:34:56.1763,1907-12-25 -1907-05-05 12:34:56.1764,1907-12-25 -1907-05-05 12:34:56.1765,1907-12-25 -1907-05-05 12:34:56.1766,1907-12-25 -1907-05-05 12:34:56.1767,1907-12-25 -1907-05-05 12:34:56.1768,1907-12-25 -1907-05-05 12:34:56.1769,1907-12-25 -1907-05-05 12:34:56.177,1907-12-25 -1907-05-05 12:34:56.1771,1907-12-25 -1907-05-05 12:34:56.1772,1907-12-25 -1907-05-05 12:34:56.1773,1907-12-25 -1907-05-05 12:34:56.1774,1907-12-25 -1907-05-05 12:34:56.1775,1907-12-25 -1907-05-05 12:34:56.1776,1907-12-25 -1907-05-05 12:34:56.1777,1907-12-25 -1907-05-05 12:34:56.1778,1907-12-25 -1907-05-05 12:34:56.1779,1907-12-25 -1907-05-05 12:34:56.178,1907-12-25 -1907-05-05 12:34:56.1781,1907-12-25 -1907-05-05 12:34:56.1782,1907-12-25 -1907-05-05 12:34:56.1783,1907-12-25 -1907-05-05 12:34:56.1784,1907-12-25 -1907-05-05 12:34:56.1785,1907-12-25 -1907-05-05 12:34:56.1786,1907-12-25 -1907-05-05 12:34:56.1787,1907-12-25 -1907-05-05 12:34:56.1788,1907-12-25 -1907-05-05 12:34:56.1789,1907-12-25 -1907-05-05 12:34:56.179,1907-12-25 -1907-05-05 12:34:56.1791,1907-12-25 -1907-05-05 12:34:56.1792,1907-12-25 -1907-05-05 12:34:56.1793,1907-12-25 -1907-05-05 12:34:56.1794,1907-12-25 -1907-05-05 12:34:56.1795,1907-12-25 -1907-05-05 12:34:56.1796,1907-12-25 -1907-05-05 12:34:56.1797,1907-12-25 -1907-05-05 12:34:56.1798,1907-12-25 -1907-05-05 12:34:56.1799,1907-12-25 -1907-05-05 12:34:56.18,1907-12-25 -1907-05-05 12:34:56.1801,1907-12-25 -1907-05-05 12:34:56.1802,1907-12-25 -1907-05-05 12:34:56.1803,1907-12-25 -1907-05-05 12:34:56.1804,1907-12-25 -1907-05-05 12:34:56.1805,1907-12-25 -1907-05-05 12:34:56.1806,1907-12-25 -1907-05-05 12:34:56.1807,1907-12-25 -1907-05-05 12:34:56.1808,1907-12-25 -1907-05-05 12:34:56.1809,1907-12-25 -1907-05-05 12:34:56.181,1907-12-25 -1907-05-05 12:34:56.1811,1907-12-25 -1907-05-05 12:34:56.1812,1907-12-25 -1907-05-05 12:34:56.1813,1907-12-25 -1907-05-05 12:34:56.1814,1907-12-25 -1907-05-05 12:34:56.1815,1907-12-25 -1907-05-05 12:34:56.1816,1907-12-25 -1907-05-05 12:34:56.1817,1907-12-25 -1907-05-05 12:34:56.1818,1907-12-25 -1907-05-05 12:34:56.1819,1907-12-25 -1907-05-05 12:34:56.182,1907-12-25 -1907-05-05 12:34:56.1821,1907-12-25 -1907-05-05 12:34:56.1822,1907-12-25 -1907-05-05 12:34:56.1823,1907-12-25 -1907-05-05 12:34:56.1824,1907-12-25 -1907-05-05 12:34:56.1825,1907-12-25 -1907-05-05 12:34:56.1826,1907-12-25 -1907-05-05 12:34:56.1827,1907-12-25 -1907-05-05 12:34:56.1828,1907-12-25 -1907-05-05 12:34:56.1829,1907-12-25 -1907-05-05 12:34:56.183,1907-12-25 -1907-05-05 12:34:56.1831,1907-12-25 -1907-05-05 12:34:56.1832,1907-12-25 -1907-05-05 12:34:56.1833,1907-12-25 -1907-05-05 12:34:56.1834,1907-12-25 -1907-05-05 12:34:56.1835,1907-12-25 -1907-05-05 12:34:56.1836,1907-12-25 -1907-05-05 12:34:56.1837,1907-12-25 -1907-05-05 12:34:56.1838,1907-12-25 -1907-05-05 12:34:56.1839,1907-12-25 -1907-05-05 12:34:56.184,1907-12-25 -1907-05-05 12:34:56.1841,1907-12-25 -1907-05-05 12:34:56.1842,1907-12-25 -1907-05-05 12:34:56.1843,1907-12-25 -1907-05-05 12:34:56.1844,1907-12-25 -1907-05-05 12:34:56.1845,1907-12-25 -1907-05-05 12:34:56.1846,1907-12-25 -1907-05-05 12:34:56.1847,1907-12-25 -1907-05-05 12:34:56.1848,1907-12-25 -1907-05-05 12:34:56.1849,1907-12-25 -1907-05-05 12:34:56.185,1907-12-25 -1907-05-05 12:34:56.1851,1907-12-25 -1907-05-05 12:34:56.1852,1907-12-25 -1907-05-05 12:34:56.1853,1907-12-25 -1907-05-05 12:34:56.1854,1907-12-25 -1907-05-05 12:34:56.1855,1907-12-25 -1907-05-05 12:34:56.1856,1907-12-25 -1907-05-05 12:34:56.1857,1907-12-25 -1907-05-05 12:34:56.1858,1907-12-25 -1907-05-05 12:34:56.1859,1907-12-25 -1907-05-05 12:34:56.186,1907-12-25 -1907-05-05 12:34:56.1861,1907-12-25 -1907-05-05 12:34:56.1862,1907-12-25 -1907-05-05 12:34:56.1863,1907-12-25 -1907-05-05 12:34:56.1864,1907-12-25 -1907-05-05 12:34:56.1865,1907-12-25 -1907-05-05 12:34:56.1866,1907-12-25 -1907-05-05 12:34:56.1867,1907-12-25 -1907-05-05 12:34:56.1868,1907-12-25 -1907-05-05 12:34:56.1869,1907-12-25 -1907-05-05 12:34:56.187,1907-12-25 -1907-05-05 12:34:56.1871,1907-12-25 -1907-05-05 12:34:56.1872,1907-12-25 -1907-05-05 12:34:56.1873,1907-12-25 -1907-05-05 12:34:56.1874,1907-12-25 -1907-05-05 12:34:56.1875,1907-12-25 -1907-05-05 12:34:56.1876,1907-12-25 -1907-05-05 12:34:56.1877,1907-12-25 -1907-05-05 12:34:56.1878,1907-12-25 -1907-05-05 12:34:56.1879,1907-12-25 -1907-05-05 12:34:56.188,1907-12-25 -1907-05-05 12:34:56.1881,1907-12-25 -1907-05-05 12:34:56.1882,1907-12-25 -1907-05-05 12:34:56.1883,1907-12-25 -1907-05-05 12:34:56.1884,1907-12-25 -1907-05-05 12:34:56.1885,1907-12-25 -1907-05-05 12:34:56.1886,1907-12-25 -1907-05-05 12:34:56.1887,1907-12-25 -1907-05-05 12:34:56.1888,1907-12-25 -1907-05-05 12:34:56.1889,1907-12-25 -1907-05-05 12:34:56.189,1907-12-25 -1907-05-05 12:34:56.1891,1907-12-25 -1907-05-05 12:34:56.1892,1907-12-25 -1907-05-05 12:34:56.1893,1907-12-25 -1907-05-05 12:34:56.1894,1907-12-25 -1907-05-05 12:34:56.1895,1907-12-25 -1907-05-05 12:34:56.1896,1907-12-25 -1907-05-05 12:34:56.1897,1907-12-25 -1907-05-05 12:34:56.1898,1907-12-25 -1907-05-05 12:34:56.1899,1907-12-25 -1907-05-05 12:34:56.19,1907-12-25 -1907-05-05 12:34:56.1901,1907-12-25 -1907-05-05 12:34:56.1902,1907-12-25 -1907-05-05 12:34:56.1903,1907-12-25 -1907-05-05 12:34:56.1904,1907-12-25 -1907-05-05 12:34:56.1905,1907-12-25 -1907-05-05 12:34:56.1906,1907-12-25 -1907-05-05 12:34:56.1907,1907-12-25 -1907-05-05 12:34:56.1908,1907-12-25 -1907-05-05 12:34:56.1909,1907-12-25 -1907-05-05 12:34:56.191,1907-12-25 -1907-05-05 12:34:56.1911,1907-12-25 -1907-05-05 12:34:56.1912,1907-12-25 -1907-05-05 12:34:56.1913,1907-12-25 -1907-05-05 12:34:56.1914,1907-12-25 -1907-05-05 12:34:56.1915,1907-12-25 -1907-05-05 12:34:56.1916,1907-12-25 -1907-05-05 12:34:56.1917,1907-12-25 -1907-05-05 12:34:56.1918,1907-12-25 -1907-05-05 12:34:56.1919,1907-12-25 -1907-05-05 12:34:56.192,1907-12-25 -1907-05-05 12:34:56.1921,1907-12-25 -1907-05-05 12:34:56.1922,1907-12-25 -1907-05-05 12:34:56.1923,1907-12-25 -1907-05-05 12:34:56.1924,1907-12-25 -1907-05-05 12:34:56.1925,1907-12-25 -1907-05-05 12:34:56.1926,1907-12-25 -1907-05-05 12:34:56.1927,1907-12-25 -1907-05-05 12:34:56.1928,1907-12-25 -1907-05-05 12:34:56.1929,1907-12-25 -1907-05-05 12:34:56.193,1907-12-25 -1907-05-05 12:34:56.1931,1907-12-25 -1907-05-05 12:34:56.1932,1907-12-25 -1907-05-05 12:34:56.1933,1907-12-25 -1907-05-05 12:34:56.1934,1907-12-25 -1907-05-05 12:34:56.1935,1907-12-25 -1907-05-05 12:34:56.1936,1907-12-25 -1907-05-05 12:34:56.1937,1907-12-25 -1907-05-05 12:34:56.1938,1907-12-25 -1907-05-05 12:34:56.1939,1907-12-25 -1907-05-05 12:34:56.194,1907-12-25 -1907-05-05 12:34:56.1941,1907-12-25 -1907-05-05 12:34:56.1942,1907-12-25 -1907-05-05 12:34:56.1943,1907-12-25 -1907-05-05 12:34:56.1944,1907-12-25 -1907-05-05 12:34:56.1945,1907-12-25 -1907-05-05 12:34:56.1946,1907-12-25 -1907-05-05 12:34:56.1947,1907-12-25 -1907-05-05 12:34:56.1948,1907-12-25 -1907-05-05 12:34:56.1949,1907-12-25 -1907-05-05 12:34:56.195,1907-12-25 -1907-05-05 12:34:56.1951,1907-12-25 -1907-05-05 12:34:56.1952,1907-12-25 -1907-05-05 12:34:56.1953,1907-12-25 -1907-05-05 12:34:56.1954,1907-12-25 -1907-05-05 12:34:56.1955,1907-12-25 -1907-05-05 12:34:56.1956,1907-12-25 -1907-05-05 12:34:56.1957,1907-12-25 -1907-05-05 12:34:56.1958,1907-12-25 -1907-05-05 12:34:56.1959,1907-12-25 -1907-05-05 12:34:56.196,1907-12-25 -1907-05-05 12:34:56.1961,1907-12-25 -1907-05-05 12:34:56.1962,1907-12-25 -1907-05-05 12:34:56.1963,1907-12-25 -1907-05-05 12:34:56.1964,1907-12-25 -1907-05-05 12:34:56.1965,1907-12-25 -1907-05-05 12:34:56.1966,1907-12-25 -1907-05-05 12:34:56.1967,1907-12-25 -1907-05-05 12:34:56.1968,1907-12-25 -1907-05-05 12:34:56.1969,1907-12-25 -1907-05-05 12:34:56.197,1907-12-25 -1907-05-05 12:34:56.1971,1907-12-25 -1907-05-05 12:34:56.1972,1907-12-25 -1907-05-05 12:34:56.1973,1907-12-25 -1907-05-05 12:34:56.1974,1907-12-25 -1907-05-05 12:34:56.1975,1907-12-25 -1907-05-05 12:34:56.1976,1907-12-25 -1907-05-05 12:34:56.1977,1907-12-25 -1907-05-05 12:34:56.1978,1907-12-25 -1907-05-05 12:34:56.1979,1907-12-25 -1907-05-05 12:34:56.198,1907-12-25 -1907-05-05 12:34:56.1981,1907-12-25 -1907-05-05 12:34:56.1982,1907-12-25 -1907-05-05 12:34:56.1983,1907-12-25 -1907-05-05 12:34:56.1984,1907-12-25 -1907-05-05 12:34:56.1985,1907-12-25 -1907-05-05 12:34:56.1986,1907-12-25 -1907-05-05 12:34:56.1987,1907-12-25 -1907-05-05 12:34:56.1988,1907-12-25 -1907-05-05 12:34:56.1989,1907-12-25 -1907-05-05 12:34:56.199,1907-12-25 -1907-05-05 12:34:56.1991,1907-12-25 -1907-05-05 12:34:56.1992,1907-12-25 -1907-05-05 12:34:56.1993,1907-12-25 -1907-05-05 12:34:56.1994,1907-12-25 -1907-05-05 12:34:56.1995,1907-12-25 -1907-05-05 12:34:56.1996,1907-12-25 -1907-05-05 12:34:56.1997,1907-12-25 -1907-05-05 12:34:56.1998,1907-12-25 -1907-05-05 12:34:56.1999,1907-12-25 -1908-05-05 12:34:56.1,1908-12-25 -1908-05-05 12:34:56.1001,1908-12-25 -1908-05-05 12:34:56.1002,1908-12-25 -1908-05-05 12:34:56.1003,1908-12-25 -1908-05-05 12:34:56.1004,1908-12-25 -1908-05-05 12:34:56.1005,1908-12-25 -1908-05-05 12:34:56.1006,1908-12-25 -1908-05-05 12:34:56.1007,1908-12-25 -1908-05-05 12:34:56.1008,1908-12-25 -1908-05-05 12:34:56.1009,1908-12-25 -1908-05-05 12:34:56.101,1908-12-25 -1908-05-05 12:34:56.1011,1908-12-25 -1908-05-05 12:34:56.1012,1908-12-25 -1908-05-05 12:34:56.1013,1908-12-25 -1908-05-05 12:34:56.1014,1908-12-25 -1908-05-05 12:34:56.1015,1908-12-25 -1908-05-05 12:34:56.1016,1908-12-25 -1908-05-05 12:34:56.1017,1908-12-25 -1908-05-05 12:34:56.1018,1908-12-25 -1908-05-05 12:34:56.1019,1908-12-25 -1908-05-05 12:34:56.102,1908-12-25 -1908-05-05 12:34:56.1021,1908-12-25 -1908-05-05 12:34:56.1022,1908-12-25 -1908-05-05 12:34:56.1023,1908-12-25 -1908-05-05 12:34:56.1024,1908-12-25 -1908-05-05 12:34:56.1025,1908-12-25 -1908-05-05 12:34:56.1026,1908-12-25 -1908-05-05 12:34:56.1027,1908-12-25 -1908-05-05 12:34:56.1028,1908-12-25 -1908-05-05 12:34:56.1029,1908-12-25 -1908-05-05 12:34:56.103,1908-12-25 -1908-05-05 12:34:56.1031,1908-12-25 -1908-05-05 12:34:56.1032,1908-12-25 -1908-05-05 12:34:56.1033,1908-12-25 -1908-05-05 12:34:56.1034,1908-12-25 -1908-05-05 12:34:56.1035,1908-12-25 -1908-05-05 12:34:56.1036,1908-12-25 -1908-05-05 12:34:56.1037,1908-12-25 -1908-05-05 12:34:56.1038,1908-12-25 -1908-05-05 12:34:56.1039,1908-12-25 -1908-05-05 12:34:56.104,1908-12-25 -1908-05-05 12:34:56.1041,1908-12-25 -1908-05-05 12:34:56.1042,1908-12-25 -1908-05-05 12:34:56.1043,1908-12-25 -1908-05-05 12:34:56.1044,1908-12-25 -1908-05-05 12:34:56.1045,1908-12-25 -1908-05-05 12:34:56.1046,1908-12-25 -1908-05-05 12:34:56.1047,1908-12-25 -1908-05-05 12:34:56.1048,1908-12-25 -1908-05-05 12:34:56.1049,1908-12-25 -1908-05-05 12:34:56.105,1908-12-25 -1908-05-05 12:34:56.1051,1908-12-25 -1908-05-05 12:34:56.1052,1908-12-25 -1908-05-05 12:34:56.1053,1908-12-25 -1908-05-05 12:34:56.1054,1908-12-25 -1908-05-05 12:34:56.1055,1908-12-25 -1908-05-05 12:34:56.1056,1908-12-25 -1908-05-05 12:34:56.1057,1908-12-25 -1908-05-05 12:34:56.1058,1908-12-25 -1908-05-05 12:34:56.1059,1908-12-25 -1908-05-05 12:34:56.106,1908-12-25 -1908-05-05 12:34:56.1061,1908-12-25 -1908-05-05 12:34:56.1062,1908-12-25 -1908-05-05 12:34:56.1063,1908-12-25 -1908-05-05 12:34:56.1064,1908-12-25 -1908-05-05 12:34:56.1065,1908-12-25 -1908-05-05 12:34:56.1066,1908-12-25 -1908-05-05 12:34:56.1067,1908-12-25 -1908-05-05 12:34:56.1068,1908-12-25 -1908-05-05 12:34:56.1069,1908-12-25 -1908-05-05 12:34:56.107,1908-12-25 -1908-05-05 12:34:56.1071,1908-12-25 -1908-05-05 12:34:56.1072,1908-12-25 -1908-05-05 12:34:56.1073,1908-12-25 -1908-05-05 12:34:56.1074,1908-12-25 -1908-05-05 12:34:56.1075,1908-12-25 -1908-05-05 12:34:56.1076,1908-12-25 -1908-05-05 12:34:56.1077,1908-12-25 -1908-05-05 12:34:56.1078,1908-12-25 -1908-05-05 12:34:56.1079,1908-12-25 -1908-05-05 12:34:56.108,1908-12-25 -1908-05-05 12:34:56.1081,1908-12-25 -1908-05-05 12:34:56.1082,1908-12-25 -1908-05-05 12:34:56.1083,1908-12-25 -1908-05-05 12:34:56.1084,1908-12-25 -1908-05-05 12:34:56.1085,1908-12-25 -1908-05-05 12:34:56.1086,1908-12-25 -1908-05-05 12:34:56.1087,1908-12-25 -1908-05-05 12:34:56.1088,1908-12-25 -1908-05-05 12:34:56.1089,1908-12-25 -1908-05-05 12:34:56.109,1908-12-25 -1908-05-05 12:34:56.1091,1908-12-25 -1908-05-05 12:34:56.1092,1908-12-25 -1908-05-05 12:34:56.1093,1908-12-25 -1908-05-05 12:34:56.1094,1908-12-25 -1908-05-05 12:34:56.1095,1908-12-25 -1908-05-05 12:34:56.1096,1908-12-25 -1908-05-05 12:34:56.1097,1908-12-25 -1908-05-05 12:34:56.1098,1908-12-25 -1908-05-05 12:34:56.1099,1908-12-25 -1908-05-05 12:34:56.11,1908-12-25 -1908-05-05 12:34:56.1101,1908-12-25 -1908-05-05 12:34:56.1102,1908-12-25 -1908-05-05 12:34:56.1103,1908-12-25 -1908-05-05 12:34:56.1104,1908-12-25 -1908-05-05 12:34:56.1105,1908-12-25 -1908-05-05 12:34:56.1106,1908-12-25 -1908-05-05 12:34:56.1107,1908-12-25 -1908-05-05 12:34:56.1108,1908-12-25 -1908-05-05 12:34:56.1109,1908-12-25 -1908-05-05 12:34:56.111,1908-12-25 -1908-05-05 12:34:56.1111,1908-12-25 -1908-05-05 12:34:56.1112,1908-12-25 -1908-05-05 12:34:56.1113,1908-12-25 -1908-05-05 12:34:56.1114,1908-12-25 -1908-05-05 12:34:56.1115,1908-12-25 -1908-05-05 12:34:56.1116,1908-12-25 -1908-05-05 12:34:56.1117,1908-12-25 -1908-05-05 12:34:56.1118,1908-12-25 -1908-05-05 12:34:56.1119,1908-12-25 -1908-05-05 12:34:56.112,1908-12-25 -1908-05-05 12:34:56.1121,1908-12-25 -1908-05-05 12:34:56.1122,1908-12-25 -1908-05-05 12:34:56.1123,1908-12-25 -1908-05-05 12:34:56.1124,1908-12-25 -1908-05-05 12:34:56.1125,1908-12-25 -1908-05-05 12:34:56.1126,1908-12-25 -1908-05-05 12:34:56.1127,1908-12-25 -1908-05-05 12:34:56.1128,1908-12-25 -1908-05-05 12:34:56.1129,1908-12-25 -1908-05-05 12:34:56.113,1908-12-25 -1908-05-05 12:34:56.1131,1908-12-25 -1908-05-05 12:34:56.1132,1908-12-25 -1908-05-05 12:34:56.1133,1908-12-25 -1908-05-05 12:34:56.1134,1908-12-25 -1908-05-05 12:34:56.1135,1908-12-25 -1908-05-05 12:34:56.1136,1908-12-25 -1908-05-05 12:34:56.1137,1908-12-25 -1908-05-05 12:34:56.1138,1908-12-25 -1908-05-05 12:34:56.1139,1908-12-25 -1908-05-05 12:34:56.114,1908-12-25 -1908-05-05 12:34:56.1141,1908-12-25 -1908-05-05 12:34:56.1142,1908-12-25 -1908-05-05 12:34:56.1143,1908-12-25 -1908-05-05 12:34:56.1144,1908-12-25 -1908-05-05 12:34:56.1145,1908-12-25 -1908-05-05 12:34:56.1146,1908-12-25 -1908-05-05 12:34:56.1147,1908-12-25 -1908-05-05 12:34:56.1148,1908-12-25 -1908-05-05 12:34:56.1149,1908-12-25 -1908-05-05 12:34:56.115,1908-12-25 -1908-05-05 12:34:56.1151,1908-12-25 -1908-05-05 12:34:56.1152,1908-12-25 -1908-05-05 12:34:56.1153,1908-12-25 -1908-05-05 12:34:56.1154,1908-12-25 -1908-05-05 12:34:56.1155,1908-12-25 -1908-05-05 12:34:56.1156,1908-12-25 -1908-05-05 12:34:56.1157,1908-12-25 -1908-05-05 12:34:56.1158,1908-12-25 -1908-05-05 12:34:56.1159,1908-12-25 -1908-05-05 12:34:56.116,1908-12-25 -1908-05-05 12:34:56.1161,1908-12-25 -1908-05-05 12:34:56.1162,1908-12-25 -1908-05-05 12:34:56.1163,1908-12-25 -1908-05-05 12:34:56.1164,1908-12-25 -1908-05-05 12:34:56.1165,1908-12-25 -1908-05-05 12:34:56.1166,1908-12-25 -1908-05-05 12:34:56.1167,1908-12-25 -1908-05-05 12:34:56.1168,1908-12-25 -1908-05-05 12:34:56.1169,1908-12-25 -1908-05-05 12:34:56.117,1908-12-25 -1908-05-05 12:34:56.1171,1908-12-25 -1908-05-05 12:34:56.1172,1908-12-25 -1908-05-05 12:34:56.1173,1908-12-25 -1908-05-05 12:34:56.1174,1908-12-25 -1908-05-05 12:34:56.1175,1908-12-25 -1908-05-05 12:34:56.1176,1908-12-25 -1908-05-05 12:34:56.1177,1908-12-25 -1908-05-05 12:34:56.1178,1908-12-25 -1908-05-05 12:34:56.1179,1908-12-25 -1908-05-05 12:34:56.118,1908-12-25 -1908-05-05 12:34:56.1181,1908-12-25 -1908-05-05 12:34:56.1182,1908-12-25 -1908-05-05 12:34:56.1183,1908-12-25 -1908-05-05 12:34:56.1184,1908-12-25 -1908-05-05 12:34:56.1185,1908-12-25 -1908-05-05 12:34:56.1186,1908-12-25 -1908-05-05 12:34:56.1187,1908-12-25 -1908-05-05 12:34:56.1188,1908-12-25 -1908-05-05 12:34:56.1189,1908-12-25 -1908-05-05 12:34:56.119,1908-12-25 -1908-05-05 12:34:56.1191,1908-12-25 -1908-05-05 12:34:56.1192,1908-12-25 -1908-05-05 12:34:56.1193,1908-12-25 -1908-05-05 12:34:56.1194,1908-12-25 -1908-05-05 12:34:56.1195,1908-12-25 -1908-05-05 12:34:56.1196,1908-12-25 -1908-05-05 12:34:56.1197,1908-12-25 -1908-05-05 12:34:56.1198,1908-12-25 -1908-05-05 12:34:56.1199,1908-12-25 -1908-05-05 12:34:56.12,1908-12-25 -1908-05-05 12:34:56.1201,1908-12-25 -1908-05-05 12:34:56.1202,1908-12-25 -1908-05-05 12:34:56.1203,1908-12-25 -1908-05-05 12:34:56.1204,1908-12-25 -1908-05-05 12:34:56.1205,1908-12-25 -1908-05-05 12:34:56.1206,1908-12-25 -1908-05-05 12:34:56.1207,1908-12-25 -1908-05-05 12:34:56.1208,1908-12-25 -1908-05-05 12:34:56.1209,1908-12-25 -1908-05-05 12:34:56.121,1908-12-25 -1908-05-05 12:34:56.1211,1908-12-25 -1908-05-05 12:34:56.1212,1908-12-25 -1908-05-05 12:34:56.1213,1908-12-25 -1908-05-05 12:34:56.1214,1908-12-25 -1908-05-05 12:34:56.1215,1908-12-25 -1908-05-05 12:34:56.1216,1908-12-25 -1908-05-05 12:34:56.1217,1908-12-25 -1908-05-05 12:34:56.1218,1908-12-25 -1908-05-05 12:34:56.1219,1908-12-25 -1908-05-05 12:34:56.122,1908-12-25 -1908-05-05 12:34:56.1221,1908-12-25 -1908-05-05 12:34:56.1222,1908-12-25 -1908-05-05 12:34:56.1223,1908-12-25 -1908-05-05 12:34:56.1224,1908-12-25 -1908-05-05 12:34:56.1225,1908-12-25 -1908-05-05 12:34:56.1226,1908-12-25 -1908-05-05 12:34:56.1227,1908-12-25 -1908-05-05 12:34:56.1228,1908-12-25 -1908-05-05 12:34:56.1229,1908-12-25 -1908-05-05 12:34:56.123,1908-12-25 -1908-05-05 12:34:56.1231,1908-12-25 -1908-05-05 12:34:56.1232,1908-12-25 -1908-05-05 12:34:56.1233,1908-12-25 -1908-05-05 12:34:56.1234,1908-12-25 -1908-05-05 12:34:56.1235,1908-12-25 -1908-05-05 12:34:56.1236,1908-12-25 -1908-05-05 12:34:56.1237,1908-12-25 -1908-05-05 12:34:56.1238,1908-12-25 -1908-05-05 12:34:56.1239,1908-12-25 -1908-05-05 12:34:56.124,1908-12-25 -1908-05-05 12:34:56.1241,1908-12-25 -1908-05-05 12:34:56.1242,1908-12-25 -1908-05-05 12:34:56.1243,1908-12-25 -1908-05-05 12:34:56.1244,1908-12-25 -1908-05-05 12:34:56.1245,1908-12-25 -1908-05-05 12:34:56.1246,1908-12-25 -1908-05-05 12:34:56.1247,1908-12-25 -1908-05-05 12:34:56.1248,1908-12-25 -1908-05-05 12:34:56.1249,1908-12-25 -1908-05-05 12:34:56.125,1908-12-25 -1908-05-05 12:34:56.1251,1908-12-25 -1908-05-05 12:34:56.1252,1908-12-25 -1908-05-05 12:34:56.1253,1908-12-25 -1908-05-05 12:34:56.1254,1908-12-25 -1908-05-05 12:34:56.1255,1908-12-25 -1908-05-05 12:34:56.1256,1908-12-25 -1908-05-05 12:34:56.1257,1908-12-25 -1908-05-05 12:34:56.1258,1908-12-25 -1908-05-05 12:34:56.1259,1908-12-25 -1908-05-05 12:34:56.126,1908-12-25 -1908-05-05 12:34:56.1261,1908-12-25 -1908-05-05 12:34:56.1262,1908-12-25 -1908-05-05 12:34:56.1263,1908-12-25 -1908-05-05 12:34:56.1264,1908-12-25 -1908-05-05 12:34:56.1265,1908-12-25 -1908-05-05 12:34:56.1266,1908-12-25 -1908-05-05 12:34:56.1267,1908-12-25 -1908-05-05 12:34:56.1268,1908-12-25 -1908-05-05 12:34:56.1269,1908-12-25 -1908-05-05 12:34:56.127,1908-12-25 -1908-05-05 12:34:56.1271,1908-12-25 -1908-05-05 12:34:56.1272,1908-12-25 -1908-05-05 12:34:56.1273,1908-12-25 -1908-05-05 12:34:56.1274,1908-12-25 -1908-05-05 12:34:56.1275,1908-12-25 -1908-05-05 12:34:56.1276,1908-12-25 -1908-05-05 12:34:56.1277,1908-12-25 -1908-05-05 12:34:56.1278,1908-12-25 -1908-05-05 12:34:56.1279,1908-12-25 -1908-05-05 12:34:56.128,1908-12-25 -1908-05-05 12:34:56.1281,1908-12-25 -1908-05-05 12:34:56.1282,1908-12-25 -1908-05-05 12:34:56.1283,1908-12-25 -1908-05-05 12:34:56.1284,1908-12-25 -1908-05-05 12:34:56.1285,1908-12-25 -1908-05-05 12:34:56.1286,1908-12-25 -1908-05-05 12:34:56.1287,1908-12-25 -1908-05-05 12:34:56.1288,1908-12-25 -1908-05-05 12:34:56.1289,1908-12-25 -1908-05-05 12:34:56.129,1908-12-25 -1908-05-05 12:34:56.1291,1908-12-25 -1908-05-05 12:34:56.1292,1908-12-25 -1908-05-05 12:34:56.1293,1908-12-25 -1908-05-05 12:34:56.1294,1908-12-25 -1908-05-05 12:34:56.1295,1908-12-25 -1908-05-05 12:34:56.1296,1908-12-25 -1908-05-05 12:34:56.1297,1908-12-25 -1908-05-05 12:34:56.1298,1908-12-25 -1908-05-05 12:34:56.1299,1908-12-25 -1908-05-05 12:34:56.13,1908-12-25 -1908-05-05 12:34:56.1301,1908-12-25 -1908-05-05 12:34:56.1302,1908-12-25 -1908-05-05 12:34:56.1303,1908-12-25 -1908-05-05 12:34:56.1304,1908-12-25 -1908-05-05 12:34:56.1305,1908-12-25 -1908-05-05 12:34:56.1306,1908-12-25 -1908-05-05 12:34:56.1307,1908-12-25 -1908-05-05 12:34:56.1308,1908-12-25 -1908-05-05 12:34:56.1309,1908-12-25 -1908-05-05 12:34:56.131,1908-12-25 -1908-05-05 12:34:56.1311,1908-12-25 -1908-05-05 12:34:56.1312,1908-12-25 -1908-05-05 12:34:56.1313,1908-12-25 -1908-05-05 12:34:56.1314,1908-12-25 -1908-05-05 12:34:56.1315,1908-12-25 -1908-05-05 12:34:56.1316,1908-12-25 -1908-05-05 12:34:56.1317,1908-12-25 -1908-05-05 12:34:56.1318,1908-12-25 -1908-05-05 12:34:56.1319,1908-12-25 -1908-05-05 12:34:56.132,1908-12-25 -1908-05-05 12:34:56.1321,1908-12-25 -1908-05-05 12:34:56.1322,1908-12-25 -1908-05-05 12:34:56.1323,1908-12-25 -1908-05-05 12:34:56.1324,1908-12-25 -1908-05-05 12:34:56.1325,1908-12-25 -1908-05-05 12:34:56.1326,1908-12-25 -1908-05-05 12:34:56.1327,1908-12-25 -1908-05-05 12:34:56.1328,1908-12-25 -1908-05-05 12:34:56.1329,1908-12-25 -1908-05-05 12:34:56.133,1908-12-25 -1908-05-05 12:34:56.1331,1908-12-25 -1908-05-05 12:34:56.1332,1908-12-25 -1908-05-05 12:34:56.1333,1908-12-25 -1908-05-05 12:34:56.1334,1908-12-25 -1908-05-05 12:34:56.1335,1908-12-25 -1908-05-05 12:34:56.1336,1908-12-25 -1908-05-05 12:34:56.1337,1908-12-25 -1908-05-05 12:34:56.1338,1908-12-25 -1908-05-05 12:34:56.1339,1908-12-25 -1908-05-05 12:34:56.134,1908-12-25 -1908-05-05 12:34:56.1341,1908-12-25 -1908-05-05 12:34:56.1342,1908-12-25 -1908-05-05 12:34:56.1343,1908-12-25 -1908-05-05 12:34:56.1344,1908-12-25 -1908-05-05 12:34:56.1345,1908-12-25 -1908-05-05 12:34:56.1346,1908-12-25 -1908-05-05 12:34:56.1347,1908-12-25 -1908-05-05 12:34:56.1348,1908-12-25 -1908-05-05 12:34:56.1349,1908-12-25 -1908-05-05 12:34:56.135,1908-12-25 -1908-05-05 12:34:56.1351,1908-12-25 -1908-05-05 12:34:56.1352,1908-12-25 -1908-05-05 12:34:56.1353,1908-12-25 -1908-05-05 12:34:56.1354,1908-12-25 -1908-05-05 12:34:56.1355,1908-12-25 -1908-05-05 12:34:56.1356,1908-12-25 -1908-05-05 12:34:56.1357,1908-12-25 -1908-05-05 12:34:56.1358,1908-12-25 -1908-05-05 12:34:56.1359,1908-12-25 -1908-05-05 12:34:56.136,1908-12-25 -1908-05-05 12:34:56.1361,1908-12-25 -1908-05-05 12:34:56.1362,1908-12-25 -1908-05-05 12:34:56.1363,1908-12-25 -1908-05-05 12:34:56.1364,1908-12-25 -1908-05-05 12:34:56.1365,1908-12-25 -1908-05-05 12:34:56.1366,1908-12-25 -1908-05-05 12:34:56.1367,1908-12-25 -1908-05-05 12:34:56.1368,1908-12-25 -1908-05-05 12:34:56.1369,1908-12-25 -1908-05-05 12:34:56.137,1908-12-25 -1908-05-05 12:34:56.1371,1908-12-25 -1908-05-05 12:34:56.1372,1908-12-25 -1908-05-05 12:34:56.1373,1908-12-25 -1908-05-05 12:34:56.1374,1908-12-25 -1908-05-05 12:34:56.1375,1908-12-25 -1908-05-05 12:34:56.1376,1908-12-25 -1908-05-05 12:34:56.1377,1908-12-25 -1908-05-05 12:34:56.1378,1908-12-25 -1908-05-05 12:34:56.1379,1908-12-25 -1908-05-05 12:34:56.138,1908-12-25 -1908-05-05 12:34:56.1381,1908-12-25 -1908-05-05 12:34:56.1382,1908-12-25 -1908-05-05 12:34:56.1383,1908-12-25 -1908-05-05 12:34:56.1384,1908-12-25 -1908-05-05 12:34:56.1385,1908-12-25 -1908-05-05 12:34:56.1386,1908-12-25 -1908-05-05 12:34:56.1387,1908-12-25 -1908-05-05 12:34:56.1388,1908-12-25 -1908-05-05 12:34:56.1389,1908-12-25 -1908-05-05 12:34:56.139,1908-12-25 -1908-05-05 12:34:56.1391,1908-12-25 -1908-05-05 12:34:56.1392,1908-12-25 -1908-05-05 12:34:56.1393,1908-12-25 -1908-05-05 12:34:56.1394,1908-12-25 -1908-05-05 12:34:56.1395,1908-12-25 -1908-05-05 12:34:56.1396,1908-12-25 -1908-05-05 12:34:56.1397,1908-12-25 -1908-05-05 12:34:56.1398,1908-12-25 -1908-05-05 12:34:56.1399,1908-12-25 -1908-05-05 12:34:56.14,1908-12-25 -1908-05-05 12:34:56.1401,1908-12-25 -1908-05-05 12:34:56.1402,1908-12-25 -1908-05-05 12:34:56.1403,1908-12-25 -1908-05-05 12:34:56.1404,1908-12-25 -1908-05-05 12:34:56.1405,1908-12-25 -1908-05-05 12:34:56.1406,1908-12-25 -1908-05-05 12:34:56.1407,1908-12-25 -1908-05-05 12:34:56.1408,1908-12-25 -1908-05-05 12:34:56.1409,1908-12-25 -1908-05-05 12:34:56.141,1908-12-25 -1908-05-05 12:34:56.1411,1908-12-25 -1908-05-05 12:34:56.1412,1908-12-25 -1908-05-05 12:34:56.1413,1908-12-25 -1908-05-05 12:34:56.1414,1908-12-25 -1908-05-05 12:34:56.1415,1908-12-25 -1908-05-05 12:34:56.1416,1908-12-25 -1908-05-05 12:34:56.1417,1908-12-25 -1908-05-05 12:34:56.1418,1908-12-25 -1908-05-05 12:34:56.1419,1908-12-25 -1908-05-05 12:34:56.142,1908-12-25 -1908-05-05 12:34:56.1421,1908-12-25 -1908-05-05 12:34:56.1422,1908-12-25 -1908-05-05 12:34:56.1423,1908-12-25 -1908-05-05 12:34:56.1424,1908-12-25 -1908-05-05 12:34:56.1425,1908-12-25 -1908-05-05 12:34:56.1426,1908-12-25 -1908-05-05 12:34:56.1427,1908-12-25 -1908-05-05 12:34:56.1428,1908-12-25 -1908-05-05 12:34:56.1429,1908-12-25 -1908-05-05 12:34:56.143,1908-12-25 -1908-05-05 12:34:56.1431,1908-12-25 -1908-05-05 12:34:56.1432,1908-12-25 -1908-05-05 12:34:56.1433,1908-12-25 -1908-05-05 12:34:56.1434,1908-12-25 -1908-05-05 12:34:56.1435,1908-12-25 -1908-05-05 12:34:56.1436,1908-12-25 -1908-05-05 12:34:56.1437,1908-12-25 -1908-05-05 12:34:56.1438,1908-12-25 -1908-05-05 12:34:56.1439,1908-12-25 -1908-05-05 12:34:56.144,1908-12-25 -1908-05-05 12:34:56.1441,1908-12-25 -1908-05-05 12:34:56.1442,1908-12-25 -1908-05-05 12:34:56.1443,1908-12-25 -1908-05-05 12:34:56.1444,1908-12-25 -1908-05-05 12:34:56.1445,1908-12-25 -1908-05-05 12:34:56.1446,1908-12-25 -1908-05-05 12:34:56.1447,1908-12-25 -1908-05-05 12:34:56.1448,1908-12-25 -1908-05-05 12:34:56.1449,1908-12-25 -1908-05-05 12:34:56.145,1908-12-25 -1908-05-05 12:34:56.1451,1908-12-25 -1908-05-05 12:34:56.1452,1908-12-25 -1908-05-05 12:34:56.1453,1908-12-25 -1908-05-05 12:34:56.1454,1908-12-25 -1908-05-05 12:34:56.1455,1908-12-25 -1908-05-05 12:34:56.1456,1908-12-25 -1908-05-05 12:34:56.1457,1908-12-25 -1908-05-05 12:34:56.1458,1908-12-25 -1908-05-05 12:34:56.1459,1908-12-25 -1908-05-05 12:34:56.146,1908-12-25 -1908-05-05 12:34:56.1461,1908-12-25 -1908-05-05 12:34:56.1462,1908-12-25 -1908-05-05 12:34:56.1463,1908-12-25 -1908-05-05 12:34:56.1464,1908-12-25 -1908-05-05 12:34:56.1465,1908-12-25 -1908-05-05 12:34:56.1466,1908-12-25 -1908-05-05 12:34:56.1467,1908-12-25 -1908-05-05 12:34:56.1468,1908-12-25 -1908-05-05 12:34:56.1469,1908-12-25 -1908-05-05 12:34:56.147,1908-12-25 -1908-05-05 12:34:56.1471,1908-12-25 -1908-05-05 12:34:56.1472,1908-12-25 -1908-05-05 12:34:56.1473,1908-12-25 -1908-05-05 12:34:56.1474,1908-12-25 -1908-05-05 12:34:56.1475,1908-12-25 -1908-05-05 12:34:56.1476,1908-12-25 -1908-05-05 12:34:56.1477,1908-12-25 -1908-05-05 12:34:56.1478,1908-12-25 -1908-05-05 12:34:56.1479,1908-12-25 -1908-05-05 12:34:56.148,1908-12-25 -1908-05-05 12:34:56.1481,1908-12-25 -1908-05-05 12:34:56.1482,1908-12-25 -1908-05-05 12:34:56.1483,1908-12-25 -1908-05-05 12:34:56.1484,1908-12-25 -1908-05-05 12:34:56.1485,1908-12-25 -1908-05-05 12:34:56.1486,1908-12-25 -1908-05-05 12:34:56.1487,1908-12-25 -1908-05-05 12:34:56.1488,1908-12-25 -1908-05-05 12:34:56.1489,1908-12-25 -1908-05-05 12:34:56.149,1908-12-25 -1908-05-05 12:34:56.1491,1908-12-25 -1908-05-05 12:34:56.1492,1908-12-25 -1908-05-05 12:34:56.1493,1908-12-25 -1908-05-05 12:34:56.1494,1908-12-25 -1908-05-05 12:34:56.1495,1908-12-25 -1908-05-05 12:34:56.1496,1908-12-25 -1908-05-05 12:34:56.1497,1908-12-25 -1908-05-05 12:34:56.1498,1908-12-25 -1908-05-05 12:34:56.1499,1908-12-25 -1908-05-05 12:34:56.15,1908-12-25 -1908-05-05 12:34:56.1501,1908-12-25 -1908-05-05 12:34:56.1502,1908-12-25 -1908-05-05 12:34:56.1503,1908-12-25 -1908-05-05 12:34:56.1504,1908-12-25 -1908-05-05 12:34:56.1505,1908-12-25 -1908-05-05 12:34:56.1506,1908-12-25 -1908-05-05 12:34:56.1507,1908-12-25 -1908-05-05 12:34:56.1508,1908-12-25 -1908-05-05 12:34:56.1509,1908-12-25 -1908-05-05 12:34:56.151,1908-12-25 -1908-05-05 12:34:56.1511,1908-12-25 -1908-05-05 12:34:56.1512,1908-12-25 -1908-05-05 12:34:56.1513,1908-12-25 -1908-05-05 12:34:56.1514,1908-12-25 -1908-05-05 12:34:56.1515,1908-12-25 -1908-05-05 12:34:56.1516,1908-12-25 -1908-05-05 12:34:56.1517,1908-12-25 -1908-05-05 12:34:56.1518,1908-12-25 -1908-05-05 12:34:56.1519,1908-12-25 -1908-05-05 12:34:56.152,1908-12-25 -1908-05-05 12:34:56.1521,1908-12-25 -1908-05-05 12:34:56.1522,1908-12-25 -1908-05-05 12:34:56.1523,1908-12-25 -1908-05-05 12:34:56.1524,1908-12-25 -1908-05-05 12:34:56.1525,1908-12-25 -1908-05-05 12:34:56.1526,1908-12-25 -1908-05-05 12:34:56.1527,1908-12-25 -1908-05-05 12:34:56.1528,1908-12-25 -1908-05-05 12:34:56.1529,1908-12-25 -1908-05-05 12:34:56.153,1908-12-25 -1908-05-05 12:34:56.1531,1908-12-25 -1908-05-05 12:34:56.1532,1908-12-25 -1908-05-05 12:34:56.1533,1908-12-25 -1908-05-05 12:34:56.1534,1908-12-25 -1908-05-05 12:34:56.1535,1908-12-25 -1908-05-05 12:34:56.1536,1908-12-25 -1908-05-05 12:34:56.1537,1908-12-25 -1908-05-05 12:34:56.1538,1908-12-25 -1908-05-05 12:34:56.1539,1908-12-25 -1908-05-05 12:34:56.154,1908-12-25 -1908-05-05 12:34:56.1541,1908-12-25 -1908-05-05 12:34:56.1542,1908-12-25 -1908-05-05 12:34:56.1543,1908-12-25 -1908-05-05 12:34:56.1544,1908-12-25 -1908-05-05 12:34:56.1545,1908-12-25 -1908-05-05 12:34:56.1546,1908-12-25 -1908-05-05 12:34:56.1547,1908-12-25 -1908-05-05 12:34:56.1548,1908-12-25 -1908-05-05 12:34:56.1549,1908-12-25 -1908-05-05 12:34:56.155,1908-12-25 -1908-05-05 12:34:56.1551,1908-12-25 -1908-05-05 12:34:56.1552,1908-12-25 -1908-05-05 12:34:56.1553,1908-12-25 -1908-05-05 12:34:56.1554,1908-12-25 -1908-05-05 12:34:56.1555,1908-12-25 -1908-05-05 12:34:56.1556,1908-12-25 -1908-05-05 12:34:56.1557,1908-12-25 -1908-05-05 12:34:56.1558,1908-12-25 -1908-05-05 12:34:56.1559,1908-12-25 -1908-05-05 12:34:56.156,1908-12-25 -1908-05-05 12:34:56.1561,1908-12-25 -1908-05-05 12:34:56.1562,1908-12-25 -1908-05-05 12:34:56.1563,1908-12-25 -1908-05-05 12:34:56.1564,1908-12-25 -1908-05-05 12:34:56.1565,1908-12-25 -1908-05-05 12:34:56.1566,1908-12-25 -1908-05-05 12:34:56.1567,1908-12-25 -1908-05-05 12:34:56.1568,1908-12-25 -1908-05-05 12:34:56.1569,1908-12-25 -1908-05-05 12:34:56.157,1908-12-25 -1908-05-05 12:34:56.1571,1908-12-25 -1908-05-05 12:34:56.1572,1908-12-25 -1908-05-05 12:34:56.1573,1908-12-25 -1908-05-05 12:34:56.1574,1908-12-25 -1908-05-05 12:34:56.1575,1908-12-25 -1908-05-05 12:34:56.1576,1908-12-25 -1908-05-05 12:34:56.1577,1908-12-25 -1908-05-05 12:34:56.1578,1908-12-25 -1908-05-05 12:34:56.1579,1908-12-25 -1908-05-05 12:34:56.158,1908-12-25 -1908-05-05 12:34:56.1581,1908-12-25 -1908-05-05 12:34:56.1582,1908-12-25 -1908-05-05 12:34:56.1583,1908-12-25 -1908-05-05 12:34:56.1584,1908-12-25 -1908-05-05 12:34:56.1585,1908-12-25 -1908-05-05 12:34:56.1586,1908-12-25 -1908-05-05 12:34:56.1587,1908-12-25 -1908-05-05 12:34:56.1588,1908-12-25 -1908-05-05 12:34:56.1589,1908-12-25 -1908-05-05 12:34:56.159,1908-12-25 -1908-05-05 12:34:56.1591,1908-12-25 -1908-05-05 12:34:56.1592,1908-12-25 -1908-05-05 12:34:56.1593,1908-12-25 -1908-05-05 12:34:56.1594,1908-12-25 -1908-05-05 12:34:56.1595,1908-12-25 -1908-05-05 12:34:56.1596,1908-12-25 -1908-05-05 12:34:56.1597,1908-12-25 -1908-05-05 12:34:56.1598,1908-12-25 -1908-05-05 12:34:56.1599,1908-12-25 -1908-05-05 12:34:56.16,1908-12-25 -1908-05-05 12:34:56.1601,1908-12-25 -1908-05-05 12:34:56.1602,1908-12-25 -1908-05-05 12:34:56.1603,1908-12-25 -1908-05-05 12:34:56.1604,1908-12-25 -1908-05-05 12:34:56.1605,1908-12-25 -1908-05-05 12:34:56.1606,1908-12-25 -1908-05-05 12:34:56.1607,1908-12-25 -1908-05-05 12:34:56.1608,1908-12-25 -1908-05-05 12:34:56.1609,1908-12-25 -1908-05-05 12:34:56.161,1908-12-25 -1908-05-05 12:34:56.1611,1908-12-25 -1908-05-05 12:34:56.1612,1908-12-25 -1908-05-05 12:34:56.1613,1908-12-25 -1908-05-05 12:34:56.1614,1908-12-25 -1908-05-05 12:34:56.1615,1908-12-25 -1908-05-05 12:34:56.1616,1908-12-25 -1908-05-05 12:34:56.1617,1908-12-25 -1908-05-05 12:34:56.1618,1908-12-25 -1908-05-05 12:34:56.1619,1908-12-25 -1908-05-05 12:34:56.162,1908-12-25 -1908-05-05 12:34:56.1621,1908-12-25 -1908-05-05 12:34:56.1622,1908-12-25 -1908-05-05 12:34:56.1623,1908-12-25 -1908-05-05 12:34:56.1624,1908-12-25 -1908-05-05 12:34:56.1625,1908-12-25 -1908-05-05 12:34:56.1626,1908-12-25 -1908-05-05 12:34:56.1627,1908-12-25 -1908-05-05 12:34:56.1628,1908-12-25 -1908-05-05 12:34:56.1629,1908-12-25 -1908-05-05 12:34:56.163,1908-12-25 -1908-05-05 12:34:56.1631,1908-12-25 -1908-05-05 12:34:56.1632,1908-12-25 -1908-05-05 12:34:56.1633,1908-12-25 -1908-05-05 12:34:56.1634,1908-12-25 -1908-05-05 12:34:56.1635,1908-12-25 -1908-05-05 12:34:56.1636,1908-12-25 -1908-05-05 12:34:56.1637,1908-12-25 -1908-05-05 12:34:56.1638,1908-12-25 -1908-05-05 12:34:56.1639,1908-12-25 -1908-05-05 12:34:56.164,1908-12-25 -1908-05-05 12:34:56.1641,1908-12-25 -1908-05-05 12:34:56.1642,1908-12-25 -1908-05-05 12:34:56.1643,1908-12-25 -1908-05-05 12:34:56.1644,1908-12-25 -1908-05-05 12:34:56.1645,1908-12-25 -1908-05-05 12:34:56.1646,1908-12-25 -1908-05-05 12:34:56.1647,1908-12-25 -1908-05-05 12:34:56.1648,1908-12-25 -1908-05-05 12:34:56.1649,1908-12-25 -1908-05-05 12:34:56.165,1908-12-25 -1908-05-05 12:34:56.1651,1908-12-25 -1908-05-05 12:34:56.1652,1908-12-25 -1908-05-05 12:34:56.1653,1908-12-25 -1908-05-05 12:34:56.1654,1908-12-25 -1908-05-05 12:34:56.1655,1908-12-25 -1908-05-05 12:34:56.1656,1908-12-25 -1908-05-05 12:34:56.1657,1908-12-25 -1908-05-05 12:34:56.1658,1908-12-25 -1908-05-05 12:34:56.1659,1908-12-25 -1908-05-05 12:34:56.166,1908-12-25 -1908-05-05 12:34:56.1661,1908-12-25 -1908-05-05 12:34:56.1662,1908-12-25 -1908-05-05 12:34:56.1663,1908-12-25 -1908-05-05 12:34:56.1664,1908-12-25 -1908-05-05 12:34:56.1665,1908-12-25 -1908-05-05 12:34:56.1666,1908-12-25 -1908-05-05 12:34:56.1667,1908-12-25 -1908-05-05 12:34:56.1668,1908-12-25 -1908-05-05 12:34:56.1669,1908-12-25 -1908-05-05 12:34:56.167,1908-12-25 -1908-05-05 12:34:56.1671,1908-12-25 -1908-05-05 12:34:56.1672,1908-12-25 -1908-05-05 12:34:56.1673,1908-12-25 -1908-05-05 12:34:56.1674,1908-12-25 -1908-05-05 12:34:56.1675,1908-12-25 -1908-05-05 12:34:56.1676,1908-12-25 -1908-05-05 12:34:56.1677,1908-12-25 -1908-05-05 12:34:56.1678,1908-12-25 -1908-05-05 12:34:56.1679,1908-12-25 -1908-05-05 12:34:56.168,1908-12-25 -1908-05-05 12:34:56.1681,1908-12-25 -1908-05-05 12:34:56.1682,1908-12-25 -1908-05-05 12:34:56.1683,1908-12-25 -1908-05-05 12:34:56.1684,1908-12-25 -1908-05-05 12:34:56.1685,1908-12-25 -1908-05-05 12:34:56.1686,1908-12-25 -1908-05-05 12:34:56.1687,1908-12-25 -1908-05-05 12:34:56.1688,1908-12-25 -1908-05-05 12:34:56.1689,1908-12-25 -1908-05-05 12:34:56.169,1908-12-25 -1908-05-05 12:34:56.1691,1908-12-25 -1908-05-05 12:34:56.1692,1908-12-25 -1908-05-05 12:34:56.1693,1908-12-25 -1908-05-05 12:34:56.1694,1908-12-25 -1908-05-05 12:34:56.1695,1908-12-25 -1908-05-05 12:34:56.1696,1908-12-25 -1908-05-05 12:34:56.1697,1908-12-25 -1908-05-05 12:34:56.1698,1908-12-25 -1908-05-05 12:34:56.1699,1908-12-25 -1908-05-05 12:34:56.17,1908-12-25 -1908-05-05 12:34:56.1701,1908-12-25 -1908-05-05 12:34:56.1702,1908-12-25 -1908-05-05 12:34:56.1703,1908-12-25 -1908-05-05 12:34:56.1704,1908-12-25 -1908-05-05 12:34:56.1705,1908-12-25 -1908-05-05 12:34:56.1706,1908-12-25 -1908-05-05 12:34:56.1707,1908-12-25 -1908-05-05 12:34:56.1708,1908-12-25 -1908-05-05 12:34:56.1709,1908-12-25 -1908-05-05 12:34:56.171,1908-12-25 -1908-05-05 12:34:56.1711,1908-12-25 -1908-05-05 12:34:56.1712,1908-12-25 -1908-05-05 12:34:56.1713,1908-12-25 -1908-05-05 12:34:56.1714,1908-12-25 -1908-05-05 12:34:56.1715,1908-12-25 -1908-05-05 12:34:56.1716,1908-12-25 -1908-05-05 12:34:56.1717,1908-12-25 -1908-05-05 12:34:56.1718,1908-12-25 -1908-05-05 12:34:56.1719,1908-12-25 -1908-05-05 12:34:56.172,1908-12-25 -1908-05-05 12:34:56.1721,1908-12-25 -1908-05-05 12:34:56.1722,1908-12-25 -1908-05-05 12:34:56.1723,1908-12-25 -1908-05-05 12:34:56.1724,1908-12-25 -1908-05-05 12:34:56.1725,1908-12-25 -1908-05-05 12:34:56.1726,1908-12-25 -1908-05-05 12:34:56.1727,1908-12-25 -1908-05-05 12:34:56.1728,1908-12-25 -1908-05-05 12:34:56.1729,1908-12-25 -1908-05-05 12:34:56.173,1908-12-25 -1908-05-05 12:34:56.1731,1908-12-25 -1908-05-05 12:34:56.1732,1908-12-25 -1908-05-05 12:34:56.1733,1908-12-25 -1908-05-05 12:34:56.1734,1908-12-25 -1908-05-05 12:34:56.1735,1908-12-25 -1908-05-05 12:34:56.1736,1908-12-25 -1908-05-05 12:34:56.1737,1908-12-25 -1908-05-05 12:34:56.1738,1908-12-25 -1908-05-05 12:34:56.1739,1908-12-25 -1908-05-05 12:34:56.174,1908-12-25 -1908-05-05 12:34:56.1741,1908-12-25 -1908-05-05 12:34:56.1742,1908-12-25 -1908-05-05 12:34:56.1743,1908-12-25 -1908-05-05 12:34:56.1744,1908-12-25 -1908-05-05 12:34:56.1745,1908-12-25 -1908-05-05 12:34:56.1746,1908-12-25 -1908-05-05 12:34:56.1747,1908-12-25 -1908-05-05 12:34:56.1748,1908-12-25 -1908-05-05 12:34:56.1749,1908-12-25 -1908-05-05 12:34:56.175,1908-12-25 -1908-05-05 12:34:56.1751,1908-12-25 -1908-05-05 12:34:56.1752,1908-12-25 -1908-05-05 12:34:56.1753,1908-12-25 -1908-05-05 12:34:56.1754,1908-12-25 -1908-05-05 12:34:56.1755,1908-12-25 -1908-05-05 12:34:56.1756,1908-12-25 -1908-05-05 12:34:56.1757,1908-12-25 -1908-05-05 12:34:56.1758,1908-12-25 -1908-05-05 12:34:56.1759,1908-12-25 -1908-05-05 12:34:56.176,1908-12-25 -1908-05-05 12:34:56.1761,1908-12-25 -1908-05-05 12:34:56.1762,1908-12-25 -1908-05-05 12:34:56.1763,1908-12-25 -1908-05-05 12:34:56.1764,1908-12-25 -1908-05-05 12:34:56.1765,1908-12-25 -1908-05-05 12:34:56.1766,1908-12-25 -1908-05-05 12:34:56.1767,1908-12-25 -1908-05-05 12:34:56.1768,1908-12-25 -1908-05-05 12:34:56.1769,1908-12-25 -1908-05-05 12:34:56.177,1908-12-25 -1908-05-05 12:34:56.1771,1908-12-25 -1908-05-05 12:34:56.1772,1908-12-25 -1908-05-05 12:34:56.1773,1908-12-25 -1908-05-05 12:34:56.1774,1908-12-25 -1908-05-05 12:34:56.1775,1908-12-25 -1908-05-05 12:34:56.1776,1908-12-25 -1908-05-05 12:34:56.1777,1908-12-25 -1908-05-05 12:34:56.1778,1908-12-25 -1908-05-05 12:34:56.1779,1908-12-25 -1908-05-05 12:34:56.178,1908-12-25 -1908-05-05 12:34:56.1781,1908-12-25 -1908-05-05 12:34:56.1782,1908-12-25 -1908-05-05 12:34:56.1783,1908-12-25 -1908-05-05 12:34:56.1784,1908-12-25 -1908-05-05 12:34:56.1785,1908-12-25 -1908-05-05 12:34:56.1786,1908-12-25 -1908-05-05 12:34:56.1787,1908-12-25 -1908-05-05 12:34:56.1788,1908-12-25 -1908-05-05 12:34:56.1789,1908-12-25 -1908-05-05 12:34:56.179,1908-12-25 -1908-05-05 12:34:56.1791,1908-12-25 -1908-05-05 12:34:56.1792,1908-12-25 -1908-05-05 12:34:56.1793,1908-12-25 -1908-05-05 12:34:56.1794,1908-12-25 -1908-05-05 12:34:56.1795,1908-12-25 -1908-05-05 12:34:56.1796,1908-12-25 -1908-05-05 12:34:56.1797,1908-12-25 -1908-05-05 12:34:56.1798,1908-12-25 -1908-05-05 12:34:56.1799,1908-12-25 -1908-05-05 12:34:56.18,1908-12-25 -1908-05-05 12:34:56.1801,1908-12-25 -1908-05-05 12:34:56.1802,1908-12-25 -1908-05-05 12:34:56.1803,1908-12-25 -1908-05-05 12:34:56.1804,1908-12-25 -1908-05-05 12:34:56.1805,1908-12-25 -1908-05-05 12:34:56.1806,1908-12-25 -1908-05-05 12:34:56.1807,1908-12-25 -1908-05-05 12:34:56.1808,1908-12-25 -1908-05-05 12:34:56.1809,1908-12-25 -1908-05-05 12:34:56.181,1908-12-25 -1908-05-05 12:34:56.1811,1908-12-25 -1908-05-05 12:34:56.1812,1908-12-25 -1908-05-05 12:34:56.1813,1908-12-25 -1908-05-05 12:34:56.1814,1908-12-25 -1908-05-05 12:34:56.1815,1908-12-25 -1908-05-05 12:34:56.1816,1908-12-25 -1908-05-05 12:34:56.1817,1908-12-25 -1908-05-05 12:34:56.1818,1908-12-25 -1908-05-05 12:34:56.1819,1908-12-25 -1908-05-05 12:34:56.182,1908-12-25 -1908-05-05 12:34:56.1821,1908-12-25 -1908-05-05 12:34:56.1822,1908-12-25 -1908-05-05 12:34:56.1823,1908-12-25 -1908-05-05 12:34:56.1824,1908-12-25 -1908-05-05 12:34:56.1825,1908-12-25 -1908-05-05 12:34:56.1826,1908-12-25 -1908-05-05 12:34:56.1827,1908-12-25 -1908-05-05 12:34:56.1828,1908-12-25 -1908-05-05 12:34:56.1829,1908-12-25 -1908-05-05 12:34:56.183,1908-12-25 -1908-05-05 12:34:56.1831,1908-12-25 -1908-05-05 12:34:56.1832,1908-12-25 -1908-05-05 12:34:56.1833,1908-12-25 -1908-05-05 12:34:56.1834,1908-12-25 -1908-05-05 12:34:56.1835,1908-12-25 -1908-05-05 12:34:56.1836,1908-12-25 -1908-05-05 12:34:56.1837,1908-12-25 -1908-05-05 12:34:56.1838,1908-12-25 -1908-05-05 12:34:56.1839,1908-12-25 -1908-05-05 12:34:56.184,1908-12-25 -1908-05-05 12:34:56.1841,1908-12-25 -1908-05-05 12:34:56.1842,1908-12-25 -1908-05-05 12:34:56.1843,1908-12-25 -1908-05-05 12:34:56.1844,1908-12-25 -1908-05-05 12:34:56.1845,1908-12-25 -1908-05-05 12:34:56.1846,1908-12-25 -1908-05-05 12:34:56.1847,1908-12-25 -1908-05-05 12:34:56.1848,1908-12-25 -1908-05-05 12:34:56.1849,1908-12-25 -1908-05-05 12:34:56.185,1908-12-25 -1908-05-05 12:34:56.1851,1908-12-25 -1908-05-05 12:34:56.1852,1908-12-25 -1908-05-05 12:34:56.1853,1908-12-25 -1908-05-05 12:34:56.1854,1908-12-25 -1908-05-05 12:34:56.1855,1908-12-25 -1908-05-05 12:34:56.1856,1908-12-25 -1908-05-05 12:34:56.1857,1908-12-25 -1908-05-05 12:34:56.1858,1908-12-25 -1908-05-05 12:34:56.1859,1908-12-25 -1908-05-05 12:34:56.186,1908-12-25 -1908-05-05 12:34:56.1861,1908-12-25 -1908-05-05 12:34:56.1862,1908-12-25 -1908-05-05 12:34:56.1863,1908-12-25 -1908-05-05 12:34:56.1864,1908-12-25 -1908-05-05 12:34:56.1865,1908-12-25 -1908-05-05 12:34:56.1866,1908-12-25 -1908-05-05 12:34:56.1867,1908-12-25 -1908-05-05 12:34:56.1868,1908-12-25 -1908-05-05 12:34:56.1869,1908-12-25 -1908-05-05 12:34:56.187,1908-12-25 -1908-05-05 12:34:56.1871,1908-12-25 -1908-05-05 12:34:56.1872,1908-12-25 -1908-05-05 12:34:56.1873,1908-12-25 -1908-05-05 12:34:56.1874,1908-12-25 -1908-05-05 12:34:56.1875,1908-12-25 -1908-05-05 12:34:56.1876,1908-12-25 -1908-05-05 12:34:56.1877,1908-12-25 -1908-05-05 12:34:56.1878,1908-12-25 -1908-05-05 12:34:56.1879,1908-12-25 -1908-05-05 12:34:56.188,1908-12-25 -1908-05-05 12:34:56.1881,1908-12-25 -1908-05-05 12:34:56.1882,1908-12-25 -1908-05-05 12:34:56.1883,1908-12-25 -1908-05-05 12:34:56.1884,1908-12-25 -1908-05-05 12:34:56.1885,1908-12-25 -1908-05-05 12:34:56.1886,1908-12-25 -1908-05-05 12:34:56.1887,1908-12-25 -1908-05-05 12:34:56.1888,1908-12-25 -1908-05-05 12:34:56.1889,1908-12-25 -1908-05-05 12:34:56.189,1908-12-25 -1908-05-05 12:34:56.1891,1908-12-25 -1908-05-05 12:34:56.1892,1908-12-25 -1908-05-05 12:34:56.1893,1908-12-25 -1908-05-05 12:34:56.1894,1908-12-25 -1908-05-05 12:34:56.1895,1908-12-25 -1908-05-05 12:34:56.1896,1908-12-25 -1908-05-05 12:34:56.1897,1908-12-25 -1908-05-05 12:34:56.1898,1908-12-25 -1908-05-05 12:34:56.1899,1908-12-25 -1908-05-05 12:34:56.19,1908-12-25 -1908-05-05 12:34:56.1901,1908-12-25 -1908-05-05 12:34:56.1902,1908-12-25 -1908-05-05 12:34:56.1903,1908-12-25 -1908-05-05 12:34:56.1904,1908-12-25 -1908-05-05 12:34:56.1905,1908-12-25 -1908-05-05 12:34:56.1906,1908-12-25 -1908-05-05 12:34:56.1907,1908-12-25 -1908-05-05 12:34:56.1908,1908-12-25 -1908-05-05 12:34:56.1909,1908-12-25 -1908-05-05 12:34:56.191,1908-12-25 -1908-05-05 12:34:56.1911,1908-12-25 -1908-05-05 12:34:56.1912,1908-12-25 -1908-05-05 12:34:56.1913,1908-12-25 -1908-05-05 12:34:56.1914,1908-12-25 -1908-05-05 12:34:56.1915,1908-12-25 -1908-05-05 12:34:56.1916,1908-12-25 -1908-05-05 12:34:56.1917,1908-12-25 -1908-05-05 12:34:56.1918,1908-12-25 -1908-05-05 12:34:56.1919,1908-12-25 -1908-05-05 12:34:56.192,1908-12-25 -1908-05-05 12:34:56.1921,1908-12-25 -1908-05-05 12:34:56.1922,1908-12-25 -1908-05-05 12:34:56.1923,1908-12-25 -1908-05-05 12:34:56.1924,1908-12-25 -1908-05-05 12:34:56.1925,1908-12-25 -1908-05-05 12:34:56.1926,1908-12-25 -1908-05-05 12:34:56.1927,1908-12-25 -1908-05-05 12:34:56.1928,1908-12-25 -1908-05-05 12:34:56.1929,1908-12-25 -1908-05-05 12:34:56.193,1908-12-25 -1908-05-05 12:34:56.1931,1908-12-25 -1908-05-05 12:34:56.1932,1908-12-25 -1908-05-05 12:34:56.1933,1908-12-25 -1908-05-05 12:34:56.1934,1908-12-25 -1908-05-05 12:34:56.1935,1908-12-25 -1908-05-05 12:34:56.1936,1908-12-25 -1908-05-05 12:34:56.1937,1908-12-25 -1908-05-05 12:34:56.1938,1908-12-25 -1908-05-05 12:34:56.1939,1908-12-25 -1908-05-05 12:34:56.194,1908-12-25 -1908-05-05 12:34:56.1941,1908-12-25 -1908-05-05 12:34:56.1942,1908-12-25 -1908-05-05 12:34:56.1943,1908-12-25 -1908-05-05 12:34:56.1944,1908-12-25 -1908-05-05 12:34:56.1945,1908-12-25 -1908-05-05 12:34:56.1946,1908-12-25 -1908-05-05 12:34:56.1947,1908-12-25 -1908-05-05 12:34:56.1948,1908-12-25 -1908-05-05 12:34:56.1949,1908-12-25 -1908-05-05 12:34:56.195,1908-12-25 -1908-05-05 12:34:56.1951,1908-12-25 -1908-05-05 12:34:56.1952,1908-12-25 -1908-05-05 12:34:56.1953,1908-12-25 -1908-05-05 12:34:56.1954,1908-12-25 -1908-05-05 12:34:56.1955,1908-12-25 -1908-05-05 12:34:56.1956,1908-12-25 -1908-05-05 12:34:56.1957,1908-12-25 -1908-05-05 12:34:56.1958,1908-12-25 -1908-05-05 12:34:56.1959,1908-12-25 -1908-05-05 12:34:56.196,1908-12-25 -1908-05-05 12:34:56.1961,1908-12-25 -1908-05-05 12:34:56.1962,1908-12-25 -1908-05-05 12:34:56.1963,1908-12-25 -1908-05-05 12:34:56.1964,1908-12-25 -1908-05-05 12:34:56.1965,1908-12-25 -1908-05-05 12:34:56.1966,1908-12-25 -1908-05-05 12:34:56.1967,1908-12-25 -1908-05-05 12:34:56.1968,1908-12-25 -1908-05-05 12:34:56.1969,1908-12-25 -1908-05-05 12:34:56.197,1908-12-25 -1908-05-05 12:34:56.1971,1908-12-25 -1908-05-05 12:34:56.1972,1908-12-25 -1908-05-05 12:34:56.1973,1908-12-25 -1908-05-05 12:34:56.1974,1908-12-25 -1908-05-05 12:34:56.1975,1908-12-25 -1908-05-05 12:34:56.1976,1908-12-25 -1908-05-05 12:34:56.1977,1908-12-25 -1908-05-05 12:34:56.1978,1908-12-25 -1908-05-05 12:34:56.1979,1908-12-25 -1908-05-05 12:34:56.198,1908-12-25 -1908-05-05 12:34:56.1981,1908-12-25 -1908-05-05 12:34:56.1982,1908-12-25 -1908-05-05 12:34:56.1983,1908-12-25 -1908-05-05 12:34:56.1984,1908-12-25 -1908-05-05 12:34:56.1985,1908-12-25 -1908-05-05 12:34:56.1986,1908-12-25 -1908-05-05 12:34:56.1987,1908-12-25 -1908-05-05 12:34:56.1988,1908-12-25 -1908-05-05 12:34:56.1989,1908-12-25 -1908-05-05 12:34:56.199,1908-12-25 -1908-05-05 12:34:56.1991,1908-12-25 -1908-05-05 12:34:56.1992,1908-12-25 -1908-05-05 12:34:56.1993,1908-12-25 -1908-05-05 12:34:56.1994,1908-12-25 -1908-05-05 12:34:56.1995,1908-12-25 -1908-05-05 12:34:56.1996,1908-12-25 -1908-05-05 12:34:56.1997,1908-12-25 -1908-05-05 12:34:56.1998,1908-12-25 -1908-05-05 12:34:56.1999,1908-12-25 -1909-05-05 12:34:56.1,1909-12-25 -1909-05-05 12:34:56.1001,1909-12-25 -1909-05-05 12:34:56.1002,1909-12-25 -1909-05-05 12:34:56.1003,1909-12-25 -1909-05-05 12:34:56.1004,1909-12-25 -1909-05-05 12:34:56.1005,1909-12-25 -1909-05-05 12:34:56.1006,1909-12-25 -1909-05-05 12:34:56.1007,1909-12-25 -1909-05-05 12:34:56.1008,1909-12-25 -1909-05-05 12:34:56.1009,1909-12-25 -1909-05-05 12:34:56.101,1909-12-25 -1909-05-05 12:34:56.1011,1909-12-25 -1909-05-05 12:34:56.1012,1909-12-25 -1909-05-05 12:34:56.1013,1909-12-25 -1909-05-05 12:34:56.1014,1909-12-25 -1909-05-05 12:34:56.1015,1909-12-25 -1909-05-05 12:34:56.1016,1909-12-25 -1909-05-05 12:34:56.1017,1909-12-25 -1909-05-05 12:34:56.1018,1909-12-25 -1909-05-05 12:34:56.1019,1909-12-25 -1909-05-05 12:34:56.102,1909-12-25 -1909-05-05 12:34:56.1021,1909-12-25 -1909-05-05 12:34:56.1022,1909-12-25 -1909-05-05 12:34:56.1023,1909-12-25 -1909-05-05 12:34:56.1024,1909-12-25 -1909-05-05 12:34:56.1025,1909-12-25 -1909-05-05 12:34:56.1026,1909-12-25 -1909-05-05 12:34:56.1027,1909-12-25 -1909-05-05 12:34:56.1028,1909-12-25 -1909-05-05 12:34:56.1029,1909-12-25 -1909-05-05 12:34:56.103,1909-12-25 -1909-05-05 12:34:56.1031,1909-12-25 -1909-05-05 12:34:56.1032,1909-12-25 -1909-05-05 12:34:56.1033,1909-12-25 -1909-05-05 12:34:56.1034,1909-12-25 -1909-05-05 12:34:56.1035,1909-12-25 -1909-05-05 12:34:56.1036,1909-12-25 -1909-05-05 12:34:56.1037,1909-12-25 -1909-05-05 12:34:56.1038,1909-12-25 -1909-05-05 12:34:56.1039,1909-12-25 -1909-05-05 12:34:56.104,1909-12-25 -1909-05-05 12:34:56.1041,1909-12-25 -1909-05-05 12:34:56.1042,1909-12-25 -1909-05-05 12:34:56.1043,1909-12-25 -1909-05-05 12:34:56.1044,1909-12-25 -1909-05-05 12:34:56.1045,1909-12-25 -1909-05-05 12:34:56.1046,1909-12-25 -1909-05-05 12:34:56.1047,1909-12-25 -1909-05-05 12:34:56.1048,1909-12-25 -1909-05-05 12:34:56.1049,1909-12-25 -1909-05-05 12:34:56.105,1909-12-25 -1909-05-05 12:34:56.1051,1909-12-25 -1909-05-05 12:34:56.1052,1909-12-25 -1909-05-05 12:34:56.1053,1909-12-25 -1909-05-05 12:34:56.1054,1909-12-25 -1909-05-05 12:34:56.1055,1909-12-25 -1909-05-05 12:34:56.1056,1909-12-25 -1909-05-05 12:34:56.1057,1909-12-25 -1909-05-05 12:34:56.1058,1909-12-25 -1909-05-05 12:34:56.1059,1909-12-25 -1909-05-05 12:34:56.106,1909-12-25 -1909-05-05 12:34:56.1061,1909-12-25 -1909-05-05 12:34:56.1062,1909-12-25 -1909-05-05 12:34:56.1063,1909-12-25 -1909-05-05 12:34:56.1064,1909-12-25 -1909-05-05 12:34:56.1065,1909-12-25 -1909-05-05 12:34:56.1066,1909-12-25 -1909-05-05 12:34:56.1067,1909-12-25 -1909-05-05 12:34:56.1068,1909-12-25 -1909-05-05 12:34:56.1069,1909-12-25 -1909-05-05 12:34:56.107,1909-12-25 -1909-05-05 12:34:56.1071,1909-12-25 -1909-05-05 12:34:56.1072,1909-12-25 -1909-05-05 12:34:56.1073,1909-12-25 -1909-05-05 12:34:56.1074,1909-12-25 -1909-05-05 12:34:56.1075,1909-12-25 -1909-05-05 12:34:56.1076,1909-12-25 -1909-05-05 12:34:56.1077,1909-12-25 -1909-05-05 12:34:56.1078,1909-12-25 -1909-05-05 12:34:56.1079,1909-12-25 -1909-05-05 12:34:56.108,1909-12-25 -1909-05-05 12:34:56.1081,1909-12-25 -1909-05-05 12:34:56.1082,1909-12-25 -1909-05-05 12:34:56.1083,1909-12-25 -1909-05-05 12:34:56.1084,1909-12-25 -1909-05-05 12:34:56.1085,1909-12-25 -1909-05-05 12:34:56.1086,1909-12-25 -1909-05-05 12:34:56.1087,1909-12-25 -1909-05-05 12:34:56.1088,1909-12-25 -1909-05-05 12:34:56.1089,1909-12-25 -1909-05-05 12:34:56.109,1909-12-25 -1909-05-05 12:34:56.1091,1909-12-25 -1909-05-05 12:34:56.1092,1909-12-25 -1909-05-05 12:34:56.1093,1909-12-25 -1909-05-05 12:34:56.1094,1909-12-25 -1909-05-05 12:34:56.1095,1909-12-25 -1909-05-05 12:34:56.1096,1909-12-25 -1909-05-05 12:34:56.1097,1909-12-25 -1909-05-05 12:34:56.1098,1909-12-25 -1909-05-05 12:34:56.1099,1909-12-25 -1909-05-05 12:34:56.11,1909-12-25 -1909-05-05 12:34:56.1101,1909-12-25 -1909-05-05 12:34:56.1102,1909-12-25 -1909-05-05 12:34:56.1103,1909-12-25 -1909-05-05 12:34:56.1104,1909-12-25 -1909-05-05 12:34:56.1105,1909-12-25 -1909-05-05 12:34:56.1106,1909-12-25 -1909-05-05 12:34:56.1107,1909-12-25 -1909-05-05 12:34:56.1108,1909-12-25 -1909-05-05 12:34:56.1109,1909-12-25 -1909-05-05 12:34:56.111,1909-12-25 -1909-05-05 12:34:56.1111,1909-12-25 -1909-05-05 12:34:56.1112,1909-12-25 -1909-05-05 12:34:56.1113,1909-12-25 -1909-05-05 12:34:56.1114,1909-12-25 -1909-05-05 12:34:56.1115,1909-12-25 -1909-05-05 12:34:56.1116,1909-12-25 -1909-05-05 12:34:56.1117,1909-12-25 -1909-05-05 12:34:56.1118,1909-12-25 -1909-05-05 12:34:56.1119,1909-12-25 -1909-05-05 12:34:56.112,1909-12-25 -1909-05-05 12:34:56.1121,1909-12-25 -1909-05-05 12:34:56.1122,1909-12-25 -1909-05-05 12:34:56.1123,1909-12-25 -1909-05-05 12:34:56.1124,1909-12-25 -1909-05-05 12:34:56.1125,1909-12-25 -1909-05-05 12:34:56.1126,1909-12-25 -1909-05-05 12:34:56.1127,1909-12-25 -1909-05-05 12:34:56.1128,1909-12-25 -1909-05-05 12:34:56.1129,1909-12-25 -1909-05-05 12:34:56.113,1909-12-25 -1909-05-05 12:34:56.1131,1909-12-25 -1909-05-05 12:34:56.1132,1909-12-25 -1909-05-05 12:34:56.1133,1909-12-25 -1909-05-05 12:34:56.1134,1909-12-25 -1909-05-05 12:34:56.1135,1909-12-25 -1909-05-05 12:34:56.1136,1909-12-25 -1909-05-05 12:34:56.1137,1909-12-25 -1909-05-05 12:34:56.1138,1909-12-25 -1909-05-05 12:34:56.1139,1909-12-25 -1909-05-05 12:34:56.114,1909-12-25 -1909-05-05 12:34:56.1141,1909-12-25 -1909-05-05 12:34:56.1142,1909-12-25 -1909-05-05 12:34:56.1143,1909-12-25 -1909-05-05 12:34:56.1144,1909-12-25 -1909-05-05 12:34:56.1145,1909-12-25 -1909-05-05 12:34:56.1146,1909-12-25 -1909-05-05 12:34:56.1147,1909-12-25 -1909-05-05 12:34:56.1148,1909-12-25 -1909-05-05 12:34:56.1149,1909-12-25 -1909-05-05 12:34:56.115,1909-12-25 -1909-05-05 12:34:56.1151,1909-12-25 -1909-05-05 12:34:56.1152,1909-12-25 -1909-05-05 12:34:56.1153,1909-12-25 -1909-05-05 12:34:56.1154,1909-12-25 -1909-05-05 12:34:56.1155,1909-12-25 -1909-05-05 12:34:56.1156,1909-12-25 -1909-05-05 12:34:56.1157,1909-12-25 -1909-05-05 12:34:56.1158,1909-12-25 -1909-05-05 12:34:56.1159,1909-12-25 -1909-05-05 12:34:56.116,1909-12-25 -1909-05-05 12:34:56.1161,1909-12-25 -1909-05-05 12:34:56.1162,1909-12-25 -1909-05-05 12:34:56.1163,1909-12-25 -1909-05-05 12:34:56.1164,1909-12-25 -1909-05-05 12:34:56.1165,1909-12-25 -1909-05-05 12:34:56.1166,1909-12-25 -1909-05-05 12:34:56.1167,1909-12-25 -1909-05-05 12:34:56.1168,1909-12-25 -1909-05-05 12:34:56.1169,1909-12-25 -1909-05-05 12:34:56.117,1909-12-25 -1909-05-05 12:34:56.1171,1909-12-25 -1909-05-05 12:34:56.1172,1909-12-25 -1909-05-05 12:34:56.1173,1909-12-25 -1909-05-05 12:34:56.1174,1909-12-25 -1909-05-05 12:34:56.1175,1909-12-25 -1909-05-05 12:34:56.1176,1909-12-25 -1909-05-05 12:34:56.1177,1909-12-25 -1909-05-05 12:34:56.1178,1909-12-25 -1909-05-05 12:34:56.1179,1909-12-25 -1909-05-05 12:34:56.118,1909-12-25 -1909-05-05 12:34:56.1181,1909-12-25 -1909-05-05 12:34:56.1182,1909-12-25 -1909-05-05 12:34:56.1183,1909-12-25 -1909-05-05 12:34:56.1184,1909-12-25 -1909-05-05 12:34:56.1185,1909-12-25 -1909-05-05 12:34:56.1186,1909-12-25 -1909-05-05 12:34:56.1187,1909-12-25 -1909-05-05 12:34:56.1188,1909-12-25 -1909-05-05 12:34:56.1189,1909-12-25 -1909-05-05 12:34:56.119,1909-12-25 -1909-05-05 12:34:56.1191,1909-12-25 -1909-05-05 12:34:56.1192,1909-12-25 -1909-05-05 12:34:56.1193,1909-12-25 -1909-05-05 12:34:56.1194,1909-12-25 -1909-05-05 12:34:56.1195,1909-12-25 -1909-05-05 12:34:56.1196,1909-12-25 -1909-05-05 12:34:56.1197,1909-12-25 -1909-05-05 12:34:56.1198,1909-12-25 -1909-05-05 12:34:56.1199,1909-12-25 -1909-05-05 12:34:56.12,1909-12-25 -1909-05-05 12:34:56.1201,1909-12-25 -1909-05-05 12:34:56.1202,1909-12-25 -1909-05-05 12:34:56.1203,1909-12-25 -1909-05-05 12:34:56.1204,1909-12-25 -1909-05-05 12:34:56.1205,1909-12-25 -1909-05-05 12:34:56.1206,1909-12-25 -1909-05-05 12:34:56.1207,1909-12-25 -1909-05-05 12:34:56.1208,1909-12-25 -1909-05-05 12:34:56.1209,1909-12-25 -1909-05-05 12:34:56.121,1909-12-25 -1909-05-05 12:34:56.1211,1909-12-25 -1909-05-05 12:34:56.1212,1909-12-25 -1909-05-05 12:34:56.1213,1909-12-25 -1909-05-05 12:34:56.1214,1909-12-25 -1909-05-05 12:34:56.1215,1909-12-25 -1909-05-05 12:34:56.1216,1909-12-25 -1909-05-05 12:34:56.1217,1909-12-25 -1909-05-05 12:34:56.1218,1909-12-25 -1909-05-05 12:34:56.1219,1909-12-25 -1909-05-05 12:34:56.122,1909-12-25 -1909-05-05 12:34:56.1221,1909-12-25 -1909-05-05 12:34:56.1222,1909-12-25 -1909-05-05 12:34:56.1223,1909-12-25 -1909-05-05 12:34:56.1224,1909-12-25 -1909-05-05 12:34:56.1225,1909-12-25 -1909-05-05 12:34:56.1226,1909-12-25 -1909-05-05 12:34:56.1227,1909-12-25 -1909-05-05 12:34:56.1228,1909-12-25 -1909-05-05 12:34:56.1229,1909-12-25 -1909-05-05 12:34:56.123,1909-12-25 -1909-05-05 12:34:56.1231,1909-12-25 -1909-05-05 12:34:56.1232,1909-12-25 -1909-05-05 12:34:56.1233,1909-12-25 -1909-05-05 12:34:56.1234,1909-12-25 -1909-05-05 12:34:56.1235,1909-12-25 -1909-05-05 12:34:56.1236,1909-12-25 -1909-05-05 12:34:56.1237,1909-12-25 -1909-05-05 12:34:56.1238,1909-12-25 -1909-05-05 12:34:56.1239,1909-12-25 -1909-05-05 12:34:56.124,1909-12-25 -1909-05-05 12:34:56.1241,1909-12-25 -1909-05-05 12:34:56.1242,1909-12-25 -1909-05-05 12:34:56.1243,1909-12-25 -1909-05-05 12:34:56.1244,1909-12-25 -1909-05-05 12:34:56.1245,1909-12-25 -1909-05-05 12:34:56.1246,1909-12-25 -1909-05-05 12:34:56.1247,1909-12-25 -1909-05-05 12:34:56.1248,1909-12-25 -1909-05-05 12:34:56.1249,1909-12-25 -1909-05-05 12:34:56.125,1909-12-25 -1909-05-05 12:34:56.1251,1909-12-25 -1909-05-05 12:34:56.1252,1909-12-25 -1909-05-05 12:34:56.1253,1909-12-25 -1909-05-05 12:34:56.1254,1909-12-25 -1909-05-05 12:34:56.1255,1909-12-25 -1909-05-05 12:34:56.1256,1909-12-25 -1909-05-05 12:34:56.1257,1909-12-25 -1909-05-05 12:34:56.1258,1909-12-25 -1909-05-05 12:34:56.1259,1909-12-25 -1909-05-05 12:34:56.126,1909-12-25 -1909-05-05 12:34:56.1261,1909-12-25 -1909-05-05 12:34:56.1262,1909-12-25 -1909-05-05 12:34:56.1263,1909-12-25 -1909-05-05 12:34:56.1264,1909-12-25 -1909-05-05 12:34:56.1265,1909-12-25 -1909-05-05 12:34:56.1266,1909-12-25 -1909-05-05 12:34:56.1267,1909-12-25 -1909-05-05 12:34:56.1268,1909-12-25 -1909-05-05 12:34:56.1269,1909-12-25 -1909-05-05 12:34:56.127,1909-12-25 -1909-05-05 12:34:56.1271,1909-12-25 -1909-05-05 12:34:56.1272,1909-12-25 -1909-05-05 12:34:56.1273,1909-12-25 -1909-05-05 12:34:56.1274,1909-12-25 -1909-05-05 12:34:56.1275,1909-12-25 -1909-05-05 12:34:56.1276,1909-12-25 -1909-05-05 12:34:56.1277,1909-12-25 -1909-05-05 12:34:56.1278,1909-12-25 -1909-05-05 12:34:56.1279,1909-12-25 -1909-05-05 12:34:56.128,1909-12-25 -1909-05-05 12:34:56.1281,1909-12-25 -1909-05-05 12:34:56.1282,1909-12-25 -1909-05-05 12:34:56.1283,1909-12-25 -1909-05-05 12:34:56.1284,1909-12-25 -1909-05-05 12:34:56.1285,1909-12-25 -1909-05-05 12:34:56.1286,1909-12-25 -1909-05-05 12:34:56.1287,1909-12-25 -1909-05-05 12:34:56.1288,1909-12-25 -1909-05-05 12:34:56.1289,1909-12-25 -1909-05-05 12:34:56.129,1909-12-25 -1909-05-05 12:34:56.1291,1909-12-25 -1909-05-05 12:34:56.1292,1909-12-25 -1909-05-05 12:34:56.1293,1909-12-25 -1909-05-05 12:34:56.1294,1909-12-25 -1909-05-05 12:34:56.1295,1909-12-25 -1909-05-05 12:34:56.1296,1909-12-25 -1909-05-05 12:34:56.1297,1909-12-25 -1909-05-05 12:34:56.1298,1909-12-25 -1909-05-05 12:34:56.1299,1909-12-25 -1909-05-05 12:34:56.13,1909-12-25 -1909-05-05 12:34:56.1301,1909-12-25 -1909-05-05 12:34:56.1302,1909-12-25 -1909-05-05 12:34:56.1303,1909-12-25 -1909-05-05 12:34:56.1304,1909-12-25 -1909-05-05 12:34:56.1305,1909-12-25 -1909-05-05 12:34:56.1306,1909-12-25 -1909-05-05 12:34:56.1307,1909-12-25 -1909-05-05 12:34:56.1308,1909-12-25 -1909-05-05 12:34:56.1309,1909-12-25 -1909-05-05 12:34:56.131,1909-12-25 -1909-05-05 12:34:56.1311,1909-12-25 -1909-05-05 12:34:56.1312,1909-12-25 -1909-05-05 12:34:56.1313,1909-12-25 -1909-05-05 12:34:56.1314,1909-12-25 -1909-05-05 12:34:56.1315,1909-12-25 -1909-05-05 12:34:56.1316,1909-12-25 -1909-05-05 12:34:56.1317,1909-12-25 -1909-05-05 12:34:56.1318,1909-12-25 -1909-05-05 12:34:56.1319,1909-12-25 -1909-05-05 12:34:56.132,1909-12-25 -1909-05-05 12:34:56.1321,1909-12-25 -1909-05-05 12:34:56.1322,1909-12-25 -1909-05-05 12:34:56.1323,1909-12-25 -1909-05-05 12:34:56.1324,1909-12-25 -1909-05-05 12:34:56.1325,1909-12-25 -1909-05-05 12:34:56.1326,1909-12-25 -1909-05-05 12:34:56.1327,1909-12-25 -1909-05-05 12:34:56.1328,1909-12-25 -1909-05-05 12:34:56.1329,1909-12-25 -1909-05-05 12:34:56.133,1909-12-25 -1909-05-05 12:34:56.1331,1909-12-25 -1909-05-05 12:34:56.1332,1909-12-25 -1909-05-05 12:34:56.1333,1909-12-25 -1909-05-05 12:34:56.1334,1909-12-25 -1909-05-05 12:34:56.1335,1909-12-25 -1909-05-05 12:34:56.1336,1909-12-25 -1909-05-05 12:34:56.1337,1909-12-25 -1909-05-05 12:34:56.1338,1909-12-25 -1909-05-05 12:34:56.1339,1909-12-25 -1909-05-05 12:34:56.134,1909-12-25 -1909-05-05 12:34:56.1341,1909-12-25 -1909-05-05 12:34:56.1342,1909-12-25 -1909-05-05 12:34:56.1343,1909-12-25 -1909-05-05 12:34:56.1344,1909-12-25 -1909-05-05 12:34:56.1345,1909-12-25 -1909-05-05 12:34:56.1346,1909-12-25 -1909-05-05 12:34:56.1347,1909-12-25 -1909-05-05 12:34:56.1348,1909-12-25 -1909-05-05 12:34:56.1349,1909-12-25 -1909-05-05 12:34:56.135,1909-12-25 -1909-05-05 12:34:56.1351,1909-12-25 -1909-05-05 12:34:56.1352,1909-12-25 -1909-05-05 12:34:56.1353,1909-12-25 -1909-05-05 12:34:56.1354,1909-12-25 -1909-05-05 12:34:56.1355,1909-12-25 -1909-05-05 12:34:56.1356,1909-12-25 -1909-05-05 12:34:56.1357,1909-12-25 -1909-05-05 12:34:56.1358,1909-12-25 -1909-05-05 12:34:56.1359,1909-12-25 -1909-05-05 12:34:56.136,1909-12-25 -1909-05-05 12:34:56.1361,1909-12-25 -1909-05-05 12:34:56.1362,1909-12-25 -1909-05-05 12:34:56.1363,1909-12-25 -1909-05-05 12:34:56.1364,1909-12-25 -1909-05-05 12:34:56.1365,1909-12-25 -1909-05-05 12:34:56.1366,1909-12-25 -1909-05-05 12:34:56.1367,1909-12-25 -1909-05-05 12:34:56.1368,1909-12-25 -1909-05-05 12:34:56.1369,1909-12-25 -1909-05-05 12:34:56.137,1909-12-25 -1909-05-05 12:34:56.1371,1909-12-25 -1909-05-05 12:34:56.1372,1909-12-25 -1909-05-05 12:34:56.1373,1909-12-25 -1909-05-05 12:34:56.1374,1909-12-25 -1909-05-05 12:34:56.1375,1909-12-25 -1909-05-05 12:34:56.1376,1909-12-25 -1909-05-05 12:34:56.1377,1909-12-25 -1909-05-05 12:34:56.1378,1909-12-25 -1909-05-05 12:34:56.1379,1909-12-25 -1909-05-05 12:34:56.138,1909-12-25 -1909-05-05 12:34:56.1381,1909-12-25 -1909-05-05 12:34:56.1382,1909-12-25 -1909-05-05 12:34:56.1383,1909-12-25 -1909-05-05 12:34:56.1384,1909-12-25 -1909-05-05 12:34:56.1385,1909-12-25 -1909-05-05 12:34:56.1386,1909-12-25 -1909-05-05 12:34:56.1387,1909-12-25 -1909-05-05 12:34:56.1388,1909-12-25 -1909-05-05 12:34:56.1389,1909-12-25 -1909-05-05 12:34:56.139,1909-12-25 -1909-05-05 12:34:56.1391,1909-12-25 -1909-05-05 12:34:56.1392,1909-12-25 -1909-05-05 12:34:56.1393,1909-12-25 -1909-05-05 12:34:56.1394,1909-12-25 -1909-05-05 12:34:56.1395,1909-12-25 -1909-05-05 12:34:56.1396,1909-12-25 -1909-05-05 12:34:56.1397,1909-12-25 -1909-05-05 12:34:56.1398,1909-12-25 -1909-05-05 12:34:56.1399,1909-12-25 -1909-05-05 12:34:56.14,1909-12-25 -1909-05-05 12:34:56.1401,1909-12-25 -1909-05-05 12:34:56.1402,1909-12-25 -1909-05-05 12:34:56.1403,1909-12-25 -1909-05-05 12:34:56.1404,1909-12-25 -1909-05-05 12:34:56.1405,1909-12-25 -1909-05-05 12:34:56.1406,1909-12-25 -1909-05-05 12:34:56.1407,1909-12-25 -1909-05-05 12:34:56.1408,1909-12-25 -1909-05-05 12:34:56.1409,1909-12-25 -1909-05-05 12:34:56.141,1909-12-25 -1909-05-05 12:34:56.1411,1909-12-25 -1909-05-05 12:34:56.1412,1909-12-25 -1909-05-05 12:34:56.1413,1909-12-25 -1909-05-05 12:34:56.1414,1909-12-25 -1909-05-05 12:34:56.1415,1909-12-25 -1909-05-05 12:34:56.1416,1909-12-25 -1909-05-05 12:34:56.1417,1909-12-25 -1909-05-05 12:34:56.1418,1909-12-25 -1909-05-05 12:34:56.1419,1909-12-25 -1909-05-05 12:34:56.142,1909-12-25 -1909-05-05 12:34:56.1421,1909-12-25 -1909-05-05 12:34:56.1422,1909-12-25 -1909-05-05 12:34:56.1423,1909-12-25 -1909-05-05 12:34:56.1424,1909-12-25 -1909-05-05 12:34:56.1425,1909-12-25 -1909-05-05 12:34:56.1426,1909-12-25 -1909-05-05 12:34:56.1427,1909-12-25 -1909-05-05 12:34:56.1428,1909-12-25 -1909-05-05 12:34:56.1429,1909-12-25 -1909-05-05 12:34:56.143,1909-12-25 -1909-05-05 12:34:56.1431,1909-12-25 -1909-05-05 12:34:56.1432,1909-12-25 -1909-05-05 12:34:56.1433,1909-12-25 -1909-05-05 12:34:56.1434,1909-12-25 -1909-05-05 12:34:56.1435,1909-12-25 -1909-05-05 12:34:56.1436,1909-12-25 -1909-05-05 12:34:56.1437,1909-12-25 -1909-05-05 12:34:56.1438,1909-12-25 -1909-05-05 12:34:56.1439,1909-12-25 -1909-05-05 12:34:56.144,1909-12-25 -1909-05-05 12:34:56.1441,1909-12-25 -1909-05-05 12:34:56.1442,1909-12-25 -1909-05-05 12:34:56.1443,1909-12-25 -1909-05-05 12:34:56.1444,1909-12-25 -1909-05-05 12:34:56.1445,1909-12-25 -1909-05-05 12:34:56.1446,1909-12-25 -1909-05-05 12:34:56.1447,1909-12-25 -1909-05-05 12:34:56.1448,1909-12-25 -1909-05-05 12:34:56.1449,1909-12-25 -1909-05-05 12:34:56.145,1909-12-25 -1909-05-05 12:34:56.1451,1909-12-25 -1909-05-05 12:34:56.1452,1909-12-25 -1909-05-05 12:34:56.1453,1909-12-25 -1909-05-05 12:34:56.1454,1909-12-25 -1909-05-05 12:34:56.1455,1909-12-25 -1909-05-05 12:34:56.1456,1909-12-25 -1909-05-05 12:34:56.1457,1909-12-25 -1909-05-05 12:34:56.1458,1909-12-25 -1909-05-05 12:34:56.1459,1909-12-25 -1909-05-05 12:34:56.146,1909-12-25 -1909-05-05 12:34:56.1461,1909-12-25 -1909-05-05 12:34:56.1462,1909-12-25 -1909-05-05 12:34:56.1463,1909-12-25 -1909-05-05 12:34:56.1464,1909-12-25 -1909-05-05 12:34:56.1465,1909-12-25 -1909-05-05 12:34:56.1466,1909-12-25 -1909-05-05 12:34:56.1467,1909-12-25 -1909-05-05 12:34:56.1468,1909-12-25 -1909-05-05 12:34:56.1469,1909-12-25 -1909-05-05 12:34:56.147,1909-12-25 -1909-05-05 12:34:56.1471,1909-12-25 -1909-05-05 12:34:56.1472,1909-12-25 -1909-05-05 12:34:56.1473,1909-12-25 -1909-05-05 12:34:56.1474,1909-12-25 -1909-05-05 12:34:56.1475,1909-12-25 -1909-05-05 12:34:56.1476,1909-12-25 -1909-05-05 12:34:56.1477,1909-12-25 -1909-05-05 12:34:56.1478,1909-12-25 -1909-05-05 12:34:56.1479,1909-12-25 -1909-05-05 12:34:56.148,1909-12-25 -1909-05-05 12:34:56.1481,1909-12-25 -1909-05-05 12:34:56.1482,1909-12-25 -1909-05-05 12:34:56.1483,1909-12-25 -1909-05-05 12:34:56.1484,1909-12-25 -1909-05-05 12:34:56.1485,1909-12-25 -1909-05-05 12:34:56.1486,1909-12-25 -1909-05-05 12:34:56.1487,1909-12-25 -1909-05-05 12:34:56.1488,1909-12-25 -1909-05-05 12:34:56.1489,1909-12-25 -1909-05-05 12:34:56.149,1909-12-25 -1909-05-05 12:34:56.1491,1909-12-25 -1909-05-05 12:34:56.1492,1909-12-25 -1909-05-05 12:34:56.1493,1909-12-25 -1909-05-05 12:34:56.1494,1909-12-25 -1909-05-05 12:34:56.1495,1909-12-25 -1909-05-05 12:34:56.1496,1909-12-25 -1909-05-05 12:34:56.1497,1909-12-25 -1909-05-05 12:34:56.1498,1909-12-25 -1909-05-05 12:34:56.1499,1909-12-25 -1909-05-05 12:34:56.15,1909-12-25 -1909-05-05 12:34:56.1501,1909-12-25 -1909-05-05 12:34:56.1502,1909-12-25 -1909-05-05 12:34:56.1503,1909-12-25 -1909-05-05 12:34:56.1504,1909-12-25 -1909-05-05 12:34:56.1505,1909-12-25 -1909-05-05 12:34:56.1506,1909-12-25 -1909-05-05 12:34:56.1507,1909-12-25 -1909-05-05 12:34:56.1508,1909-12-25 -1909-05-05 12:34:56.1509,1909-12-25 -1909-05-05 12:34:56.151,1909-12-25 -1909-05-05 12:34:56.1511,1909-12-25 -1909-05-05 12:34:56.1512,1909-12-25 -1909-05-05 12:34:56.1513,1909-12-25 -1909-05-05 12:34:56.1514,1909-12-25 -1909-05-05 12:34:56.1515,1909-12-25 -1909-05-05 12:34:56.1516,1909-12-25 -1909-05-05 12:34:56.1517,1909-12-25 -1909-05-05 12:34:56.1518,1909-12-25 -1909-05-05 12:34:56.1519,1909-12-25 -1909-05-05 12:34:56.152,1909-12-25 -1909-05-05 12:34:56.1521,1909-12-25 -1909-05-05 12:34:56.1522,1909-12-25 -1909-05-05 12:34:56.1523,1909-12-25 -1909-05-05 12:34:56.1524,1909-12-25 -1909-05-05 12:34:56.1525,1909-12-25 -1909-05-05 12:34:56.1526,1909-12-25 -1909-05-05 12:34:56.1527,1909-12-25 -1909-05-05 12:34:56.1528,1909-12-25 -1909-05-05 12:34:56.1529,1909-12-25 -1909-05-05 12:34:56.153,1909-12-25 -1909-05-05 12:34:56.1531,1909-12-25 -1909-05-05 12:34:56.1532,1909-12-25 -1909-05-05 12:34:56.1533,1909-12-25 -1909-05-05 12:34:56.1534,1909-12-25 -1909-05-05 12:34:56.1535,1909-12-25 -1909-05-05 12:34:56.1536,1909-12-25 -1909-05-05 12:34:56.1537,1909-12-25 -1909-05-05 12:34:56.1538,1909-12-25 -1909-05-05 12:34:56.1539,1909-12-25 -1909-05-05 12:34:56.154,1909-12-25 -1909-05-05 12:34:56.1541,1909-12-25 -1909-05-05 12:34:56.1542,1909-12-25 -1909-05-05 12:34:56.1543,1909-12-25 -1909-05-05 12:34:56.1544,1909-12-25 -1909-05-05 12:34:56.1545,1909-12-25 -1909-05-05 12:34:56.1546,1909-12-25 -1909-05-05 12:34:56.1547,1909-12-25 -1909-05-05 12:34:56.1548,1909-12-25 -1909-05-05 12:34:56.1549,1909-12-25 -1909-05-05 12:34:56.155,1909-12-25 -1909-05-05 12:34:56.1551,1909-12-25 -1909-05-05 12:34:56.1552,1909-12-25 -1909-05-05 12:34:56.1553,1909-12-25 -1909-05-05 12:34:56.1554,1909-12-25 -1909-05-05 12:34:56.1555,1909-12-25 -1909-05-05 12:34:56.1556,1909-12-25 -1909-05-05 12:34:56.1557,1909-12-25 -1909-05-05 12:34:56.1558,1909-12-25 -1909-05-05 12:34:56.1559,1909-12-25 -1909-05-05 12:34:56.156,1909-12-25 -1909-05-05 12:34:56.1561,1909-12-25 -1909-05-05 12:34:56.1562,1909-12-25 -1909-05-05 12:34:56.1563,1909-12-25 -1909-05-05 12:34:56.1564,1909-12-25 -1909-05-05 12:34:56.1565,1909-12-25 -1909-05-05 12:34:56.1566,1909-12-25 -1909-05-05 12:34:56.1567,1909-12-25 -1909-05-05 12:34:56.1568,1909-12-25 -1909-05-05 12:34:56.1569,1909-12-25 -1909-05-05 12:34:56.157,1909-12-25 -1909-05-05 12:34:56.1571,1909-12-25 -1909-05-05 12:34:56.1572,1909-12-25 -1909-05-05 12:34:56.1573,1909-12-25 -1909-05-05 12:34:56.1574,1909-12-25 -1909-05-05 12:34:56.1575,1909-12-25 -1909-05-05 12:34:56.1576,1909-12-25 -1909-05-05 12:34:56.1577,1909-12-25 -1909-05-05 12:34:56.1578,1909-12-25 -1909-05-05 12:34:56.1579,1909-12-25 -1909-05-05 12:34:56.158,1909-12-25 -1909-05-05 12:34:56.1581,1909-12-25 -1909-05-05 12:34:56.1582,1909-12-25 -1909-05-05 12:34:56.1583,1909-12-25 -1909-05-05 12:34:56.1584,1909-12-25 -1909-05-05 12:34:56.1585,1909-12-25 -1909-05-05 12:34:56.1586,1909-12-25 -1909-05-05 12:34:56.1587,1909-12-25 -1909-05-05 12:34:56.1588,1909-12-25 -1909-05-05 12:34:56.1589,1909-12-25 -1909-05-05 12:34:56.159,1909-12-25 -1909-05-05 12:34:56.1591,1909-12-25 -1909-05-05 12:34:56.1592,1909-12-25 -1909-05-05 12:34:56.1593,1909-12-25 -1909-05-05 12:34:56.1594,1909-12-25 -1909-05-05 12:34:56.1595,1909-12-25 -1909-05-05 12:34:56.1596,1909-12-25 -1909-05-05 12:34:56.1597,1909-12-25 -1909-05-05 12:34:56.1598,1909-12-25 -1909-05-05 12:34:56.1599,1909-12-25 -1909-05-05 12:34:56.16,1909-12-25 -1909-05-05 12:34:56.1601,1909-12-25 -1909-05-05 12:34:56.1602,1909-12-25 -1909-05-05 12:34:56.1603,1909-12-25 -1909-05-05 12:34:56.1604,1909-12-25 -1909-05-05 12:34:56.1605,1909-12-25 -1909-05-05 12:34:56.1606,1909-12-25 -1909-05-05 12:34:56.1607,1909-12-25 -1909-05-05 12:34:56.1608,1909-12-25 -1909-05-05 12:34:56.1609,1909-12-25 -1909-05-05 12:34:56.161,1909-12-25 -1909-05-05 12:34:56.1611,1909-12-25 -1909-05-05 12:34:56.1612,1909-12-25 -1909-05-05 12:34:56.1613,1909-12-25 -1909-05-05 12:34:56.1614,1909-12-25 -1909-05-05 12:34:56.1615,1909-12-25 -1909-05-05 12:34:56.1616,1909-12-25 -1909-05-05 12:34:56.1617,1909-12-25 -1909-05-05 12:34:56.1618,1909-12-25 -1909-05-05 12:34:56.1619,1909-12-25 -1909-05-05 12:34:56.162,1909-12-25 -1909-05-05 12:34:56.1621,1909-12-25 -1909-05-05 12:34:56.1622,1909-12-25 -1909-05-05 12:34:56.1623,1909-12-25 -1909-05-05 12:34:56.1624,1909-12-25 -1909-05-05 12:34:56.1625,1909-12-25 -1909-05-05 12:34:56.1626,1909-12-25 -1909-05-05 12:34:56.1627,1909-12-25 -1909-05-05 12:34:56.1628,1909-12-25 -1909-05-05 12:34:56.1629,1909-12-25 -1909-05-05 12:34:56.163,1909-12-25 -1909-05-05 12:34:56.1631,1909-12-25 -1909-05-05 12:34:56.1632,1909-12-25 -1909-05-05 12:34:56.1633,1909-12-25 -1909-05-05 12:34:56.1634,1909-12-25 -1909-05-05 12:34:56.1635,1909-12-25 -1909-05-05 12:34:56.1636,1909-12-25 -1909-05-05 12:34:56.1637,1909-12-25 -1909-05-05 12:34:56.1638,1909-12-25 -1909-05-05 12:34:56.1639,1909-12-25 -1909-05-05 12:34:56.164,1909-12-25 -1909-05-05 12:34:56.1641,1909-12-25 -1909-05-05 12:34:56.1642,1909-12-25 -1909-05-05 12:34:56.1643,1909-12-25 -1909-05-05 12:34:56.1644,1909-12-25 -1909-05-05 12:34:56.1645,1909-12-25 -1909-05-05 12:34:56.1646,1909-12-25 -1909-05-05 12:34:56.1647,1909-12-25 -1909-05-05 12:34:56.1648,1909-12-25 -1909-05-05 12:34:56.1649,1909-12-25 -1909-05-05 12:34:56.165,1909-12-25 -1909-05-05 12:34:56.1651,1909-12-25 -1909-05-05 12:34:56.1652,1909-12-25 -1909-05-05 12:34:56.1653,1909-12-25 -1909-05-05 12:34:56.1654,1909-12-25 -1909-05-05 12:34:56.1655,1909-12-25 -1909-05-05 12:34:56.1656,1909-12-25 -1909-05-05 12:34:56.1657,1909-12-25 -1909-05-05 12:34:56.1658,1909-12-25 -1909-05-05 12:34:56.1659,1909-12-25 -1909-05-05 12:34:56.166,1909-12-25 -1909-05-05 12:34:56.1661,1909-12-25 -1909-05-05 12:34:56.1662,1909-12-25 -1909-05-05 12:34:56.1663,1909-12-25 -1909-05-05 12:34:56.1664,1909-12-25 -1909-05-05 12:34:56.1665,1909-12-25 -1909-05-05 12:34:56.1666,1909-12-25 -1909-05-05 12:34:56.1667,1909-12-25 -1909-05-05 12:34:56.1668,1909-12-25 -1909-05-05 12:34:56.1669,1909-12-25 -1909-05-05 12:34:56.167,1909-12-25 -1909-05-05 12:34:56.1671,1909-12-25 -1909-05-05 12:34:56.1672,1909-12-25 -1909-05-05 12:34:56.1673,1909-12-25 -1909-05-05 12:34:56.1674,1909-12-25 -1909-05-05 12:34:56.1675,1909-12-25 -1909-05-05 12:34:56.1676,1909-12-25 -1909-05-05 12:34:56.1677,1909-12-25 -1909-05-05 12:34:56.1678,1909-12-25 -1909-05-05 12:34:56.1679,1909-12-25 -1909-05-05 12:34:56.168,1909-12-25 -1909-05-05 12:34:56.1681,1909-12-25 -1909-05-05 12:34:56.1682,1909-12-25 -1909-05-05 12:34:56.1683,1909-12-25 -1909-05-05 12:34:56.1684,1909-12-25 -1909-05-05 12:34:56.1685,1909-12-25 -1909-05-05 12:34:56.1686,1909-12-25 -1909-05-05 12:34:56.1687,1909-12-25 -1909-05-05 12:34:56.1688,1909-12-25 -1909-05-05 12:34:56.1689,1909-12-25 -1909-05-05 12:34:56.169,1909-12-25 -1909-05-05 12:34:56.1691,1909-12-25 -1909-05-05 12:34:56.1692,1909-12-25 -1909-05-05 12:34:56.1693,1909-12-25 -1909-05-05 12:34:56.1694,1909-12-25 -1909-05-05 12:34:56.1695,1909-12-25 -1909-05-05 12:34:56.1696,1909-12-25 -1909-05-05 12:34:56.1697,1909-12-25 -1909-05-05 12:34:56.1698,1909-12-25 -1909-05-05 12:34:56.1699,1909-12-25 -1909-05-05 12:34:56.17,1909-12-25 -1909-05-05 12:34:56.1701,1909-12-25 -1909-05-05 12:34:56.1702,1909-12-25 -1909-05-05 12:34:56.1703,1909-12-25 -1909-05-05 12:34:56.1704,1909-12-25 -1909-05-05 12:34:56.1705,1909-12-25 -1909-05-05 12:34:56.1706,1909-12-25 -1909-05-05 12:34:56.1707,1909-12-25 -1909-05-05 12:34:56.1708,1909-12-25 -1909-05-05 12:34:56.1709,1909-12-25 -1909-05-05 12:34:56.171,1909-12-25 -1909-05-05 12:34:56.1711,1909-12-25 -1909-05-05 12:34:56.1712,1909-12-25 -1909-05-05 12:34:56.1713,1909-12-25 -1909-05-05 12:34:56.1714,1909-12-25 -1909-05-05 12:34:56.1715,1909-12-25 -1909-05-05 12:34:56.1716,1909-12-25 -1909-05-05 12:34:56.1717,1909-12-25 -1909-05-05 12:34:56.1718,1909-12-25 -1909-05-05 12:34:56.1719,1909-12-25 -1909-05-05 12:34:56.172,1909-12-25 -1909-05-05 12:34:56.1721,1909-12-25 -1909-05-05 12:34:56.1722,1909-12-25 -1909-05-05 12:34:56.1723,1909-12-25 -1909-05-05 12:34:56.1724,1909-12-25 -1909-05-05 12:34:56.1725,1909-12-25 -1909-05-05 12:34:56.1726,1909-12-25 -1909-05-05 12:34:56.1727,1909-12-25 -1909-05-05 12:34:56.1728,1909-12-25 -1909-05-05 12:34:56.1729,1909-12-25 -1909-05-05 12:34:56.173,1909-12-25 -1909-05-05 12:34:56.1731,1909-12-25 -1909-05-05 12:34:56.1732,1909-12-25 -1909-05-05 12:34:56.1733,1909-12-25 -1909-05-05 12:34:56.1734,1909-12-25 -1909-05-05 12:34:56.1735,1909-12-25 -1909-05-05 12:34:56.1736,1909-12-25 -1909-05-05 12:34:56.1737,1909-12-25 -1909-05-05 12:34:56.1738,1909-12-25 -1909-05-05 12:34:56.1739,1909-12-25 -1909-05-05 12:34:56.174,1909-12-25 -1909-05-05 12:34:56.1741,1909-12-25 -1909-05-05 12:34:56.1742,1909-12-25 -1909-05-05 12:34:56.1743,1909-12-25 -1909-05-05 12:34:56.1744,1909-12-25 -1909-05-05 12:34:56.1745,1909-12-25 -1909-05-05 12:34:56.1746,1909-12-25 -1909-05-05 12:34:56.1747,1909-12-25 -1909-05-05 12:34:56.1748,1909-12-25 -1909-05-05 12:34:56.1749,1909-12-25 -1909-05-05 12:34:56.175,1909-12-25 -1909-05-05 12:34:56.1751,1909-12-25 -1909-05-05 12:34:56.1752,1909-12-25 -1909-05-05 12:34:56.1753,1909-12-25 -1909-05-05 12:34:56.1754,1909-12-25 -1909-05-05 12:34:56.1755,1909-12-25 -1909-05-05 12:34:56.1756,1909-12-25 -1909-05-05 12:34:56.1757,1909-12-25 -1909-05-05 12:34:56.1758,1909-12-25 -1909-05-05 12:34:56.1759,1909-12-25 -1909-05-05 12:34:56.176,1909-12-25 -1909-05-05 12:34:56.1761,1909-12-25 -1909-05-05 12:34:56.1762,1909-12-25 -1909-05-05 12:34:56.1763,1909-12-25 -1909-05-05 12:34:56.1764,1909-12-25 -1909-05-05 12:34:56.1765,1909-12-25 -1909-05-05 12:34:56.1766,1909-12-25 -1909-05-05 12:34:56.1767,1909-12-25 -1909-05-05 12:34:56.1768,1909-12-25 -1909-05-05 12:34:56.1769,1909-12-25 -1909-05-05 12:34:56.177,1909-12-25 -1909-05-05 12:34:56.1771,1909-12-25 -1909-05-05 12:34:56.1772,1909-12-25 -1909-05-05 12:34:56.1773,1909-12-25 -1909-05-05 12:34:56.1774,1909-12-25 -1909-05-05 12:34:56.1775,1909-12-25 -1909-05-05 12:34:56.1776,1909-12-25 -1909-05-05 12:34:56.1777,1909-12-25 -1909-05-05 12:34:56.1778,1909-12-25 -1909-05-05 12:34:56.1779,1909-12-25 -1909-05-05 12:34:56.178,1909-12-25 -1909-05-05 12:34:56.1781,1909-12-25 -1909-05-05 12:34:56.1782,1909-12-25 -1909-05-05 12:34:56.1783,1909-12-25 -1909-05-05 12:34:56.1784,1909-12-25 -1909-05-05 12:34:56.1785,1909-12-25 -1909-05-05 12:34:56.1786,1909-12-25 -1909-05-05 12:34:56.1787,1909-12-25 -1909-05-05 12:34:56.1788,1909-12-25 -1909-05-05 12:34:56.1789,1909-12-25 -1909-05-05 12:34:56.179,1909-12-25 -1909-05-05 12:34:56.1791,1909-12-25 -1909-05-05 12:34:56.1792,1909-12-25 -1909-05-05 12:34:56.1793,1909-12-25 -1909-05-05 12:34:56.1794,1909-12-25 -1909-05-05 12:34:56.1795,1909-12-25 -1909-05-05 12:34:56.1796,1909-12-25 -1909-05-05 12:34:56.1797,1909-12-25 -1909-05-05 12:34:56.1798,1909-12-25 -1909-05-05 12:34:56.1799,1909-12-25 -1909-05-05 12:34:56.18,1909-12-25 -1909-05-05 12:34:56.1801,1909-12-25 -1909-05-05 12:34:56.1802,1909-12-25 -1909-05-05 12:34:56.1803,1909-12-25 -1909-05-05 12:34:56.1804,1909-12-25 -1909-05-05 12:34:56.1805,1909-12-25 -1909-05-05 12:34:56.1806,1909-12-25 -1909-05-05 12:34:56.1807,1909-12-25 -1909-05-05 12:34:56.1808,1909-12-25 -1909-05-05 12:34:56.1809,1909-12-25 -1909-05-05 12:34:56.181,1909-12-25 -1909-05-05 12:34:56.1811,1909-12-25 -1909-05-05 12:34:56.1812,1909-12-25 -1909-05-05 12:34:56.1813,1909-12-25 -1909-05-05 12:34:56.1814,1909-12-25 -1909-05-05 12:34:56.1815,1909-12-25 -1909-05-05 12:34:56.1816,1909-12-25 -1909-05-05 12:34:56.1817,1909-12-25 -1909-05-05 12:34:56.1818,1909-12-25 -1909-05-05 12:34:56.1819,1909-12-25 -1909-05-05 12:34:56.182,1909-12-25 -1909-05-05 12:34:56.1821,1909-12-25 -1909-05-05 12:34:56.1822,1909-12-25 -1909-05-05 12:34:56.1823,1909-12-25 -1909-05-05 12:34:56.1824,1909-12-25 -1909-05-05 12:34:56.1825,1909-12-25 -1909-05-05 12:34:56.1826,1909-12-25 -1909-05-05 12:34:56.1827,1909-12-25 -1909-05-05 12:34:56.1828,1909-12-25 -1909-05-05 12:34:56.1829,1909-12-25 -1909-05-05 12:34:56.183,1909-12-25 -1909-05-05 12:34:56.1831,1909-12-25 -1909-05-05 12:34:56.1832,1909-12-25 -1909-05-05 12:34:56.1833,1909-12-25 -1909-05-05 12:34:56.1834,1909-12-25 -1909-05-05 12:34:56.1835,1909-12-25 -1909-05-05 12:34:56.1836,1909-12-25 -1909-05-05 12:34:56.1837,1909-12-25 -1909-05-05 12:34:56.1838,1909-12-25 -1909-05-05 12:34:56.1839,1909-12-25 -1909-05-05 12:34:56.184,1909-12-25 -1909-05-05 12:34:56.1841,1909-12-25 -1909-05-05 12:34:56.1842,1909-12-25 -1909-05-05 12:34:56.1843,1909-12-25 -1909-05-05 12:34:56.1844,1909-12-25 -1909-05-05 12:34:56.1845,1909-12-25 -1909-05-05 12:34:56.1846,1909-12-25 -1909-05-05 12:34:56.1847,1909-12-25 -1909-05-05 12:34:56.1848,1909-12-25 -1909-05-05 12:34:56.1849,1909-12-25 -1909-05-05 12:34:56.185,1909-12-25 -1909-05-05 12:34:56.1851,1909-12-25 -1909-05-05 12:34:56.1852,1909-12-25 -1909-05-05 12:34:56.1853,1909-12-25 -1909-05-05 12:34:56.1854,1909-12-25 -1909-05-05 12:34:56.1855,1909-12-25 -1909-05-05 12:34:56.1856,1909-12-25 -1909-05-05 12:34:56.1857,1909-12-25 -1909-05-05 12:34:56.1858,1909-12-25 -1909-05-05 12:34:56.1859,1909-12-25 -1909-05-05 12:34:56.186,1909-12-25 -1909-05-05 12:34:56.1861,1909-12-25 -1909-05-05 12:34:56.1862,1909-12-25 -1909-05-05 12:34:56.1863,1909-12-25 -1909-05-05 12:34:56.1864,1909-12-25 -1909-05-05 12:34:56.1865,1909-12-25 -1909-05-05 12:34:56.1866,1909-12-25 -1909-05-05 12:34:56.1867,1909-12-25 -1909-05-05 12:34:56.1868,1909-12-25 -1909-05-05 12:34:56.1869,1909-12-25 -1909-05-05 12:34:56.187,1909-12-25 -1909-05-05 12:34:56.1871,1909-12-25 -1909-05-05 12:34:56.1872,1909-12-25 -1909-05-05 12:34:56.1873,1909-12-25 -1909-05-05 12:34:56.1874,1909-12-25 -1909-05-05 12:34:56.1875,1909-12-25 -1909-05-05 12:34:56.1876,1909-12-25 -1909-05-05 12:34:56.1877,1909-12-25 -1909-05-05 12:34:56.1878,1909-12-25 -1909-05-05 12:34:56.1879,1909-12-25 -1909-05-05 12:34:56.188,1909-12-25 -1909-05-05 12:34:56.1881,1909-12-25 -1909-05-05 12:34:56.1882,1909-12-25 -1909-05-05 12:34:56.1883,1909-12-25 -1909-05-05 12:34:56.1884,1909-12-25 -1909-05-05 12:34:56.1885,1909-12-25 -1909-05-05 12:34:56.1886,1909-12-25 -1909-05-05 12:34:56.1887,1909-12-25 -1909-05-05 12:34:56.1888,1909-12-25 -1909-05-05 12:34:56.1889,1909-12-25 -1909-05-05 12:34:56.189,1909-12-25 -1909-05-05 12:34:56.1891,1909-12-25 -1909-05-05 12:34:56.1892,1909-12-25 -1909-05-05 12:34:56.1893,1909-12-25 -1909-05-05 12:34:56.1894,1909-12-25 -1909-05-05 12:34:56.1895,1909-12-25 -1909-05-05 12:34:56.1896,1909-12-25 -1909-05-05 12:34:56.1897,1909-12-25 -1909-05-05 12:34:56.1898,1909-12-25 -1909-05-05 12:34:56.1899,1909-12-25 -1909-05-05 12:34:56.19,1909-12-25 -1909-05-05 12:34:56.1901,1909-12-25 -1909-05-05 12:34:56.1902,1909-12-25 -1909-05-05 12:34:56.1903,1909-12-25 -1909-05-05 12:34:56.1904,1909-12-25 -1909-05-05 12:34:56.1905,1909-12-25 -1909-05-05 12:34:56.1906,1909-12-25 -1909-05-05 12:34:56.1907,1909-12-25 -1909-05-05 12:34:56.1908,1909-12-25 -1909-05-05 12:34:56.1909,1909-12-25 -1909-05-05 12:34:56.191,1909-12-25 -1909-05-05 12:34:56.1911,1909-12-25 -1909-05-05 12:34:56.1912,1909-12-25 -1909-05-05 12:34:56.1913,1909-12-25 -1909-05-05 12:34:56.1914,1909-12-25 -1909-05-05 12:34:56.1915,1909-12-25 -1909-05-05 12:34:56.1916,1909-12-25 -1909-05-05 12:34:56.1917,1909-12-25 -1909-05-05 12:34:56.1918,1909-12-25 -1909-05-05 12:34:56.1919,1909-12-25 -1909-05-05 12:34:56.192,1909-12-25 -1909-05-05 12:34:56.1921,1909-12-25 -1909-05-05 12:34:56.1922,1909-12-25 -1909-05-05 12:34:56.1923,1909-12-25 -1909-05-05 12:34:56.1924,1909-12-25 -1909-05-05 12:34:56.1925,1909-12-25 -1909-05-05 12:34:56.1926,1909-12-25 -1909-05-05 12:34:56.1927,1909-12-25 -1909-05-05 12:34:56.1928,1909-12-25 -1909-05-05 12:34:56.1929,1909-12-25 -1909-05-05 12:34:56.193,1909-12-25 -1909-05-05 12:34:56.1931,1909-12-25 -1909-05-05 12:34:56.1932,1909-12-25 -1909-05-05 12:34:56.1933,1909-12-25 -1909-05-05 12:34:56.1934,1909-12-25 -1909-05-05 12:34:56.1935,1909-12-25 -1909-05-05 12:34:56.1936,1909-12-25 -1909-05-05 12:34:56.1937,1909-12-25 -1909-05-05 12:34:56.1938,1909-12-25 -1909-05-05 12:34:56.1939,1909-12-25 -1909-05-05 12:34:56.194,1909-12-25 -1909-05-05 12:34:56.1941,1909-12-25 -1909-05-05 12:34:56.1942,1909-12-25 -1909-05-05 12:34:56.1943,1909-12-25 -1909-05-05 12:34:56.1944,1909-12-25 -1909-05-05 12:34:56.1945,1909-12-25 -1909-05-05 12:34:56.1946,1909-12-25 -1909-05-05 12:34:56.1947,1909-12-25 -1909-05-05 12:34:56.1948,1909-12-25 -1909-05-05 12:34:56.1949,1909-12-25 -1909-05-05 12:34:56.195,1909-12-25 -1909-05-05 12:34:56.1951,1909-12-25 -1909-05-05 12:34:56.1952,1909-12-25 -1909-05-05 12:34:56.1953,1909-12-25 -1909-05-05 12:34:56.1954,1909-12-25 -1909-05-05 12:34:56.1955,1909-12-25 -1909-05-05 12:34:56.1956,1909-12-25 -1909-05-05 12:34:56.1957,1909-12-25 -1909-05-05 12:34:56.1958,1909-12-25 -1909-05-05 12:34:56.1959,1909-12-25 -1909-05-05 12:34:56.196,1909-12-25 -1909-05-05 12:34:56.1961,1909-12-25 -1909-05-05 12:34:56.1962,1909-12-25 -1909-05-05 12:34:56.1963,1909-12-25 -1909-05-05 12:34:56.1964,1909-12-25 -1909-05-05 12:34:56.1965,1909-12-25 -1909-05-05 12:34:56.1966,1909-12-25 -1909-05-05 12:34:56.1967,1909-12-25 -1909-05-05 12:34:56.1968,1909-12-25 -1909-05-05 12:34:56.1969,1909-12-25 -1909-05-05 12:34:56.197,1909-12-25 -1909-05-05 12:34:56.1971,1909-12-25 -1909-05-05 12:34:56.1972,1909-12-25 -1909-05-05 12:34:56.1973,1909-12-25 -1909-05-05 12:34:56.1974,1909-12-25 -1909-05-05 12:34:56.1975,1909-12-25 -1909-05-05 12:34:56.1976,1909-12-25 -1909-05-05 12:34:56.1977,1909-12-25 -1909-05-05 12:34:56.1978,1909-12-25 -1909-05-05 12:34:56.1979,1909-12-25 -1909-05-05 12:34:56.198,1909-12-25 -1909-05-05 12:34:56.1981,1909-12-25 -1909-05-05 12:34:56.1982,1909-12-25 -1909-05-05 12:34:56.1983,1909-12-25 -1909-05-05 12:34:56.1984,1909-12-25 -1909-05-05 12:34:56.1985,1909-12-25 -1909-05-05 12:34:56.1986,1909-12-25 -1909-05-05 12:34:56.1987,1909-12-25 -1909-05-05 12:34:56.1988,1909-12-25 -1909-05-05 12:34:56.1989,1909-12-25 -1909-05-05 12:34:56.199,1909-12-25 -1909-05-05 12:34:56.1991,1909-12-25 -1909-05-05 12:34:56.1992,1909-12-25 -1909-05-05 12:34:56.1993,1909-12-25 -1909-05-05 12:34:56.1994,1909-12-25 -1909-05-05 12:34:56.1995,1909-12-25 -1909-05-05 12:34:56.1996,1909-12-25 -1909-05-05 12:34:56.1997,1909-12-25 -1909-05-05 12:34:56.1998,1909-12-25 -1909-05-05 12:34:56.1999,1909-12-25 diff --git a/flink-connectors/flink-orc/src/test/resources/decimal.dat b/flink-connectors/flink-orc/src/test/resources/decimal.dat deleted file mode 100644 index 42010b518c91e..0000000000000 --- a/flink-connectors/flink-orc/src/test/resources/decimal.dat +++ /dev/null @@ -1,6000 +0,0 @@ --1000.5 --999.6 --998.7 --997.8 --996.9 --995.1 --994.11 --993.12 --992.13 --991.14 --990.15 --989.16 --988.17 --987.18 --986.19 --985.2 --984.21 --983.22 --982.23 --981.24 --980.25 --979.26 --978.27 --977.28 --976.29 --975.3 --974.31 --973.32 --972.33 --971.34 --970.35 --969.36 --968.37 --967.38 --966.39 --965.4 --964.41 --963.42 --962.43 --961.44 --960.45 --959.46 --958.47 --957.48 --956.49 --955.5 --954.51 --953.52 --952.53 --951.54 --950.55 --949.56 --948.57 --947.58 --946.59 --945.6 --944.61 --943.62 --942.63 --941.64 --940.65 --939.66 --938.67 --937.68 --936.69 --935.7 --934.71 --933.72 --932.73 --931.74 --930.75 --929.76 --928.77 --927.78 --926.79 --925.8 --924.81 --923.82 --922.83 --921.84 --920.85 --919.86 --918.87 --917.88 --916.89 --915.9 --914.91 --913.92 --912.93 --911.94 --910.95 --909.96 --908.97 --907.98 --906.99 --905.1 --904.101 --903.102 --902.103 --901.104 --900.105 --899.106 --898.107 --897.108 --896.109 --895.11 --894.111 --893.112 --892.113 --891.114 --890.115 --889.116 --888.117 --887.118 --886.119 --885.12 --884.121 --883.122 --882.123 --881.124 --880.125 --879.126 --878.127 --877.128 --876.129 --875.13 --874.131 --873.132 --872.133 --871.134 --870.135 --869.136 --868.137 --867.138 --866.139 --865.14 --864.141 --863.142 --862.143 --861.144 --860.145 --859.146 --858.147 --857.148 --856.149 --855.15 --854.151 --853.152 --852.153 --851.154 --850.155 --849.156 --848.157 --847.158 --846.159 --845.16 --844.161 --843.162 --842.163 --841.164 --840.165 --839.166 --838.167 --837.168 --836.169 --835.17 --834.171 --833.172 --832.173 --831.174 --830.175 --829.176 --828.177 --827.178 --826.179 --825.18 --824.181 --823.182 --822.183 --821.184 --820.185 --819.186 --818.187 --817.188 --816.189 --815.19 --814.191 --813.192 --812.193 --811.194 --810.195 --809.196 --808.197 --807.198 --806.199 --805.2 --804.201 --803.202 --802.203 --801.204 --800.205 --799.206 --798.207 --797.208 --796.209 --795.21 --794.211 --793.212 --792.213 --791.214 --790.215 --789.216 --788.217 --787.218 --786.219 --785.22 --784.221 --783.222 --782.223 --781.224 --780.225 --779.226 --778.227 --777.228 --776.229 --775.23 --774.231 --773.232 --772.233 --771.234 --770.235 --769.236 --768.237 --767.238 --766.239 --765.24 --764.241 --763.242 --762.243 --761.244 --760.245 --759.246 --758.247 --757.248 --756.249 --755.25 --754.251 --753.252 --752.253 --751.254 --750.255 --749.256 --748.257 --747.258 --746.259 --745.26 --744.261 --743.262 --742.263 --741.264 --740.265 --739.266 --738.267 --737.268 --736.269 --735.27 --734.271 --733.272 --732.273 --731.274 --730.275 --729.276 --728.277 --727.278 --726.279 --725.28 --724.281 --723.282 --722.283 --721.284 --720.285 --719.286 --718.287 --717.288 --716.289 --715.29 --714.291 --713.292 --712.293 --711.294 --710.295 --709.296 --708.297 --707.298 --706.299 --705.3 --704.301 --703.302 --702.303 --701.304 --700.305 --699.306 --698.307 --697.308 --696.309 --695.31 --694.311 --693.312 --692.313 --691.314 --690.315 --689.316 --688.317 --687.318 --686.319 --685.32 --684.321 --683.322 --682.323 --681.324 --680.325 --679.326 --678.327 --677.328 --676.329 --675.33 --674.331 --673.332 --672.333 --671.334 --670.335 --669.336 --668.337 --667.338 --666.339 --665.34 --664.341 --663.342 --662.343 --661.344 --660.345 --659.346 --658.347 --657.348 --656.349 --655.35 --654.351 --653.352 --652.353 --651.354 --650.355 --649.356 --648.357 --647.358 --646.359 --645.36 --644.361 --643.362 --642.363 --641.364 --640.365 --639.366 --638.367 --637.368 --636.369 --635.37 --634.371 --633.372 --632.373 --631.374 --630.375 --629.376 --628.377 --627.378 --626.379 --625.38 --624.381 --623.382 --622.383 --621.384 --620.385 --619.386 --618.387 --617.388 --616.389 --615.39 --614.391 --613.392 --612.393 --611.394 --610.395 --609.396 --608.397 --607.398 --606.399 --605.4 --604.401 --603.402 --602.403 --601.404 --600.405 --599.406 --598.407 --597.408 --596.409 --595.41 --594.411 --593.412 --592.413 --591.414 --590.415 --589.416 --588.417 --587.418 --586.419 --585.42 --584.421 --583.422 --582.423 --581.424 --580.425 --579.426 --578.427 --577.428 --576.429 --575.43 --574.431 --573.432 --572.433 --571.434 --570.435 --569.436 --568.437 --567.438 --566.439 --565.44 --564.441 --563.442 --562.443 --561.444 --560.445 --559.446 --558.447 --557.448 --556.449 --555.45 --554.451 --553.452 --552.453 --551.454 --550.455 --549.456 --548.457 --547.458 --546.459 --545.46 --544.461 --543.462 --542.463 --541.464 --540.465 --539.466 --538.467 --537.468 --536.469 --535.47 --534.471 --533.472 --532.473 --531.474 --530.475 --529.476 --528.477 --527.478 --526.479 --525.48 --524.481 --523.482 --522.483 --521.484 --520.485 --519.486 --518.487 --517.488 --516.489 --515.49 --514.491 --513.492 --512.493 --511.494 --510.495 --509.496 --508.497 --507.498 --506.499 --505.5 --504.501 --503.502 --502.503 --501.504 --500.505 --499.506 --498.507 --497.508 --496.509 --495.51 --494.511 --493.512 --492.513 --491.514 --490.515 --489.516 --488.517 --487.518 --486.519 --485.52 --484.521 --483.522 --482.523 --481.524 --480.525 --479.526 --478.527 --477.528 --476.529 --475.53 --474.531 --473.532 --472.533 --471.534 --470.535 --469.536 --468.537 --467.538 --466.539 --465.54 --464.541 --463.542 --462.543 --461.544 --460.545 --459.546 --458.547 --457.548 --456.549 --455.55 --454.551 --453.552 --452.553 --451.554 --450.555 --449.556 --448.557 --447.558 --446.559 --445.56 --444.561 --443.562 --442.563 --441.564 --440.565 --439.566 --438.567 --437.568 --436.569 --435.57 --434.571 --433.572 --432.573 --431.574 --430.575 --429.576 --428.577 --427.578 --426.579 --425.58 --424.581 --423.582 --422.583 --421.584 --420.585 --419.586 --418.587 --417.588 --416.589 --415.59 --414.591 --413.592 --412.593 --411.594 --410.595 --409.596 --408.597 --407.598 --406.599 --405.6 --404.601 --403.602 --402.603 --401.604 --400.605 --399.606 --398.607 --397.608 --396.609 --395.61 --394.611 --393.612 --392.613 --391.614 --390.615 --389.616 --388.617 --387.618 --386.619 --385.62 --384.621 --383.622 --382.623 --381.624 --380.625 --379.626 --378.627 --377.628 --376.629 --375.63 --374.631 --373.632 --372.633 --371.634 --370.635 --369.636 --368.637 --367.638 --366.639 --365.64 --364.641 --363.642 --362.643 --361.644 --360.645 --359.646 --358.647 --357.648 --356.649 --355.65 --354.651 --353.652 --352.653 --351.654 --350.655 --349.656 --348.657 --347.658 --346.659 --345.66 --344.661 --343.662 --342.663 --341.664 --340.665 --339.666 --338.667 --337.668 --336.669 --335.67 --334.671 --333.672 --332.673 --331.674 --330.675 --329.676 --328.677 --327.678 --326.679 --325.68 --324.681 --323.682 --322.683 --321.684 --320.685 --319.686 --318.687 --317.688 --316.689 --315.69 --314.691 --313.692 --312.693 --311.694 --310.695 --309.696 --308.697 --307.698 --306.699 --305.7 --304.701 --303.702 --302.703 --301.704 --300.705 --299.706 --298.707 --297.708 --296.709 --295.71 --294.711 --293.712 --292.713 --291.714 --290.715 --289.716 --288.717 --287.718 --286.719 --285.72 --284.721 --283.722 --282.723 --281.724 --280.725 --279.726 --278.727 --277.728 --276.729 --275.73 --274.731 --273.732 --272.733 --271.734 --270.735 --269.736 --268.737 --267.738 --266.739 --265.74 --264.741 --263.742 --262.743 --261.744 --260.745 --259.746 --258.747 --257.748 --256.749 --255.75 --254.751 --253.752 --252.753 --251.754 --250.755 --249.756 --248.757 --247.758 --246.759 --245.76 --244.761 --243.762 --242.763 --241.764 --240.765 --239.766 --238.767 --237.768 --236.769 --235.77 --234.771 --233.772 --232.773 --231.774 --230.775 --229.776 --228.777 --227.778 --226.779 --225.78 --224.781 --223.782 --222.783 --221.784 --220.785 --219.786 --218.787 --217.788 --216.789 --215.79 --214.791 --213.792 --212.793 --211.794 --210.795 --209.796 --208.797 --207.798 --206.799 --205.8 --204.801 --203.802 --202.803 --201.804 --200.805 --199.806 --198.807 --197.808 --196.809 --195.81 --194.811 --193.812 --192.813 --191.814 --190.815 --189.816 --188.817 --187.818 --186.819 --185.82 --184.821 --183.822 --182.823 --181.824 --180.825 --179.826 --178.827 --177.828 --176.829 --175.83 --174.831 --173.832 --172.833 --171.834 --170.835 --169.836 --168.837 --167.838 --166.839 --165.84 --164.841 --163.842 --162.843 --161.844 --160.845 --159.846 --158.847 --157.848 --156.849 --155.85 --154.851 --153.852 --152.853 --151.854 --150.855 --149.856 --148.857 --147.858 --146.859 --145.86 --144.861 --143.862 --142.863 --141.864 --140.865 --139.866 --138.867 --137.868 --136.869 --135.87 --134.871 --133.872 --132.873 --131.874 --130.875 --129.876 --128.877 --127.878 --126.879 --125.88 --124.881 --123.882 --122.883 --121.884 --120.885 --119.886 --118.887 --117.888 --116.889 --115.89 --114.891 --113.892 --112.893 --111.894 --110.895 --109.896 --108.897 --107.898 --106.899 --105.9 --104.901 --103.902 --102.903 --101.904 --100.905 --99.906 --98.907 --97.908 --96.909 --95.91 --94.911 --93.912 --92.913 --91.914 --90.915 --89.916 --88.917 --87.918 --86.919 --85.92 --84.921 --83.922 --82.923 --81.924 --80.925 --79.926 --78.927 --77.928 --76.929 --75.93 --74.931 --73.932 --72.933 --71.934 --70.935 --69.936 --68.937 --67.938 --66.939 --65.94 --64.941 --63.942 --62.943 --61.944 --60.945 --59.946 --58.947 --57.948 --56.949 --55.95 --54.951 --53.952 --52.953 --51.954 --50.955 --49.956 --48.957 --47.958 --46.959 --45.96 --44.961 --43.962 --42.963 --41.964 --40.965 --39.966 --38.967 --37.968 --36.969 --35.97 --34.971 --33.972 --32.973 --31.974 --30.975 --29.976 --28.977 --27.978 --26.979 --25.98 --24.981 --23.982 --22.983 --21.984 --20.985 --19.986 --18.987 --17.988 --16.989 --15.99 --14.991 --13.992 --12.993 --11.994 --10.995 --9.996 --8.997 --7.998 --6.999 --5.1 --4.1001 --3.1002 --2.1003 --1.1004 -0.1005 -1.1006 -2.1007 -3.1008 -4.1009 -5.101 -6.1011 -7.1012 -8.1013 -9.1014 -10.1015 -11.1016 -12.1017 -13.1018 -14.1019 -15.102 -16.1021 -17.1022 -18.1023 -19.1024 -20.1025 -21.1026 -22.1027 -23.1028 -24.1029 -25.103 -26.1031 -27.1032 -28.1033 -29.1034 -30.1035 -31.1036 -32.1037 -33.1038 -34.1039 -35.104 -36.1041 -37.1042 -38.1043 -39.1044 -40.1045 -41.1046 -42.1047 -43.1048 -44.1049 -45.105 -46.1051 -47.1052 -48.1053 -49.1054 -50.1055 -51.1056 -52.1057 -53.1058 -54.1059 -55.106 -56.1061 -57.1062 -58.1063 -59.1064 -60.1065 -61.1066 -62.1067 -63.1068 -64.1069 -65.107 -66.1071 -67.1072 -68.1073 -69.1074 -70.1075 -71.1076 -72.1077 -73.1078 -74.1079 -75.108 -76.1081 -77.1082 -78.1083 -79.1084 -80.1085 -81.1086 -82.1087 -83.1088 -84.1089 -85.109 -86.1091 -87.1092 -88.1093 -89.1094 -90.1095 -91.1096 -92.1097 -93.1098 -94.1099 -95.11 -96.1101 -97.1102 -98.1103 -99.1104 -100.1105 -101.1106 -102.1107 -103.1108 -104.1109 -105.111 -106.1111 -107.1112 -108.1113 -109.1114 -110.1115 -111.1116 -112.1117 -113.1118 -114.1119 -115.112 -116.1121 -117.1122 -118.1123 -119.1124 -120.1125 -121.1126 -122.1127 -123.1128 -124.1129 -125.113 -126.1131 -127.1132 -128.1133 -129.1134 -130.1135 -131.1136 -132.1137 -133.1138 -134.1139 -135.114 -136.1141 -137.1142 -138.1143 -139.1144 -140.1145 -141.1146 -142.1147 -143.1148 -144.1149 -145.115 -146.1151 -147.1152 -148.1153 -149.1154 -150.1155 -151.1156 -152.1157 -153.1158 -154.1159 -155.116 -156.1161 -157.1162 -158.1163 -159.1164 -160.1165 -161.1166 -162.1167 -163.1168 -164.1169 -165.117 -166.1171 -167.1172 -168.1173 -169.1174 -170.1175 -171.1176 -172.1177 -173.1178 -174.1179 -175.118 -176.1181 -177.1182 -178.1183 -179.1184 -180.1185 -181.1186 -182.1187 -183.1188 -184.1189 -185.119 -186.1191 -187.1192 -188.1193 -189.1194 -190.1195 -191.1196 -192.1197 -193.1198 -194.1199 -195.12 -196.1201 -197.1202 -198.1203 -199.1204 -200.1205 -201.1206 -202.1207 -203.1208 -204.1209 -205.121 -206.1211 -207.1212 -208.1213 -209.1214 -210.1215 -211.1216 -212.1217 -213.1218 -214.1219 -215.122 -216.1221 -217.1222 -218.1223 -219.1224 -220.1225 -221.1226 -222.1227 -223.1228 -224.1229 -225.123 -226.1231 -227.1232 -228.1233 -229.1234 -230.1235 -231.1236 -232.1237 -233.1238 -234.1239 -235.124 -236.1241 -237.1242 -238.1243 -239.1244 -240.1245 -241.1246 -242.1247 -243.1248 -244.1249 -245.125 -246.1251 -247.1252 -248.1253 -249.1254 -250.1255 -251.1256 -252.1257 -253.1258 -254.1259 -255.126 -256.1261 -257.1262 -258.1263 -259.1264 -260.1265 -261.1266 -262.1267 -263.1268 -264.1269 -265.127 -266.1271 -267.1272 -268.1273 -269.1274 -270.1275 -271.1276 -272.1277 -273.1278 -274.1279 -275.128 -276.1281 -277.1282 -278.1283 -279.1284 -280.1285 -281.1286 -282.1287 -283.1288 -284.1289 -285.129 -286.1291 -287.1292 -288.1293 -289.1294 -290.1295 -291.1296 -292.1297 -293.1298 -294.1299 -295.13 -296.1301 -297.1302 -298.1303 -299.1304 -300.1305 -301.1306 -302.1307 -303.1308 -304.1309 -305.131 -306.1311 -307.1312 -308.1313 -309.1314 -310.1315 -311.1316 -312.1317 -313.1318 -314.1319 -315.132 -316.1321 -317.1322 -318.1323 -319.1324 -320.1325 -321.1326 -322.1327 -323.1328 -324.1329 -325.133 -326.1331 -327.1332 -328.1333 -329.1334 -330.1335 -331.1336 -332.1337 -333.1338 -334.1339 -335.134 -336.1341 -337.1342 -338.1343 -339.1344 -340.1345 -341.1346 -342.1347 -343.1348 -344.1349 -345.135 -346.1351 -347.1352 -348.1353 -349.1354 -350.1355 -351.1356 -352.1357 -353.1358 -354.1359 -355.136 -356.1361 -357.1362 -358.1363 -359.1364 -360.1365 -361.1366 -362.1367 -363.1368 -364.1369 -365.137 -366.1371 -367.1372 -368.1373 -369.1374 -370.1375 -371.1376 -372.1377 -373.1378 -374.1379 -375.138 -376.1381 -377.1382 -378.1383 -379.1384 -380.1385 -381.1386 -382.1387 -383.1388 -384.1389 -385.139 -386.1391 -387.1392 -388.1393 -389.1394 -390.1395 -391.1396 -392.1397 -393.1398 -394.1399 -395.14 -396.1401 -397.1402 -398.1403 -399.1404 -400.1405 -401.1406 -402.1407 -403.1408 -404.1409 -405.141 -406.1411 -407.1412 -408.1413 -409.1414 -410.1415 -411.1416 -412.1417 -413.1418 -414.1419 -415.142 -416.1421 -417.1422 -418.1423 -419.1424 -420.1425 -421.1426 -422.1427 -423.1428 -424.1429 -425.143 -426.1431 -427.1432 -428.1433 -429.1434 -430.1435 -431.1436 -432.1437 -433.1438 -434.1439 -435.144 -436.1441 -437.1442 -438.1443 -439.1444 -440.1445 -441.1446 -442.1447 -443.1448 -444.1449 -445.145 -446.1451 -447.1452 -448.1453 -449.1454 -450.1455 -451.1456 -452.1457 -453.1458 -454.1459 -455.146 -456.1461 -457.1462 -458.1463 -459.1464 -460.1465 -461.1466 -462.1467 -463.1468 -464.1469 -465.147 -466.1471 -467.1472 -468.1473 -469.1474 -470.1475 -471.1476 -472.1477 -473.1478 -474.1479 -475.148 -476.1481 -477.1482 -478.1483 -479.1484 -480.1485 -481.1486 -482.1487 -483.1488 -484.1489 -485.149 -486.1491 -487.1492 -488.1493 -489.1494 -490.1495 -491.1496 -492.1497 -493.1498 -494.1499 -495.15 -496.1501 -497.1502 -498.1503 -499.1504 -500.1505 -501.1506 -502.1507 -503.1508 -504.1509 -505.151 -506.1511 -507.1512 -508.1513 -509.1514 -510.1515 -511.1516 -512.1517 -513.1518 -514.1519 -515.152 -516.1521 -517.1522 -518.1523 -519.1524 -520.1525 -521.1526 -522.1527 -523.1528 -524.1529 -525.153 -526.1531 -527.1532 -528.1533 -529.1534 -530.1535 -531.1536 -532.1537 -533.1538 -534.1539 -535.154 -536.1541 -537.1542 -538.1543 -539.1544 -540.1545 -541.1546 -542.1547 -543.1548 -544.1549 -545.155 -546.1551 -547.1552 -548.1553 -549.1554 -550.1555 -551.1556 -552.1557 -553.1558 -554.1559 -555.156 -556.1561 -557.1562 -558.1563 -559.1564 -560.1565 -561.1566 -562.1567 -563.1568 -564.1569 -565.157 -566.1571 -567.1572 -568.1573 -569.1574 -570.1575 -571.1576 -572.1577 -573.1578 -574.1579 -575.158 -576.1581 -577.1582 -578.1583 -579.1584 -580.1585 -581.1586 -582.1587 -583.1588 -584.1589 -585.159 -586.1591 -587.1592 -588.1593 -589.1594 -590.1595 -591.1596 -592.1597 -593.1598 -594.1599 -595.16 -596.1601 -597.1602 -598.1603 -599.1604 -600.1605 -601.1606 -602.1607 -603.1608 -604.1609 -605.161 -606.1611 -607.1612 -608.1613 -609.1614 -610.1615 -611.1616 -612.1617 -613.1618 -614.1619 -615.162 -616.1621 -617.1622 -618.1623 -619.1624 -620.1625 -621.1626 -622.1627 -623.1628 -624.1629 -625.163 -626.1631 -627.1632 -628.1633 -629.1634 -630.1635 -631.1636 -632.1637 -633.1638 -634.1639 -635.164 -636.1641 -637.1642 -638.1643 -639.1644 -640.1645 -641.1646 -642.1647 -643.1648 -644.1649 -645.165 -646.1651 -647.1652 -648.1653 -649.1654 -650.1655 -651.1656 -652.1657 -653.1658 -654.1659 -655.166 -656.1661 -657.1662 -658.1663 -659.1664 -660.1665 -661.1666 -662.1667 -663.1668 -664.1669 -665.167 -666.1671 -667.1672 -668.1673 -669.1674 -670.1675 -671.1676 -672.1677 -673.1678 -674.1679 -675.168 -676.1681 -677.1682 -678.1683 -679.1684 -680.1685 -681.1686 -682.1687 -683.1688 -684.1689 -685.169 -686.1691 -687.1692 -688.1693 -689.1694 -690.1695 -691.1696 -692.1697 -693.1698 -694.1699 -695.17 -696.1701 -697.1702 -698.1703 -699.1704 -700.1705 -701.1706 -702.1707 -703.1708 -704.1709 -705.171 -706.1711 -707.1712 -708.1713 -709.1714 -710.1715 -711.1716 -712.1717 -713.1718 -714.1719 -715.172 -716.1721 -717.1722 -718.1723 -719.1724 -720.1725 -721.1726 -722.1727 -723.1728 -724.1729 -725.173 -726.1731 -727.1732 -728.1733 -729.1734 -730.1735 -731.1736 -732.1737 -733.1738 -734.1739 -735.174 -736.1741 -737.1742 -738.1743 -739.1744 -740.1745 -741.1746 -742.1747 -743.1748 -744.1749 -745.175 -746.1751 -747.1752 -748.1753 -749.1754 -750.1755 -751.1756 -752.1757 -753.1758 -754.1759 -755.176 -756.1761 -757.1762 -758.1763 -759.1764 -760.1765 -761.1766 -762.1767 -763.1768 -764.1769 -765.177 -766.1771 -767.1772 -768.1773 -769.1774 -770.1775 -771.1776 -772.1777 -773.1778 -774.1779 -775.178 -776.1781 -777.1782 -778.1783 -779.1784 -780.1785 -781.1786 -782.1787 -783.1788 -784.1789 -785.179 -786.1791 -787.1792 -788.1793 -789.1794 -790.1795 -791.1796 -792.1797 -793.1798 -794.1799 -795.18 -796.1801 -797.1802 -798.1803 -799.1804 -800.1805 -801.1806 -802.1807 -803.1808 -804.1809 -805.181 -806.1811 -807.1812 -808.1813 -809.1814 -810.1815 -811.1816 -812.1817 -813.1818 -814.1819 -815.182 -816.1821 -817.1822 -818.1823 -819.1824 -820.1825 -821.1826 -822.1827 -823.1828 -824.1829 -825.183 -826.1831 -827.1832 -828.1833 -829.1834 -830.1835 -831.1836 -832.1837 -833.1838 -834.1839 -835.184 -836.1841 -837.1842 -838.1843 -839.1844 -840.1845 -841.1846 -842.1847 -843.1848 -844.1849 -845.185 -846.1851 -847.1852 -848.1853 -849.1854 -850.1855 -851.1856 -852.1857 -853.1858 -854.1859 -855.186 -856.1861 -857.1862 -858.1863 -859.1864 -860.1865 -861.1866 -862.1867 -863.1868 -864.1869 -865.187 -866.1871 -867.1872 -868.1873 -869.1874 -870.1875 -871.1876 -872.1877 -873.1878 -874.1879 -875.188 -876.1881 -877.1882 -878.1883 -879.1884 -880.1885 -881.1886 -882.1887 -883.1888 -884.1889 -885.189 -886.1891 -887.1892 -888.1893 -889.1894 -890.1895 -891.1896 -892.1897 -893.1898 -894.1899 -895.19 -896.1901 -897.1902 -898.1903 -899.1904 -900.1905 -901.1906 -902.1907 -903.1908 -904.1909 -905.191 -906.1911 -907.1912 -908.1913 -909.1914 -910.1915 -911.1916 -912.1917 -913.1918 -914.1919 -915.192 -916.1921 -917.1922 -918.1923 -919.1924 -920.1925 -921.1926 -922.1927 -923.1928 -924.1929 -925.193 -926.1931 -927.1932 -928.1933 -929.1934 -930.1935 -931.1936 -932.1937 -933.1938 -934.1939 -935.194 -936.1941 -937.1942 -938.1943 -939.1944 -940.1945 -941.1946 -942.1947 -943.1948 -944.1949 -945.195 -946.1951 -947.1952 -948.1953 -949.1954 -950.1955 -951.1956 -952.1957 -953.1958 -954.1959 -955.196 -956.1961 -957.1962 -958.1963 -959.1964 -960.1965 -961.1966 -962.1967 -963.1968 -964.1969 -965.197 -966.1971 -967.1972 -968.1973 -969.1974 -970.1975 -971.1976 -972.1977 -973.1978 -974.1979 -975.198 -976.1981 -977.1982 -978.1983 -979.1984 -980.1985 -981.1986 -982.1987 -983.1988 -984.1989 -985.199 -986.1991 -987.1992 -988.1993 -989.1994 -990.1995 -991.1996 -992.1997 -993.1998 -994.1999 -995.2 -996.2001 -997.2002 -998.2003 -999.2004 -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -0.1 -1.2 -2.3 -3.4 -4.5 -5.6 -6.7 -7.8 -8.9 -9.1 -10.11 -11.12 -12.13 -13.14 -14.15 -15.16 -16.17 -17.18 -18.19 -19.2 -20.21 -21.22 -22.23 -23.24 -24.25 -25.26 -26.27 -27.28 -28.29 -29.3 -30.31 -31.32 -32.33 -33.34 -34.35 -35.36 -36.37 -37.38 -38.39 -39.4 -40.41 -41.42 -42.43 -43.44 -44.45 -45.46 -46.47 -47.48 -48.49 -49.5 -50.51 -51.52 -52.53 -53.54 -54.55 -55.56 -56.57 -57.58 -58.59 -59.6 -60.61 -61.62 -62.63 -63.64 -64.65 -65.66 -66.67 -67.68 -68.69 -69.7 -70.71 -71.72 -72.73 -73.74 -74.75 -75.76 -76.77 -77.78 -78.79 -79.8 -80.81 -81.82 -82.83 -83.84 -84.85 -85.86 -86.87 -87.88 -88.89 -89.9 -90.91 -91.92 -92.93 -93.94 -94.95 -95.96 -96.97 -97.98 -98.99 -99.1 -100.101 -101.102 -102.103 -103.104 -104.105 -105.106 -106.107 -107.108 -108.109 -109.11 -110.111 -111.112 -112.113 -113.114 -114.115 -115.116 -116.117 -117.118 -118.119 -119.12 -120.121 -121.122 -122.123 -123.124 -124.125 -125.126 -126.127 -127.128 -128.129 -129.13 -130.131 -131.132 -132.133 -133.134 -134.135 -135.136 -136.137 -137.138 -138.139 -139.14 -140.141 -141.142 -142.143 -143.144 -144.145 -145.146 -146.147 -147.148 -148.149 -149.15 -150.151 -151.152 -152.153 -153.154 -154.155 -155.156 -156.157 -157.158 -158.159 -159.16 -160.161 -161.162 -162.163 -163.164 -164.165 -165.166 -166.167 -167.168 -168.169 -169.17 -170.171 -171.172 -172.173 -173.174 -174.175 -175.176 -176.177 -177.178 -178.179 -179.18 -180.181 -181.182 -182.183 -183.184 -184.185 -185.186 -186.187 -187.188 -188.189 -189.19 -190.191 -191.192 -192.193 -193.194 -194.195 -195.196 -196.197 -197.198 -198.199 -199.2 -200.201 -201.202 -202.203 -203.204 -204.205 -205.206 -206.207 -207.208 -208.209 -209.21 -210.211 -211.212 -212.213 -213.214 -214.215 -215.216 -216.217 -217.218 -218.219 -219.22 -220.221 -221.222 -222.223 -223.224 -224.225 -225.226 -226.227 -227.228 -228.229 -229.23 -230.231 -231.232 -232.233 -233.234 -234.235 -235.236 -236.237 -237.238 -238.239 -239.24 -240.241 -241.242 -242.243 -243.244 -244.245 -245.246 -246.247 -247.248 -248.249 -249.25 -250.251 -251.252 -252.253 -253.254 -254.255 -255.256 -256.257 -257.258 -258.259 -259.26 -260.261 -261.262 -262.263 -263.264 -264.265 -265.266 -266.267 -267.268 -268.269 -269.27 -270.271 -271.272 -272.273 -273.274 -274.275 -275.276 -276.277 -277.278 -278.279 -279.28 -280.281 -281.282 -282.283 -283.284 -284.285 -285.286 -286.287 -287.288 -288.289 -289.29 -290.291 -291.292 -292.293 -293.294 -294.295 -295.296 -296.297 -297.298 -298.299 -299.3 -300.301 -301.302 -302.303 -303.304 -304.305 -305.306 -306.307 -307.308 -308.309 -309.31 -310.311 -311.312 -312.313 -313.314 -314.315 -315.316 -316.317 -317.318 -318.319 -319.32 -320.321 -321.322 -322.323 -323.324 -324.325 -325.326 -326.327 -327.328 -328.329 -329.33 -330.331 -331.332 -332.333 -333.334 -334.335 -335.336 -336.337 -337.338 -338.339 -339.34 -340.341 -341.342 -342.343 -343.344 -344.345 -345.346 -346.347 -347.348 -348.349 -349.35 -350.351 -351.352 -352.353 -353.354 -354.355 -355.356 -356.357 -357.358 -358.359 -359.36 -360.361 -361.362 -362.363 -363.364 -364.365 -365.366 -366.367 -367.368 -368.369 -369.37 -370.371 -371.372 -372.373 -373.374 -374.375 -375.376 -376.377 -377.378 -378.379 -379.38 -380.381 -381.382 -382.383 -383.384 -384.385 -385.386 -386.387 -387.388 -388.389 -389.39 -390.391 -391.392 -392.393 -393.394 -394.395 -395.396 -396.397 -397.398 -398.399 -399.4 -400.401 -401.402 -402.403 -403.404 -404.405 -405.406 -406.407 -407.408 -408.409 -409.41 -410.411 -411.412 -412.413 -413.414 -414.415 -415.416 -416.417 -417.418 -418.419 -419.42 -420.421 -421.422 -422.423 -423.424 -424.425 -425.426 -426.427 -427.428 -428.429 -429.43 -430.431 -431.432 -432.433 -433.434 -434.435 -435.436 -436.437 -437.438 -438.439 -439.44 -440.441 -441.442 -442.443 -443.444 -444.445 -445.446 -446.447 -447.448 -448.449 -449.45 -450.451 -451.452 -452.453 -453.454 -454.455 -455.456 -456.457 -457.458 -458.459 -459.46 -460.461 -461.462 -462.463 -463.464 -464.465 -465.466 -466.467 -467.468 -468.469 -469.47 -470.471 -471.472 -472.473 -473.474 -474.475 -475.476 -476.477 -477.478 -478.479 -479.48 -480.481 -481.482 -482.483 -483.484 -484.485 -485.486 -486.487 -487.488 -488.489 -489.49 -490.491 -491.492 -492.493 -493.494 -494.495 -495.496 -496.497 -497.498 -498.499 -499.5 -500.501 -501.502 -502.503 -503.504 -504.505 -505.506 -506.507 -507.508 -508.509 -509.51 -510.511 -511.512 -512.513 -513.514 -514.515 -515.516 -516.517 -517.518 -518.519 -519.52 -520.521 -521.522 -522.523 -523.524 -524.525 -525.526 -526.527 -527.528 -528.529 -529.53 -530.531 -531.532 -532.533 -533.534 -534.535 -535.536 -536.537 -537.538 -538.539 -539.54 -540.541 -541.542 -542.543 -543.544 -544.545 -545.546 -546.547 -547.548 -548.549 -549.55 -550.551 -551.552 -552.553 -553.554 -554.555 -555.556 -556.557 -557.558 -558.559 -559.56 -560.561 -561.562 -562.563 -563.564 -564.565 -565.566 -566.567 -567.568 -568.569 -569.57 -570.571 -571.572 -572.573 -573.574 -574.575 -575.576 -576.577 -577.578 -578.579 -579.58 -580.581 -581.582 -582.583 -583.584 -584.585 -585.586 -586.587 -587.588 -588.589 -589.59 -590.591 -591.592 -592.593 -593.594 -594.595 -595.596 -596.597 -597.598 -598.599 -599.6 -600.601 -601.602 -602.603 -603.604 -604.605 -605.606 -606.607 -607.608 -608.609 -609.61 -610.611 -611.612 -612.613 -613.614 -614.615 -615.616 -616.617 -617.618 -618.619 -619.62 -620.621 -621.622 -622.623 -623.624 -624.625 -625.626 -626.627 -627.628 -628.629 -629.63 -630.631 -631.632 -632.633 -633.634 -634.635 -635.636 -636.637 -637.638 -638.639 -639.64 -640.641 -641.642 -642.643 -643.644 -644.645 -645.646 -646.647 -647.648 -648.649 -649.65 -650.651 -651.652 -652.653 -653.654 -654.655 -655.656 -656.657 -657.658 -658.659 -659.66 -660.661 -661.662 -662.663 -663.664 -664.665 -665.666 -666.667 -667.668 -668.669 -669.67 -670.671 -671.672 -672.673 -673.674 -674.675 -675.676 -676.677 -677.678 -678.679 -679.68 -680.681 -681.682 -682.683 -683.684 -684.685 -685.686 -686.687 -687.688 -688.689 -689.69 -690.691 -691.692 -692.693 -693.694 -694.695 -695.696 -696.697 -697.698 -698.699 -699.7 -700.701 -701.702 -702.703 -703.704 -704.705 -705.706 -706.707 -707.708 -708.709 -709.71 -710.711 -711.712 -712.713 -713.714 -714.715 -715.716 -716.717 -717.718 -718.719 -719.72 -720.721 -721.722 -722.723 -723.724 -724.725 -725.726 -726.727 -727.728 -728.729 -729.73 -730.731 -731.732 -732.733 -733.734 -734.735 -735.736 -736.737 -737.738 -738.739 -739.74 -740.741 -741.742 -742.743 -743.744 -744.745 -745.746 -746.747 -747.748 -748.749 -749.75 -750.751 -751.752 -752.753 -753.754 -754.755 -755.756 -756.757 -757.758 -758.759 -759.76 -760.761 -761.762 -762.763 -763.764 -764.765 -765.766 -766.767 -767.768 -768.769 -769.77 -770.771 -771.772 -772.773 -773.774 -774.775 -775.776 -776.777 -777.778 -778.779 -779.78 -780.781 -781.782 -782.783 -783.784 -784.785 -785.786 -786.787 -787.788 -788.789 -789.79 -790.791 -791.792 -792.793 -793.794 -794.795 -795.796 -796.797 -797.798 -798.799 -799.8 -800.801 -801.802 -802.803 -803.804 -804.805 -805.806 -806.807 -807.808 -808.809 -809.81 -810.811 -811.812 -812.813 -813.814 -814.815 -815.816 -816.817 -817.818 -818.819 -819.82 -820.821 -821.822 -822.823 -823.824 -824.825 -825.826 -826.827 -827.828 -828.829 -829.83 -830.831 -831.832 -832.833 -833.834 -834.835 -835.836 -836.837 -837.838 -838.839 -839.84 -840.841 -841.842 -842.843 -843.844 -844.845 -845.846 -846.847 -847.848 -848.849 -849.85 -850.851 -851.852 -852.853 -853.854 -854.855 -855.856 -856.857 -857.858 -858.859 -859.86 -860.861 -861.862 -862.863 -863.864 -864.865 -865.866 -866.867 -867.868 -868.869 -869.87 -870.871 -871.872 -872.873 -873.874 -874.875 -875.876 -876.877 -877.878 -878.879 -879.88 -880.881 -881.882 -882.883 -883.884 -884.885 -885.886 -886.887 -887.888 -888.889 -889.89 -890.891 -891.892 -892.893 -893.894 -894.895 -895.896 -896.897 -897.898 -898.899 -899.9 -900.901 -901.902 -902.903 -903.904 -904.905 -905.906 -906.907 -907.908 -908.909 -909.91 -910.911 -911.912 -912.913 -913.914 -914.915 -915.916 -916.917 -917.918 -918.919 -919.92 -920.921 -921.922 -922.923 -923.924 -924.925 -925.926 -926.927 -927.928 -928.929 -929.93 -930.931 -931.932 -932.933 -933.934 -934.935 -935.936 -936.937 -937.938 -938.939 -939.94 -940.941 -941.942 -942.943 -943.944 -944.945 -945.946 -946.947 -947.948 -948.949 -949.95 -950.951 -951.952 -952.953 -953.954 -954.955 -955.956 -956.957 -957.958 -958.959 -959.96 -960.961 -961.962 -962.963 -963.964 -964.965 -965.966 -966.967 -967.968 -968.969 -969.97 -970.971 -971.972 -972.973 -973.974 -974.975 -975.976 -976.977 -977.978 -978.979 -979.98 -980.981 -981.982 -982.983 -983.984 -984.985 -985.986 -986.987 -987.988 -988.989 -989.99 -990.991 -991.992 -992.993 -993.994 -994.995 -995.996 -996.997 -997.998 -998.999 -999.1 -1000.1001 -1001.1002 -1002.1003 -1003.1004 -1004.1005 -1005.1006 -1006.1007 -1007.1008 -1008.1009 -1009.101 -1010.1011 -1011.1012 -1012.1013 -1013.1014 -1014.1015 -1015.1016 -1016.1017 -1017.1018 -1018.1019 -1019.102 -1020.1021 -1021.1022 -1022.1023 -1023.1024 -1024.1025 -1025.1026 -1026.1027 -1027.1028 -1028.1029 -1029.103 -1030.1031 -1031.1032 -1032.1033 -1033.1034 -1034.1035 -1035.1036 -1036.1037 -1037.1038 -1038.1039 -1039.104 -1040.1041 -1041.1042 -1042.1043 -1043.1044 -1044.1045 -1045.1046 -1046.1047 -1047.1048 -1048.1049 -1049.105 -1050.1051 -1051.1052 -1052.1053 -1053.1054 -1054.1055 -1055.1056 -1056.1057 -1057.1058 -1058.1059 -1059.106 -1060.1061 -1061.1062 -1062.1063 -1063.1064 -1064.1065 -1065.1066 -1066.1067 -1067.1068 -1068.1069 -1069.107 -1070.1071 -1071.1072 -1072.1073 -1073.1074 -1074.1075 -1075.1076 -1076.1077 -1077.1078 -1078.1079 -1079.108 -1080.1081 -1081.1082 -1082.1083 -1083.1084 -1084.1085 -1085.1086 -1086.1087 -1087.1088 -1088.1089 -1089.109 -1090.1091 -1091.1092 -1092.1093 -1093.1094 -1094.1095 -1095.1096 -1096.1097 -1097.1098 -1098.1099 -1099.11 -1100.1101 -1101.1102 -1102.1103 -1103.1104 -1104.1105 -1105.1106 -1106.1107 -1107.1108 -1108.1109 -1109.111 -1110.1111 -1111.1112 -1112.1113 -1113.1114 -1114.1115 -1115.1116 -1116.1117 -1117.1118 -1118.1119 -1119.112 -1120.1121 -1121.1122 -1122.1123 -1123.1124 -1124.1125 -1125.1126 -1126.1127 -1127.1128 -1128.1129 -1129.113 -1130.1131 -1131.1132 -1132.1133 -1133.1134 -1134.1135 -1135.1136 -1136.1137 -1137.1138 -1138.1139 -1139.114 -1140.1141 -1141.1142 -1142.1143 -1143.1144 -1144.1145 -1145.1146 -1146.1147 -1147.1148 -1148.1149 -1149.115 -1150.1151 -1151.1152 -1152.1153 -1153.1154 -1154.1155 -1155.1156 -1156.1157 -1157.1158 -1158.1159 -1159.116 -1160.1161 -1161.1162 -1162.1163 -1163.1164 -1164.1165 -1165.1166 -1166.1167 -1167.1168 -1168.1169 -1169.117 -1170.1171 -1171.1172 -1172.1173 -1173.1174 -1174.1175 -1175.1176 -1176.1177 -1177.1178 -1178.1179 -1179.118 -1180.1181 -1181.1182 -1182.1183 -1183.1184 -1184.1185 -1185.1186 -1186.1187 -1187.1188 -1188.1189 -1189.119 -1190.1191 -1191.1192 -1192.1193 -1193.1194 -1194.1195 -1195.1196 -1196.1197 -1197.1198 -1198.1199 -1199.12 -1200.1201 -1201.1202 -1202.1203 -1203.1204 -1204.1205 -1205.1206 -1206.1207 -1207.1208 -1208.1209 -1209.121 -1210.1211 -1211.1212 -1212.1213 -1213.1214 -1214.1215 -1215.1216 -1216.1217 -1217.1218 -1218.1219 -1219.122 -1220.1221 -1221.1222 -1222.1223 -1223.1224 -1224.1225 -1225.1226 -1226.1227 -1227.1228 -1228.1229 -1229.123 -1230.1231 -1231.1232 -1232.1233 -1233.1234 -1234.1235 -1235.1236 -1236.1237 -1237.1238 -1238.1239 -1239.124 -1240.1241 -1241.1242 -1242.1243 -1243.1244 -1244.1245 -1245.1246 -1246.1247 -1247.1248 -1248.1249 -1249.125 -1250.1251 -1251.1252 -1252.1253 -1253.1254 -1254.1255 -1255.1256 -1256.1257 -1257.1258 -1258.1259 -1259.126 -1260.1261 -1261.1262 -1262.1263 -1263.1264 -1264.1265 -1265.1266 -1266.1267 -1267.1268 -1268.1269 -1269.127 -1270.1271 -1271.1272 -1272.1273 -1273.1274 -1274.1275 -1275.1276 -1276.1277 -1277.1278 -1278.1279 -1279.128 -1280.1281 -1281.1282 -1282.1283 -1283.1284 -1284.1285 -1285.1286 -1286.1287 -1287.1288 -1288.1289 -1289.129 -1290.1291 -1291.1292 -1292.1293 -1293.1294 -1294.1295 -1295.1296 -1296.1297 -1297.1298 -1298.1299 -1299.13 -1300.1301 -1301.1302 -1302.1303 -1303.1304 -1304.1305 -1305.1306 -1306.1307 -1307.1308 -1308.1309 -1309.131 -1310.1311 -1311.1312 -1312.1313 -1313.1314 -1314.1315 -1315.1316 -1316.1317 -1317.1318 -1318.1319 -1319.132 -1320.1321 -1321.1322 -1322.1323 -1323.1324 -1324.1325 -1325.1326 -1326.1327 -1327.1328 -1328.1329 -1329.133 -1330.1331 -1331.1332 -1332.1333 -1333.1334 -1334.1335 -1335.1336 -1336.1337 -1337.1338 -1338.1339 -1339.134 -1340.1341 -1341.1342 -1342.1343 -1343.1344 -1344.1345 -1345.1346 -1346.1347 -1347.1348 -1348.1349 -1349.135 -1350.1351 -1351.1352 -1352.1353 -1353.1354 -1354.1355 -1355.1356 -1356.1357 -1357.1358 -1358.1359 -1359.136 -1360.1361 -1361.1362 -1362.1363 -1363.1364 -1364.1365 -1365.1366 -1366.1367 -1367.1368 -1368.1369 -1369.137 -1370.1371 -1371.1372 -1372.1373 -1373.1374 -1374.1375 -1375.1376 -1376.1377 -1377.1378 -1378.1379 -1379.138 -1380.1381 -1381.1382 -1382.1383 -1383.1384 -1384.1385 -1385.1386 -1386.1387 -1387.1388 -1388.1389 -1389.139 -1390.1391 -1391.1392 -1392.1393 -1393.1394 -1394.1395 -1395.1396 -1396.1397 -1397.1398 -1398.1399 -1399.14 -1400.1401 -1401.1402 -1402.1403 -1403.1404 -1404.1405 -1405.1406 -1406.1407 -1407.1408 -1408.1409 -1409.141 -1410.1411 -1411.1412 -1412.1413 -1413.1414 -1414.1415 -1415.1416 -1416.1417 -1417.1418 -1418.1419 -1419.142 -1420.1421 -1421.1422 -1422.1423 -1423.1424 -1424.1425 -1425.1426 -1426.1427 -1427.1428 -1428.1429 -1429.143 -1430.1431 -1431.1432 -1432.1433 -1433.1434 -1434.1435 -1435.1436 -1436.1437 -1437.1438 -1438.1439 -1439.144 -1440.1441 -1441.1442 -1442.1443 -1443.1444 -1444.1445 -1445.1446 -1446.1447 -1447.1448 -1448.1449 -1449.145 -1450.1451 -1451.1452 -1452.1453 -1453.1454 -1454.1455 -1455.1456 -1456.1457 -1457.1458 -1458.1459 -1459.146 -1460.1461 -1461.1462 -1462.1463 -1463.1464 -1464.1465 -1465.1466 -1466.1467 -1467.1468 -1468.1469 -1469.147 -1470.1471 -1471.1472 -1472.1473 -1473.1474 -1474.1475 -1475.1476 -1476.1477 -1477.1478 -1478.1479 -1479.148 -1480.1481 -1481.1482 -1482.1483 -1483.1484 -1484.1485 -1485.1486 -1486.1487 -1487.1488 -1488.1489 -1489.149 -1490.1491 -1491.1492 -1492.1493 -1493.1494 -1494.1495 -1495.1496 -1496.1497 -1497.1498 -1498.1499 -1499.15 -1500.1501 -1501.1502 -1502.1503 -1503.1504 -1504.1505 -1505.1506 -1506.1507 -1507.1508 -1508.1509 -1509.151 -1510.1511 -1511.1512 -1512.1513 -1513.1514 -1514.1515 -1515.1516 -1516.1517 -1517.1518 -1518.1519 -1519.152 -1520.1521 -1521.1522 -1522.1523 -1523.1524 -1524.1525 -1525.1526 -1526.1527 -1527.1528 -1528.1529 -1529.153 -1530.1531 -1531.1532 -1532.1533 -1533.1534 -1534.1535 -1535.1536 -1536.1537 -1537.1538 -1538.1539 -1539.154 -1540.1541 -1541.1542 -1542.1543 -1543.1544 -1544.1545 -1545.1546 -1546.1547 -1547.1548 -1548.1549 -1549.155 -1550.1551 -1551.1552 -1552.1553 -1553.1554 -1554.1555 -1555.1556 -1556.1557 -1557.1558 -1558.1559 -1559.156 -1560.1561 -1561.1562 -1562.1563 -1563.1564 -1564.1565 -1565.1566 -1566.1567 -1567.1568 -1568.1569 -1569.157 -1570.1571 -1571.1572 -1572.1573 -1573.1574 -1574.1575 -1575.1576 -1576.1577 -1577.1578 -1578.1579 -1579.158 -1580.1581 -1581.1582 -1582.1583 -1583.1584 -1584.1585 -1585.1586 -1586.1587 -1587.1588 -1588.1589 -1589.159 -1590.1591 -1591.1592 -1592.1593 -1593.1594 -1594.1595 -1595.1596 -1596.1597 -1597.1598 -1598.1599 -1599.16 -1600.1601 -1601.1602 -1602.1603 -1603.1604 -1604.1605 -1605.1606 -1606.1607 -1607.1608 -1608.1609 -1609.161 -1610.1611 -1611.1612 -1612.1613 -1613.1614 -1614.1615 -1615.1616 -1616.1617 -1617.1618 -1618.1619 -1619.162 -1620.1621 -1621.1622 -1622.1623 -1623.1624 -1624.1625 -1625.1626 -1626.1627 -1627.1628 -1628.1629 -1629.163 -1630.1631 -1631.1632 -1632.1633 -1633.1634 -1634.1635 -1635.1636 -1636.1637 -1637.1638 -1638.1639 -1639.164 -1640.1641 -1641.1642 -1642.1643 -1643.1644 -1644.1645 -1645.1646 -1646.1647 -1647.1648 -1648.1649 -1649.165 -1650.1651 -1651.1652 -1652.1653 -1653.1654 -1654.1655 -1655.1656 -1656.1657 -1657.1658 -1658.1659 -1659.166 -1660.1661 -1661.1662 -1662.1663 -1663.1664 -1664.1665 -1665.1666 -1666.1667 -1667.1668 -1668.1669 -1669.167 -1670.1671 -1671.1672 -1672.1673 -1673.1674 -1674.1675 -1675.1676 -1676.1677 -1677.1678 -1678.1679 -1679.168 -1680.1681 -1681.1682 -1682.1683 -1683.1684 -1684.1685 -1685.1686 -1686.1687 -1687.1688 -1688.1689 -1689.169 -1690.1691 -1691.1692 -1692.1693 -1693.1694 -1694.1695 -1695.1696 -1696.1697 -1697.1698 -1698.1699 -1699.17 -1700.1701 -1701.1702 -1702.1703 -1703.1704 -1704.1705 -1705.1706 -1706.1707 -1707.1708 -1708.1709 -1709.171 -1710.1711 -1711.1712 -1712.1713 -1713.1714 -1714.1715 -1715.1716 -1716.1717 -1717.1718 -1718.1719 -1719.172 -1720.1721 -1721.1722 -1722.1723 -1723.1724 -1724.1725 -1725.1726 -1726.1727 -1727.1728 -1728.1729 -1729.173 -1730.1731 -1731.1732 -1732.1733 -1733.1734 -1734.1735 -1735.1736 -1736.1737 -1737.1738 -1738.1739 -1739.174 -1740.1741 -1741.1742 -1742.1743 -1743.1744 -1744.1745 -1745.1746 -1746.1747 -1747.1748 -1748.1749 -1749.175 -1750.1751 -1751.1752 -1752.1753 -1753.1754 -1754.1755 -1755.1756 -1756.1757 -1757.1758 -1758.1759 -1759.176 -1760.1761 -1761.1762 -1762.1763 -1763.1764 -1764.1765 -1765.1766 -1766.1767 -1767.1768 -1768.1769 -1769.177 -1770.1771 -1771.1772 -1772.1773 -1773.1774 -1774.1775 -1775.1776 -1776.1777 -1777.1778 -1778.1779 -1779.178 -1780.1781 -1781.1782 -1782.1783 -1783.1784 -1784.1785 -1785.1786 -1786.1787 -1787.1788 -1788.1789 -1789.179 -1790.1791 -1791.1792 -1792.1793 -1793.1794 -1794.1795 -1795.1796 -1796.1797 -1797.1798 -1798.1799 -1799.18 -1800.1801 -1801.1802 -1802.1803 -1803.1804 -1804.1805 -1805.1806 -1806.1807 -1807.1808 -1808.1809 -1809.181 -1810.1811 -1811.1812 -1812.1813 -1813.1814 -1814.1815 -1815.1816 -1816.1817 -1817.1818 -1818.1819 -1819.182 -1820.1821 -1821.1822 -1822.1823 -1823.1824 -1824.1825 -1825.1826 -1826.1827 -1827.1828 -1828.1829 -1829.183 -1830.1831 -1831.1832 -1832.1833 -1833.1834 -1834.1835 -1835.1836 -1836.1837 -1837.1838 -1838.1839 -1839.184 -1840.1841 -1841.1842 -1842.1843 -1843.1844 -1844.1845 -1845.1846 -1846.1847 -1847.1848 -1848.1849 -1849.185 -1850.1851 -1851.1852 -1852.1853 -1853.1854 -1854.1855 -1855.1856 -1856.1857 -1857.1858 -1858.1859 -1859.186 -1860.1861 -1861.1862 -1862.1863 -1863.1864 -1864.1865 -1865.1866 -1866.1867 -1867.1868 -1868.1869 -1869.187 -1870.1871 -1871.1872 -1872.1873 -1873.1874 -1874.1875 -1875.1876 -1876.1877 -1877.1878 -1878.1879 -1879.188 -1880.1881 -1881.1882 -1882.1883 -1883.1884 -1884.1885 -1885.1886 -1886.1887 -1887.1888 -1888.1889 -1889.189 -1890.1891 -1891.1892 -1892.1893 -1893.1894 -1894.1895 -1895.1896 -1896.1897 -1897.1898 -1898.1899 -1899.19 -1900.1901 -1901.1902 -1902.1903 -1903.1904 -1904.1905 -1905.1906 -1906.1907 -1907.1908 -1908.1909 -1909.191 -1910.1911 -1911.1912 -1912.1913 -1913.1914 -1914.1915 -1915.1916 -1916.1917 -1917.1918 -1918.1919 -1919.192 -1920.1921 -1921.1922 -1922.1923 -1923.1924 -1924.1925 -1925.1926 -1926.1927 -1927.1928 -1928.1929 -1929.193 -1930.1931 -1931.1932 -1932.1933 -1933.1934 -1934.1935 -1935.1936 -1936.1937 -1937.1938 -1938.1939 -1939.194 -1940.1941 -1941.1942 -1942.1943 -1943.1944 -1944.1945 -1945.1946 -1946.1947 -1947.1948 -1948.1949 -1949.195 -1950.1951 -1951.1952 -1952.1953 -1953.1954 -1954.1955 -1955.1956 -1956.1957 -1957.1958 -1958.1959 -1959.196 -1960.1961 -1961.1962 -1962.1963 -1963.1964 -1964.1965 -1965.1966 -1966.1967 -1967.1968 -1968.1969 -1969.197 -1970.1971 -1971.1972 -1972.1973 -1973.1974 -1974.1975 -1975.1976 -1976.1977 -1977.1978 -1978.1979 -1979.198 -1980.1981 -1981.1982 -1982.1983 -1983.1984 -1984.1985 -1985.1986 -1986.1987 -1987.1988 -1988.1989 -1989.199 -1990.1991 -1991.1992 -1992.1993 -1993.1994 -1994.1995 -1995.1996 -1996.1997 -1997.1998 -1998.1999 -1999.2 diff --git a/flink-connectors/flink-orc/src/test/resources/demo-11-none.orc b/flink-connectors/flink-orc/src/test/resources/demo-11-none.orc deleted file mode 100644 index 1d1d714a846259ec5b2b0471e55eec94efd7671d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5147970 zcmeFa0dP}wzxVk+NtzP^LK;vG)JM49WqD_3M|PZdo$c<9Gj|TF`_3-xJNLdUJ3H&n z%f5Df?{bA5c6rr(_Sx2mu>>eJV1Q6-janc;sX=Ou8cL8_qSP9oK!94K)*7IcDgh$a z{hm|oQ%Y#lOWM@>of|(VoFq;1%{kvY^UVKqe*eSYtJXZb(Na3!dcV~mgnb(B+N~CY zt$eZ4v?V#&Iz!(*6}umvWR;$12X z7y3-reEsps+6&DVVH1kTY*(>lD3+Y~&t)o>%FM-*+fpofti_UdO0iTqwOI1milu6M zvE-joEY&+?@B0WPxr|@?2qn2a0*po;gggj&5V8r` zgls}KA)Am*$R=bHvI*IQY(h36n~+U-{kQ*##vr7Hi~eEBQdyW}ktYkIEQBm1Sr}wt z=1vB}gMlXko8foeiZ>=to+itaSB|K(p7p#R;||6$pRe^>PMt@Y^Y%76N&qGyW! z^{Rh-cG;>I{-NmKtz7Xm?%Z0Wu8K}s6)n?LQ9n}^RdZBPI$sstkEr7GW2$I)TopT? zP(}G-Ra{@9iuR{e(Nv_0+GkaiGI_{_hb`}hC&e=Yy-S3ak2Z~cfve5z4; zsBp+ACfJ)=%@#8R3|Zew%1nK0P#(~?##>vHGGG69o-#|{nw7cwR#v9#Tg$EOBy44i zWRsNbl77da)M~FZ*bIshTO+mx{Q=`|4;XJfU{vz-2TaPj$@(iyiba2gNm2C&@{~OW z?2v3`<$3)HX1qfClV;^XeZQ=%*0+{h+eyl5yTKy#|4ewD!LBh+r`*5S)c+hj_cw*+ zHnGLMSQlHGu7CZvgLIjDmS{>ZrI%=0&R(1~tVCniC7LOkL?c6@F(~Z|Gqq^ki=nOj zYiKin4{he(K%4m=q0RhH&}RN;Xfyu{w3+`E+RVR&HuJwhn|T`A%>NE;=HEk``9Gk| z{1dd970_n>PiQl%pw0Z>&}RMvw3+9j&HO*mX8vDjGyfm7nGy-!j05Oh12|p*oNEEs zI-qO=;NA#$UIo0HfXdeZ-xi>Hn^Y+Iw@X~Fzyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m_W`Gpj7kQ0);+{h$}X@c8zP-xOR>Oj{!tKXYR62hQFLd;t~o3B zS$q|etu`SkO=ph%SJ^ge?lzvLBpyp)Wf;)kb9|-RO8f$<^EzleQqVEB%bwI2hXbS@IkARL4kZ1t9!a&a_ zK(Y}?H37X5z;O(4egU|?1j>#B?vsG$6yQArRGtNV=Yi_40RPuO%{M^cJD~1+I5)u` z=-h~$uczP#U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R}{5Hg=Z-x(76>(Kj z&Q}%jAml;FgOH1SxyYAI$R=bHvI*IQY(h36n~+V&CQRepOth8eoIk?(BfsnUBmD3j zrmeIt7{>V;YTHf%a$rpxVuB zU8BidySe`^+RfoyXgAuN8_AZ1c{aCcH-Us?Yp6FUO~DWTA=}zb=Y{kGZL0RH8G4{i zdHEpi#)P>(Rm~Y|H~3OeSS|v=E+EtmgnNL-pMXdbXub|aQ$XuYAl3`Cl^CGdIDn2d zK*9-hl>$93Ah{k$l>xowfMXNjd<}4I0m`-k?(Kl*b-?=uQ28d{dkd(32k`FzYIXsE z-9X)5Ah-_*y$gi*1C0j^g;L~w16MRK0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XB zkn;qn-B7!sb|XZea@2P$aCIA3w{dkFSGRGUFPo4}$R=bHvI*IQY(h36n~+V&Cd_#g zrqOOV>ID6p3q3m{L#ESi@TH)z1cAs$KywI)HUO<*AodB+)(FI# zfQ|@|_#EhJ26|e6WE4o90D4;i#~HwR7I2*h%Dw{JUjv?R0PlA|<@bQ^2cY@_;J*aa zTm}MHfVv-n;8h@W4G7-=8h-{NzW~j@0?`+Z_6S4`}glxhz+KqwZVK^R!6IM83B`XtFXfMHh7$?V47PXsvs3fX5{g^5m z9#_TACsa|sSQXcosG|KTRWuc;qW0N|s?n$;{qC}+CH0Fp1MgrS%@76-9=b;f2Lo#| zux|c2E={3k&RFi@M(bwWSvLWLZq=9-ZK%ztS~s(-1|jUzEv57AQ!P=a(bj$f*3HfG zm!xcKHw|~0cGG=xyJCGv{}>8B#>3OCrYBdfeA+HAS+RV@%I8)n4ci83Hx`>!X+M>* zc7tyPg{2gTxq!CyK)ej-C$?p8~B%f!Jq2+c6;i z1<>&&w420nYBz#wI+y?xU;<2l2`~XBzyz286JP>NfC(@GCcp%kK+Y4Ob~8{n5Q1wu zxVnw2+qk-otJ~OwY(h36n~+V&CS((`3E6~fLN+0rkWHBLCQPH*W$ z7HCiYgK9SgFj~wu!>u7V<7?Q>zTvbRO~;Wi^rdg;%_L>D-Cz;=e7IZ z`k#a6TJ!8iNG{gfw43+|qpjO#Qku@+^k!YVslS+KovGV3`I_W2#nLI;Fp*0sS$A^w z$g~@)ZHm%RVa#;e4Zal=mR2Bf8t94vJ?DUA8<1)TdgFlOJHYup;Q9e5y8yT^0iMf% z_X<$?BjCFVR9^%9H-MU-fxs_7-LF9KMH3X8mw@ogK;tVwWG&FV4v205S~mi*SAn)o zK>RhJV+)Yj26Sx)dR_;TZNfC(@GCcp%k z025#WOn}-AwHr>m;j|l0yD^yto(EiF$t9LtV#y_zIbUMQgOCRy4?_OGWB$KmHX)mk zO~@u>6Q;F5F7yArFY~9p#KBQKi^tDdoJH-XfD>*;G~EZ@&7fQUm}oc|%(@wY zFO!K`vs80xrVMdgraemOX57uXsU1$b>62};E#36^Otz9MkZwYqfvih6-HG(lO~bMM zGl!IJN)8U1bt53%w68T;bU9|Qh_K+Pc_a2TjN0t7z=LPvq{XF%gIAo2y!{3Q@Q z4z!*GVyA$%GeG<-&~Y9}dm3xQ$K7CS((` z3E6~fLN+0rkWI)YWD~LpbKZn$tQ!*-%8#Fg^0MI1VdT%jc{e-}WO1)2LKdwXRdmX# zXql#p`kAVznxl%+`KsuCL=~qWQ$@q$Dyehw@7O_jrv;;`f7zj=)-MdltWZknc!o4RGH8JU;{8Ux3PA0pE*xP;6cT{4WDFuKSZvlySfUX@t&n_Uj8%XU1diMd20N{KNaMb~2^?*AF zcs|N2l)RxlUPqJ(FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^xoF_o-hT09a8xFhS zup17$VH2_m*@SFDHX)mkO~@u>6S4`}gls}K;W%r;G}?{ej1tZ$p?XEvhpx{AbA7U? z-OPhdqKc*>Rn$JKimj_vQSuk4IBO)hIO~D>%|KNJKs~1rYSKU*V^KOQWS}RKEfYu0 zNWB?56=k*l#nn5En1SDYls0 z;#rq&?l$bEBpyZ5@FGoa=e z5cmS9`w|Eq2SO)-@F}423=laBG@l2eUjeOO1F>&_w(o%W_dv%FK;i<>bqVOX3?#1r zsULyftAOJ=;7kFon?PAF;4U%a^Em+T8lch%_)39lm$^{#uQ%s>IRQTa6JP>NfC(@G zCcp%k025#WOn?b60Vco%m;e)C0+epJtcc5sxU6WLEi2+BrFlu|oL^F!2O$qa9)z6m zzzGj*LN+0rkWI)YWD~LpbKZn$q#N@DTWU^&h6W7{8X7dLwJb_EGH2XmJl`iDQqmB{ zN5QB$I7@{wgLyZDhr*1K_R{}nb^j$NZ%XOLHpHZx97#964Q5;ISx7fu?@nc1y6L=- ze$q|Vo;5>Hy6N_rtkbOqA?){AO6S`xm?mUWN<30p6DMDP;$-cKW{a>1MZjR$pcyP1 z0spH&%_bo58c??d2yO#H+kx=wK;s)g0%1M$5;$37tOF3`0f z=s5r+-v?430KFdqju7B%09;|9>=VG<2zZ(RZv?3P9Pl**)h&QO3e=nc0NfC(@GCcp%k025#WOd#h8(7K^@L+ggt4Xqnq)|^eq zCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMk)$a6sRc3f+vE+6Ja8q2wAjl7I3^x z#-n`}i0(&|7XJE z2D`>QopS$PQ~z`DTx*`)2tma^8*04)fn!;P$2I~y{p{aQ2;Yn8M z3A?mNDPKHj??St@RNFgGf9@RZxwEa4AD#Qcs<{h`o>^5?WS5t$SiWNAb1Rg)xi<~T zqeHtlEfs^@8xw+Pk7e%O;A_ESIS&NC0zzK{;ctM(?|{hnK=Thk^a9X&35Z<=+O7cc zAAydmK;jzEbpz=68A$#Dq<#f@UzFk5tO1-(z*P#Axd8Wiz*7cz%YjNa;M)vTdjS7d zpvDUXDuB95AXo*2d_eeZps`vmlp;IjoUbb22VeqBfC(@GCcp%k025#WOn?b60Vco% zm;e)C0!)DJ4ObU&brDw=jkDE7ys$JcES>WUOYYi7~C$%jv(iqns&qTz8>?0iBM<%?BueTgdC zpHf9rkt%ARRq5yS`8l#^Suge5Hv{irCe5gtI0Zt=K)p$$+bq@881pFImT7;P-p#o4 zZUP3~r!nh(O*-$U&$r38cN5+v+gh%{yLsy+f7ZR5hP%wX>AtyLu|A}GJBsG*Ot+ez zT)Fb;zY06$<%7H%v(2J3ykW?A<_*3U9G2ZcWG~RX4~V`CwC)FD2Y|Nsf%peN$A>`T zW1#CG&~pe#9tKiJfZk65M+9(w4!D|uvKGJ{1w1DJZ!1uF8t}z{>T`g<4X9}c0&$@3 zTOim0ggSw60%*JlM7n_HZXntNwEhI|CYGdk!tO?V2 zH+h`)Aq!6XFjKqWiNF(KB1wo@^ll0@l|&Wwn}Mndpi@>w%QWnni5+uPQ956xnxkt; z^PnXa3q+J&z3JC*mSx+)S(K4mA&b*uGHQCmF z#G*8rAn9$~F|1QnK#k?2&ml7MoS6`Zja#244#f%MBp* zGtl-65dRhEc+mpK<|Ux(WuWI3Ah{Mutpj>D0FH9N=>}Yzfie%^-U@iUfVTputOR^j zK(!C>zYWw>1A(1DogWD90YWuExE5#(0Fn2A<~kr+53~k>*hfHH2#7ZT9brqMl=#HL zONlZ8Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaaiz^91PKaAgr!7I9_KI9pl7%S!XI z(mB7ZG!H@^gggj2<$+Tk*o16CHX)mkO~@u>6Xv`L(|9*jPN@E6gKqg_;^Aa4?Pg>~&1%h=vEIu~)m~2PX53jfwZmCAqciO$e#Bzy z-ZV*R`Yv`R>()(oBE5CfaBTm~A+4K+ZG)!WSZ!03l3m7(r`_Od!C?6mNE`*aJ_CA= z0m(0b)R#c-alp|EI8Otv7*KW&aJK=TcEB44D!&DM9YA#_;7$%4`ar3(Y43!VtF;E6DCPJ}F4Hym%1*=U~v(f-sR z-Dc1~myM2nlP?Sz7&rH>&rB&6Be<>Bo=oFr+!;4}a%0@|Y?@>%iNm;QY`l_n(ZIX`JB5Xntuva!|_R4EOY75Z24RBNd z&Pu>l1(f*!_uGJ{8u0D}D*b?O4^UkL_-lch01$W&sH+2l^*|^Hgg*irLqMbfXbuC> zPk`1&Al3x5MS%F{Ku0r>XaTySK+g#v*$Sji1HCc8aUO7f1>?r`HH{m=NfC(@GCcp%k025#WOd#h8(7556BCaXonxb*Griho5=H;YwemQ9#gggj& z5OTT$r#r9-*@SFDHX)mkO~@w9c@w5FZn(yN;;gZ!F(C{7nsn%lJP~jrWYM^p2a`k< zO+~7xeO47)SF57rFR*adNUCUA57ci4sw#j^8M~%o&rIx?ql(h`s_1@16{jClMZ@Fj zA7EZiA-?-S&t#G2oD3d&GjLC~Bpfv(>t+PP%&>M%zGl>jVQ=%a=VVj7fnQWCOvB#h zY0nw{<7M|bg+f@(aBPGk>sv`F(6DqY>{k|a#+&u7?fJ=l?IzZF=A`P)}TLN{Otkbtp|)sp8kMIIX78*^Hl}>08D@hFaajO1egF5U;<2l2`~XBzyz286JP>NfC6S4`}gls}KA)Am*$R=bHvI*IQr1p&cp}Jx zC&I)z5whss$bVp$W~3fbKHQ`sRFi^HQ*o9Gk-@~9(Q;9i-9@ljnwcjvtVlDWClvlZ zt(!jE#&XNP$+2~_`j&Omx6PJy{;rfXX_+R}WMN0slupO$Z1y0Ciy?_z4hd1j0=~V+4qN4m39d z(H5XJ3dBwTZLL83G|&+P66b)fHlU{+NXCKGw?JUI1K|fU?Vg`wHOs5%69G zDz5>)8$k8Xux|Xn(7NHo8z#U6m;e)C0!)AjFaajO1egF5U;<2l2`~XBkoyE^-EdtI z*A;PH(KuUI#0yLF!qPdvurv=s9)vsyIp=|M9@vCzLN+0rkWI)YWE1AR3Da0NTxUOV z*4fjVkOhBDdUSa_5pW`8(Yjf{DK{BU_*sAnKm9{gFnVbS0|yT+%c_Kfx644h$>#8x zA->&$0lp24#t~^VS$Q(uo6INP{8=A~W5$deZHSIpc~Ez5rM(~vwmw0qNXz?9yz$#~=qz86H6lF1Nl96-$)Am9Y*N`asY2(1UgWk6#& z5OD*|n}MhYXx$3Lyg*w85U&I}s(^$K=z1IIsRoiefs`NU-2*uG0nT>;*M6Yv0N{Qf z@O%JxKLjd227CvB>O+A4Fi>*@2z(0E9i3b#1wWh2tBEoJCcp%k025#WOn?b60Vco% zm;e)C0!)AjFaaiz^8_g0P`;sjL-~gC4KHubCS((`3E6~fLN+0rkWI)YWD~Lp*@SGu zan^)sthxQbp~vDg~WBK}QxX>!p7EX5bwRr5Qq2xpx((Ko~P{Z^l}K(H)qayy<9-_Zj$` z4!mC|1e_JKO(?Rym6QT~Yfv80x5isrlQLibcb+m!-}G6+-UJMIQwgHY z^Vk}&9W36A&d{5{=aX#>-YH5`Y_T3@gn3byUrSvvVzW$dc<1g9G7J=VeKw|k^lUSO8U<(k60^t)t zV=E9j4K&Aq=sBRZ4T!Y^ZE+y}Ezr>cBszhv1kiI4NOl3KZlJdZa9jnP*8tZIpzLSB z{R`mv74W_|1(MB6fbV6X`W3*x7N}VV1U3M58-d`fKxh*Xehp~cGNn+8Y@3qvB?bHd zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2~fP@(jqP`;?kmVwzP;Bm*&N#bAEAY z9)vsyc@T2e17|(33E6~fLN+0rkWI)Y%y|>05pTHEe&Q^(r#OMJ4>X8q5b;F7iI7F{ zrciT9R8hYfsHy-uWmU9H!=9PgF-H}p^HsVzx|cK$UQ)3zs=_!>$++|WvTRE@i!!rr zRBir@dCYFBwfCZR0~=?2SvUJ~W8H+kQ*14rux@sL-I;akrv74{b*8SwrT!dnRR2fS(K7JhRm%Sd@mR*6+omCXs!aHKA`n&AXW{u?F8a}pkoh^r~$fa zft~=6d=E&~0loEr<3qstG2l7~lpO-xhXKzK!22msc@*${22>vd{9gbyUjl*SK;20o zcnS!e0m5g2#`8erE1>ynAo>l^`W>vB*!Q$<1ebI$0Vco%m;e)C0!)AjFaajO1egF5 zU;<2l2{3`2CqV0l)(x#2S~s+AcxiJsA)Am*$R=bHvI*IQY(h36n~+V&CS((ivnEVq z-AJC;!?i69G}2*wF>B4p9JDd32k5smkOb~EUfKPDDV2E%R!3pe?i7GoLJ z+dS>>)4UmX=1uKz=8ZN7M}mOTH}__evf6I22>m}3_8aUP^K{Dndrke%!E>#7b|VBB z>nz=cnXb7oQ!UZXDYo{y`;?~j$4X4r8Tvz0@zBGQtkM&9X^~RCc+lR3c4?`$cb@*- zIofk)TPHs{_k~q+7ZyFUs;Fqd*qTIo^QPh0{+UCXH|6Dn=G|CqR;9gr1m+FC7c7

O}i`k9SFV-gx&zcZvu^P0g-or<{dzE7tp#Ji0uX1_5tyCfsXz6 z6-tQ%_vL&|0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WG;g@Jh--_uwrHHK zE#hURd0FY4Usjq2ArC?xgq-rgDGzKyHX)mkO~@u>6S4_&-h^q)8?LpVIBV@`PRN44 zCS5u+PXwF@Su}4r;3l)-J_VxvsX@xkpnonK9Rnv{NEui+LkTv^G(ETCV`HAAz>3 zK>QleaRW&F40Qbh^!y4WUz}PfrCyrK)eKC42`~XBzyz286JP>NfC(@GCcp%k025#W zOn?dGJOL^|-(>BN(7YLU=1q=<-Sn)PYAZPo^QQUj+N_&5-FF#wQxXp;Lz*|Wo)skz_9^vZUkJf0%e;3 z_iKP>3*g-bRBi`+uLIR@0RA_Dnzw+!J3!qIAh-(%?FPbofyRA63T&E*2xc6>2XT@khPKm*AUXR*V@M zhOBQTr9j^rln3;!@z&O)%-8>&r_9o~W@WCvm6iMTt>xBsk}};YVj5Li*Jv*_Dd#5ZuP`YV{S_uf)gQ=HhShFX;{}o}+XHVL z$EVuL-<+m2?JIA}x^~lfA^pIcsy%Cl9(dER&1ALfGR+iCrja4j7?kn{Z?|sDmc3@ zfif@Pt^hoh(+VYT)if?>U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WOd#h8(7K^@ zL+ggt4Xqnq+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMpt>D_s*dLJn3V^0;YLwr{Xp;l5PBa7e*iRo2t+;xnhyfeLqO|cAa(?3`xJ;D1v)+h632k9FMysef#h)@ zbrR@31vp}W^Bmx61IpR~cO3A13wS$#%1*$S0IDxSyzzHYyy3VTCcp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaaiz`vfT7aA^^j7IA6OI9po8t4i~#(mB7XG!H@^gggj26Xv`L(}*`*YCmz7+Ebj61%FMdbP`VloCsMIZ}K6NsN(cvs%Usz z6+53$MfqY?TwkJ!_NP?QRHTa9XI08MeR7U0TGmTxRTK=Q87SS{dH)_YoB|;-@NP1a zWzv5BA$AsMU+u0_Z3?u9sNO)S8Gh!?SgJPxgRa<^mFMrG-e~)?Kl8?Sd77>EW1G@+ z<@;+{S8p2bGV`YU=61#UkUrB!!AzTnr&~==u3Y)FU0$+c`HGd#tx!r1nr_!@OmL7| z-jQzKIn>Y_d@pz`R{;NyK+RPka1E%t0R(>rLcajvUxCIKZSZVf0-9e2qOSn0Yk}B0 zplt&X-w1TP3M4iGU9SN>TY%&?AhjLneI0OA0Zt#_dK)OK2HZOVk00>v0V-<%UoB7_ z0Q~O(HFZFs9;gf23Z>vjwwzzilOKQyFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zx;J!h=-$x1p?kv%oU;kpgls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)=)PZnIP&&B#? zS}QyeWWf_*;+zOsbZ-hZokSJ&n}Mndpi@>w%QWnni5+uPQ956xo};Tt^Pnab3!x4- z5@uPpJsb?7$w0ihs~$6iIWwY%mfx<~EE;p=29nS3bl^?jpqsH7c(ZRf>t=KY-UL3j z*&5d6D^1bk8}qC)bQ5N(X2MLeN>A9OMN0YNL3P7gDV9#zhPF~l)}5R^GVA6y!DhPt*Y4B)ng!yNS*cA( z)_i^EWNoL}B5XntFjzj;43>jH@DLC>41|vWjh_OMqd@a#K=c^U`UMdC5@31*E%IAUxCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO z1ah7LtsAZ@;<_TPD;j6(ig;mZURXNk7nbHh$b*mvA?G}B&I6l}O~@u>6S4`}glxi` zH(?s%C`LG*&FDNVKcJy6A+33;m?7_W+2i6G)IBx381wV+D+^=cn(6||hc!1`uK-3GgRsgX|psfms z`+$zO?S)dJ+RoJsOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3FJHhDmPqI#6?A1 zR5Z>O74d@7yr6W>FDT7}kOv_TLe6*KdZ^AUn4Hwx@oJICj zCS<{1lMN%Bl?dEQiZc5@IWk~I&)-!0*jny_qDG3`hpLB!o z1%+i7kk}1$?FD-F0m*lP)PA7%0N|(xoI$|#5l|Ka+zo&y40t~QDjNY`6Hpxi{GS6g z%|M_9sEY!@6F{gH2%iQTV?g8_(A)+@+kw_N5c?Kr>j2`NKt}>dTm-thfSzt3*#qq+ z^%J!lPP$MfZ7eU8)`SyZm8Yxs^)A$ zHX)mkO~@u>6S4`}gls}KA)Am*$R-?TO_)ZzG0TGM^tn!7qOigfK^8m_CeDeFMeRoZ z18X!RRfzJTCG|Tucit%&H4A5{kTRHdb63x%K=Wg!jK-}%`?EA}et+i8^H#G(vRRbX z`gW4C+HSB2{XY}-8|)hMbjtmEP5sZobFF!Hqos7db(Ym2gnc^h+NWB2uG(!SpG;Sp zS{;|OZr*fWNI&qVYR{UX2i}yI4>E5A%$xR|(qQw(qSRIxtoiyEnyh^xvqjj1B4Dvx z)hw24KTVRZDkO~659|4Yofb-Dw zLdkV_dd@H9$q&E;m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28%^R*Q;@TpvEgENQ zi+EjWUROHj*Olf$$b*mvA;&y$%mbT{O~@u>6S4`}glxi`H(?s{hHLF7&RTn#6SCm1 zNtZ56Wasl|{!I&ofTUuEi6QG-Nh#2`2IT>LYrM5JDf9Jz z=P9%Fty!6?Z)N3veQUY3ouo{+%FtUR<*=mRF(|d#D-AY-V#L;ntwDdl_}c@HRLJ**ZgaSf;{Zd3cgldcrO(Qpy((+Ply$E!FnU)1Ny>d+u!OTSZstux^61d!P1Poo)vo{jDVk3s!>2JQE&t+n@x}rTsi8i-c!RG6 ziKP*6H34N2!2LPkX$HJ4KxGv0odBv^0sm>BCI$q~0d;LaupJ1+f$+CLV+RoF1ey~- z^divO1;o06wjLn<6VQm^b0tBcQBM@ zpmcNR{d?4K3WOm8@8+LlqrfOB7KVGcWkIDGk&jcT$vgj?Z+b7Ux_9qp29y^w?2bPF zW^}!qfI;_b%!)SI#sJ@DY`hz{dxov59p25>SY_6|n})kgyy?EVU9mo-do_yY?M%0t zo?N-|>Awm)4ci8JHwK$YseLwc?*?BB4$Ef1>j5ga0zNNLT>w7?~4rr?f;z6L}BOnn1x*C9dvVs;J)#R8;_-vMO4pVb4tL zn4^l)`6|^MT}zq=EeU>#s#z-O?VG;+%d+j@EXqi_xfgfl)@x<$tC=#tEzq7!?*=~2 z@bhlQ(!1F=oOh%7I1(hF)%tdlvf6I22>m}3_8aUP^K{Dndrke%!E>#7b|Wko>nvS( znXV0znQHO1&#=|j+^;mCUUEMq8wZeB19Uloo>Cy`0#fUN-ZH@PD&X7%xLyOwwgB#JfM+}4eI2NL1Ms~G zRKErI-vMfN0D)aV-R}DfrQqKCxu$^$FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zoF_o>hT;vy8;Um+Z+LNYHX)mkO~@u>6S4`}gls}KA)Am*$R=bHj8{^b`yKIIqTX@cOt!Z({OD6%ptX#l7pt( z4I6FfPNfC(@GCcp%8p8&NR zE-d20A}%Z%XA6sXS!rHYI_H;_=0V7VkOv{BJaEbbn~+V&CS((`3E6~f!kjl@8tsM) z?I+GcdukK1;IB!M&dd`5Cqfpr8xFY1Y`9N>Xn*QX~5Q107{Rq8#XQ13jC8qz6cC1$w=J z<8{FK2H<)VD0>TVzXN!70N!0dY7pUF`_}>L;_5*NfC(@GCcp$} z-O#$Bbwlfh)(tOg&L(6NvI*IQY(h36n~+V&CS((`3E6~f!g1DwX{;L|3og^=GW|UI zDm)Q5^lBmvy~?6>GY>Y2Dw>K^QTwbawysu1$zNdNtdUgFvL2}43{+JBoicV!!=9Pg zF-H}p^HtIPh$>D$rizBg)jz<{oI-r}fx^ck&HpeAam$v7qh@5@jNROs)tdWaRfUzq zZpNW@lcQlb(Z~a~_N_COri!(_S=VmvHteP(9#V$XZmRYS8g^sVv>T@}bL|G-3ku8U zK&%;PYXRa>pyLFPXa%}X13fVyc@9Xm0ln>j;~T*F9pL&NDEk3$UjRIp0Pkg>@(SSl z5vaZj_^$yqH-Ny;K;17u@K+%8;!G$uF9D4&1CdvN=Cwd{9niV~h;0PgUIpTtfR5K@ z7D|aNGr6LH2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6ft)8m?S>1BxUh%|i^kc) zB3@IP*OborHKlnF@*w0v$Po`5@xUfz6S4`}gls}KA)7GgO_)Zzp>aavgzA-n#>qr7 zPWWr$*UX}JBmaRlnvp6*`OuOEYBz)K7mS*Pvs4%}7HyXp4bq1{YT8k$F--QascVetZq3ZSbJ z=&1sdJ|Oir&|3{Sb^*@afNL*MwhwT>3wZVe-UC48`+)BQp!!3=|1nT=5C|Ls>J9_J zBS7d=Abb>P{0xX31Dd}8qF(~7$AQ>MpzRb8KLd1}1rq0huCIWeuYu$@&~8%SQM(a2 zUsAyjzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%sNGPzp>{*;~=hy;q_3juN z^w~B-k@c;l6zE&S_?B<}j1OtBnUy?jjo3bjtz?sx=k={cS*>p;DXZ-Ui_rfw;c% z^^ZA4`@edNF{*Yi-6-Y;JgC3 zegw*{0`6;o=LX>Y8L0dP@cjx@zo@{oc?qa_83?=r)U5@A>wwS(AiNQ1d=-dn0-9d~ zqFaF0Z9r^0(DphIe*@@v6G*%TbiD)g>;RIxfYffFcQ4?m1)Kq;P;$Mea9INrU;<2l z2`~XBzyz286JP>NfC(@GCcp%k025#WIZuG@4ObU&brDw=jkDE7ys|W}ES>W!OY(273)K~E~7w~ zd3d_j^yJEwPycmVPI>tq-i<*i|IcstZmc$|(&99x^=|OB;IO<8xIO^NJ_Ot!1D=C` z_YhEd81Nkdsy_w%M}eBpfWR@J?h7FJB@j9egiivEr+~;Ap!qBiJrA^g1;oAv+P(qe z-vJ%p1BoAit_wiVB_MejNL>MXe*_#!zNfC(@GCcp%k025#WOn?b60Vco%m;k*SdN=fL=-tq};q}eggls}KA)Am*$R=bH zvI*IQY(h36n~+U7&YCcdcaw+f?c12C6E6 zPFWQ#)39eIcFa*l>3o%Hj;D>(H-Rv9AyBVE%H}02a*{Z&TcT-h=KI`61{lz@%OkJ$MFOb*=biE7o><5wufYkdy?+1Y6BfuE~ zTn#{37;t|Acp3q36Hplee4hi=&G2sgE%a^#7j-ZJCcp%k025#WOn?b60Vco%m;e)C z0!)AjFoB#WK<|bti@36gD~rb2$|7D?nwOQ%`DLYf5b_}8LC7f&obtdXWD~Lp*@SFD zHX)lZ=S`T#yWvXviL=t4-h?dpYtp0>JP~jrWYN1R;EbCQ&G&(LGw7B-CLT@(({4s) z)U4K=8PmPoRPE)oZZO5hs62qdGG_Q3+RPfWEl-)HZ_UbFeJd+tIqjx)IO}F~rrr3y zoMo$hW46-t;@elVZryYz(pxtT$M(-0(zXFcF811ie_pBt#&4EQ}j%~l}b1?nnh7fQj( z**RZOzz@I#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28wHslW3RHP-Eh85 zX0v??MEg^Ne49c4TsAuT4MxgjpxoTMKVvQyBe<>Bo=oK?Bjx6PUA?g=ecMUObbVTl zq#Ty?I|ijzlXVO>gJQ(ih^;|?!1&t(##;{#h-wXQ6n?Ud_AoLCp-T^f30wTMC=Dk34 zAJFx24OGVf|2d$h4G6RYb#Wm0EfDGe!ks{40?JL~B9$9Xx?uuLfC(@GCcp%k025#W zOn?b60Vco%m;e)C0=Z9s$_*D4aZwQ$6^*k+MZBOiFDRY!3rh1K88K|lNI%VvdhCMT}V~#3H=c}Uo5mlUiOcf1}tABurIfeM{#oBi1?vWs%^bNh4 zq^!0ZEJFX!g#8A)#yp*J|6Wu7bMRbip4|vf#yU$^W~Sq=eX1pVX|}E9!vdx0@Xv2& z-MqQmu$z*2NEzC^2@D!`W3gG4@}?1(H~3z#SS|yRD?sy)K=dlmdJTx(0NQ>A;=ceL zzXFLD3t-v21oXTNBwqniYk}T%fTIj>mIE#~P_`LxdjQW?!0QDnD*#_5P+bN1eL&6I zK%g3^+X)2yKxhvTt^pcrfk*&oeh-M&0j>1~g;Fe7!0U)I0Vco%m;e)C0!)AjFaajO z1egF5U;<2l2`~XBkn;p+-q5_Ec|-Gt<_#}w&L(6NvI*IQY(h36n~+V&CS((`3E6~f z!g1DwY0Mk5EGS-3ypSkf@I;UWPlSncB4p9Lk^jIZ%}6bxe3(i7+RdGJ3Pw%DSt>*Z z18;^GWrj6r(mr9?UG$ozWA5C*@+s8bRE+VhPzZfsjZkELD=7u~)}TC~Z;iLMCPf=t zW47ffv-GW5nX7N*TZ3-KX5h`R+D)HslWpxL`eA{seeHuv)7#&Fly&W<^FsQ8H&uJq z3_b9s%2$CV|Lx zpg9FZZvw5oK&<3Jd_D&dUjuYFA1ss-r4QzOK>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e)C0@QA}u!swbxUguPEiB@7rFmWHoL^U(2O$qa9)uk8z%dVOLN+0rkWI)Y zWD~LpbKZn$v>Pt8pEwKcsZGd&za~YxJe~+R5wfV=EZ~rvj0gNIz<{5A;RBPG?lf-( z?pc;q3kPqPfq64Vvu74)?#qGn3V^ym29#yIx}zLYag_ApMrez^6?X6Cf_t9j}9&0v>X^D-%Npg z(|%$E@(sQhM3!|xVgt~%5$JgpNNxgBuK~SV0EZWFRsgO_psWgT`vA||fVUc`+zI&n zK=mHLUjx+C0)YTf_Z|?e148vcI0!U;1VlnWa{~|!1FfF`u|}Y+35Z94j?aNaGtkun z^hAN=3CK68R?0VmD?69~6JP>NfC(@GCcp%k025#WOn?b60Vco%m_W`GpnOC5hVl*N z8_GAlzB!waO~@u>6S4`}gls}KA)Am*$R=bHvI)mo6Q+@G46@*AeXiCQ)K+*R$bu)r z#5obNDBt8mC{e}f$5heqxGHu&p^Eaws<^&H741){qNzv~wa=;)bovAxS+uN|`t_TE zcQBP^pm=lV{d-h#3WPBO_vV%l(=W!1_MJkCH{){ZO~9ZFHfCM0G1%@DZ$@Y8P0y(Z zZ6(!zQkptK7qc$jG~8wCP4~_1iuEC@K?u8IDV_iDbgSvfl`EgN%S%=)U$OGJ6-u|y zWVP#$Pr>7M*&;OY#y)uH%@n2OOQSVkf8u2AiDo=eC;|@4Da~Oy1EkIZz2^Z(JK&51 zu5W>|4#3?BcoKm3B2d`{__~4W9>D(-P?H1#*MYhe5WES5dVz4spWxUyfXEu4*$G5T zfmRm~TMx9A0r7I6!wn=h16>}VXDg8O0;vk1w-Ru?2{_;SQ=#N~=TA9bQNRzt1egF5 zU;<2l2`~XBzyz286JP>NfC(@GCcp%k0KFToEaJ){t}Gg7D~ot-X+6>%eI5FC^P9s)hrn3 zKVx)Tt-X;|9Wei9v^5|6{ncap9s9y86apTL85V}DZzZKb-x@NOZf5AuwPqiV<5Qwg&wH<8KcbZ#`hd?3!Vw-R#SabmOT0ldb$GNH?GQ z*G#g`(EXLEng}z=Dm`JB7AfV62kl*GmzHXK=jqR#qdj-Fb@HQgUsyGFVbL?Iii-Z~ z&eUJbv(D6?mai!>Q!Jgb4Q-{AtUEb-q|%K+Y5BX`rJLU-;G~mo@U;znXpllD| zt^qu?fHweCz6bc~fa-d{9|USX0s=NL<40x_Uy7B%<>4w8@m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzyz2;?h~MNL+OUn4W%1OH@vVpn~+V&CS((`3E6~f zLN+0rkWI)YWD~Lp$5|7mk!~beaNdoHtMsX>@I;UWPXyzHIT5lb-4t-Z&4`Bkz`7Z9 z%O4X7CxcNpgJqli;pCq?NAiqx&pcrm1Jh>ETj(Qp27lSnc8z{t;!W*v;?47LdnB7h z>6>~pNm*?-ScLwc3HuFpjd?od{=KID=is^4Ji8J8i*=T+!c5mxn5h=`Pk*vig&$Iy zc6utZF5Yw}(u+3@$M(-0QoJcIH{CAT7;Gk`)+yb-b13r$-wPH?67XIJDpP>(CQ#iA z_)8vwW#a$>Yk)c@5G(~kE+D)fXed_d3J zK(ZQ0?F4%LfMYM<+y}Vc1M}fu@K%^CDJ`F@;K6S4`}gls}KA)Am*$R=bHvI)mo6Q)sageVmp;q7d2_d6Hzo0qGNgG^>lrlc z#*8UH-5U*=54*wlg2mznf}4Sm2MBKk8ofZI0%)!TqE$ev4~V@Dv{eJ~oj`{lNbCW+ zYJi?vAQ=Es?*YAafa3$e`61x?7$`dkxDNrI!+`e)Q28m~I|@{P2KbKwHD3UMFM+z_ zK=33GIt7H!0F7s1-bBvRyb)a1!33B96JP>NfC(@GCcp%k025#WOn?b60Vco%a-IOq z8?G(l+9Iwk8fR;Zcui?uQ#$9@l;%OmgOCRyM?7%E1DlXd$R=bHvI*IQY{Hy3VH)#> zYwah_T6>xkvf!^tm(IWw0VhHh%^Ud-Y|@O>BFcxE)UVy#d8c61M4Y94@b8Ibp<|<8}o%tr~+D+$$^aF3I_N*Ct;7!A} zLE4SQW>s2Fj6l1=_kzOG4n*QW^S3~>18D68VhNz_A`tHaI=X>G576}!(31p`*MU?D z=)DOzeg&K_&V^$05>WOs;C==0tOdO5fXWSkZzE9sD&XG))Vu}+wg7e8fZ%o@^g0lJ z1895`h`a?fzXL>f0Ij>`7D}<*b926+fFFPfFaajO1egF5U;<2l2`~XBzyz286JP>N zfC(@GYB$twsNGPzp?1TIo3jbogls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)kmC=0IB z=Q@28g%zF%vfzm@aZZFRYBviwrV^C_fR~l>v#fYsDTZ8_9@wW$zw;mXYdGlxePnzNCXn)eIJczAilad3$z7*_I6nnkM}e}>0QWJ#^9A7j5~w^5_)Y@VrvU#Mpyn(PI1ki) z1q8naLf-)4?|{bdfyfU)^93M!3240x#I68sKLYWqK*u$hH;EfGZ#eLV2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?dGJ^`9HTwBDoMO<4n&ej(3y3)L^bk46U&4Z8!ArC^1 zdEl4_HX)mkO~@u>6S4`}ggI}*H0BN0+E1Ld_B1DC!C#Xuox~FXCqfp@n|#b7I^88)g8*RU2v)poUCMm1!28+=D zGhx5Mt}#!i+`re<{~SEmnrAm!O6OZ=Sq(zir{k`DswJM9YwK?M8>MN(yPK@m8Tvz0 z@zBGQtkM&9X^~RCc+lR3c4?`$cb@*-Iofk)TPHs{_k~q+7ZyFUs;I~=FIlmC#meVa zC=GX+deeP#yJCGvf4!o;e!A842M4(~0`5&i#US_Qx1DzVFHONfC(@GCcp%k025#WOn?b60Vco%a-IO? z8_GA7Zz$hTzTqX#*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djeL_Q3$E7ZYJD@c z6`lyP;E6DCPJ}GVH-(x{qKf*>Kvf0MDXXGo8urY@jyb9*ov+f*(cPqZaFdFKP=y$% z-Q0P9S++%-MH!hlqdt9Rx#rp6yWhIuQ;osMn{;pb*U!-ZVX}K?`RzaPBB7ZqW}8rC zeJd#i`qpr3)Xm68-pm-qkvIE>({4s*b&pK0gV)8X7 zW{Ra#wxO+*l65C%53Ake>o1(Fz0eGqMkvCXX4h@XjW&F!d@=NuCQV<70IAP`-e$n@ zCEz>`xK09PrvUdEz;hPxo(C$w0(@Ts)!zXA?|_=`fxr(y-31_c2?$*V!dHOCAA!hK zp!pgQy#ciT48(o`+I|J%FFp*t<|QEUGSKx3(6bgut^-mVfZmOO!won$KU^reJP+r5 zF#$gS6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e)C0#t6esECV-xTt8HEh^$=rFmKD zoL^R&2O$qa9)z6oz$p)GLN+0rkWI)YWD~LpbKZn$lp8LxpE!%`sZ7X%za}L*!4m-| zLKc;q0?xP@(R`mf-H!a*{ftpka59*7GwLSFpzmJzn|w29|ET^`e^bmB8q-_hJv%qk zbw|gd^lc|8qv_n#=Ek{kY$*-MQ&bq<3x_j_sd0q;u2lGg;HvG#N`b z_*M{DwgIl~K-uen`whVJCg6PwsC)NfC(@GCcp%k025#WOn?b60Vco%a-IOC8%j5nZYbSQ zy5Tj=*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djdWv@1y|{FmA-+x3Qq)C@I;t6 zCqfpb8xFR~Y^YCxXn$&Gn`GJO$T#`In1OIJrVa{}7icI$r!g9j{!WFP8Dl8i?8%LA zpD9QHDIg(V~r3YV671&L<}`Ztx-b^SWA#vB2)|*Yt&c+)EYHn z#QJ@n19l;Vl1kciyU&^UIN>B|l4s6&`p^4uUa1Fx^g}@AVPN19AiD|3JqqNv07F{= zTMJ-s1sq<$xgBu%0QU~S;|E%H0p0-MYy0PVoj>@`6<pJY2g&lJg(KJsHgEuSU_^pbFEKvRd#^u!GyRQ+3 z;oD;M{z$`R$|d0_qqJ@ot3HfztZ#RyZ!yx>AB#u-POY2k$I-f}(3l(Vo`1Hq-TNuK=Na zK=@T4x*zC#4T!xCbRPubhk)M0K;lgxc?3wk1*DGwnYV#~cYy2(Aonhie-9XXAFzE0 z*wcWcA8=*>*J;2#0C)z0mMq}?0`TPk|5+fA2ZHB-_90j|p~ipVf(9nQ1egF5U;<2l z2`~XBzyz286JP>NfC(@GCcp$Lo&c>IS~s+AXx-4d;Wf?Kgls}KA)Am*$R=bHvI*IQ zY(h36n~+U7!J4p$byF<~F4N~SeI0!jo(PiQi7`a+0?3fO!ZjYYSlkW@y1kbF~~P)VvSLrr){g`+1j>Ro~vyoc|wUd zf5C^+SxoY3Z97F?ZPlBE_MZ{^^;VTt8l~aA#^LASx#lXX0mEa=v$Z)gGt@aU(@nv9 z|HTq{6XH$wkYlQOruNWuJap3(vu=@9w^(joQm}WSRkuvtTcVPDwIH#q1w!`$ z;roH;13>3`Aod{8{SXj;80dWjNNfUPJKzb{Xc?=kM9LPQaWVT!1prX4+DV+5R3xt2Y^r~5bgq^F`)B}f2r5Sx+&g> ziZ80*2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0u*n!w1`WKxU^`3EiK|zrFm89 zieFWl2O$qa9)uk7z#$K8LN+0rkWI)YWD~LpE8c`f#2YTPpFB(LDNaa&za~|>DxL^9 z5y~ju%;$)klE?eZ$9SJLLU;a#pW$-%=Hfle%j)6a?MiTOO0sM2P;D7gS#FEfzeDk6 z;)*wag;=Aaa3+O%u#NP&b-vJUQfaJSC>OCO+K9KnU82AXtehlP30rIDS zp)-JO5U^(f#}|Mz2e{4x?mXZ*2eb?U-o`peHa5V&4hYzRU=z^p074spuoH+j1D!4) zwh`!d1M$s3ucxkFm)KUvH4RLF2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgJOPR~ z6mKZrP`sgd!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g@kTERuGHsBeL-b~ zCxRq+B21nWp^V~94P+8U9KTf&kp+s_u}BfkOB8W-sUrGrQ$$yTBHHd$DCeA)b0pET z;evNFEa05KgPAlJOE;I_zeWwGR_IIcZhnYOQ?I%+=4-goyP0s_O;E4-HPx~@*k=5^ zoA_gOmcCQ)Zk~TKRCezsa)p66gXgx&<{xVVb7Xa3&J45hPb*jc`EQeQB3lZ)8?$Aa zJouEMeK`R9P4J%Ftpuy+8C5a4_n zaD@SP1n@+GmIHvd6YzBb{umH=0|<5l?L9y!4up>a(O#hQI1oz!-6w%~66oy%5-A}0 zA&^P~>3(=OnGC%f4!mIkOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p-fZh#P7I9?} zR~AjMl|{U+G_NaN@#{+SAml;FgOFn$IOc&($R=bHvI*IQY(h3+#hb8*cf*zTlV_zp zy$MP1*Q7~j* z48Hgixoguy@v?h29jB|zvoyV?26~NknyFv1ps?h|^~Y`;op&SP-9(P)%~tKNO;i7x zWD*v=Jb3-3`}Ek?yaW=j1YM@H7D}4#2ws@Hqj0GZ1hA!Hqz>8whO%!X6;H4d`qEVy!^87l>~MdVN4*2axmw zsa-%i0A$*Lfgq559>}%-q+XZr_({bt<;f4g1egF5U;<2l2`~XBzyz286JP>NfC(@G zCcp%k0L2@MHxzFu-cY>Z#m(7-Y(h36n~+V&CS((`3E6~fLN+0rkWDzjny`p?qmu+z z>T{*OfyxR`1WE8jm^>#!8O57g4!IfCfFGDQ1-F`U5phZwcQXoOrWChk+10F?!FkuJbb>jfc?IC^*YD>^SGda2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6 zfr=+U>xS!!xUPuniYC~)B3@ORSCy{#Ri$|l@*w0v$RQ6L^1vo!6S4`}gls}KA)Bz` zO<2Ub;X3=tv(BE@ge3TD(xa>9iGUNKjMfe3+mtrjr&jdcHnPW|`5?N2(Ki?W^M{SW zsS%*Dl?hmu<#Jy4iI#>1J$Z-PoGu zS(tL0xdwW6=?SYq3uA}2Sj%Ooqiy;3+N62@iw402qc~d zlI=jM14xH}%*(()7|2F|TolM30ERjN+d;s72yh$*oNofIBY^uYNH?Bilx{@D7gX>A zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GN;i~lDBV!Hp>)H`nzISngls}KA)Am* z$R=bHvI*IQY(h36n{a|PVG-#@NP?^Mxk|r^x(ZJON$^CNJSRdKrJFj)B#P*2P(<6E zirBnb5smji#960PM9&7GVpJY2g&lJg(KJsHgEuSU_^pbFEKvRdM&{Jx zyRW%uzRTibF3QDrN4Y#4r6loY{B)Gns`_G7gz@4pJ3jYm-AvR`H?H1!me%b*mAg9P zTgqam(kM%B96rYok{6x#=drB_v0jsu=YZ4@kZ$}b6dM~b zunx%Dfm{=icK|~h0NX=={b9iI2;kfVxE=-ETL8~ipye^Z`#9ix;-~dG|C2xEiUuaY z1egF5U;<2l2`~XBzyz286JP>NfC(@GCcp$Lo&dEQE-d20A}%bNU<-?QNoihEy5g6V z=0V7VkOv_rJaEDTn~+V&CS((`3E6~f!iqOx5$%Qx?I+JddukJs;IB!MPR|noCqfyu z8|fcdqZzG2R0A#PV(q5je(ji9ILm~ggn2iRV8-=UPwhs&!CY;swiqsKjq*J0-&OK# zZCfqR)wYs+y|x{nVK;488+NlAA4X>}$>+CI4%m(X_O}7YJAm^9;CdHuzXy2U2UX953iAn+Uz z+yk`l1wtNfC(@GCcp%k z025#WOn~wYmltt)5tkQDu;oR(v@|a*UGYmx^C09w$b*oR9ysZNO~@u>6S4`}gls}K zVa1!Uh>~3D5ph4i-;!zPJ}YbH#HDS6mk4kMMM@TV#gvyG%r!a*`Y4T@;HQ=y=9UeJ+5&j#JFesl31Or{%PgP zKU<}xEACja@~#zfb8~@sW3rg#o-?BmZ}7b!u|$E`0ie4Rh<5?KF(B~NfC(@GCcp$Lo&d!giZ>K*DBe)K;T6u=gls}KA)Am*$R=bHvI*IQY(h36 zn~+U7!J4p$cvFQd<#VMzSL#z);fWv#9Z!Twb0U;cys1}R5=C@u1X{NN{gNVjuEU;L z*fB>DP4g7GIhvPL2QO)jKt#pen_&-UdATK=#U)ucSGQ~GR8PiS##@d0*C^ghIPqrp z)x?{znRpX-{@l`c65`Fv_kLY=@uuT+m3fxt*VMqTu}(AfOBNKC+_?VOjiVE9%$8|# zThLH!x=pctkXW3m#L^5TTtIRoka7d*%|ONj3~U3kEkLdn$a{gI?SSoRz`hf3JOenN z1zfuU_j7<}574p~@V)@}UIhFv0fAS5;69-JRUot<2)_nIUk5r50NP(_&Orqs!wTL%5Vkh>l`)BI27nc73s)fqHX%(_KZ-D0_UNx|NQR^2jnZ=Lqs zIqGw7G*7){?%k{AE^N4cRYSwy#?}mGifcEK!+U0psNF=i6wJF3&~763%jVq}<;edj zuHE2!L19S(nLc131!O-2a%mvn4-91h+Zn+ADd6}FaDEQBz69J~0iLgcmTv&>w}9_E zz<=*ApxCShg7*RK_XD8^fbe=C`XJEx5DNfC(@GCcp%k025#WOrYWkP`jaa zL+ysz4YeCy+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35#eqMoDm;KG*5%DXj2B z;M}XpH14O2+6@QXls4R_R`lIg;M)}ZbNSflH#I_Ef^u{1{)|{7MsZuMKKYuI8%2G& zaa6a}>hrJZO@F}4>xF8uR9l2BXOPanXfeu|n^Ro5!S{l`vJ1!ufT1?Pwg<591spE`&KCjKOMv?oz_Sl%c@^;P z2Yjyq{?~!PK_GYtXg>^u-UPx&faqI5=P@AmHqiYJ5I+I*z6&JY1Cs9psSkkkM?mId zVBixVdkV;%0rH;$L!SY*F93TE%8lbJl^agFVFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wl}~`m4Hp%0Q4tpvO|V5pyr48MC|&UjO7kG(LCAxU^Bp+fflbIJWD~Lp*@SFD zHetn^u!wR)+l00WwJQT{lgVV8@YlqzSw`ii4l0Qvx*8PGcBdjXuU16kJmuOY`ghDtEniAX0Yo=1RkE8dG6; zWb-CyEEH;ta^G#EFmLd^V6l7yIKBm(-vO?B{}qzoDskk1>6VzwO;4x{MU+K#*-g_2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?b60h%{7Z)o1oyrFr+3!AeE*@SFDHX)mkO~@u>6S4`}gls}KA)9c5HDM9+ zrdkpdFDPEYHkOWVJ$#Wu<(Y%phlR&MRqll(?iWt0E5yx*;L}Y;?b}Ujv^Abgz zU8>N|(afY8m`N9#HwE`=$4taoCPWG5%~*|rS$+k*X4xob)fCleM!S>R%~c29TvfYS zJto?X>-B%Nv~K#P+!bj%UUu!K|5WjTH?6zYjXdzCai6hJv8k3LUR~kN5hmW?i@{+z z2zU+wEr$W`n}F{K;C~AU90P)H1MTksp%Xy(T_E}%(D^NfC(@GCQ$JN=-qH-5my#*Wzhs% zS;PxV^TN^%csB+A)O3}y&BMXlm7v}jN2{^?u;r8 z6wvuJ5ZeiKKLf;{1$uV_iRXai9w4Nfc6dT8`?LtZ)o4}3g>J>HX)mkO~@u>6S4`}gls}KA)Am* z$R?a%O<2Uf(PLpO=Y4RwKD`y52$G<;A|}m=P)7TvhLdhepYkJ#o(=G6pxqSSFqfu4 zzA5ob}q8c{E%$u>9dlUTNmzKzLH_KhNrl#rUnP$BZ*6F6Ec{fcl>lRsci{<7e1$!4- zb<5Pfb=q_1sL#F8JoT2jcdweeu;KPq4GmUl>54m6th{T59J#{Wo56EiW%G}<*UReb zXPAwDTDkJie|zDe_cHUwB)9F=k7Di(z8EZ)kATp}K=>0NdJ5<~1H?WBx<3Qrp98&L z0*SAH6V7UKeY-nTr~j025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5sCWW2Z@9LIYm2zHXo9UR;^n1zdFhH@UYZ9X4?-S$Ywc-HVDQ7FF(>$I^4FBa4>u#rXx`MTGKnHOHUh2N zfPP64J=bB+EbN%0h^Bc8Zgy8jyNNz`v!&-dXg42x{&?B7n~u{}=2@CuQvd{ z`NP1_n}972*pC8^Uch-Aa3uiuNx+i?TKWKQ3h;dh_|rh39|&fE_R~OU00?6amoAzG zI==v7IiUM25YGd>=b+srhN#_e;0+UC0!)AjFaajO1egF5U;<2l2`~XBzyz286R3Ow z)NZKVP`jaaL+yqaH)j*F3E6~fLN+0rkWI)YWD~Lp*@SFDHsJ(o!Xny@?pjuMQ*q#w z6xt!QL)ak`aAsH;wVPVFB#LNWqKLCg714K_BDxwB(RQaIHm_Dh<2~?j)cyDH^j4?-eX2JTPQ7WnntG#7!qHhw^7+X(Q{>fFy-8^Q z8L?k)RavD`8s2Lheh!{%uCf|nz?f%iGRzEBhM8`Pe|NK`@9=!N>%e<2lwG|U%oJB| zB8T_P8d1GzZZ4R3W426_2g8O@OuWGtgT`|2d}uanf#iKa>V6>o0FYS^3_J*A9|Cd@ z1NldQp-q5oGhp`sj%|Rm1#qN zfC(@GCcp%k025#WRByPrh>MH3xM+ecF5-oyd12{_Us#$4ArC?xgq-uhIS*_?HX)mk zO~@u>6S4^_-h@Te8za>TsuNTvCZIaOk0c2L&&?lvZkAEK;fR~k#{1MlODb?~3jVo# zY!sXtp({bV8Og9&uKF?4%6nU;{y{o76VAEWbv5VaylYc#=O%S{zGZOhujH<$4!v1+ z=cegQap$Jz#Oc|ib8ZBjo5)MLLg%JhZa$@gtsHjsY@7&-yil7PJraHIg|hkz>$xcdQ52531Acn1LAAmGmefiHky4ro6M zgz`Z691tA>Ivallzs3f1uLI(CptlJ~IDq5^Ams$o%|ONl3~U6lZXmb$SM|EQ=T}_L zzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wr7pmRg#hRzM08#*_ z6S4`}gls}KA)Am*$R=bHvI!?x6Bco91V`3zWDUJ5x*b2*?O=2hGrBT5H+39wQ}TG9 zdeJd!jnEC>7OVFQr{7$>r(7D2Qj&Kw3aw_os?3OMzo8&TrQXa}J)P=nzv2JuYlMEB zZ}mbITBwLDkbO7ivE)^uSzMV@JvFjht< z@7HN}^m3c}O4au;U~9luuRUP+{sF^<2MltR_JC17IaPawQ8sC>Fv^PdK$Tqb#GCVz zZ^||CCcE`lmc|V18~Y0%m)*X((!`s_R9GI-zG-tCFLi1JvW?VTD&JIFs^zxK733Ry zFNiFU0r|&)p(g;_cEIif96JD~A8_pg+yTJT2DAhL@AH7K9q@Mmfe;XU8E6j!p$HI; z0?`9NXD1Nr0=i>B{0*SD8%Xp3$vBWY3Z#31%yD2K0c1}Cxg?P91BOz7?IXbcG2|P^ zCzNkQ#TQoa126$5zyz286JP>NfC(@GCcp%k025#WOn?b60m?UAUc}`^TwXN6mKX85 z(!8#8#jh*PgOCRy4?>Q4;Ft$CA)Am*$R=bHvI*IQ6>q{K@=Z18YH+Rw1uP0!6tE_l zMN~%lMuJeHh}JoZXqu;p!J8Fv{8mLo7ARuJB1JSWQN-D$3I!ca(5VrdR}Y&vm*1%! zGZ$x>(3ddu=2~Q$A0o0qM!9y4X0(5y{_j(}nF%$f+A=}3o3R;sW6S)?(%kWHa@WV+ z@5-*-^q(p|^rm&!x{-(8M79)YH+qXvZvB5pq21tnL18%!I0gXcAmGXZ?k@mO4rnAm#+Rn}N6s=-mh;+(2?Okn#ZOZ9t|4 z7-$8uULdy}$oqhy9f0i_!2T@Y*bO+J16+Fm_uhZ2*Lhy}H?C-40!)AjFaajO1egF5 zU;<2l2`~XBzyz286JP>NpyCNoyP z6S4`}gcGa@i)c57Nw(OW4hSZig}ZstQJQAFcCis)abQ$)`OpkpJ@x((=; zuT*7Moq48NFNAfv zscGI#Q_Q+WR^4K`c}cjOqZvf$LAld_T#(~&Tpt~1{9|w99K;k5jOaiGsAe{m-9|8kuAlnb*GC=+`Ff;(z zJ_GEZ1CB2N=U0I1Yry>t;Q1D4`3~^jdka3_+FR;%{`+pJ_@V-S04Bf$m;e)C0!)Aj zFaajO1egF5U;<2l2`~XBzyzq?aB&eA7jbdX1Y2Ch3rq9D(iOk3G!H@^gggj2=Yew` z*o16CHX)mkO~@u>6IQ$li>NnxdJ6Ou=qX^)I!^?i2$ShVD5H8)!znkVPxz5kHEGyT zx%dud(p;?FTz>x=MVwk8N-%G(?#h(n(-b_Xc0|3VR-HgI-nYL~`R4j@9D5VgYm!a1 z{1IOPD12GTKy$y)B0KKh1!V4s~11TSn z-T`F%z`!mb8vt@`Kt2czJrCGk0PHUUj+X%ED}ZYs;C>bG><3z21H7*TzJq}O5D+*F z1m6VOk3hZ&y+!#(aAgM*U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0+eqk-%!4x zd_(z$*EnYrvI*IQY(h36n~+V&CS((`3E6~fLN?(9Yr-P(O%(^la8L}r6M83;$vfe% ziC?pf@=ZNQ-IO}!r%voxv_=R;h>NwG%kM8Qw}`X2B=cs>4VXJr$;LE%Tddw6^EWMc z3u-r4oq4nSYTC`%%)AL6y~Pso{#x#Gq&v&5-E^F;GSAZNni^H|nP%#jEEvosH?BW+ zpNg&n-bfxvR?qX9FRW?4CMjaw}AaSz;W-dq1dbiT=xO)`vK1bK+Af-`yk+Z2=G4)1Reo` zn}GI5fzTEpycLK(26R6D>v~=6iCNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0@QA}u!swbxUgt~EiB@trFm)TieFlq2O$qa9)z6qz)25mLN+0rkWI)YWD~Lp zE8c`fv>P3#eMrK{b^5d}cp~sbm`o008MT{QxFm{bUZRMzOBK;~nPWKe-pyd9xOWpdyl2*k-p!!5VBC$_VwQXC zhEaGo_+oHawgWLA(7gkQ`+?qFKq3Gn+kjLMNIwr`+JS)%AR7X5F9Z27Fcbl7`vLoF zfa7(*c@S_N0^ElI&znHY5y1Nv;5!ER-v$Ej0KpSL`@2BsJs|u(5d8q?{0NAB40L}2 z#7_aeXW-o=KBafVaW_nW2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!(7T~`L+^&( z4ZRy)+MG?uCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35$3)R89<3PNYs9$T>eaX7tQ%9cWnwuuyDH<{#0P(E=?nk6-1WvQxw1PqO=pTbH$5j#&mOIF zGfi$gG#cjyUkv_A7D#*nBy&LOERfCvnRCFv5Rh&Bclb3nAioY6vIDjU0Q-8t@gU%Q z2yi_NxE}#Ln}C)_0q+*Tw-xX|1_T}lf=>YLPXeK*fbi2mbSKdH3=n%3=-v&)p96aL z0ExXo@&zFEB9MOR-|KamSN@%s5oH2QfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Od z3DCLWsv@o`;;NzvwyKDil;$O+D}G679)vsyc@T2K11CJN3E6~fLN+0rkWI)YtauX^ zac+bpIPRvJi}a~n@I;UVPlT#Tb0U<{xvArPo04by)Wb_!BMig0#p?aS(Ki?GDVK($ zjMBSVtQs-X#`<=L`WEzVCY*Oup;)VHb<>y% z%OiR>ZEmC4s{Q?G`1@ALBrMb9$p0xEbz_qIp3-0XM7>3i4+n)MtST%KAQJ@!4glFs zAlC)tW5CcGfbC7degts51vrlZuD1dAJAmf|(DE+eeGl-x5BNU-0v`dvkAe12fY2!* zd6S4^>SQ8e}Zn)5X@+`EcHX#ZAniT2uJP~jrlu^5pIO3+{ z@jf+T^Xg&e=JGqWV`kwj6S@+{-CWVLsa5@$k@TP+{AjKEXRm19)T$4e#^P41{#pGv z-|B@rN2b~$WJ%lVL9#MS{ z1GWZi_1Xi5?;kK+c)%c6X%86XlT)=<7-f_83ZtxO4^+ulm2dun7w9Y|`TTZ@yxOWa z3GF{4_Uo-Gt29c(dyT`-!E?=3Rs&QP^K8v(nW0)O(@oh8zp*r)fPCZJ>qZ`W)7*TSe502eZ@E;yF(Hgd#}(uod@qPBP9Wb547mW?CcyqE;Mf8 zAh{n%y#}OT2QmkNfkQy{Fpzr_$R7cQ-U4jBfc-e+8%Kijjo``-Ccp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaah|@dPN}aCs4z7jb#f1Y2IjOH1?8(iOk7G!H@^gggj2>4B3T z*o16CHX)mkO~@u>6IQ$li^w+yNpQ73SL++8t?)#U1W$y?b0U;czL^i9L=lbmD58Iz zP7ysDfR2qo>o%ZY!mjJEXBKvpEU74y^l#1=gEy}cy7M>8(YoBfxp>d=vbs2UyAu4H zk_?+WRAZ(*=E`FA?~I9FGs2&(r+HJddGl9I&8dc~qslzh@{cs*Mv_-wFmKLp%Vpl! zPW;Bwyz^GM>(j>YxS2PR+`*B}o7P7P%p1MMDEIw;qcCsqyT^ zfahbNL?K=qn)nH4yy<==>IleFt>kdn+uPwLtHEK;nKN z`2dhw52POiG7kX*4+GgpfZQe^|0poD1+Z-c>@9$!6>xe1*LJ||yR}~D*>P*d*A(yr zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GCP4Fs<_*monm06Wc!_g1A)Am*$R=bH zvI*IQY(h36n~+V&CS(&%uqG^G-f*q`cF5Nle z%o}_$SS-%~o@arU-GKKwz_$nR?*#%c0Kpf5_LqRrD?oT35PcQs+z-TF1G--a;s=4= zLqOs%kbDzJ9RbpB0hwdKz}rCf9UylC$iE8=y$9G*fc-NfC(@GCcp%k025#WOn?b60Vco%DxUz&8?G(l+9Iwk znqX^-cxh=~TDszwmgYgogOCRyCp~b|1DlXd$R=bHvI*IQY{H5+VG;ADN)lYE&!zg+ zbXIsGU}OfJl}U3Vl+nDY=a`#PNBq=5Ov11jMO9Ub`#0zJFE6)+gUK=_XgAkl&s=yd zPLMUCy7b$8^~p4EN-}S*hXhk?G0Eq*Q{@Z&V{khYwTUkvHD2Znd<%_8)TB zbH~q=-Mr~IU1gr72{ttlY^>8v{gMT9x#Y(6$8H>*d87Uct&RFom^b)huvoqZ{NDh9 zZ-L-NM_0n27B5WWwH-Vby>0L0b<-46oshk)LPfy5&~aubkx6i9CYGFyRx$AIkP zK<)`3|0FQ<6kyu{*!_TG7vKy4t~S6O1U%0JE$x7}1Mr0a|I0ui30AW)#XyDRRxSt7$dM$6dOC=2Ne}Y2i;W)_X7)2&zi8MaYu2)ycKm zRxjV6ZLdD@W<0%{wySwJ=Y5-UdpE(?{=*V^%e_A=9!u#GhLlRGsUc1WYsN} zo0k;qU1-%UQ}@_iZ5Y4v;(nq}~P6?*W_x)lOY_3g6~C}F4?-SD1q!_!&C973eYcIQcPt+p|E5NW5}cc{9XnH}$}Z-DTaCJBY~T0tzfbMvs^e~U zT}`_gn{hYMM}BMRc^}%%=er*(yLQubrnq*~bK>;u(P=kk%QU&~0mCT9-QbHsVc7)4 z9tFC$0P(Fr?_)sXaUl5wka`kGKLuo-1_pKl*=K;B83J4{ z1MV>3i2yB8zQ@on}l|g=%aQc zxTb>%FaajO1egF5U;<2l2`~XBzyz286JP>NfC*GQ0ctnYZm8W*yP6S4`}gls}KA)9c5HDM9$hO63t;Hq{C0g_;JwAKtf5hmt|P)6;h4myb< zx*8PGcBdjXuU16kJy3Dh>EPmQ06I1Tt=jo^`_8^ zDbQrf_|Mg{%o$+dQy}{pkoz3Se+dkI1=#X{ z{T$#J0-TNYaBOUVdmZ4h11(K}*8%u80DdPBXa<5VpnW3{as%PbK-2?tZUbU1KzAz; z_X54!frJl8?f_DLAib-;UY7~fSA0bQKL8V80!)AjFaajO1egF5U;<2l2`~XBzyz28 z6JP@LZn(0DD~q_YXo9UQ;#H-2Rq2XfRhkDO4?-S<9P+>+4{SmLK*q{%ZApA91Yi0CqB#yZ$dBjf*yrg02=JGqWW2WIO6NVB- z-oSzx*IPZk8~FxvwW-=-xUewLDkbO7ivE)^uSzMV@JvOgf8B-mlZ{ z=;b!`m3oU_HehSORjahDMG+g>b^~Y#l0)=I_s<1o_9);u0604VR~O)p0iHL2mTth?1Nh>A|0odX1%k(c_5=_*351hC zv=8V^0kIE(?lchZ2YNF=;xv#P08)cMItygJ00weE_AHRgL%YeJqjtk_H%x#DFaajO z1egF5U;<2l2`~XBzyz286JP>Npz;Y&yP6S4`}gcGa@i)c4APH3D^y{e*mB?+Dglj=k$qjoc&BW_9_?=xQv-n>Q_&ficg zmzy^i?^#|}3kPpkx_Kk1Ld@UB@{H;I`RW^)&fg^U&NxlonXkUlNVA5<;?a7c*ZdhF zOWIZ^*J@jR$^Oku?fK*E-~1JCs<%|b*in_9YWYXl>MWA{7j0|0u$>~Ww(3nn`_G8S z^;VTt8l~aA#^LASx#lXX0XB!Q|@wq{^7X!H<8@Ik^P&V zZI}5sdO5P-Qvb$mG0Q#nEBH6~UT|5y1M>I&CtRDgfXxorn*fIcaBcuxPQcv^cw9is zM!@R^e47Ek2MBBff-OLMD-iMm;q5@w2XyWLVt$}|7Z48sy=_1u2qd2eQtd#x1IUDc zftP`77|2C{d=wZu0N7p!><9m|UgtRUpS+?d6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e)C0u@hy{tZ_cafK0A7)`JhM!dQ-uP$Bjt4s4B0qrH7KI(PKA!nc}GVQJsWhx2F}HIFqmd!%jFtvoLZqPLBA=Y+$_7A1#~T%%5qhJ z87JRn_>Jl*-e?nXjPl8;npb0#O`2C@lojoPDmkcHLOO^xs$f%0yjiV1Zj#S$%O&2} z-uO>Tv*&km*H_7NWfyNER~URVcy6m~{;}r&$nbw|nqfBnY30g4TcxEd?pU$%t`%}~ zv++{NMsG37t!s6c?i|s)!54$Y(hWFz0B0O<9R=LIfaf^Sk^sCX0bdgE_W^+v5d08m zPXnQTAe;fBr-9A^AT|heXMy+^KyMC6oCT73AaxE%4*{9R-@&r60oipx&JN_8fFTE9 zdl0Zc1UMcBoR0vmO@RB+-_`3pTYgvZH3j?tOn?b60Vco%m;e)C0!)AjFaajO1egF5 zU;<2l3DCTuc|-Gt<_*moUgexk$R=bHvI*IQY(h36n~+V&CS((`3E6}btO<*lH#%1QcFa*k(>#T8jwUA6K}=dB z5K(dYW>~~oUTzC#aY@?E)u+JJsiKUrjJF!~uhG0QRa*=fM&B6ad5~JFEmiVtZCfqR z)wYs+y|x{nkvF@qX5M^1@}|7zjmz^pOKU&Oo8z&KR`X2FhMBI~FjLIBMONKnxp_&! z-i21(GIejA_S`w@b8j?Hy=Cs*tL83jxP4Va!`}p&j?-1vp_G5tKZNT{s;5q@g z-vvDH0WI$X-VXrZM=)>vAJe=MT-Lz^m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyvCu z0L>e&E#lfDt}U8iYm0bkXlXO zxo?Ah6ccao#o(}H0RL$qFaQJxf%YsA`T_{&faqDEGY`bh0o_ADyz#%_*w}!?Iv{BW zQcXbG0c17+15O~@4CGuuej_mC25egZ`&Pj57~p&ya6JLIp9DNl0WD7h-kpH&8NmN6 z5ZDa_p99+W{8zm$wD-R%zM_C1fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)AjFadfu z^ls?g(7T~`!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g?}jVwC(lZIdJ~f1 zuSt`xnkNEIgfee{2z?38&9w_OrZr*|x7F&C>D){> z=Vn)BoSR_#e_0~y{#)+ycn_4_xoJ96+_~vFaeDUXoEwwHEJu!v#<{^4gTK-NghD|0 zWgr>`IwL?V3UnU;;+;Tm7m$bn$v1#hH<0cDGI3zwD3I+1a>s#u0vI|8*xm!|?*onx z0Ov=5>tn$E3E(*ew44FFp8~$m0RQJe;7cI*70~`Q5c&oPe+xvv13K^h?|NNq?SFGQ z0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfX)q96>(J&R~1dLRYkm{G%qP# z@k>hcAml;FgOC#*IN^a!$R=bHvI*IQY(h3+#hb8*b0Z|dMfzN%UqxGmCxRq+B21nW zp^VN=9b6JcbTufV?M_8(Uag46d*I=$(hayz2>tMFv3kF7^v%V4%BA5bC3!caP-d>`*VL5e)YOf;cLUR> zUVYQQ{R-oAzh0;|OSMJFlD5^!wc1uM-=J*`7q&)up7!r5dA7E#mgj0)Nxoj&nl5ao z$TQ87NoUc?`*qqKz1*h0Qg6}A25b%3>a_<9-#=iu@PI+C(jG9%C#Pz!Fv=$F6-HUn z9;lM9I_~B#c!3U6U(S!cnIf;Y>P89wq|F-nRq27G+;K{P9H&+^W)0hg&Bda&Tf^j!y%QU&wZ5YM48+NfC(@GCcp%k025#WOn~YQ)f=ieRBx!>@Y3dNLN+0r zkWI)YWD~Lp*@SFDHX)mkO~@vkU`<#=z2Rc}$+Os=>VzctYf`4u^F+XjP)7Ae`Uh5N zMynCkKusD!^r#&(6K9zaC5*fotLZb#Rn=wM<+o+(&aryaEB$49H`ngnocC?Y?cK!V z|841OT_AUz{QC2(F-Jx1E~a%J_%%!z(606O#!(NfqWVm>IZD60Q(uh@hRZ^3~+r8 zxW5EEUjZ#&1Kw`{-?xDOJ0NiH0ys8nf%f}=(EUL80U){_=zI`}Jp^<=48$J+dN%=y zM}g!PAhi`pKenJ=mw9{vmozW|Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaah|@dW7I zaAgr!7I9_K1Y23eD@*gr(iOk5G!H@^gggj2=z)VC*o16CHX)mkO~@u>6IQ$li+DE% zNpPV)7wQ{ntnfsT1W$y?b0U<{yP3~1HzkkwnU4`aYlJv|L#K@3;NYRjKN*00f*@K3YG+JX2FyrbA`9X^L65$f{c`H!msJyU?mzrtYoNo;yc< z?v3WDx6HkJ)!c;*x36kwuu4l;+_7TiT`S}(2{@75!I1@=p1nqMjrRAas#;C8Nnm`D zxOD8zG&nhZN3WpY;A=r;@dBCcz<>|P?f`OrAioP33IMj}0Q(-mu@`W@0JvTR+%Ey1 zSAdp%fcI6vw;%Ao1_WLQf(L>2LqO;-5PlPg9sxSv0%FI2?ze&XJ3#LVAn`7cd=E&y z52QZ;G9Lj09|PG>fZQqQH~BNvZ#edb2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bg zJ^|`C)NiQYP`{ym!%LjA3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g{e}zfC(nX= z>JyUSuSua!#}ff3LK*d&8Ym@-IDV@lA`29;W04}7mnh=wQbqLLriiWvMYP?iP|-QB z=t!bxgKk(nx%dtS(~K;GT%(LrD-0#rHy4bUi$Bg~)h8~?O& z<)8m1>>{1$M12e|M3A4oQ9 zftLFK@BM)90l>c=2s{V`9|GDR211Vj;Y~pFQJ`}R5ZelLKL*4f2YR0X5>Eojr-0Pc zKzb*Tc?KAG7Rc@fa?b(zJ;2ajz}5lSL;q8+bG-aN6~CesSaM!8evd`VPups?%%9Ipb- z{ebH=!2LSlIS8~I0=$O--vbN- z@42FZ2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fr=+U?S|S7wHs6Hc%uETY|Tq5b4pXisfI68tqO(g~gjI1$RI-PCf% z&8X)4z`H59)r^aVQ^K?xSTL8|YSe!k$MjaG{(Y)9C8;;(2i=U%z?-(KsW<8%937_b zoF9BMMP6;yn}qhC5&QL4l~o$0;l0M;=is^KDysn^i+Q%@w9HVQmgy$f`rlhx-+_A5 zx8tp{t2cw0;_6N0@Sa&CsyEHe#!D?5BPIYvj*dpX!54$Z@*v=O2xxg2@IC_gHUa)e zfxs3ZxD{xB3VZ@9RKi;K9pXo4*+;)SJoVd;urSegeR4?-S< zob$jr4{SmR@OC1^Jz88*vRKc>38w`J-dq;oUjoSR)&b8gPNHsyA1yzl(p z()RQp+9nrpbLr4Q8t*5lvGiA_;?W z^m6kO_$%+I{>ll!|1J=C4+y>ww0{7EJ_5oY1JO@_&Qn0_4AA{45dRG5{TxVq2_(M) zQeOkNfC(@GCcp%k025#WOn?b60Vco%m;e)~cmi~8 z=-kk`p>sp$hL<*H6S4`}gls}KA)Am*$R=bHvI*IQY(h5S1Z%<~&J9=DPo7owbS5Oh zUy~M{fhPh^gfcoeb#O@((bb@cwmTKEd9@-M?}3N2PN#^T4M4|6pmiJ2FJaep*fR?| z<|v|Ro+1WsR>bjJ6%kpW`~!^5smFI;BXq;J#p?aS={FbeDVK($l;quvLaUjtDl?{Q zze(yJp?Wjn)SC*8y9qx12TSBjs5jomwKL5#HHl?9B$k_|n01S+y2a2~3id9v>XxZ{ z>$K<2QJ;IGdFm~5?_M=`VZ-gK8XBz9(iL~CSb5h9`AXw%8dG6;MD?c4T`=y(WHHN) z$Npz;Y&z2V{_E-vEY zq6xORh*y>7Ri!I_RcRiCJP3IZa>xURJg^Dbgls}KA)Am*$R@0K6Bbc#swKg-`dq88 zqqM>kK@vO>CeMjbM)gMe2UclDs}a>eO&a!YF27SdW+u)up)X6eDl& z#o(}f1;oAvy1xP9-vYhg0f~G62*+kEkh%{@-w$LS00!0r*$08#LqPsvVCWIR<_7GW z0fz^0ZUbB`fV&m&c!8GffY%53b^v}q5ZDC-13-Hl5DEg}=YeQD(AfdRLO}P+Ks*fe zM*di@OGN)z@r!x#126$5zyz286JP>NfC(@GCcp%k025#WOn?b60VY82hTaXm8+teN zZg_cfHX)mkO~@u>6S4`}gls}KA)Am*$R=bHPOv5{;@xni{p49`Pj5mJ{55IPRq;f? ziBLxGWE}Hy7_&URDhUZ&!kPGirCHsBW{kWTEDcarbUu z`qZm$df`_Xw>$MhZ?1-7BVf~B&tCw%kwuTE^qdZUhca=O_+g8hSwXGyyuWd~i zwo~MpX33%+H$WzSU)B0h-$eZtXTD9Rj(=dF;D&f~K%7gz8G?x9U#_}4FcpXR{1X729^kE?LCNOXW z$i4;Sjsf|%fuVN*+eyHl1RQ;UGX=Ol1l(!B(+{*{0PktQHvsqtfj|}regU-SfY4bW zoCl)kfX*Qx)~GtXQK(8H0GyzEmklFyGol3nf)2wh+0~25ZOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;-6Sfa(nw7jbbB7Z**i#YMcbG_NdO@heO7Aml;FgOGzBIOu^* z$R=bHvI*IQY(h3+#hb8*dZU*F*XnbvzM!ejKl<>Y|;@{M&Kc(?Xw_0I@z4G^>-~?4)NC)NSFWBm_EmUyS{W^hZW1cqQ^Ce=9fZo(L z#ayGkW~%zpt4#uOmAG^sjuHM%>oy%^mWNfDyzz|iA>%?sGK z0}dbH+yS`!fO{9<2>>l^fHw&Eo(KHxK%fH%hJf~$flwF-M}TM)=sW<#I)UykARYsH z-vAQbK(Yr&#ewuuAkzyB90#%qAa@c1PCiKiM^t=q1wQ~2U;<2l2`~XBzyz286JP>N zfC(@GCcp%k0282qLji{Z4h0+vIK0j|n~+V&CS((`3E6~fLN+0rkWI)YWD~LpCs-2} z5pcNVe)252r$8YI{+d+kj64x=B9u|UsaKs6MRaTgu*zH!{gNVjuEU;L*fB>DP4g5w zI+~wU2R{iOilQnfBdH(D%Pr$9F3G;Rq9b!9cV@Kv@3{1fFIKNfC(@G zCcp%k025#WOn?bgJON5KTvo(oMO;=i!Il;A;?lghbj2?&&4Z8!ArC^%df==FHX)mk zO~@u>6S4`}gcWbXBGQdc5?rOvRr&_%Dm)P+!4qNfoCsxiyt3M3Hxv*uD4 z<#HjXED}x$GjB!`fiC}4*S@Eo){T6FX3-cfY>o0f?cY`MY;9XD&(*e)Jf7A~TV<>p zTVSE3`5dg9`?u!HZru!Kid#34!+U0pXx$8YjYVhJl&;<2dqH7o0~|rX`8?oi2izTi zCj_*-40ywUF9P_ZK;Qrn>;&4ofKUtwzX3$MfzBQv76-bI0`Xp;_c)M90Lha;DhZ_f zfJ_P)_z=jZfm}b3&j3TG0o$j5{WHMvIpF*faD4^1zlL_>`G(pJXWcLXCcp%k025#W zOn?b60Vco%m;e)C0!)AjFoDV^K<$Rw4YeC;H`H!;U2`@en~+V&CS((`3E6~fLN+0r zkWI)YWD`!XCM=@eaH0L=S!hpfLK6HnDbiK*M8JtqM(u{fZAu&LQ!DyzyIi@sc+c{& z(Qj&mp#`&=;>u0WiPN)3tK3w}jqmG9S8njVpsx%8p2q)$USk8i>j0k}@HYVg2N2uNfC(@GCcp%k z025#WOn?b60Vco%sN8T-5f>G4QPBijRKyEP^McY9zo0Y^LLP)X2sz(@^BvfPY(h36 zn~+V&CS(&gog#WR0392F)@?w)gk9HR&n)biqll(?iWt0E5yx*;L}Y>T4=^#O9^ZY9Fbv-o ztM|nSw{lrH$|$v)#R3XW^;qBTP~T#tuRj)#QoE@#OD3JgD4(1Pl}C>|CTtDZD%t~8 za)l<{cw_%-Y1_O=?mC#=S$6H_N|SCHQ(<{T?WWCLFzKe+Vv_rw(wDB?;Cn$~c?0lw z1A!hO7zf&q0-;_Yd>n`-fXj`yr4>1Id0Ml>ySHfy@9fFbHI`K<*15 zp96-@0=91e`?rAOJHUDGA}BU%0r!1?=YF8&0l>Q+@I46l9|8go1Hngt_DzfGb)iQW zaYX|YU;<2l2`~XBzyz286JP>NfC(@GCcp%k025#W6;FWL4YeC;H`H#Z-SDdBY(h36 zn~+V&CS((`3E6~fLN+0rkWI)YoM25@M7!Za`^mG=p4x;Y_-j(6)AK~YiBLxEM*0WV zXhy3L)j&%cc5W`eQ#)oB&N87ZVcN|VJ)2t9k1>tKtycZBG;d7R7K40)Hqgc>&(pS5 z@@#EeEzi}ql02RRZ?0CZ3Q~LKx{kE?E~UFfL=e4*aaj5K&lN$2Z7A2rQ;S zv0cr_aYk*pU1dLF=&vU?vA+#x^NpznxhsQ}zlAh%0 z>C=DSkMlYN_>KVnqd?#|5c~uPodDWS!n_H8M)O7#eN_cN025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5pn1c!MO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFD zHX)mkO~@uJdJ|?bZ*)e%rTSc|ucx!ZKLn%TAHt944HEuxxC=z%s!0G+`4#DThrW zi^C7dqV+*pY@I2Ky4kY0I7b%U56hyxN*2wJ$+UB>+BrtivGSUDGicykeSoPn*IPF? zKE6#4r$T6R5N~cpr>RurnQ^zVQM@TR@y4fB1sj8;jJ7G9coTj7AxpRIpQQHFZ=B7$ zc++~5kvIL9)=B1{sgrjkOy0S7f?5C2!iA4mjdK>vU$F4;1ybwU4Dm)N;>}vE*{Z&2 zoHDM*Xc88!)G*mRK|S^p<(SE&x9FvY^%__#uPYXdABeR8T>&8eHqa9U60JaA2uQvQ zq}qUVJ1`IiZ2JKFM}T8LQ2R09JP5cB0q!F}{ZYVk9BBLm@SXsgP6EEq0RQJe;0qvl z3J9G6+P(zBXMxB$AbK9?ya2?$2D-ih;+KJ*D==>oOaH0pYYO-Qm;e)C0!)AjFaajO z1egF5U;<2l2`~XBzyz286QFs+wMAT8#I;2wwzi13mgcRci+*cqUWB{|c@c8b11CMO z3E6~fLN+0rkWI)YEP4}WF>lI@f=l(eRNp{ng?|VbnL%geNArh}NAspqkx69HvKqiD zb6NBnWzlg5j!eXX$+D=KCR5H)#iXeala>grEVLQgO~#M&@@?VF%1OJq9ed{5dyR^M zGXj%l`bb_p#Bat-G6b9ZHeGqKzQ9|h(8FUfScGI$_Zq1}-D{=0)xGZ8UN22ke=d_I zse6MoMco^vyVSi&x>Mbc!OupM#-fq-Xw(NwwI-qd)8n|-s<2F@aqy^q z@HKd?xy-7Ap<X0 zI>V}&s~nxGzIL+m+I!4n@1OF-qA4?~9$i#b^_@f0a=OesQGHpta<*}%UZVw_B~`CD zc+ar%%?+MS|6SjAZmH`Pn&{qQ_9YAU&kgf#=>Hym+!2Ueocmb$g z3pm#St``CK2B3Z;;MoK;ZU($tfTouL-z$LsRUoh(2)+h{b^vW}0O2=*$Xh^kC(!v0 z5ZeWG?FQm|fS&h&#QQ+sULg4aklOc8m74TNbZ-O~b}#`Zzyz286JP>NfC(@GCcp%k z025#WOn?b6fubis_lE8b-5a_$bZ>Zvb2cHHkWI)YWD~Lp*@SFDHX)mkO~@u>6P8#L zW^r%0+Me18{VPH3#3=ZO@T2-e$fJ8x!AUnGn(}j_AW~is7bl0AHzSZ{hVWnppPD zkzKoK-M3@nklIc4E`6qAqnA4Vka=tf?FQcq3QH779{>hA0oxJ4eiU#V2Wme7oF@R+ zNx=OXQ2#mL`2uJ>1$fT@OL3xLfH*w_EOQsb!qdC?aX@B=UbCcp%k025#W zOn?b60Vco%m;e)C0!)AjFaah&?S>1BxUh%|i%M)^5$`O`J4+Y+&eFUHc@gp=3E6~fLN+0rkWEwb=YeGqHjq2t^S7ZDP&^{}68N4@nvC%`*Z+8Hqj1WF$U(Uo!MvEEzeUz(r)G0q%9WurqkG;Pe?P68 zVTavp8_v4z2G*H#KLnTQ?m?Pfr@wy2%(~W00CRXhxvj z;EO?F*$6l`0kxX}=N7>AGT?p%sDBmkYzG=&1H3zcrZ)iJn}Gifa6o3wg+(b z0NfC(@GCcp%Wp8&NRYB$twsNGPz z;hoLdgls}KA)Am*$R=bHvI*IQY(h36n~+UdVojJuyWy&K?X9h9rx0KijE>ftj(-TH z`G=54?Pe-;5?QoY$)fo&S*%$si|W5a#aX72MaN2@Wi`;S9_TgV&>c835eFvAqGp;b z`tOs);Rj^V`k?#|Fg2$V-+gcx3?9tNX&y`-OdL5U^#&G9fwslUi43wZ+~~!O_K_c` zcXJm8%or>`>S;I5Ge5U9?EGh`z5mmWyn8n{n|4zj4@yINHx1kL=I?no1`E8K#7Mjw zd@(pIUjpv4K>azua~^2C0C>L!n!W*imjVA3Ah7hG;n+M0gq{N0o(96pfyfFV`V7$d zED(DR=voEDp9gwg01|6~zI8zIMIf~SNN)rNHUYLqz`hl5c!An&fU^m3H3M!RQ2#pM z@dJ%5|6Hl@2L73M5@iBRfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Oj3DCRY$|9~T z;>w~DTUo@rO7pJLMZc>wFG5~~ya+kufkPhHgls}KA)Am*$R=bH7QG3xcsB;lhcODe z1Deuw2dG4G;tjo?Tl?G0qjzKc2li-2>JXK~OS;~>$#`5bY8uX5q03?94J?>~Zk6-Mi^Mm3`h#!?tBZ z&%5dO+~D1qqz0dM1oLk2#o(~K19*1D(|5O^O5?gc_00B!q#@JB#oKM?&G z=sXC-4gp<9fcR0M=Qxn~1n4^fBu@gV&w%vjz`z%P?KEKT104N8Z4z*P1-Mdx`yx=E z20WL5#sR=v{R@0P8{k_8`0c-_)C6jNQS=oB`~XaV2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?c{yP6S4`}geBI5S-cx6 zCsa=8UX{_kG7A17{HXpA^61@6=Zu@2=le{@e4iykcXdOpWOz3jKhDdmhJ(}PuHL|D zTp~<`*`_Ne(iV8*)SEK1(WJ5HrDJ1Z_Gt0IguM=XS$(2RYS!X_2E$$croO-+m8mZ< zNIzAN8>PRhdy}+S-H(wLTeT*k{?p@qTC2h`mBzuN`oY)WwdOLb&QvqaJjtvT!a4yD zt>aC;<-f4B9)f-2d%Jp~`A+qz@p$UqF=ow7t7eu|H#_6#46A0Ya&)Tt+R4gm?=g?P zf65b!rp%~%bWv56)i`It`~?diUm)GgzG+SE9ooL>Sf63vm@HNfC(@GCcp%k023&B0<>?qzKH9KxW1^w))(>S(!9BJ z(Qhuzi;x!~FG9|G;H(EWA)Am*$R=bHvI*IQMQ_3^_KlWrGw9Qj~GtC>5!BSdtZ!py5s(GUrHd+`sndZ$^ z<0jwcP3X`sEFGJFDYdWKx+d@DP3ukO-t=EuCz*eynmdwW?o2T2A6mHZk?)k9y1ESW z#%viUb??-TVD1gR7%Y|}K=>#SISxcW0Xk0rv6Dd8XF&XOpyvx9aSG@=10=r$QfGnm zIbh&CV7my|(}3d=P&)uPtA7d0#s;{T0rhskQv)NfC(@G zCP4Fs<_*monm06Wc#m^7A)Am*$R=bHvI*IQY(h36n~+V&CS((qSQBP3Z#eNo&>kpR zdw^0Dr6@{KMxnojKZHD*HmQJ8)zo4osFs%`};Ejw&Wig_yKN zAfoK@&7g=gFW(lvRScMwg9o0fv#78_^UwAb|CQ@(6<9fz5%4(1k!H-13Lj*5U{rbju24$ zF5qkfTRuyNsC%t+x4JK_i8sx|c{igo@g{!mmzMtB zv!wQeSKiONchjH9?%lNR+c9xS@1}LF{`-oJuxO>`d$TGwSr5WtIj1-*=YhlppzmuS z`3;b|45Y6B150PYu~`P#?SP{OsC59&m4K@jaMuC#PQbGoXmkPIH9(Ua@T~{@^+2Ej z2zr3fOF&yA5Z($zyg+mt(AfmUnt?7K5Pu!$@dJq#pf3O<-v&~_S(Tb}>nz?(lnF2a zCcp%k025#WOn?b60Vco%m;e)C0!)AjFoB{cK<|d$4ZRzBH}r0JcXKu&n~+V&CS((` z3E6~fLN+0rkWI)YWD}NH6K3&lZe^AN-2u7-g>eU{L>YySLvQE?-ohV39=#imxXEq2 zPlf1yI77I}_;KFo_&4Q3n}c)n!xLz3!GMAOatrUe&9~gXb91Na=9r|r)cqLjjV6sn zBkj?s51@fe!!ug=AUf=I*lX1%bl*OqyY_@mDpQ}(OUK5l@6byo^&NUiR-Y)7wiU;@ zN$#FysjizXwfj$fl6U8(=1g|yrsL@8Nh5P^v>0Ji_h?qVChI}?E4vkcWe<>k4;XkK zu!RBphkzpj)OG;QDBwB(xI2OR!+<9SG#&%IT|iSe;EMzPPk}%W5bOm)383vX5bgsa z{XjGcbbbZIQb5;5Af5(#E&+)Fps#v1KA#OpEd$baV4w!DtpM!L%&ycpo}FFv)dc(i zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCLWsv@o`;;Nz&TUEqcO7oV|MZcvq zFG5~~ya+krffF9sgls}KA)Am*$R=bH7QG3xI5&a<4FwtsH1t~NwcLW&Qp%T29-W)1 za7kp*UL}j>$7Hc)u`H_p4i9IUMiw0_ftJ-k!+N0Ch(mYa$V42NEQ^|Hvgp507Ka~@ zMeBp|KfuhKN__Y0r@PEjCbwjczPbKLzBC*;C+}utV`r}aA~!yU>644AGPOt+OfH`- zzpwdWsW-!pyZI|z9gW2#T^)NfMp|sunuPjKkKXC9 z9G$AZcCzx?d(30+pYp__DKn}bT~t+NHO^Tuf5F1X7f8LQvd_F}*tTrwnK#wDGQ=AL zjH8bJ?2gS%#2b7wNG#oeI}X%;3V3>e#$Ldi0Gdt%zCOU;4+N4x@GBsc0@^MD;WQAr z1Vjgb&gx%5vatbO%Ye8Y=&1n`4xn!(kgNq#bwJt)46FuhF97znfMXp{`y$}n0Jt^+ z?oB}bX27!rXngrsl^X9Wzbg8Y0)7A{zyz286JP>NfC(@GCcp%k025#WOn?b60Vco% zDBf^s5tkNmX;FzSE#keUd2i{W-&>j&AumE+gdFw2Q4efFHX)mkO~@u>6S4`5-h^4i z8(m>mfg1%+q~TlDq5P0mApreny@5+SZ`_@C>ba+CGJyu5li zI9(3zO%^@oMq%dmk4#sT8`IUzs63#4V=`ED(%l#(W3cF@Y3jaAnxyUx(iC-Xl!~I? z{0%49S`1Pd_B!l;ioM2SltyRnjc41hEX`@?H%FSb6imNqP3;|8zv*}*L%%UWziDmJ zW;JcH9)!xWO;K5z0BDf#d-o)d{2z0|PO@b{w#O0ys_pwI>1RXMpQ-!2JbKe+uxN0UEyqyk~)? zbAazW;J*OmORU@R9FE=Jb1RP;bmq>yx^y zmQB`!&{)!n#&QV=3;@CEe}QIW1KO4WVLK410iq6|b0rX~1-j~hxD)7E4J2GZ-x?t4 z22$&RbUiT80N6GG_RWA}3sCzq;Cuyey$ZOu1NE-~o*h8r8-Vvspy@5Zw-fNc0|a&f z!QDV;5774Bzf@|%@Ba&LD#`?y025#WOn?b60Vco%m;e)C0!)AjFaajO1eie46QFv- z#YJ3P#KlD=wz!BlnC1uYUiU?ya*tS?X7f0Td+{l~l#qn-J?f+uwaQ#|pUuQp* zckia+WTl#A_HctY9r-0zoKxjG8wgL!014Nz$qR#=HtA1Uni9P@8qOU06 z2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0!)D34ZRzBH}r1k-SGD2Y(h36n~+V& zCS((`3E6~fLN+0rkWI)YEU_lc;@xNpvk07S3*8nfCsa<@HU%4fl}GQUf@5w*G~x&1 zO~zJU5DzDZfj61TO}Qe*j2qQ$s`C5k-jtks(>$De^H+5yj!C*Y`(})^*s3)N^`G9P zv1qLd%TyW%kLm|sgV&nNtU4$#=1HmzGeNOo#+#z9Ut79Ag?n@9Y+v5poBl+0_oj8< zj)_CMH+6LxGjD|A-aM(v9B*TgS`XZCP%pvnfyd%fJeD;;%nfv{2jcZWPXmzf0DUh3 z$wnZx6-ax5fo*_oJ79keaO?nT-vFF%0q92UB`j=CqT~$xHpNDbZ-O~b}#`Zzyz286JP>N zfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5pONcTT2)H*3!HPc@gp=3E6~fLN+0rkWEo%{)br5&7NbD!*_j>1LkttRcVKT;&%Ib<)vzwo>TfsTeFm zGOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlOjnZA}-gIq0274pS7LBw=qdw3|&B{Bq z7OkYiUWdI_eM0x`6S`|p=%h0B3B7b|tojbUWK!Rumt^&cGHKgz(#=)bCg0Lc{L^1s z`hEW@wI4rjpJcvM^>N0-$GLZmSu@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMk zGpZh4R8?g)&RH;j!NSKENHu4&OE(=yPfr?Fy0NOif1Kh58BGG&_TTq?=f+?$NX?0x zIXC!X@K<_(L@&^n0FtMHR3DJ;2L_UW?L1(=064w|YQF)TmjTxmz`gWe;nzF~c%A|p zp9Z|kfu z1xUUMq_+QSr6&E_zZQKp0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R} z(7B;=L+6I~HfIyE3E6~fLN+0rkWI)YWD~Lp*@SFDHerc1VHW2`6lMW9y%u^c)J>?H zux$!9_bQLh%~ZG~vS_c8Me}2_ShH9b)qjVFvrHq4j+H>mY5*f2=;n0XQ7Kv`E)kl+ zZI<#lbNtQqNAjiN$T@j8BhYH5E6R-a_BZ^WFTYw%F2AkvhpFD&2|dPODU5nkqclG9G^KPo+L1{?!rrD)8e^0&9TJ%!K{~d{XgD(b+<#izK2L@UI z+fKm#4&c}Y)b0kHdjQvafct%*elOtp0BGC?cs~M~_5;3;0slcDa0mz<0YXQCw&Otf z6CiQ|h@J#GKLcW)16^ML@l!z086fc`(03L{o&!?nf%FAn;A_Bk39t`9y>V3k8&@?j z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ1k?--f(dd7Z-7HQHd=s;$5YASLveP zRhkzeFG5~~9P+>+4{Sm6DG zjS)Ht+?vU!2)qacHvplHK-(rDycvjW0irJhov#3~SAnkWK>RhJX9tjY1L%7b zNWKN6b^_^lfPr0rEdNfC(@G zCcp%k025#WOn?b60Vco%DBe)Kp?E{_hT;wHZ_XxU6S4`}gls}KA)Am*$R=bHvI*IQ zY{C+2!YtyAt}v^>DYsB=p>sm#gl$u>*;jcKZ>Dq3P0llZrbA2`oD-vtd--nu&5a(; zyu5liI9(3z&8_$|OB8?Rd)us>jhgw&_?bd)KT!APu42r*`5R8Eg=$iUy$<`IVz03n zrO|P3oFDz0rQyZjNbS`dBL#DBT2p(6c5kX1Gu#_x@=)F75xF<`X7E_{1MZK3`h$Sy z5YTu8@E!%4jsw0=0RIUfa1sc9282Eb+P(n7r+~;AAo?ZHc@~JB1G>%w@e4rD*FfSM zpzkt}yaJ?_{sx}Slfb}JfUO3wI{?Q@ptcrp)&VXj;9d>Xy8zD`pwa!CN{x5@Z@8?1 z2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5$`R{drKGn z-qO4Xc@gp=3E6~fLN+0rkWEyh@cTsFe8E|;*_nzh=_S~ zZ^}97CifvfM$xenx=ohpG8jv9qkVJzk=rzJDul>Ey%`k?rb3ZxOgGyql&41JyhHq2 z+Bd^azVT^Q%f=x6^(OX>a$Exm<*J185Wd51@F(iD9dncIn4=r5y zh}Ae}!TbdaA73C<@5-=mOct}$zgzqLaX3RyzQH$x$+7|PZUmY(0lv+Ee+v+J83?`t zgkA;OwgchUfXEIY`UcSXCJ=iI=-LUy-vN4d0g2r}-yR_O9*}w;Nbdy(J^*YHz}^8k zqCo8dz}X474g>BOP=5^YbODXsfHw{_eG2$`0Dmv+n?Qp04JY3)0Vco%m;e)C0!)Aj zFaajO1egF5U;<2l2{3`;CqVm#_6_YD+BdXsc$afFA)Am*$R=bHvI*IQY(h36n~+V& zCS((qSQBQkZ^{a@6r6?&4Ht?h6i?VT1)G4CNBgEy5lUpyvKnYu5A+&k(QyZkOvHi7 zvZ$FRQ_xWbovC8$%q2p|LSz+i1`p25w}~?=C-vs0g3QgtnUNl!uZTA0!EKiEnBI-a zV9`l;LuD~o^wKnSUnWgb_XcT-x;ILNoO-i;IPYe3rr!8Y{l?Pz)*Pulbn$ZDy_=TP zW#);hUQ@1EKI2TiMhn_Xs$OyMo{@PsW_ULpcHKz48+0vv0AS~uWa z54h?9cLPxG0X#1Ojg5eJE70TxeA@tj6A)+yf<7ShI?(2yQ>h8J%qjZ)Joy2b025#W zOn?b60Vco%m;e)C0!)AjFaajO1egF5pm)QSMO<0Ll|?1CvWPdA=FO#xesgJFguDoO z5pvc8XFae9*@SFDHX)mkO~@uJdJ|^xZZxzZXhYD3D3}ew6LHK|VI;&ndN&o2Nn}ws zTNW4R$fEmUS+rNlqWLjdtXV9J>c2z8Sq2-2a?Uj|3F1x0R$dSfCx@9g-zhPJZp^4Y zsHAvvO}Hs7@us;r;!WtSIhKy6e=D_b`t>bto${fX@2P3yiL6NeOU`aKx~Z^l{1N!)oyMX9!pmPrpdk^S(ABgV-dOiRW`+&ZWfaHE4^)Zk>2n-woY%#!o z3~+P-wcUU-4!AxA+&w^jFW^Z4ji&)`AJEhf_>zGCD6X{ zY(VF--&Sg3_TO?%0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;;%?fZ`3s8;Um+ zZz$gI=H_fdHX)mkO~@u>6S4`}gls}KA)Am*$R;eYCd?w<=nJz7oN^207CI+%PS`dD z8+?^V@rDy_a+~f`0VyfNxXJi&-slK86S4`}ghg+{EY6Lf z2|*KrCPcwZ2%d=3wF)C3=Fz#C$^kby5BI4QEfbdr-QYG$d7L@>=K3S~(s1MvdN;Ea zBW9=%mX~MeE2sF*yP2gtHjVZ+Upa-oz+0uzn+@=5gk)6r8mU6vYjYlUbEo=xlXRE5 zAA`Noq_JqEJsS0aR%)JxXS5crq{Ci^y;glf_w5t9YftE;GW7|)bZo5p4!vYj-=UXe z^@%d6NW*U82Y+kn-!xZh|7_=bdG~H^HteQ49+ZakZkka5Z6&sVqB(=8O z%)7xCgTrzNNE`wBjsnT!K#i zQVnw}HEGXWE@@x_On?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm z8{XHPO~@u>6S4`}gls}KA)Am*$R=bHvI*IQCDw#lyc3Rg!1+hno2A0sWcYE44@ zr^j)vRbiP*WG*L%7~cprsSr%mg*$bo3<^-^RC|Xp2|M* zreWK%p-0}-)#<;l*=RAzrmkA^{ewfhH~41oST+OcEx^FbfNdLKZvq_6K&=mOz7Dwj zfV%~#4*;IGfyN- zrwd4Q1ATEI`6-a<0n)v|KmxFx0qkGGy>XnSd&8MGOn?b60Vco%m;e)C0!)AjFaajO z1egF5U;<2__zBRx;p!r;F5>E<5?fuwTTAoS(nY_uG%rG4guDnj>4B3T*o16CHX)mk zO~@u>6BfM*v$!`p8WA)iXhamuh~SAhWUDYDVjkU_>F`NpQT=yW^e)rLqGKh{vKnYu z5A+&w=nfpAp942(I^3isLUVP)wSJ?1bN!Kdc~x<6x*YVIoD`b*iZGL(u`#oh-$VBX z<8XA>#^2~AWq6IjQYKAO_XcT-x;I`McT?c8H-E#)wH5;glr1mYV&KJzRX-(}N+P!I5mEqp#5k&vHBXV!>&ETX4An65C+kkWvFwhLxb^!J_0LPm^?OT9zC*XSLca<9VuHO~?j-LDgOn?b6 z0Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCWvdqekz?hV}=-s7B2$R=bHvI*IQY(h36 zn~+V&CS((`3E6}t)`VHy8*O10g41uI-$Lz#+6mjHVDqo?=-!lb)=lnnevFEnG-%&k z{{WL|uJ>+ke0-ZG4hGERpx)eyP*bVsGltvPsNR&EdgIfoij6^1#@iH5y>Yhw&eBl( zd#SzV%E}7!o$5rK@tBBn?-;XYrd2abs+*m0bcR(kS2;RWeeGoBwfC6E-aqAuMN?)} zJ-Vo>%4(dmVE%%Ik1vo~Z!-3#|I#|i{4@3alJfouX8l787d~>`8`oy2HwMi6=~%8E z$=DlwGiWR!!2K>z-v)TvfyOZ4{Sau10KN{u9|ZyjfM6#OIt;YMfbcOO(gj4jfzCJ( z`xNNv0ph(tPXb7s2KxGdWIvEf0_m@SffQi-8nAx@I4%RVR{-bI-{bQ=3AmpE>YoNY z%YnufzpvDIpZPskH824tzyz286JP>NfC(@GCcp%k025#WOn?b60VYuN1gPF{aS<06 zadAz+n$;LN+0rkWI)YWD~Lpi{6A;)SEI25fmaQ zL=;Sj;E6bAt1up79@U#lPP)l;%Fk4>b>=v-g=H#@gGcp)ufc20WmX+T7xN_5d6|HR*6}7!?e8tk$DrPP zk$5TZ>P^e(GV?_BY~{+?#+iDJ7L4YSs#hGmXJqP)$zqn8BO_98@XesH)B#>6(6k!x zxd8tfAm9dq>w!=`(AEHiJwW6oAle9YZUtgqplcftZvuLnfrJm}dmTvnfm92S4gdph z1GZg&eK+9P1Ju3;INt|cdjaa?z^AXUvAMkz*G#vzdhXDT(s5gP5RBuGlmsRis zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@Gsy9?`sNPV$p?bqRoU;kpgls}KA)Am* z$R=bHvI*IQY(h36o3O;1FpGMlDa<->>Mhh;Xr0hHVcQgJ`c)p)n+o_OvZ$Lai;Ht) z(fzP2+N)&I{Fp4(ES5#}-(}IeOe53J`PR>Ydy}!17evL$VdhPq$Ir}F6c_Ul+dSpj zJfC#fUr6z$%xr`hqnD12g*T(c0~7W->}BQ?DSGV=!C3=&Hh5a@!1XNP zeh#Q#1$dqZ8eagsYk{V9fbT`XzX1qr1cIA@&}N`*%O5H=;g|owH4RLF2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?a#Jpqb0Tw27XMO<1`VoQs7YiZtEy6Cr-=0(VhkQX5* zJ#f+kn~+V&CS((`3E6~f!lE}}7V$<;9fCRpb%=uL5IhlwY!${r%%gY%vBn6m<_=l3 zOq4~#WLeZqlSTi1vN-&JELtCwDdwnBQU#=>4C5x_$9bb8;FJrIgLE^LATxwJljBoU z6~$&;##XKzp?8BpIHfi4X4`Py&FBog33>lu>9`2*=9SQayn8n_XR>=Y9Y;@38ku(^ z;N4V*Zt!l*Qpe7WDL8t#Oc<&IhsCQnEZcx^6A)|sE0O)xeNCbht zRv;MyQttxkHejF~uzdj7_W_QNfZF|l^JBnu5O5y?>W=`Pqd?NfC&^o0eUy|Zs^_6yP6S4`}gls}KA)BzonlOuZ zBMP$!oNf!<7AhxHPS`dD8+?^V?`A58+~hpqrxI?`5+Me+S<2(g`8U@e$(M>F=j7gG ziZL0UOdfw+tSC1EpQ@1J&0=)`kE|#_`T}p2LXV*`28)o4>RuyNsC%t+x4JJa@uo-v zZ=x6fVCioAcd314)9Jj6H#Zx2QymXVLy9*I+w|sgvsMV}SX0e3tHC7jUr~I2+>O}+ z?Z%$r;tXmx_+C(0(m?DI&@}+WtN$H}jSWaF1N!VhvIa;wfb>dWpcb$_2iR8uj^}~e z7Xas4z_ku=zX;TC06ZIk#!Y~CGtjgJ@VyN9UjYKI0>SM-=ry2i2M~S(h`b3z-vT;! z0NfC(@GCcp%k025#W zOn}-A7Z!105f>Jf*uo;-RhoB|F8W=ic@gp=TT-XE~5q0rWisB%cLR&;7AdlV0^lE@@x_ zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm8{XQSO~@u>6S4`} zgls}KA)Am*$R=bHvI*IQCDw#lyc=C%7J<`kq1!^`gvtrqreI^Q^61@6hf5-h>c7jP zcbP^O9V>yB)j-2~px1~)ci_lG9LU*Ek$w*%hSfTkUQ?+w8JCJ=ZF2<`+z?*MJPfbec0vImI12Xwv<#P$MR z9{}-vK+i`&Vn5LLF_1h6qz(b;Bf!8>z}5xWyP@AW;?!?M(U(^6126$5zyz286JP>N zfC(@GCcp%k025#WOn?b60qQqgV8jJRTwqjU3ygSoY2IDB=y#XqMaYYg7a@l|aM%Nz zkWI)YWD~Lp*@SGuqBmg{{YFbEf>H#fh=NHGJQ3$?6~;x(qkdBkr9>8oACN`sgR_-MSGPjnjibI>NN65zq?VmOh+&8W&a6+)APeFG6DbDOIi z(T~zLPx;MMZ*o#^t`58@jC$kKs!EMPRcy3SZZg%ItIAEj)f?N%KU(Tudsu4Ues)LR z)tlCv48G~Vv`#YrOf`TcWpK^}v;Lul3m^GzVoqIMhI*q%AkF(nq~740L1Xz0aC{Eb zegQa70j@KE`%9qyEZ{i@G@b{%7l5X(0pB-(|1uD`0tA;n49(_ApzSFj{4@|*4n$V~ zozDQVXMwKgfcPq)=XoIU0?@Y>NUj4?F9PWez`#bp_7Y%k1RPs|S})+-2Dq9YuGF}j zALgnCCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO1d5&j)f=ieRBx!>P`%+@&e?=) zLN+0rkWI)YWD~Lp*@SFDHX)mkO;}=0m_@xQE6h4@>Mhh;Xr0hHVcQgJ0#+W?n@SG5 z$#u}rRG3Li1m*!{w{HeboO$`yaAxJ?-3;&7OjS%7Lmszs<+oA2DLM6K`*7;b=uEwF zzV@)C;R@87y5k?@UA<{JU1pxB8aCz1jGb|&UZVxmxuoh92k#k~dZWx9s&i>aGW7=E z3>wQ0!2JeL|0dvh3uxR4c;5kzJZYfu z64V>-0M#2#yw{&>J1kcad8nB7nRuJ zBHmn@H5J4fLU_t~> z#4%fi@euQ<-c)eX&4{M_z`e=X$_t|6nylGvVG4sY`F-!fi5s5eWW{_Bx{s$zR zCxND?0N>Mqe>o6X0R*1`LeBzi&jI08K;(HK`U23o7Kp6_x?Tk08-SjTKw=Znw;4!o z0a7mm=~sY(R{>iyVD|xz*MV9;;A{b00l@t>P#*+5tw3W4@V*N)wE@0%z#smPN=@Ly z|0w$1Joy2b025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5pm;;^hT;vy8;Uo)zd4(b zO~@u>6S4`}gls}KA)Am*$R=bHvI$G93A2bd`ogRNr`$rhh0Y0`6ShskW?$t|ypct( zQ5GF{$f9MUEE*=uqGp;b`tOs);Rj^V`k+iPM-_7_MEAoP#!beL^F~L&DHqxtq?Z#v+`#dvR8BhlXrKJPrc&t52FNu+GOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlO zjnZA}-gIq0279APW6?-^H0lGb)U3QyYtc$N>~+{{)hBe{KB2qzgib0`pU_Lk#;Who zOD6RldP!EFD3i7oN4oLt`wvU&x<5(n(bvDpyL3}?CcAXgarE@0kx4gZ%Q&fHgKi|# zZt%?@uzUmr_5;C>fzUyq?GO+?0z{4i(c?hpCqV23&~*}se+Kk?4kW$+`c47KGeGJ~ zAbl1XI0x8Lfc+xiNCUN(0OtVUs{RuM8yir+4Di^2#u~uu0Gd_;zFNRv2Lzlza5WHe z0c~skRH+HO|HSnSOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2^2j6N;h0q#AQWX zR#alkig;6L-c-8iHO~@u>6S4`}glxj1H(?g(Mo@*I3PBa3 zU@8Pp#PM2%F%a`8-Asi{B8&DaSu{T;i#3a7QT=y_ILkCJaaIB?tAU2~0Og!Yh)GL? zc5s`eJkA_`bN!Kgc{p-T;!QpW&y*{E%eY~;smg2edEQ)p8{L~>-J4+t-uxA^j>ck= zt`5E#BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignN?@1nP#43)(T;rfQQ!crqH@SSvtOe zd$Xfq)&FC@Q+;YYp1OC8Su@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMkGpZh4 zR8?g)&RH;j!NSKENH-gJQymXVL%TP=jDa@-?oIWD8wTEtlNwwZLXO^|mr#E2Sk@^X z%ZosG0}$B=L^lDQn}OICpzCEI{tD3ZDv;O?^t}cocL1q3fb^Tdz*~UrZNMG`9IZfY z2yng&xY_`BJ5V16JRbs$5y0C4G(`d50l?o01P%ki7!W!Jv~>aDZXgl|qMrhtJ#cSg zy>xFx(brY*126$5zyz286JP>NfC(@GCcp%k025#WOn?b60lGJIZ|L68y`g)6S4`}gls}KA)BzonlOudV<^l*aQZFuTd18-J7L=tZ17bc-5VnZ z-Q+yvryOq5pm=lRgNjj8apnq?LXnzECU>Nptc5ZIsn&7z+DT}*8v_U(6}1#x`3uNfX@y1*8_oiAlLwe zJV4t^K)4ZzYz3lTpmQ4#YXZ8Ofw&LodHp{tH3|QJa!CUdU;<2l2`~XBzyz286JP>N zfC(@GCcp%k025#WMNfd<4ObR%Wf4~vmDtK6-ddWsmM;3OrFjwZBIHHLNe`U#z$Ro9 zvI*IQY(h36o3Q9jn8mx%(T1Q6K^vlAHUv+^AzOu!5cBBWOy``NoM-$@7yb7Q>L>W0 z>y7Ho^+)FARl~vQa!_xw*fE12IFApeU-#Ydzu2s{&Zr)&{9)QRcVg0v0h4C#Qukx9 zH)6JoM%trMA84iKX?R8pGen2I4tuTogznoXbl0BHNx6@``J4JH4S0w0D-F_5vDa9P z(&&u6iNE=umj0*aN$uw^gbHTgw5IkBZQpdfkzwB`8DrGW1l7p-iDKlKOhel@_+l_w z-U1RkfxdTu^C~sz zW%G)@u7Dqa2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b60opgTZ)o4pzM*}?JDjr# z*@SFDHX)mkO~@u>6S4`}gls}KA)BzonlOufqb@t={y!Ek*{_t<`-0;Lz#~z8N%@r-AfxU|;odN15rQF*#g7@K-b$q zJP7o(0*MgN_b!lZ15)ikIt&ba2-rRX?E9hKI6kI&Be<@E2`~XBzyz286JP>NfC(@G zCcp%k025#WOn?a#JprmWTwKJ(MO<7|VvCD-e`(%dy6E?p=0(VhkQX7xJ#gFun~+V& zCS((`3E6~f!lE}}7WJl#LIi~f3K0bpB6uQB+A55Pm`C-dQn5*7(Xtw7SP%3XWzlg5 zj!eXX$+D=KCezMQ&7`R?la>gbEOc4zo56$gt{V%JRcsllhBGTS??zTc7=6LE#mc$H z6?prBPQ1A~>83CfZ?+f5yRjXZXQ}i2S!#dtqh#K_o0ijM=839aQ?3lD8E5J>TF_Qf z^@@Y{jLf^yLqzI7J|gc1-wX~*C*U{?)W!hkF~HRYxVwS+INNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0`zX^-O#(CcSG-nw>M`KvI*IQY(h36n~+V&CS((`3E6~fLN;NEHDMO-MpKwY z;B;H)woo~ta>BML*yyV~dN&mub2Fk5KM-#+w(^2_I5`Zw$y9F26*0yzs@qiM_tU+} zY35BK-J9m&+#6*Uj>ck=uFk$0BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignNE(|UYSEP#;loX)y$IWW@j9oVb#o4j!soyJ6U<{ zJ?63ZPkCa|lo?fzE~=`!Zgfo|yL;2RZ^y(T-J80)jF~qEeDMyCb|f=z@Xg?{yac!# zf%>h0#|t!W1H4T@Q#0W60shy4fFB690HFZT_BId>0+CiA8Ui}s1!8SLS33|713e!C zi3rfw0VJb9>Hv`L1O^TRwxfXkINNfC(@GCcp%k025#WOn?b60Vco%ik|@88?G+m>LRW#DzVi?ytOoMEnW0m zOY<4Z8+)Xs%(>Q z>BiIlXG?SNzohoB8$EfKZfeeCmu@BeL+OU<8;NV>r{gTRsmyk7xLDZqCT z@TY;mB_KEegsT4wf{hIbF9RZWAX)=-I)KCf zfNdjS-vl@|1GQTK=gWZW6~O%}P`@4UyaqJx0K9JiO>Y9ew*ddn|Ekmk-uW-yOOy#P z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`8CqU_j(ha2>N;i~lcyn_$A)Am*$R=bH zvI*IQY(h36n~+V&CS((qSQBQEZbV^LfKzOt*h1fgz6sl=V1uvnDBVnjOd^Z+Dp@o? zCW|$TWl{Zih&anMvglX|w5$di)&spp9J&KXCgQ+kS=3CEMgM)WIQ)PtS|61E0S4z( z;=3;qn!#AOVi8mwgYNjjBjQRFAqw-7W-jtksQ>1}6zTkgZT30?I zwa1)+yt_C3HyL}135 zJ)&~BNrU3ejSnhDO~sijv^fmDxgD40R{WXDf?GH6eAJWbmxzLHl|pYeK(G;#QQd2# z3U#lQ?pF7@YkR#kP5rq{nxyUx(iC-Xla`Q6FffX62n)i&oNM zuftxeKB4>e3Ej0PbW)l6gkCx}R(*$FGO6#-OS1YznKZ0;Gde?WLMtD!bi^Ru>>Svi zck!n8RQ91a4cnFtJ@lq^ZH9OwAl_7CF3<$CRtW1)Of}Q2CR5gfdW%77ZqYzuS*b`Y zwLrKIh&X}hYM|2v#MS^^ZXmuM=&1)14M3j zj!?W2T-Ct@m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyyk(0L2@MHxzFu-cY>Z4bItw zY(h36n~+V&CS((`3E6~fLN+0rkWE-(O_)Wz(G_MDIOP_~Ep$%koUm;QHvB4&;>~mp zy2*LS&vXp=St7JoH~h!G(Y?9;$h^FII5=Gn?#-?EGdE7HnWY#tBA0Ex^0K0wG(Y0a zEJffk=8@%)+(ueed3FPxi_t; zy+gY<)s1?yRn=_9DVmKDnvGWKwwlY;V`G(L1{@QT_`Z!}P{f-A#2b7qNG!*ISQpUM z4aDO>&!<452k7esk_jMn8c6p61O0&Q9AG~WI4%ISUjxo>0M})}eFdmrS_R4GNucp5 z!22}Nv>foQ0Q}DYfoFl>b3kYn(DpnKegTNA1)}SK&KH5$2B2#r5Z?s!Y_6)*B(_u) zeMtd7025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-3xxU`5%i@3C?#FiHE#?rj8 zbkT1t&5Mv1AumGCdElG}HX)mkO~@u>6S4`}ghg+{EaHupIs|nH>JSCfA$TH=*eZ;L zm`CxZ95RV44nH7^)(2&=b*3!pX3OH@99eWfEQ|IkSu{T;Q_i_6=NLuDO3k2pbNvI% zq`6+Yx$*IBYB&`_ocj82n(??o`QoE;-XVT%rE$r?$0SWq_oF-Z#-~*k8-w&$?6puxGSwU9xW;0-rr?Z`7F)F@q5jk3xYnw$Or>$~ zsDAJ@c&)k2s)Mv*o}~IK6BK`CyeYo5%F>^LdUN^U(#hsK)u+bese8wmH8ZW6SyJ8X zjH5HGnz_o+sp@MdE3duBJof%6Pb``;qw3K`RaI8woCWh2EPQ-{)OwS#H~p8^N#>ua z@0XPKPcZ8rTDb6$@7`FwD?`1}TJ%!u+}zb0d@X1!ULdgz=xYL!%|OZrq+bUH{DAE( zz`hf3yaUwk0-U=6*B-$A9#H>2;Mof_egJs)0ZktPzWspzV<2!42p$4LM}W4YK=?Qi z`2>ib06I?svCn|6&w=4aR%u75=fo}QsNfC(@G zCcp%k025#WOn?b60VYuV1gPFny`g$T^@i#V?{Cf~WD~Lp*@SFDHX)mkO~@u>6S4`} zglxhRYr-t*O<7^qfm3gx-a_ky)(P9DV6(6CsNPg6Hi;}+Rs#*|fnK95I_|)ci8wG> z7B$mk+BvG3G!ynJgovvTrouFc}fP-F7=<9y|NeK%`Hofkvt26J$9 z(%q^|qnDgx;IEu)V)zEq;#`=IO%3|=G`PyRhH`a{#$DA+PyaK(oM_hGV?^$ zttnR&pK+#MqXlgxRj)XB&#=NfC(@GCcp%k025#WOn?b60Vco%ik<+i8?Gzjx+1PCDzSA%ystFxD_!*a zO7kM*MaYYgV;(r>flbIJWD~Lp*@SFDHeu15FpG7gp$kD5f-XeCTnL_s6SfMYAm-7! zso;>C5e@jcaSTg&K`fjc#@&1;#9S}OjNao)MZuwVlaqFHm#W_sX4Xw}akLxTd;e{z zTlJ{azU$T3@~++VC$eidt^0ON98$aK_vp>v({2nH{?nWofp&xM1%>5(z_Ayo{Qz+8 z16&^g?)^ah$AIS`(0B;&9s!z;0>0yb{}Ui^0tlW2LZ1O`p9A49fXFEzdIsqH5{R7z zy3PUd^FYr9An`TO_YIJ|45Y3A>7|cCv3U}(*#UbE;BWx7D*NfC(@GCcp%k025#WOn?bcyP6S4`}geBI5S+pB{Vb*|CZK2vie2tQaK%!o03y*wiQRYajtsQ(r^OGO~cx~c~@>~&SY0^ zI*y*6G*acpB-LRUP`Nt(W~?&)#$XZ_A&IPN4)m2(ioWtZ;C=z9UkiBF0gW#L-VH$0 zM!>fT@NWhJTY%upK_W-H) zfb{#oz+S-iAz+UHjt-zU3OEk{u1>&x7^sf{o?}2`7nB=sHNfC(@GCcp%k025#WOrYoqP`TlvA}%W8qM{O8RKy!f^M=wzzo9fQLSBTt2sz(@ z^BvfPY(h36n~+V&CS(&9y$Q1@H-aJrMF@%z1rs57B97K7jDMI%c835eFvAqGp;b`tOs);Rj^V`k?#|FfpeR z-+hVD4Q{iP$3uB1`Lb~25o$NH6d}em+S`2P6hnRe(Rj8}Ib%WXBAgC_kvgeHNJe$9 zkt)=^R=Qi=>#ptf(lqtwGHH^!H%L>|y;1rHm~`Vg@u;PF`(LE?rB9v6yLNN4NjKH; zpfsd*)9lKabYrrZrT%jx&~EU(ps<_-yq^J0p98)x0RJf|0xk8h}w40lH zHWi8=V;+sILix2cZ%S_9P4jRAZ~h7$NMkWci`D%YX|Yvn66!xaj%%$7%TyW%kLm|s zgV&nNtU64JF;7y5#!OI##*8=lw*SS_dKuqWgx&<&-U7lqfyg^RbQjRM8;I=zy50lg?*l!1fy4(u z-##Gu5s=ysq(24*4g$8rfIS8{jsdk@fU_HL#R2!HKz$G3=>-}SfcG@e)Cc(b0e=z* zd<6tkKNfC(@GCcp%k025#WOn?b60Vco%m_YFppn1c! zMO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFDHX)mkO~@uJdJ|?bZ*=q_ z=tIzlD3}ky6LG>;VKl@%nm5xqbdVyk^|w+q?-~{)?sKga47*_Z+L6 zV!l(Aa>hf-xp$0NGt;V>CDqN&I6A|snX4R~s=juz^4fdMWAC5x#G)xPsvccbRb@5K zSulUW!p9d#H#2WqQ+tOtZyHu*m^T9EP4x+l*{Ys&oMH_bO~Rs;I{sht1ohZYlw&6R zXEjL82@N!s%ZkQw1qd(wA80mD0@0^{&ZmLca-eGk5Pt^fc@{`K2lTB1lFtLF7l8Cy zU|=0!s|V~2fWrgSz63ZM0oPW*?FH($0iGtHu^I6CfTq_0pC9nI0D%Ayd>aS_fwop4 z90DTm0?{_0v;BW6HL>vj@P499fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj6g>f| zH&kz^-cY@vdczx>vkBRRY(h36n~+V&CS((`3E6~fLN+0ru*8}$i+ZCi%sO!DE!10R zozOaA+Z1g0RUXxwa_A(oIQ)PtS|60f)|s-Xn=Olrb7ax|uq@iEWYPSXOg-nSo?{dp zE3cV1g9^^o2N+9py>@fs zdyQ0~?zPh0>Rxwkua~B&KbJ|9)V)EPqVA2-UFzN>-Kp-!;AbNS%V?xM8ufuzYF6H< zg#e<%UWdI_eM0x`6S`|p=-|;PZ`Di3#;WhoOD6RldP!EFD3g4{&Ahof>n7jkP4t8R zv2?HfU#b12_N{q0Z(46M^QQmOI?4PqRhW?!b7z8C|IosPk9?=>w64|PDA_bj{{Ar~ zcNjp~#T$GtNG$t+*hfIuejxra&~p$-90K}|0Li03>Nt@81Q<90*ggg9J%FPZs7(OQ z(}1fFaQ6fCNx<_J(3k?e7lEcU;JXC)2Y^8J|3b2{0ik6;n;i((01*ceT?us70D<+_1^aT%bzA8a%)=uu(12y@OTan&vTxc&5SRc(>K0ZjQhnIe03aNIdhoV9cLbH zVBhSUTixDnYQRte)>1G+z*3`BtXOK0TBC+iw3aBfMyMFD)Tp%vs5NT90QFqo1-lzc zXj55jy78JFKP#-HP1e=5uKnisS>Gb+5Y!>4Lrj_u!4Kk)t;w+vO%!j+#3j*H%O;?1 z8_@66Ro5-pGXpzj>8f(BPB+K!lIFlmf-9nnE|OcnxwwCk*%HozysR4&M$h2-9hVF< z@7J3%zgoGmmdS@ss&#{pyMFQJqHr@g;?3@{#G6N8>R3Exi#N$F|JyV84aA!#UvD*C zylFXA73+@NIX>~m3TLU}Pva18a9ogBwgRc=fPw8m`gtJp z0+8JS47~(+8vx%fz~2Z|H32nYAn-a6YzAsufKUXee;a6s0*$RexD9B27l^b2(GH+3 z2DE<#e_E+8HUx(@@1ZlLEVkW2u5$AMH2FwhI6lR)Mxh&S0jiZ_a@I+y?xU;<2l z2`~XBzyz286JP>NfC(@GCcp%kK*1BBcti1q;tjS<`8czld}q(atq}aIwy2a*fx_k`N~A`rj%1|#x>#RYB8mFQbe3Q z=H2{ym*wh%!k&e4T=r2Oi{w?4KXlArNbP3Kc{fdCX*Uxy?e&rk+yT_sR?zD`Se@?GO;ESPaL;3pA_-8ht>x5@_}Vk@Y~d3TUea+G~K=CZIC_#5V)oK_IaW z=&1#gbwFPTNWBOQ)C1|AK&Ao6?gEAy0q<*oZ_nZ~i+}Ipg0Cpx7hnQRfC(@GCcp%k z025#WOn?b60Vco%m;e)C0!)D34ObR%Wf4~vO|g|lystFxD_!vWO7kS-Nyw9sV;(r> zflbIJWD~Lp*@SFDHetb=Fo$hGiHiq549VY!(-F#zyKKs=BUgywnf@ZW9!gn z8(XJ#yRn^|X*avZ(rzYZ+Ksn)v8Q_VziA!suNyR7yQw^#Tf6Bxc53GMv>OM;{Is4P zk9LCtgTm4R_#;5o+dxee2($vhHlX%hAk+@jcK{7Bpz%W>+zB*y0g*TmJq)yU1MNqF zSOVxg4#aza?p`2~1bV&#l6^p5KaffR1E+xW0FW63vT0!GN5Fdq@cj(<&jD2zfSO-` zz!U#gW(ltPH?C-40!)AjFaajO1egF5U;<2l2`~XBzyz286JP>Npx_BmyP6S4`}gj1{ub7(hea@K%TZK2vic4@pIb}HZl}Zgi7Rd9_#1pej96c}fW~3K$wJBrPo)u!;xOC+j{qQH^ z)qm1)H^!sPzqxO7Ta_xg*;P6Bu4!(|eI=F!TJ`-S_RcS{ zER?--jQ7rx_ulEg>F(K&E}uQW{GsLL-c8-E5#w%L z;@z}Z$1&~(2L^}52LvmDT0an457buy4b?zn4G`W0GzWmlW*`~_+O`4hwLq*6=nMhz z7lH12Ah8qZX#kSDfWAf`)dUQLf%NM@rWwe#07DVL`wrmS2l)2`RR@5Y_kh6rK=2?? z`vDL-1k`^7@225ndN-=zYby8!m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28y&JA9 z;>sefESh2~i+ERQ-c`Ebca`Qz$diyKA%{G0$OD^@O~@u>6S4`}glxisH(?I%#z7l` zHUw>mNwXpNL7cBOITE6Y-i`A&?9q(ZAu5KKG%Vd*{iJllG@OOXmdD5&STK|ND5H0C zY0k~mns-wm?3n#{D*+5kARwwfxss~@CZ=*DG>S$s6PfY zd=5080K#7Y&0hkMuYu?{K-;%K`$^b0vD36~IQ51JFaajO1egF5U;<2l2`~XBzyz28 z6JP>NfC&^n0opfQU&QrATwgTB))(>S(!9BJ!EY|jlaMDNPeRUm;H(EWA)Am*$R=bH zvI*IQ1#iL}_KlTJ1f2*v5tHUb@PjyJYjRXX6YZN~7$v$odXKJJ@72}L`*c-(zpltEMG74V{aIj#G85w+#C?mp{Q=n#;|btDj$|i&LsBdB`_8gqwvT$4sA~ z$0GU7G;drE&(xZG6Sf*gjl(c(te&gQn~TPc+2&2#z!Fc_+e@{M4)hzR=53+%a;AA#QDI|#Y>kxyi}{M9%0@nm^T#xi~CmNhyF#ZA(tUA z)vX$_(}C!kQWj_|1ER4E0A~d4e?LhtWK*I|_;|?JF643lI5P1cNz6!MM z2HIZ(Vtatjy+Hg8p!?0GWtPNSOACHWPksR=zyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%sNPV$p?X90hUyJ(a?U1X6S4`}gls}KA)Am*$R=bHvI*IQY{DtlggMlkqRCkY zPQ8VC3#}7cCv2NZ8-Qh^dQ&DgiLP2U0d?Deey6UwZo!@z*fC32m2-94Ifj`u2WHYr zg@|(7H^U~*BC|D|1$lWlWBWC8#FSAcK8nYh?Q>DgnIn3R-NZ++{5HoVKgyH?LW{$r zG^eq(Xr;#1s@-91^S5tqH{L%@yWN;kW6=&+j3-vDX)a!|daRlaTN}1k;|1H_U$9+z z!G*iVnF&sAkztCyMUoM;5`EPJ_Y=r0aeF00g>;4=np{KPeA(_AoerRc@BtQ0J?tx5>G6{_ge)dp9K2WEGx65)-L0^1}4A+ zm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyu1O0OcDlFXHkdE-#v5%Zqq>Y2IGC;J26N zNyw9sCn2XjaM}Z#kWI)YWD~Lp*@SGuf;V9f`Nl#ef=UFHh)Gi-_(2@BH901tiSkV; zXWfiz&JXOH5s%_Yk#X{vdNV$)W<`GD%qVkhbhDkBeDs{Me@^e_Hp91Z!M72AXJWmZ zrm?)6iJ5wn^e^)a9))-FRLw7@dpCor+}=&=p*=H3^=?|X*xgrnH&!?{t$)5!u^HXF z!GXbHSr4SDfPrctT?1q`0oedBv>EVj1$@r|{_Q~3^FYlDKwt+Dd;@WN1HyZN=Dk4V4Iug^(DoM4{tgh^2XyWS;s=24_khIvK+i!S`2o;(2uOVd415fv zKLIjF;N4_DrFWwWzNUg-fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj(7T~`L+^&( z4ZR!Q;G9j!CS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933GTi_Q_cUPPc__3zZWpCv2NZ z8-8V?cf%<+`AzsKRlN_65N<~NbJ4{3H^s`DhjW9eGQa-uD+HPqrVg>mN4UXz=-iAs z>te$H{Ct^(dDz}mp`<;yu8Huz|zG_mp!~xt2~|Cx#>D~YUY^E&8^1o{}-8c<8s+O z4y|tQh`BbSOE)+&2rLO8dmI?*0lZ%TzApj)*Fe=bK+U&6;3N<{4b*-IguVyre*hYO z0vgW%;h%x#b3o(*5d8&cd*aU!Y*qoWCxOm2KzuFGy$(n`1@t@(BsTzk8-dg_z`(OW zdJB-*3S^%HhPDIV5a4_9&t(>W{hzs>feA1HCcp%k025#WOn?b60Vco%m;e)C0!)Aj zFoA+6K` zW&CHyert)a$e?q0uYLGEc(1#t#0HbaJ=2g{ZWXEJW|#M+KYOZwf_wAPw~eN|H`khX zQ<02nqq{d@`<1h8960cb?6}+;92q>8mjVAPK-H^2&2Av@8W7wA)b0gBZvgdg0u65g zjqd>AeL(YmAaVeRz6Z3u540ZyVjlpVhk*D;K=;Q$;uE0f2$1{~==%&v9RmhF2ht~i z%ojlROJL}0z}pY_Qh@&yP&EM53<7~P+?(K!bZocz`zSYdIyks3CO+-47~z) z8v$Pv;12^;uLCvBK%fN(Mu6J4flw5vZ~gZ&OGDeg7yNdf`~pmX2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bcyy4O!E-m8HqA9kti1(J}y`>9&Z)u){JPCOca?}GyJ+KMc zgls}KA)Am*$R;d!6Xp`NtsaL~gslzR-(hR$vOvvwATR_3 zE0#mG@dBaMK)nxWs014QKzKdST(!K+5~*I!l?_aQ2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?a#JOS!A)NiQYP`{ym!+V^w3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEiZx*l z{l+>uYr(0wP;sGoLi2=eGimd$Ow@0Rp_J(A=smh>y;oN|@6%QF{kl5)fUbHU)Ky2h zu9}wURCF#XI!@KK-ZCuUT>b=;X-3#LBmQ}vGES+oXe=9n$TL9nS)gqT(7qLjJqL7d2jb5I-7f%%9YD`Z zK=Ngv?-d~RDlo7cNWTVT_5j(vz|b3jHv;(H2K-T=suied0|M^?!FHgw0|>={`VWDI zPN1<12*-iu!$71Ph#m#n56S4`} zglxisH(?I-rielWg$N1}lO{y)gE(qyay)tfT0Np#h+38>o!^gDIcbqn^)z>ZnE zs+_CS&N0lSIWUt}Dx0osmrFNSKVM|FhO;0)??x99X8NQ(R>)^@ zkrQtm+TiDwD|b3DB&aE6fx>b^6qYZ5*q1=(*FgLmp!-`OaT4e`4J5w<`o0HJKL7(i z0qHY9=4T*#4j8%scq{$_#l{QxR|8c(pr#TC_<`VhptcGKRRi@kK*J`WF#v=&1IN zfC(@GCcp#=o&dEQYB$twsNGPz;oZ&Igls}KA)Am*$R=bHvI*IQY(h36n~+U7#hNgO zc4L{GHQ-cRsJ75Jp>e{tnY77QCTcgOoN_a+2|w^|Mm&lqMZ?Ks-pxqkrdYff4C0x< z#~k_PBh{N?+2fed$D9fN{W-sUnR2)tuxym(G`1G4)Yw|{o_cee@qU+fyRn^ytNfC(@GCcp%k025#WOn?b60Vco%m;e(fd;+v@xW0(%i@3gMimfl=y`_0?>4M)| znkOMoLY{;i^}taNY(h36n~+V&CS((`2@BqYIqVxdod`M+bRs6riQoru%GTtlh$h-M zoN|-jgr8E?```%eX2d@iO^k(8tSot$H=`LgV-De2sjTCDER-LaSi|P(-w*xe>YWq4 z=by{D*)^7Pv*MC-b8%~ybCW#x7ti3q|Ij*~8GOfd=ce*>Zs(@!*r}Q0b8f62yH@dd zPQ51QPWUV5#9z4pq<#Sgp7;;=HLHNklR$P2Ftir%t_OTofWI23ssU;?0f7Jz+ziwP zfzURfz7}Yx0~$j>_(h<(9*FD&q76XXE}*>;h&2J7VIclG(A^9qT7aGikbE2Hivp=u zV4w|1zYApA|D(*3?f8#^-^-I8f(Bt_J_0tE2bms`Xy|H!wM;49C7wS%x1A9Pk|m{M|s+QJ^LP1dap29-y`t2ql5~ zuYiU=ps^nar-0^DKx6=j4gzgyp#4W6mH|4?0`V-+eI7^*0X-Ef;MsVAzSTg=2MkmK zX+Mxz4`i!=p=!Xp5%4{;qRirdb_JI;FaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zCQ$GM=-zO35my&+bL;ZWrs80_Odd0D zuE(YMHG^ad<-lBR<4p9P>z{dZJI2g7JTC3xb{e)$m&Iez4p@vQR;_6+-eL7vH5;}z zY^}x%w!gn%yYzxhyMFQJqHyEV%oT6En^$6S4`}gls}K;S_7a9O8{_a#n#;ZlT;l z=Y-A)+h)>+UzsT0%;TV&yodbE!;qhqGA>3Q_+ww~-dw(Ck*OXIJ}wXU=GXW$S6eF! z6n>8@pT}Z(n>inv|L-Z3@ME|$N^=@pi&koEt=b*N)^=%Y*TkT4c#5={#@3`Ntr+K0gslzR-(hRY&+p1K_&92J1 zcTIC!?klk@(5mkrv3GunWufeyW4w2ky!TG`O?S_JbouQ0z`Gsr zJrDR_0IGHXH7@~ymx16bK<%qQXg5&*+JBZ=8ut7rmo+c}Ccp%k025#WOn?b60Vco% zm;e)C0!)AjFaah|@C4}IaCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3>Vcyk*o16C zHX)mkO~@u>6BfJ)bGSEH9?ZEnMO1^R2Ju3m8gwJ6K_e59+F;Tvts?box0L{T!$2T5lP)Z!Uj=u{4)^H&;KuP7|k8sXWx1ynLB_ zoSMtuQ!0Lq`&y4ud5QK7Y@4ZN--NA(W#iBuxrTis`z;s%WA5ae(Ch!{Y4Sd-b!^%G zoay#W>oq3d44&VrxqoliJDS)#x4P|rT(;~_|Il_S_KmP_+@9%LMYU}_lW%ZjFj<;` zh8CbP0)*cNnxjCZ6^OP0ZSMl@?Le#p=!^mJ4}tDZAkhW%#DU~tpsyQ99R&swK>9e4 z=>f96z)%wKehv7(0sP+rRVRU((?H-mAox8{`vVaA38+5NfC(@GCcp%k025#WOn?b60Vco%m;e)?eM9?(_6_YD+Bdw*Ih&A8 z$R=bHvI*IQY(h36n~+V&CS((`38z>S=CE%lo+yeZ6i+Cg+(_aH|C;z~nrPpYiBO`e zmQ6t2HlW|BtFBwHX9jl6(pBYLoq~=bC(VJJv{ETusmr~atDi42+r(LrmwIzeLFQWG z%y{=N7SU$<@MD2Ir*|{uyqn!)c{dk*8?(Keu=in4YZttmtbecR-c8G?BKHhKuPKIJ zQ!?Gv@AROpw2E~{?i`@1 z1=PO-H0%Q!_XFVrK=XS*Npx_D6yWz?rt}NopqA9krh&Pw!&7})|b7`K0JPCOca@GT9J+KMc zgls}KA)Am*$R;d!6Xx)4EKbG2H+C-6r*^>$!Krv5*lw5$!9?$-l#_18HRT85&4@?w zqo0K2HrFkM!ae3df3xd`-s+YEc2o1;>}))cgzt&H=#- zKP`jaaL+ysz4R3AECS((` z3E6~fLN+0rkWI)YWD~Lp*@SGuDb|EJv>Pt8zi}4YQ=4!q{xvDmId~!9LNHOg;e4C? zX8V+?-UmncHY5JIXkzr6Vr9!ixf$sKjreY+|FcXE)-}nGGWo;;4A?Q$8l^dntwk#} zw${9rn_G?dyR_Sl?KEtiE{n&a9k3WrtXk7tykhlOH5;}zY^}x%w!gn%yYzxhD>7cN zYsYUge!{M~jGwS;y75AhwyQA8O}zFIPwy!xH*Y7uGF`c;Je^y)={k05=9tP&iJ{y~ z7v;tY<;JS@&c9Nx8Qr&i=S3h{5A^K>QVqbsE+E|qWSW3%7#MmT@V)`~ z-UR$_0afn+HT!_Tejs=NsC^Fzy${qM1R6d78V>>CkAUWnfygI7^a#-QDbW5I5IY8R zeh$P>0Nq~zi7$bkuYu$@K;O4;Zc-=d+;G$l6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e(fd;)ZCxT=V&inyw1imfW*9i@3k>4M);nkOMoLY{;i@W25NY(h36n~+V&CS((` z2@BqYIh-5iR9vLbMfydwRd^vd6)%Jv=Rz>ixtRl(L{}Z?@UR__08qk7m1= zrQzt~^llc6rT?r{*6V*PmR~~eCa+O9x515ZV4})ocsB(ab(1{xh-WbJsMhgZ=Xuk; zn`@1_sYph(QN5d{fc;9x#_F+at!ph;?i^jb!I42?830m)z(5*E{|ICE8qPjSMj_}8RLXXS-}3&BM3#`zl-X~ruN6+=uKwr;L| zQaWKC&O&9$W8lp-U7AucWb701D3xDJ_a-m*=Av^mMckW5AOcw+0^MR#{mo9sF zsn&lo_spBRU8_f*c~f0I!o9J399muHxZE2Y89bJ^fov2QY6ZLp0N;Cn|9zn9AW-uG z5I6(`KLTn$211_z^+$k)Pl3kIfbcP(`Eww00*HPAw0#M*e+|UG0Xn}0;wORb(?H@o zpyzua`2*1R6OcLs4EzkF&jFbWK=v14=!w7L`>h6izQ2}P{FQ$#_^JYa0Vco%m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzy#>taCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3 z>Vcyk*o16CHX)mkO~@u>6BfJ)bGSD)r{ZFLF4nixTH%G@RJ;&woD0E3_hudk-Q+#w zXC8+9tW=hZ4-C?}TEDq`&mvP*9DH0J`b}Pf&0-N|9Hu-L$nT+hGj-jY$Kcjjp_&w7 zYs2<;hHK-zZ`+?x|Sc1@al)0#Orx_eW%afEv#(}yZHTE{c@21f>u zWgXyu3aEM-sM!DnHUhzCfZAt)&=#P6E70&9(6}84KMyp&07P~G(U*X>mx1ZeL#9YkU0Qk-vfr;2fQ)B_aWf#1gg4#nm7tL^DdI2HezwCOCo5O5)w z=-w2=C(+f>dvw)$uda69r>pAwb#?XuUG+YwtB!JAH7(KU=Unu2oT_X6CG}?5!MXSZ zlW8vZZmxcQohA-O*W{tz{2HOAtPuLl^$0sZ#lDnAXv{;a$_! zo7QWLy%{{eRdfH|7@MQX*qmG4_CGFL_NRZ$%W2&*LcMW$+*8wWVJ-v{)tfT0Np#h+38>o!^gDIcbqn^)z>ZnEs+_CS&N0lSIWUt}Dyy!nBfOgt z|6F9YhO;0q@5Y4bGfuHu-1$G|$(v31+;RO9sy9V$C&U`NcKjy8tg&k@Y;D+~8!r@T zyT?**M9HyW_{$1oJ55_rVs$CwKRfnYON2!Rox^+W!|%a+-9;rfh%WA#hVybOo|fF~ z3hj8z)AR$>o98baGhMxDIaTDIVSHM#eA;wZzte-l(kj*+xpRE#joUL_Yu#oW&(s?n z88nucfQFZW##eywt3dN^Ao3az-2=4k1=`;LVs8SSZvpXlfbM-jVn5Jx07$+E^t}(H z4gv!o0O>ZkOCS{0pS6l zc@XMNBu({(Q*W366JP>NfC(@GCcp%k025#WOn?b60Vco%m_XqZpn5~~hUyK~8>%6S4`}gls}KA)Am*$R=bHvI(bH6XsBFxY(Z73H2*Q>%^&eA>2?Gf{E%) zDJR{GYswGYn-P!VNl|g~n0Ye}XC@!JW}$dB(?@wMl2<9-U>45Qnt9VSmUwefxG`J2 z3IFhzr*;2-X&v5ge=uFV8BFCCZ(0xSnK7z(Q?YNv%o_#qrg!aC;*DFYcyGiY90$e* zHKi<&Sbh+RNfC(@GCcp#=o&d!g zE-m8HA}%eOVoQs7YiZtEy5P5#=1IttkS8H0J#f+kn~+V&CS((`3E6~f!h$zp4)MnB zR5#Re^BZPl1yvBLAiOr6icXaE*IfuEiZ>i`liz@!Qq}w52;*kNKNn4mfK#kg9@5Qd zg3KuHOr9^z5yeK0c;Jgow0AQn*9@Lw5rQ0({3ufnjE8Y}l;$+H7Om9STD3cj?bK3l zc8#Up{C(!l#jRO0Z`$_%m#6FL$F+_x53l?0?%Rw3H8*2G&0W*nmitO93$*I{N9>(n zVp%AA=NRvuCGWk{ebe2uA6-6se)&Vo%galg4=i20blJm8waU}E)tjzkr)G|+-jo=> zf4U6Iak`Yps#T9LZ`>ZY*5w6S4`}gls}KA)Am* zIK`SUhk3(=?Z0wiJDmWhVssSOY`hSr=7nIQc{2w_iLN@zb=9;)SDRPps^W3jIIAtX z>RJ!9Yy#@G0sT(wx&?b?V8<+7RnFDb;2(5#^d4Qc-mCuxM(32_*e};l7KnQ?astlf zd(4t?^t|kwk!sB4U)9{_D~toz<=9Q0voq!|Ec@s5Zf-OD9G7;xv7Lsk(`E5kv;!97 ziB)Twi&w00LTuRDu(cX5*#7>4?a~W2t;l%6t{uP0_zAn_GJe9Y>Bb91T7gF1#GiiL z)B8ERn|E7ZFx|Vk*2tTRWK|)!GXbH*#IOq z0zJj;c0NP_f>_ecl6Nq;K-Ekms80hH+l1G8Q1iYKnae6n3 zi#nJ96JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wf3pm)QSMO<0Ll|@r*Wf5;I%^OP> z{KnEe33(FoB;=e2&Us)HvI*IQY(h36n~+Ud@FvXR-8eWR#;NEISf-{s@atxM&;;RM zleK1|cjNpGdo<&9h>GDQjUup=PMC(XP}%aBdNY~sQMA8$pC?~Krjo`o(DohKz+sEAlZ0<#??UB2Q*g# z5kC-J542SQ?bSf62I$-b!~;P0W*`v+dbR<{TA;5ENQHob7lCv=klFdSGE27MZv|gc zz%RfAm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28#T$w@6mKZrP`u&I&Dn%(LN+0r zkWI)YWD~Lp*@SFDHX)mkO*qAxFo$?U=Y-A)<*OpfS5CzX;fA^pOcZbCamY>H1AgXV zz|Tr$yZAt@j3VLSrA4NCIQY2y-5bbxE0uk69`odbIQGWn@YpVCICf2j*El>y+Dv2X z&}JK3=cREslRWn3F?_hy;euu5ajP5#z(5K}p8_%iKz0xq zN(0`X0N)wtH~ycg-*D~?6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e(fd;-*OxWI@D zjJUvPiY+kW?WK8p>4M*0nkOMoLY{=2_P}WmY(h36n~+V&CS((`2@BqYIrJMVM~gTW z?F&Wwf)@fWgd1sMl8O3FF_aQr9lb|at@r9`=Y6`WzF${oAJA3rgSzS{*HzOJor=yy zMaQYS)?0=JoXekJGR=g_IHk&xhkcX7xLGK6jAMcxi{v*`y_s_AP1tHEH4a0u$)(<0 zRBp^xZ@k&Rd8#{BY8~IKI%&Fk(|V1`H-qQ5YVO|~29O2==&oDc_CGFL_NNl(14|b# zUH0%&t-5-IdgJi8wASs`@l3wKkwIfQ5BP_Gs*07+Y`j2VH4yXxwUt1~57e&*8mfTC zY9L$#G;abT0U){=XbS@E+kjXt&{+q>LqPY7K%yS#*$E^YfWBQosu37y0@7h1^E!}i z28LPy?^}TH9l*a2sM-(I8~_6Ett_(y-(Sg_iZTHvzyz286JP>NfC(@GCcp%k025#W zOn?b60VYuJ1gPFny`g$T^@i#V?{dy2WD~Lp*@SFDHX)mkO~@u>6S4`}glxho)`U6K z8(Jr{PN-kmX`S3i)(QWb_-mS|-js<=qN|ooK;1T=->IvvTd-#acFfXMYA)?&&&9I5H$ZQQ~L0;a?*nZ6%F=bpPK8oeHQN5XR>do%4)Em>L-rQ+8FSp6m zn_FBp9Vey0bGrB$pua_9Km8<)qe)qOQC_XdXs zk0k~KKLlz!flwDv9|sx^1C8B4_$bhv03yeMXb;fV3$!PJ*jGSj9}w>cx>G>n6wosO zBnN@MG?4lc7{~zWvp^;bWX}UbLxA^*3JXM=RYL!h!m2gGnzh2fI$`iBVeQkx&<0`s z#tKWBrQw+hOTiZy@Cz^jCcp%k025#WOn?b60Vco%m;e)C0!)AjFaah&4TlSjxX_3T zji%T_Bi?A5H<~W^jiz}L@+9O*$hi-k`@kk-6S4`}gls}KA)Bz^O_)Q&u{aei3|bgA zS{S?#oQfC1jdLNGsNs}y>dm+&{=mZ-@hFx(n16H4hbdZ49`kR;C)ljGZiPmlbOQ&@ zPnmq^*v65$?rcv2|#(jjdC=-PpP= zZKq-DbXhzWt=8HLRH1#ftH`LJ)8~O<#$H^YXEv=@!mlDz}BxdT7s#Q7xR-Eh8r2 zxINRgikEHUnSdk529ss8upuaH+$Idy3Y+VMk&rO@qOh%A*uGO3YY=ws62=>a-A%$o zSlIKrFxf2ZYZ0a*!hyGi>8LQ%D$KSChu#%>-xvB03jH4ls}2ckJ`x5#76v~N)*cat zJ{8t~CTut+Z2VjpJ|S%WLJXbAmo#)NipxKk025#WOn?b60Vco%m;e)C0!)AjFaajO z1eid<6QH3(Lx+YA4ILUfyeT@HkWI)YWD~Lp*@SFDHX)mkO~@u>6S4`XSQF+jbhsw} z##xh3!@{Zf*Q9gj;Dvw-!9+uclX3E!3RJ3k9~`0PjQHmwH6f->v9ji2=Umsc8bh_2 zzEX|ju|nQ_T_5@@ete49IsXbH#sP=sdhMLl1lT#d3S;MlA?A2m!%x7@@xLE7-Oi~z zo!idoI(BO2`0N}7JEu2#)r_3!TG#dw%8tY1(3(aQwj4*Ll*pn3|&kOs8gsF-rzyYr?y;_*@3A2^LA-~Z3l+gFI(7!=gwNY5} zj4<%5Ft|lnyHyx^PFTNP*zmlt@daUchp_o2VdQ0D^c7*-tHSo(Pr%NJz4ip>=g`h! z0!)AjFaajO1egF5U;<2l2`~XBzyz286JP>`Pk?p~*B^2H5!WA0vGqs1O*L;*UGUpf z^CaX+$diy$BRDmJO~@u>6S4`}gls}KVZoa)hn=IGip%-AoWF=(3oitx;)QVITnHxG zIdfp0=&GY!S4~TFwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq z@6lE3z4~up4o?}5eWkJvKNiUIk>hkO-(!}rqvvJlj6<#&)5Iwj`$moVm?Q5oQ_?Ax z-!}Gv%#rsLeu~ai_H@RaqVotOBa6qSUEEI7R+Ly>%J|QY{nip;kwNG1Ui8kL;(>WQAneORaYl==qGOCU4>4fd>5@X!YbQ$;KbSd~%UAJ60 zL}$9VJBP2O>Byl$XL(&1YZi942;&i9_uIlmRM^uhOtuO8-W8_Wg##VJbWE7}P?+r$ z4s{8=p9pcwVQPXk$hP9jVDo4f8n1&sltDlrkn6sJkj|Kb&WcNd&c&@+ zf=(Q6j;FU%1f36Slcozg{U>t|(y80Edh|g$75hdAI#!QeYx;l3Bk0JXL1gg?6RU+i zK4G#_*yk6f)(Z!!gz0Kwrbd|EBpeC|y<3F7twR5E!m91Un&*Xq7lgqb!rGUFp_hgA zuLv7n6*le`hF=pl?-54!3Zrib+ujtmza@;lBkbHKjPDnA9}p(q6ZX6>Odb^Wec*+l zlRD((9G$}7BelY!3j05P0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3_zCqO}mOOUt( ziA#{C*b*e(u9~;2F8J-Lc@pv@)bg{4$Xe?;gl!TbX*RPO}oR;Z|vG!V_T%nG`0?Hwy||;1<`aK!-rcv4y_1V8@9j0 z*5YwqnxEs+%++*~FmybF+g3r-dDWktG)<>9b8vJ`r)#hMN+rh*O$YPAuG~5LARRe2 z_$*z*R9rZ4SeWh>W{wK83E|Ljq4$K)_l3~^rLgL2Va+$fz_-HSNn!12Vdy(y{rAF# zAB2rR3BzZE%|8nx=Y-J^MR#{mo9sFsn&XpQ96U?w`%U+8$Ykf&)@2{|8d!} zKmFqe2SZmmIu5Pwn{hcha%^x}YK7T4;ZR8E-68b7B=o;5ta?RQ^QthgTNr#zSi46U z+AFMoL)h@9u<3BYIx zUG+P4)pZN@%)pLWx~iP3Q{yodra4fURw|pWY?s?PS3h55wy(1wKS4(qW5zLQj}`J+ z9HNug5S_^sbaod;(DA~}@l?OL8iLMuU%YI(pwn`y$UVdGaEjsKluUQ^J3Z(ttzzAg zJI5#JI6N+`VwZJ1Lv-ZWAhMhm`o9xaeJ`x}K^XW+7(64a{aF|~C#=69Z1_dk_{3^3 zyh_;oq%g8Z7+ouDTPJLPN*H@u*ttO%-ze;UMwob3*t12L+$!vQPMF#*9C%)senFVo zAzp(y*u;D#nVn^_nkOMoLY{=29l_ZVY(h36n~+V&CS((` z2@BqYIV2ssQ*l*4SM^(|x9~!6DqaXT&V^v2q{Fc}`Hc)JRlN_6uyaQIbJ4`eI>pM8 zhoCc>i?c{f8n?NRh4Kq&=S(>}2h(93@M|2JSU6T}i?Fp}`#WR5^OBWgww)7#nB!?W zB6iMeZ+>RFol|)_x1G~NfC(@GCcp%k025#WOn?b60Vco%3ZDS&9NIawb7<$#&f)FS*@SFDHX)mkO~@u> z6S4`}gls}KA)Am*IK`SUhn>Uq_&3gaeA*RG#lI$fI~y+qTnHxGIdfp0=&GY!S4~TF zwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq@6lE3z4~up6i*qB zeWkJtKNiUIkrQ<;-(!}rqvvJlj6=YgXOdjwoODA6PEeVA=*W3N6TPQQIpDWAJW6vK zTZ{Iu#@4FcVQg)ewsvi<@$VvSrm=NsvyH7&yWQBjE^Vh_>x4&R(NY%UiB)TopR{_c znhjeUwpQZ>+uvWXU3$R=Bb91T7gFBgyH9ST3_%%(y7_; zz3Gz9wMOVvB%|7>l1@`##0VV)NvHP*%axK&&YcdAOB;OJ3Yi7R2ASnkVdOJm^q8>i zb7A`lVeAWG=a<6x*TU{^go$s3Jtu|9)55;*gsJa^13w7UKM6BugxQ~kL+6CvA)&9r z2l~Cjs@1|8pD<7<4Elw&>xH2zVSTl*p+?xaNf-_Yn>Pz1L1A>8u&q|uUgv|P6ASq` zLZ|QxFbaNuR(=5{zyz286JP>NfC(@GCcp%k025#WOn?b6fr2MMNr%gjxD1KQkfzu& zB;Ki-cd9PbCfg?bAw z1gGMKaN}GECQ3Ta->`HuUg4-164S7ubM=$b33GN9Dr+8NbgpNjjasS3{V0*moomdW52aTSY*&Syw^Vb9=z9G zRAO^g&UKf%txA>Lj;AFaR~zIUPuKTi==@N<_J6u>GhVzIFWxoHZMm<+vOuf8f5hJT zC6&PdyG!quGOQD(P`ac zcb6Eyf4Y28r%S~mnB($tmCjtdF3wm{F31B1r0Ll}EW*!i+B{)({s zRbgVcu;(>la*wcYuQ2t7aNtd0`YmDR9btB#aA?2K`>xQ}F7$T@t75{M4~2nFVX#YB z8yAKS3+uau4M&BI31RrSu(?MV=@mwk!nUu3?R~;nzpyhUjGq#A4~U+V7^I%V89GdW z2`~XBzyz286JP>NfC(@GCcp%k025#WOkfHmKs|?g4)q-BIn;A_zjQVsn~+V&CS((` z3E6~fLN+0rkWI)YWD`!YCd{Gda6$f!vml>(g;VjbN#U-D7XmH>6ZM>VoST#P%%FLg z8MIPaFFqhWUTx@HzGsoCwhlfn4?}0%icL-{XMuP(mkx9ECyV86=6q=WzsJyma!52e z%^I_IUcaGpn_>94v|EksB+bxy3?F9oI52fb#_~9{-(hR2xoX!eH++1$wT>bnyeVtOJ z@(^_LvTAaE|50|9%Awh>^(d8>=;&a&j!nD6aBu9|Tw`0L%`~YLn^cgsu2c z3!I!su+8P@$bR#u=p>=%cm|)Ugrl>!zr}P%r}Y|BbOz6F)!e@~CgNx^h38hc{g2C* z{pla$cPjSTuN;qKM>JhKEcr`1a%7NM_6t)7gahvh)9(v22Zh-WghPjf-Y%gpF7zK3 zR&@(&jtT<_Veq)HwnrH171k$(4POZx`-I_sVRK3tIVFq^2-^mQ?P+1`M`3417(XlQ z&I%Lfg*`*UWJM*|=M|<_3kQ6{bfqxkuY{zNU0+E_r|=6fDlDq7|Kk^60!)AjFaajO z1egF5U;<2l2`~XBzyz286DW8BlyoTRP|~5KLrI6XOlK3a3E6~fLN+0rkWI)YWD~Lp z*@SFDHsKU&!W@zgm*w9$%kn8%I2Hez)b8xO5O5)wDCv}mccQD7O+ei?px>#hu3NBY z26oKSRpnfr9*^NL&4I%NAw?f+$jls|5Z4=hi3Ilb*U`SZ|qA*l1 ztlueYXb?8;5{4Uv%}v5cSQvd>*w!p;ZxO~K!p^sa@u;x7RhVcK_Pi@hwhQ|@gsGTt z;6q`$Q<&)zX5+%4!$R+;Lf>bi=lG9N&#_R?VFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wg-?Kb4i_MC0TLG=O|bto)N@KXH)mWkgRVAGiYG z&`ju)GWt1}#^#JYK4&uhoTkF~IbNtap6Z6R@N>@W>o?uc8BFE&b6OAWnK7!LGZ-2% zKF8&8X{{~R{Kx0Wk-=l>5&Cy~3gfGV-9BNWQrP1cCf5u5s)VU(;XsWry-An}2(z1oLqVbU zIiYX6(Eq%!>IGrV4q@P>weWL-FR$hJoWifYDEPfu`30B&6JP>NfC(@GCcp%k025#W zOn?b60Vco%3Z4M{9Qrx*bLi*L&*81o*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IK`SU zho8e0`8UpreEJnm#lI$vI|nZWTnHxmIh>c1->jfg)%)PpZqDU<7EO$&Q><)xxH+zs zYEmC%@^PG?(|#Qv)r>Q+HO}oTIN){9o?dj6pXyW6;fA)7+N(N-PVs>ib9R zonK;ED0}A^@0}&@z0-Zu-LoHEK6`%oL(9v{OPmiZUA%PJ!%MZw)4AQ8u4AWWj@Qj` zYjqdK;pWJZ!C~1Y3^odDn}ngTu>N&nL$k24MHr3NfC(@GCcp%k025#WOn?b60Vco%ra%I8 zbGZ75tB<()Xo{^q;w`Fqi|T^kqM9cmPePuAoEX805o|&>A)Am*$R=bHvIz^`ggM+C z<+^HGqN~j-bXD;<%Pn=W@2_FX&$?brLqco?nwP^opY^|J~qg2W5&}vFNt`O85 zPgA=ee$JcEZZ_S|xz_BQieyw9)z4`PjF_F{_Dt6*+HCpH&XFU7$C4E`oEJ6@3BwhB zu-Pk&tQJOn!nR6byI&YvFYK%m#;b+hHNwOuVNXDq+$`)13RByJ1GU0*oiGy;W?vKz z)eF5Z3w^H${jUnEb_;7>69)DOgL{RwZwNzg3hUnzHoPNj+$Rk07d9X8!_SGl=jZGk z&dy;1On?b60Vco%m;e)C0!)AjFaajO1egF5U;>3tfPN1B9Qrx*bLi*rKIv>iHX)mk zO~@u>6S4`}gls}KA)Am*$R?a(O_;;a;fnklXGK2!3a8>6a5_LZ`ijP zuWM8ce`#3Gx%x@zgy}j9l_ih)IoA|&N=2lZJ`s;n`L$GZrrZdfrm;roJOaPS;&Ev! zjO{dSMTym=jQ{M|Z!HlP8FUWswGY1s?{ycI*x=>3OO44lmlT~a>>N+)rgczsf~%TL zS9JPM<{qI_w`=w2BXp{(M<_Z9icarNi@U`5_R~c$a=Mhqs#VlC(k31e|#=V4*ITiAV6m`DhFjti4L!oFT%Dk&WJN|^2w zX8MKMlyK;j(EFXx_r1{ngRtr+Va*v~;Adg*oUrzSF!YPC{)u&9!zy9plfv*CVe?vH zWSubjl(6k-Vf%)4aCBlD*U`}_{Q8Ruiz@8@_yw2%6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%3Z4KR9j-#+DkQE#nqsSvc&lpOs=DB}s^&?^laMDNCr5B{1e=ge$R=bHvI*IQ zY{G&!VGc*f=2TqN&qe)q+AX{goQfC1jdLNG=;+Mj+?>2;2F=6Fpq0vU@qxKqS8F<# z?^$H3vV)JyL(|Di#93^fShFB+L1(eN(`w3Nf&89--Z4728RK`RX}7zb5MeA@%3?gR zYE5(TiWQ=W4O<(wR^tWR-(Rp@dcmd@886ti<2M;UVb^5DkIiG(bmN60ZP?Ly%y@$X zKOt{$XureO;&EPbbIf*h+Mwrnx{~7PoEvz5(j1-E%)!weow|+ot96{N|8eC$Nxu62 zYbZK$WKdZG!q{eEXHXd5ChV>iChCMeAz|`GVPCy4wNp6IAWZKPW*UXrCgD(6=-n&y zy&?3!DXe-+So4lBuumA=FRVQv4813;e_z;eP}ul^FnmbZ{E;y7u`v3Hu`Pk@RJ z6&)%%RCK85@RsRpLN+0rkWI)YWD~Lp*@SFDHX)mkO~@vkVojJs(cz-}8)s2I6$_{0 zUz5_Eg%<)Y1QQjVVrVD2I(m<;TJP1>&iiy#eZQ{GKA@}K2X)m^uB)acIyIh)8jn+T zt-oaG468a9pJ33<<$BK5&#zP1DOJ`y44q#y(5B2hcg7R}XUtzx_RnePTr_iB+GH3y z;jyOZT%4X`wxJV;oa5rsxcw->SKPZw$ZDF#P7OTiy0QE?f4e z66XU;7cX7*@KUXH%Lqfq3Y)2E$v6xhIWm|mUkMX^!k&I%G9~OgB}@$n2L^@dv@r9d zFq;t$ofUe|34IrY{$GSuPoPmOHLHYyCxyW^!rHaM&^lrLQ^JO)g^e48;f=!PXM~Yw zh0!g-wynbU=Y+BC!p`S~@fU>MJA{dsggq|{ldlN-UR@7EC$)P$hv*c3^+mz&(8@2s z1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$GMXy|Ya64xMc4bl`_gT#AP^Ip{jzgIO+ zLY{;?2{}4~qa)aaY(h36n~+V&CS(&9ya{s{Iz>*!CH-8|@1Wbl3js4Y=(gN27lMg~ zPMJt2x@y@3)NKR$ox19}1$$;-$1Gh{&ebXL7y{EA2u$!tbTLG7dpZ~QFEZQK!PJ;M z^c)i=)Sx0(DtrEqdGclxK6hNdgoe(TGjw*3W#~L&%+fL2&`Cnh@eB@$q4Q4r8Pg4& zmQzLU8HR{c3=yYfx~t#mL1k$b>yF$xrlE7I@q_;&Kj?DVF;1vDY#E25BZmf+B`i$6 zE*xkUrdx!Wh%o!Ma40JD?iczF2>tH~tKJvZ925pV5C#tkYd;c(J{H!0B5XJ!Z2VLh z{!G|>Oc?oG7(F3u`$E|Mr7-rju=5*X{99r7Nnzr&u;)8r@_S+555m+>!htiw^v}Y~ zIZ<@77pUlPhz=8A0!)AjFaajO1egF5U;<2l2`~XBzyz286PN-CP|=~HLq&&*4iz2V zFr7`vCS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933DhqT$E2cg_@S4o#Iry5N@ao!9+!; zl!J4|H8kjIo1}PBl$|^ropHD|`B*p$#lLZk@>nFVQqXZZJT~nPI4=&2W|?bji?o@> z)}hTdwodJKV>>w`behHzbS7qmP8x2Gr{d645Oiw8Pya9XZH5$cvy8Qw=C<5dVp*V7 z-#=pS{1VGT**nL0?<{%mo$j0Np8e?Z+4IXET3%lMk4ZR#soa82>!Cd}Miq1__SxN6 z2s#drOB>9LN6?W&gUC|x6qxl2hgJ){YlObFLjO8p)l?K8sAv%>l< z!iKHF#^;3L?ZW2gg^?G8(H+9JmxS#v3uCVcJ6{#XcMH2;6DIZud-e*GZwUL|6sF!1 z4!k2w?-OSB3$q7=L+=T_9YSC1DF{0L51--;9nR2U0!)AjFaajO1egF5U;<2l2`~XB zzyz286JP>`Pk@3BmmqNo5|6S4`} zgls}KVZoa)hoED3svBxq`VBJzgUSe%5nh{4MR&^j>n;Qn1sx8}$!}m#sp@@jbRA{U z#E3e@O64Kvj3&~I;@IT*(i~B7Tq7REvWI@ol=E|TjpgS|%m5uP)ErOs)++cpzdZMn z>3&Y->D+!!*RfMG$LHs`JZ`P)E$cW2=*XeLWBEwv|5#Y{iLmB~Fz~4`_?fWwm@xFY zu>ORw;R|8om%{MZ!sc&;k#B|3lft&s!uIckvG0YQKM3PL3A@h-6F&=k&IywjgnhpV zQ%_WZ1FMATCxw|c!t7e%&^n>_|7Y+0W1Fh~_}_PbY^M}H+EtOG`}l#F5dOJW?)BcU z$#N5un`8X^g~|QH@Rxs(d=`_h5pGDhYCbW{iggw&vp|IcWfrVhp^TzqRxP#agd%fR zm{@Smiena-vueeP^S+-4oG{APLAUm^&oh+ADW@&ndG4I&U+>3xxj|^`OTK zdN;bIpX2kn>E~2_?M0J8RQ3;l04Bf$m;e)C0!)AjFaajO1egF5U;<2l2{3_*CqO@k zeh&Q{`Z@G-c(ZgiA)Am*$R=bHvI*IQY(h36n~+V&CS(&`O-Vl#KA^Qv^}PYjVm@d- zBX`WzMDqeo4Bf4Xr?9{WlmMjngsd@MtyT*{7ClA|+Ppc&=Ul<}V{ zpypULVIBMMC*#!y1U_{~dhxp&ghd~KBUDA-8q_=Wt?|;}91hU2+bkaG=Ga=lgP_y* z{DHCyI@cPY)0By+V+uO0yG9Mru|m*snkFFV;LxPVvK91g1N|?9f$d=M6)^NF7=8_m zybiYS2BUkx*j_NP5A1vsOzsD}-U3r^gFWwp=>uTzdtl~$u>V6a`w=*J2+SP@^GCqK zC*bg>pyMd$>;YZJL3bMToCLkSpsx?~XTZQIFxU@4Cp17o$3Q`c2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bgJ^>0kT!O?UNL+$cWlNBFpK9Kxy5je#=1IttkS8I>MsREd zn~+V&CS((`3E6~f!iqOx5kaSx6K51b$-q#ZlEL+j4xuu_-zICVjDn8xUo6~AR5q%U z0@H|{bM=$@N%M6U3sVV0bfhjbrH=**Iuvw-u+O$MFOY&xK)N}$w*70Rp!43(J!Kbk z2F?_ppwqf*)z}ksh5}a!IVEY?urJxgSU&{$Pm0x^O@!PZV2VeqBfC(@G zCcp%k025#WOn?b60Vco%m;e)~cmfo3DCkhop`b%Shj&Y76S4`}gls}KA)Am*$R=bH zvI*IQY(h3+l{H}zL5FS%-4se%HI%dz!3$wpT?l0qbQW@IPRSF47Rrf1D~0LeL)OaG zj?U$KmXy`kkzZH3qoY8bnKdNfC(@GCcp%k z025#WOn?b60Vco%DxUx~9WF%TLL@Fks)JgS36UXn>MEpKY?0i5I zEe~qq{6m`Pdsq_(8#U4Pm`0W7qROKPxrl#6)VchLoOE-!qI3208L%RnfNw^-g_jEP0a7t@`^d>TG>GLvG>O z7}Tsme_~YI7RXnOHlu2iTa(-x^%qQkzF@laf=LQD<0v|jKNTIQI7X@Jj8=3msyXFW zbQ03ev2}k2Mdz6#&i}=FtFFM@Bo&yuW>^gm*bR%+mIp`ez29zFjJ@;q_s+$8@379i zXWo;`=iT4<$nwTUyYkS|N0%;pe5o40#w4Ag3!7EzKlIP5`1~za^FNm@``2F|Y-zbl z(J`tm|Kmy<$1D}6)-BgkbZ}@=WqA%vJ`Z-S22(G9J!`@Ai(v0MFtY*de+kTP0tYvP zxvgM+8(4T59NrE(f}nFJ=n8@EU7#lndfPx>1oXcF2HL@32N;Ti;dj7D3~Y~s(F7R# z08Dg(od>~W672dIOm%@h-B5JWDJnXK$}hjD_-$JG126$5zyz286JP>NfC(@GCcp%k z025#WOn?bgJOL^?RCK85P|=~H!@H)l3E6~fLN+0rkWI)YWD~Lp*@SFDHX)m^%9^l< zqC-1{b_z8uGwqb=WT)`ADSw+~RCF3(ooJ$Cy@Xt$u89Ff6Wuqj;E&~X-Y4uY;B(47N4KY`vn z=sOSk3t->^7#s#eO)pA@9bjY?*zN?Q&0x$0Cf0zRZZO#bc6q?mda%a}rZ>7J$diyKA!kQ$b_AP{O~@u>6S4`}glxi!H(?P)$Djxr8ZtJCwIJ^gRL_p^opsO8pcYvNK=zRzD#Xx@?3?#td z2VkfZ3?Bp|NwEE6Fxmyiy1_&W>^ur4d%&*aU@8svoCMRoU~eCo$$WkN^!G8agy|Xz0+; z;SJN-gls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb(BYc=>9Z!EhJ_;d+oW?>%L_qX z2xT;MI5nrVi9z+E@8MBu&ZvJbnH*C`4yq}^&M}W6wT%0cuG-YD6cc!?z?*OAORwYO z|DkryY-_Eh)@D&JZfD4?VlmhZYSy4XF{*6~yNnpsLIwK@ix`GO?Aa@Q_Yi*Cm~NlPL1Hy2sR;`kWI)YWD~Lp*@P8u!XkE# zPz0Crb2)zvy%t^wir|GXeJ+GD+Bx&3b)t!bjhbkCOcNVdXrk#UY3!^rXrgj?$T-uCN;|LjP-)`;B(6 zMyE2e*F5qb`Ce;{-DGKAV6C?rg|N?-Pwh5~L;5+kmNo07r1RW`Z_6&}Tx*0*QzoX4 zE$Ku?jnJ{!tZK`_2}wFQHYu~jL01BFe*k(qLGMA(mjwMEgMltE*bRnKVE8B)=>gl1 zgV8h?I|(Lw!OlJ~nE|^_fvJA5X8=rR!QQiAW)SQj0<$@A@Fy^r2lMB_LIE7U06P8v zolmcmbUg#Qp9MY7f!^oWNlC}IdL1PlN;*t{2`~XBzyz286JP>NfC(@GCcp%k025#W zOrY`!P|~5KLrI5{4kaDlE}c!tCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrgheDBF3X=j z%kn8%D1yIDYIjCn2=YQGqokuq@kClUb2ZVtKodiEYvTC5nuy=0iJcE+V2u$q)d%|G44eX78nK!`x zb}-uk4o1P;J77Kr7UJM=0(5)`IzIwkhd}pX&~pUzeggVF1^u6afn#9sa~L|IFKFl( zD!=-o;`eCf55NSN025#WOn?b60Vco%m;e)C0!)AjFaah|@dRk-a19dIAaM;+m90VI z-Ku%F>Wbg3nkOMoLY{;i9>L)eY(h36n~+V&CS((`2`k=&MGPI2BDkcVOZv@pTX-QT zf)~Q{xe&@|=q%*uoRY@|EtCe+N+B*jWT%WWbVmJiNm+#*`E?~oIz>F1tIe7l-?I>I z&a8_M1yAVdRIjJ=Z#gl?XscCg&$!RJ-K||{f&<-Z)~(H4=sIk z>9WU{s@L*#;`u{kdpg~3j`DP7Nl&Nm=!85S9Gmo6PJ*FcFx&@5GGO~DFxn5s2EarX z>^uu52f?l(FqH#)ege~Zu=hNeDS-VK!0a$M*tA|U=K%Aoz=9JTZU!BzLFWsgYc1%0 z5%jDBy&FK^OQ3%f7}yL3w}PQOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;<2_@(Iw>p{GMnhn@~S9o{vaO~@u>6S4`}gls}KA)Am*$R=bH zvI*IQRn~+>JRPpgpFS(|=~*a(zfGEV23`pALMWrBQ^)B!rB4o0ME4rQh^ce=6FKW< zl%_N4pEqdj)C*Gyn$9J4X2hYH>?aL$bky6V_EKvzUD}$}1^T~h)Z6uKtvXNND(Y-~ zYq_+YA-9UfU^A#$gZ{**w&9aTn^85%tx0Z;`U|E%Uoc&I!6c0veAKL-oT-1ptXlL> zm{m=Gp+=1u*!d$Ao#f9{ zbT~?f2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgK>}2CxCn`hkhln`$`&E6S4`}gcWbXB8pCpBDkiXYx-*`x9~zx1TTc? zb0L&b(P@Bnq6xX8TobKZBnA{sbl)uZ%#l0hYNC08MvF(cc;<_p53Cd>O_(m1bFO~A zq};mBqSE{v4Z_T&K|fRQWCgyjn}uG!Q#$buhde zjO+p1_kz)VVC+pWu^;Sw3rxNZcD)Ox4uC!HZIFIW`uzNfC(@GCcp%k028Qq0`zm}=g`lgpF=-~w@YUevI*IQY(h36n~+V& zCS((`3E6~fLN;NQHDM7yhb!`@&x(Bd6^h_*lg6Fkg&;44GWt37QaaH@%Y&LY|Bxp7 z9@fOcMoqLmriqO!G|}{w6n0igYljlgC4or_I-?$SQ{wBCFhFOtno~EPCp2ox&vpEB zzA#Rinlti64RmzIo1xP-o}+`=ItDo(=He`!8R`nV(IWJ}X1U*J2WxaHBYVvw?~(7d z*4Rx_W3krjM$2p%EjCL^>N&PPk5@W6XT4vR-O(A!7I$>wNA}Gb)6r>ZF<+_P807$> zHs>fGXKYCa$0lW#PB48C>`j81kHP*fFxw3droh}$Fy8|fj)TK#(D5bc{0ek^4Z6Pp zJ>P=f)1dD=(0>LDoCAa3gP|Y5@Q+~RXR!SjF#0PP`wdL|4tD+lCZG07c0B{8o&|fJ z1Jlogy{o~@3t<0RuatDMFM28IP|{%nOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p- zfRYZEA#oWJmmyWzG9=!sn)j-%_`Rxm67nSENyyO=938rIkHZjFq9zZNLS{X zk0p3Xj=Cx5WAW6^&bcV){D;~(v!`%^&aUz7oQqaYx$T^c6mx7teXw(mIDRg>ozr}- zxSi8|;_U4cvvbUHT29O13E4R~Hfgb}2eV#qa3h%Wf%z?9!4D3%f{yK=^A*tbD(HR< z^t=vwcZ0q?pnoqI*arsR1Vj75@LOQyZLs}aFnR!ty$2@V2RlCmlOKUyhrrZfu;&Pv z{sip(6wG`E_8$YYpM!&6fVmT3{!7?7g|BGm7%IQ=qT=^wRU__Qk&!QUo*I}a|AbG-YDyn37JLchm@-S~)#vXsz+exjJKy(7~}unI!`jPJzSypyNBxc?NWy1Kr<) zo*zK(kD%{o(Eke<_!SKP28MnI!+(I0r(cq6e+G;`3&x%U6VHR4tHI<8VAonO^&;4_ z4oq(VdtU-Go523fV0J4wxDCv`4Cc3kg;&7gS3ySzbnbddN;O~@u>6S4`}glxi!H(?P;r&bYM)z4M^2I?)m5EQ`+VftJMWt4OjDV}Je zb*?6w7iePWZcQA&R}=C3G_msmO|(3yiSrL>6nS)!r%r5KF=FUk{iJ@_SsG+Fk7u6>~UP0+I+^u7i9-Uj{ef`J2I@I5f}J{bNGjC=&P z9|EI?!PpTn@d?=ZDVY2W>^cUfJ_mcg0MjSH-Y>z-S786wVD=ku@LMo<8q9wO7S4df z=Rn60=*)qxpFnpW^qdF11sFQM3p8{%Mu!P70Vco%m;e)C0!)AjFaajO1egF5U;<2l z2~G<0a_(9ofwLqmtROlK3a3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%=zYx1Yh zntU1-ir{aP&Rq>J1bHEp(a>2atrJZ&J*A0(RR&FTuaW3jFVVV1VnC6*ZkBs!@koPd zp){CQ3gg9x^d7I4bS~esq^!b@{JIh(oe5huMa7&&P;acIJRZf{u1B=Fkxx<5DP7X} zx3$({wAHFLa%+;?zss$`rl^x6>3F1{V{1LUQA#>PAHFkXl1@B-XlzNRDL6{fvD#*- zZJSIJ8KZ+^lQPR6pzrC8lKyAFz_VcRIWY7*7+wuVUI5$Ig3%Yj*g7z=0qlGUOl|_Z zHiN0HV9z!%{W93Q9n8D}_P+{dUjqkU2Xni@{2s8d7aZOPI@&>J2k44|?sq^>4D`l9 zUjp=h00uh2;K7Yj(g`Iua*R&pZ;{$$5S9JIAAkuk0Vco%m;e)C0!)AjFaajO1egF5 zU;<2_;t5dF;W8vHL*g=|DqDucyH)dU)fK;6HBUmGgggm3Jc7d`*o16CHX)mkO~@u> z6IQ$li%2>~MQ~L=SM>|(ExZsE!3$yfTnJ^9bm}-dr}VKw3M8fxP3Q6_a?s7?g3i^? zZ&2B(7or45rzEka6#3@z_tZngnRTs4Jzk=wW2v>7)Z3&1Q)@G;3-oP`db_@@Rp;qj zMV+m0Etj@4%40Uoidog6Yx=CMn(Eqh|HwO#KsP z)uMmGtZMoTHEP5tcNnCW^Mw9}t2G_$H`pwfbe$RM3cJxF^uK1g-)IMGbSfiz%_HxT z@3q$0O_t^b)_SW^2>WdL)NZo`q@iPL+q6k)I-l-6T6Rq*evMf=Ll-uy)_>?fh$?@O zyKb?X|G8}0zwF9GOCMdj?D3^)(}7W%j*yy8e9Kjujz#VJ(Qu`^W04`nJB(6ik;f)= zmLp*36EOTK82JorKL$oW2V-A=i4$PwmtgWMur!0h+n z;16K#M=<|0Soj4T{uOjw0G-32t7(&@+W~r3fnF!*YX5Y(h36n~+V&CS((`3E6~fLN+0rkWE-+O;|+J;lljsvoN2Ug(CRdq={qcnVdN~5$WdG z;-?|#M7EwSyP(r?w#GU~_i*YU1kJJxC^k9nOKnIT_^2nsdvKfqS1!LR5#LHmkb};!0*!3!ydJXJ(9Zc^Ad-s5uyE#`D!=ri;Wbg2nkOMoLY{=29l_ZVY(h36n~+V&CS((`2`k=&Mf4nl zBDkKP>-kL-TX-QTf)~Q{xe&^z=hRE*L=!C!YU2Dun&^9269*eL(e{`oHm=Y_(^Hxl zSY^=Y@cilUjMyn7PwJ*b*C}C!&R^!t$j>&ZPa1IG6;Nd=@})QE=iH)yerg8jv{lB> zNk}!v*8QeW`Z?>KU$xMBtM0_ygrPPwtcC~dhDB=2gQND|Z#OK)-ue1_=iWb8v>T#r>T4k$rQ<^mB#+W~*KQ|7XF^QS>oI@;h(Q_x%m~ES5{7 za*FPhB8xmUDYASACeMIf=fKqWV9yU=`bV($XE5^%*#9e-{S6%a9nAd!=AZUS7M=lz zp9LMwpwk7q)_`s|=xG7H9?-WQ^n1a;Mlk3DLtDVGAB?nu?Ex^l1B?a1#7?j?1SWTZ zU12cQ2KGdJQqW1i;p6-q3OY=H2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!P|%^E zLqUgv4h0?FE1gZqCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrghd1$F3F!hOY$jLD1yID zDtEQK5afkWMnOjkIEwUdZq`J{98I*&)kO0GO$^ZQCiw(hcI zazvdvVJbn+F+hFEhu>Sy(prZ?Pc7v`I=spU1dO+`S(3b}NC&55380-T>88CbbjP!%;17I`@ z#?FF?L9lZOOy^Tpn3t;aBFf$DHH*J=7PS&xRc24D2UR3NfC(@G zCcp%k025#WOn?b60Vco%DxUxi9U3|`bZF?%(BZw(*@SFDHX)mkO~@u>6S4`}gls}K zA)Am*SY=IE#L(fI{OPkMpN54Z_}ip&XXJ$-FN880Itu6Jlsq%2PHbE;qUT)wq<+$b zoyEdX!VH~jDmnE~X=*3qQIG$YlFomgr1OL{h72}~xQE7NP$&%l$?>Sff)J z*=rtok9@DS#%{7SFR<2IjY8OG%cpjmB`5tHTT|;6De1I-e7NkA&cK=CV{}?~tr~lb zPD{%uNvBpGdfOJ`mFA8`?u>U7-zjAld2CW<`3x)^1BX8c9cj>c5_I)~?mp0y0llX{ zUq9#{00UVtcoqx|g5e=Bk^|d+0;72_b{6IQ$li%2>qMQ~L=SM{5zx9~zx1TTc?b0L&b(pf0Q6HPQdrHO%6 z22FIYk?2@2A(z_AeTv+5v)nUB?kHJi(WdGDxljzH_`WHR|p9wpN{| zZxyv7lXU(qzuYLt#?;8INpAlxw+5SXNzN&^q~nmKVQU$Lr1Mh8H&Z6*#Pf&7mULRz zjgoXMHmln5b#Wi3sJSC$7I|z^W(k0<9iTf1dUk@|5a`LX59gK8< z?NKoL4j7Api8$Dq0FxhpU7cX+AlQ=x(;tJqU0|jg>`#H&qu^i7FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zDoB8m4kaB*I+S!M>F}=UY(h36n~+V&CS((`3E6~fLN+0rkWI)YtgcC8(n#^t`yh(xSlV)k?}ipbZV@M zG;Pf4$(g!>V^%G?f@4-S{e>DeV$@xnS``Cxrq0pvNIl2a`p#DA=$zmFTiG3*_%#OU z3|-i)TK}O>^ikzRpSy0cn*X_M*}v?{LrWiBy6o|#YJAfuN5^WLr4GGfDyriY-6>rb zd2G^U83KJd(Ek${$b-T2V5k6wFMyF@u)S%kWYhu1R)Gm8*x3vwU0~N5Fy#h&TEMgi z>|GCLykP%EFzW*cw}3f6m~RCO0dRN+=y(-$z6QEp2i?0t&mPdb7xe7|{cnPS{b2B| ztNfC(@GCcp%k025#WOn?bgJ^?y9T!q9{NL+M{s-un~+V&CS((`3E6~f!iqOx5l5#+5nR;IMg6t3 zTX-SJIUKZGrp<*=Mn|WCvvW$F8#G@kOe=*^6UNK!oU5NNDYvh)s3bw>x+R;xKB&Od znN|8@A>K?!$5Lynt~ok;#&dMw?ig(4o}&|xdXBBl<(H1m7i-u3Z`NCNZ{{X=Gc&A) z2keGLYRiM8_TFzdEXLmX`g`Z%y?0n=-ZSsX<@4@ud}MiJ<6o*x$JrX|9Q|!|@@@86 zmI1{kr`xDatB>6=F-J#8M<@R2RdaN#YFpP;J1uIw!zf)Ad2G^UiGiUw7*2qZ55V?L zFnSP-CBekUU}qPY>;}71VCpE?(*vfDgS}}ma}w0t!Uo!j*7xB}~zofLv3GhO-y~ju_*y1h1Ca&{>TC@GoiT>gP-FYMH-u-2YIH?vYYU ztxc$kzBQRY3Fo4&0sL#J&#LuZBjq04XRM5LT!iyw!f6a6hzc0*?f`EnwOY_O^nV0NB3+%m%^1onS5m=68XGFgV->I`)CiH$m5a(ES$Zc^mY; z3;GU#{`bJZ`(W@xF!T`^J_JS%gY8Ga=qF(8Q!w!v*m(?sPV#dKItB_lOn?b60Vco% zm;e)C0!)AjFaajO1egF5U;<2_@(EDT;SwY+LE;jmDqDiYdsXvZ)fK;2HBUmGgggm3 zI)bAk*o16CHX)mkO~@u>6IQ$liwHVqMNF$@>Ckj^C?^_T_-_!7)>F3<4dps6(w!WPqw~AC`3~JV(KQXFp3*;+C zX^xoW)+Dz^{RPvXFPJX9U{Y)J7tHF(nffQpD%>HH&8%wr3pMJl@%)_0nV^%9YL2Zt zxJ~*wn@=ao?&mb0EAHoXpE!H_#QYqi%&PVOnTVf*Lz5m$8cd!9yL!P?AJ~%t)2G1R zelRlt_GiKDS#WR=%ngD099Z}X9L|G|UqR<@pzC+g{RilIdYh#88PNAE=zk6jJP!s} zgP|9|@LDkPBG|qTjBWs9FM)|oVCQBqxfSf%2BuyHd$w`O-Vl#KA^Q zv^}PYjVm9Pwt*BYSHl!>Wh3OcR3 z%vZ`dW(Yc43|HHQ067nSENyxbooEyO=WD~Lp*@SFDHX)m^ z;!Rk@(5dCf8AZ@CFjS{yaDB5w=#22U$yzI;p`&nePRUb)>ZHMRxuG-adHtjbJBx*> zgef}GmzmN>0}UPZHaWhg)@Hi2HLDneQ){bHZ`ZfA>O6g`TpFD-H6wH?Wawn1oMRhG zv`Rzg%UG`LhR(p5;v;lgcdZ(Ggw9alDnqAMZ8}s(7z50Yyg8VfuT)ccrzH;3bt6S4`}gls}KVU;yu5krS^3gr}9S~aw^6u}E&T3rZbG;|hnZcfQFgBFURyH^U+ z#fPkwQHIW_e=aGjup_^&bV)}#Y%4|W)I1jAi>y=p_z#_=Gh6p{rr6W@xBOD0tyZm( zTa(=WU2Y9FMSViwTGSQ#c80pbZnOyfuUWosw1YJ|m65&Xk@v{=T5IekOY;J2z11j$ zeYSjRw^?#h(XlnX__FkLVj7z@RJ-$@EmZuZX9~#@!>E1HR)2X#t)S;cmiA>VL zu}Poh1F+Bu4j%*^he78N(De!E{uK0l26~TyzRyAb7hvE782l0peFcWU1|#2q?cajY z(_rj7FmVR#JO?Jf2fKa%Q$K<|KZEICz}{cM%x_@-?_l;1aPaAuC3DY!`DekxbKvmv zpkoc_biXV;9aqcC^mHn}2BXO!D*J~&025#WOn?b60Vco%m;e)C0!)AjFaajO1eid@ z6QHNVl}KEP#Fa=@wi1aqtmX}?D}KXjo`gIJc@lDd1m{Pv3E6~fLN+0rkWI)YtauX^ z@pOzFR-y=M8iJYzF9co)(`g1%89kjk>7Hof_`RBl-=~S44``z0K~0>0NE3YzYvN#| zCfXj;=<-~2c@)vT#xP>)T>eDPx*4VEjQZydT08Z^P=cmYM9oF8A5 zES-o^cW-KS7pIt`bJ5Kyx1-~bdXBB-D>ypayn(ViI`M1F(iyt2S+)K{cYIVhKDSuS z|6I20Uw_FxEiI!Q9gEGXcAuM&ql06UF3UR5wE=X$1bQ}s-p!zIE9l<_23`h(+riK) zVE9!q@*3FwIvCvz#`b`TyF5~f=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fyyU9M~99M9UVG4baZ&%bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh z7IAc_r%+F!r)8#|GM)4k{x;=rvy6^T1GE!ObgY+X-6AodXrlXOxo3{tF;^4K3p8px zy23PHDoiVdjHtM#GotJ)DYvh)s3bvWd=Y1Uxs{r_DHn9462v(=HC9E=xiPCJXG#^v zD4$s5)+D!@{z8qqXFNw|a^~oGq@H7I-5rpQPLm^Ac1Nejn}uHw{TZqF>sfMS!C zr8ccTcE`jV9jk4Y+7>ZQWR4DwO}Z>!gT8M-|F>Y^G#LC244nbP=fKGKVEYeX^hYrE zGnn`V?EDo>{swma4yOJ9d!7zRrk??Op9M3|f&I^e+123S3t(<7n12y0tOJKPfR2rz z(+9e?fNnqNX$8Fj(6NfC(@GCcp%k028Qq0(5k^3W=+bxC*JtRw40r)x2GG#cx;5laMDNPeM+Q;PePK zA)Am*$R=bHvI*IQ6>q{Kj*dYQR5Yk)n5by*LQn)Rgz0l3l+n?t=j@yb%?*;0&ZtM- zlsG#jOwpOxK${itYi8ZZL&L`>_N6!eei}OC8ai#`89I|QMJFKT99!FOFm%2?_i5P; zouO=TLnnS@-<&ZGo%p6vQ*?whbmGpdrs&L4n_e5G?O1FUHQr&ACW|~aX|n7ALwmvS zJ}~ko*uEc(z6HkK1{3dsod>|=dtlf5VCqA#=OZwE2<$x!W{!aUpMcpF&=*ocZQ=q3G^bUZ&Ea*QA1_r_45E#mV;h(@r9&A4kLnm6Gp~ERU zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_3KF29Lqms#4hr&7&=^&KYiBZ)38tkf17mfYIz~Z3!#jL4#(z{HZrJQ z^gTRE%^CI2C6i<7)Cpq=cFv8fHe#iiz+(m8d_#7QHgQ8}1(cxihXs}QJq-dgYHDpl zRrIYvy;I*B)!X!~>C)D$F3|s7qu#D>Yt?!BR#9i`Tg#>G47tS+9D|xQ=ueDl8$M~Y z8C8?qn&j4~zhL_F1=FP$OzOCH&aTSXIT0!5*y4w_OFJj=#dl@5bDGZ;w{yBroV|Tq zJIAgM-kF8LJBmffbenF!a(vD#Y3%eJy_TGVLz5EA?_l%~F!uCz$;2~Y=d)n)Ik4+_ zFtr-&c>zqX1$$ovGwZqaBvft+YIKnf`x71@XMfM2j~ofuAQJe1bTLX-Z1EE z1N{*&@CF!c2SXiTI0{DI0o!9>G!DiRVB!O?vva$YbCL(Qb9@fR=P&^#zyz286JP>N zfC(@GCcp%k025#WOn?b6fyyU9Ifu)SxcrFAkE(3>5${sXyHr>FF4a5UZ9DgyESqAUQNXB)BX#_ z@HEI{Unz_uk41PsdYaDVd&=eOXe9|c6L4$B6>;hy-~4r?*Lpr5@3|o{=sG^$@So}E z+&YDh&bSkFo{&F`K@NwxI7w%Qy25U>2>q{F?l;=O8lB3>Uh~L%)(io`|D^ zLz6DcVK8|F?D_;ueG2w`2BwdJy`O`bFTnm2VD?LJ@GCI)HJJYfEPM+Np9URS(0LYg z4TA0=(31nbKY_kH=synz3SjU87#aq{O*>vIBOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfQ}9w z9XdL6bm-{tX6bA~HX)mkO~@u>6S4`}gls}KA)Am*$R@0^CM@FUa8>^FS(Q)6LJ|CJ z(z-M9LXa0i866$vzu38%sBcs!9j0s9Dsq4Qq$xX#g(zW&&iG2r7(e_NFD=1wnXi5< z#!d=4a=6a;Lv*T2(77n)lv~hANjJyV_bmjSReLv;UC#kK}57B8lV7^ku zG23d@c))Pw&atQH;MkG1932z07D;u;Z87e5NuC^(T~Ac7ntY zX)t{f?Ck|JePDkEhEDbr4IKjw9VWm8m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ27LC z=x_}Z*C25XQkAVi;?1ggv+9c9tePhwPePuAoE^d05o|&>A)Am*$R=bHvI#5RghdP; zlOnjJpG*49bX#~KD1sNl^tlkqXy`2DNfC(@GCcp%kK*bZFr$bMNo(?@7dOEyoI-8J9$R=bH zvI*IQY(h36n~+V&CS((`39GCLi+DO*nLmA2=F_uK1b>?}?+m;UV>HU zP3Mw2Gvd%p_LBxWI_hn@dsAJGPQ<9YIJLTqQ_RtsoLM?Kspr_596{;mbT%I|b`}p{0*5UH14=btrI^qhnFq-ZV~RmJW_h zx-4&kh5g|0TcG0|&=~_=anPLrJs*JHPSAG{^e4f<$6&Av40VIy6c{-Qw)cS1<6tZe zCQgE#yvLovtZ^d*gptnhrq!cnEMIL=fT2xaJT?Eeg~a@z|nC% z9i*c}M~4Y80Vco%m;e)C0!)AjFaajO1egF5U;<2l2~<7-Iyzj1#8pUKg;Zs$ka){# z-m<#lx2)z#$diyKAty+1f&`n8O~@u>6S4`}glxi!H(?P+r$!N6)XzozwX|D!At-_u z!t}Wi%IN4cKs(Vy$9jp@EfNEYCc1Byd*;X;b2ZVtK%>T^D@^mH!n9JDG-0~h(Ybuj zl5+bxi%Ju8G#E3MDSNEIcg>pOM+3fb>W=Z^cQpu$F5(DP(YFTmPJL^vNN-Vj(wJ8K(WcbQkzyEyJKR4j>TqGTM82rbZ~4^ zWLX8eoS?fI^teFp8qntk{Via?0|wWFAukx-2u6Hh`xY?j2V<>ZA^>*o0FyzmYbTfr zfjzsxbQtVy12Yk@{|zwP4i0vJxhR-_2Q0+E;W+4cA9Q{Qx;_Hkhd|F^(0e2(1s&ff z6m$%gUwl#VJGAl#U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0u*#8=upt1phH22 zH%w;}vI*IQY(h36n~+V&CS((`3E6~fLN;NQHDM7!hfDIO&ysu!7K-3+lggdog&;44 zG738N9Go+up+VBn8TF`}5@Dx=5jvy2l{%O#)=7QL$Nx`B$5Lynt|>Zg<4HPC=u>tq z>cwd~Gt?D!qebX{%@%{rXa{R_DkFQ%Bkz&#wbs~8(v-2*>(b0@NHaEzNBTLoR{yI~ z(rNBHUv^1nC|g|8i67ZFXG}?_rDfC<9jk4YIutWaWQq=sP0B1O(03H{_ke-pU@#4a zPJ-cHFwzIMXTazwFxC$y2EfiNm^=%14T7m5uqOwme*$~+VCFp7UjVZgz`p-fRYZEA#oWJmmyWzG9=!tnm4Pi_|2+$67nSE zNyym|oE^a?WD~Lp*@SFDHX)m^;!RjY(lIN7tNOXB-$=cM7lI;qAxxhOp^TCa$L5qa zGN@kkJv_?J8THR4lOyZY2}22j&R8zal1VUW7EigFBV{0t(5bO1(w;G^Cud5@#wed) zypKt4HT{Jeb=P=y&P6My+;&btiaEBnv#@i%JKXYrSZ~$Uotvb(bJq;3;Q_m0k=pX$ zsJ-{w4U4gNzW(01c<&w7nfJ_la{0Xb8y{KT*l1TCTKeeHWsfgao6i-ubGlEQy?tVK zj*xav{B6UPc23cq7`D@P+9)j+d2G^RX$3<8FuVhd1i|*5U^E2Ac7cg7*x3dqBVgAX zV5%ML=>XGFu=gD>69fC>U^W2`egNh=!Tdq6kOYT61|6S(&QC$tXQ2BS==mJNfC(@G zCcp%kKoulFJBM}-?Ht-Uv~zg7bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh7O``< z9)J3*$ERJP2>v$d+nIPF$P1y2cFufhooM1DVMOL zm1O8lV3f^5$Ti}|AM$FP&kpko8*dNfC(@GCcp%kK*bZFr^A&KxTck7-2Gx%x@{q-i^gg|UQ5Iyd6o7*~o3KN|3n8~M`f`D_Eeajcm^ zlksZ94eL3#>Kczly+z;7P-p8?ZwzYIpg%FHZ42ZpMk$V%@d)4FTb*pqbPn?~t5m_F3{KTFqha9~nn zc?V3!z^*u$N`O5dfay-K_aK-_g8d(Z*)DLf8_cD^{86yb0}dYt9VbBNm!Rt_(ET;& z`3CfU3;Irj{_nuR88CPb41EuVe*hytg6%(p(OGVMQsa5|jw+5S{uF$uZOWPUhin0vQNl7`!*4G0=XU*E< zQ)cMI^M}SZbee+ZD<|WaWfV<&45ceNI5MfSxWM!pu-6S{TEKn}m|YJJdcoXAFz*8k zTfkvI=y(})ZUA)Am*$R=bHvI#5Rghdn`qawJbpKJOBZi zPZK*I&_v6FnmGTECi))M#KA^Qv^}O##_46APcIt&F z!O$s5t0_gjx%@r#&~R$6^{B^7lyoe$Hj{dr^j>P^coq!9skPOpx9i(lb)LRe}Tfu8R{?+>8wN6`N>82AMY{tAYE1H-?Akw3upr$ds_XTaFA zVB$Hj^La428ti%jOsxfbUIf$Yz}^jD<|VLy6PVo$4sHc=+ra$GAt~t;wud-JhmsBx zU;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qu0+e(p=}^+4q(e!E_e^IKvI*IQY(h36 zn~+V&CS((`3E6~fLN;NQHDM7+hs*M(&$4_<7K-3+liHn`7lOPH$|&hHz&p`I$9jp@ zEfNEYCc1Byd*;X;b2ZVtK%>W_J52MX!z6_i4Nl5f18bI)Ti97tlA&`=kLFr_%|!P< z3LnQZ@>qoD)N`s%&)GAco-;W!baK+nu{He&J*TVxx&PC8tFFx41Z8H1)$oAbut;rr zaMa%W?S{qJJ70hAT)g)V>&$!RJ-K||{f&<-Z*2U_vgtTmW1XYFtxmqpKFczo*koU+ zO{`@wPH;E`I(CE3J)mnZ=-vl<-UPk-LEl@T|7|ev zE*LxjhTa3i?}L#K!S;{9=pit67)%@iJ3j%FpMqVVfvIC)&*xzJ3$XVDnE4Xy{|d~0 z4Gw+-=Dr2el6u}E&`dkQQ)N|@NH)ldKgRVAG>ZU~3DPe-nU*^n+ zLNlpP8tCU-8k`84ayPq?ZE$-*UkL;T>rk^tu7&Sh} zYO|`{PE+ZA4vtKEEI)&;UqJV-pyxNx`#b3S1N1-rx@6!PF!(GOdJYUf4@Opl?Jt1Q zwP5T;FtHBo+yEwD0=qVWsm)-|RxrH{?0p%`YzO;a0kf}ygRg?b zehydUPoEX}^eYs>-zJT_T3!h9LMWr3!+ANS%?hd)eGgyl=3KsK$>exCb;4AFn`2!m zru5N(ujA~Ts+*m&t1@nmM`}5?*0sB&o6{28Uv@XA`CM@~r~Aa&+b8Pg%u?G9Pr%K= zkx7T;ebDzI=>G@|90G%f!O#&f{0SKO6m0(tj2;7HpM!}nz|Iq3@=LJmD=_sn*z*mT z{ub;#4Q9Rr`_F*cbKv0jVD1Mn|07uV865rvbQD161<*ANx|?=MdK{p473gz<{$?=X z0)uOINjE3t-bFWuZVnS*0!)AjFaajO1egF5U;<2l2`~XBzyz286R3OwbaS}+h^vpd z`l!lQAMqB|yhU}zZ&A&YkS8HeLQag}#0WMan~+V&CS((`3E6}dZ^9yOj!*;_^K&tO z4XqYl2#Vl^FnunBGP*hQrE{W*gN>SKdrT7>S7@T?De3F1GH9ZEjYP+KiPkL=1B%>r zv)nUB?wG5I<^`GqoDV+ktd(si^6`Z`S|2vyOy2K7#TYvk-4A?&j)%?s={OF*hQwzeeA{bf+hBttbm%#Q-V01GW+X^POft@dd$?ag*D`4tXu;(=}{W{pY z8_eth`}cy`ec<4mU~WH{e+w+U4GzBxI^v)+0lGc_-JPK4Am~kkzK=nF7Z~UUgDEg{ z6b$!(k>g-{8h%dnB>fyi<=0+R{Kl;O0hj<2U;<2l2`~XBzyz286JP>NfC(@GCcp$L zo&fzE`Z@G-=;zSS;eFECgls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb&*6&v>9ZoA zeuX0V+oW-4L*RtSu6}C%+I-|kW)WtB2E36 zevbaCaiu8#qaOd~f1agtt8VpJ)Qj60>TIiGG1v@h)}TL;ipT=_4x`Pen&j3bw?_R1 z)1NPxF1=tdBew(+LuJRv_|uvyfL+ZpN#yU`-_zh=4L zXa{R_DkFQ%Bkz&#wbs~8(qysL>r%@lOD7@)9b5eM-O|!YZ8%+aOK0Fr@hLj3yH<@o zMW?0ZDoe+xHr;cjrGw;K9=wL7gCmnR%U59ZYcTc=nD`d#JPjtl1G~C9Q+N;{SM~;01Hp=mK=TtbT~n0Gw5=G?lqvt4SHKZp9l1>2LoO( zxDgEb!0;9@;s@JX!Ds-C?En)&uyf~bY3U?GyE#RNmJSnO0!)AjFaajO1egF5U;<2l z2`~XBzyz286R3Owv~;)*iR+NK4ynr4A@OF_yjgX{Z&uBdkS8HeLe7rh>5EQ`+VftJMWwdk_a&k_|Q-c=DsX;4+;o?J1 z=DJ$cxqQ!(vPwJh>q-!HN}6f&DAbzrm@|vY^#4rg=V_p#Q?jD-Z{6Xkl~NC^pIY_r zx|*Y?D=sNI7q{h7bQ03ev336nMQ7jBe@vO86VD$SThVDRCK85P|=~HLq&%-O=lCb z3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%`#i}I(>qI@bAir{aP(w%`9g1ivQsOZ#5 z?L-sD@6|;7K27X=KoczwYU2Dun&^9269*eL(e{`|jpw4qqloS`mkgZ|Rp;UpIqK$e zJ?HA@Hz@4L$uT7uI@jaf=)#M6rMT|L^?d1sl$=p-rlB+L5S_?)Lv$v`&`C)-$JX~z zSQ$##Z0MW^z2NNqBR%KqUGzyz286JP>NfC(@GCcp%k025#WOn?b60VYuK z1gPk65fT?6aS>9LEkfeWs(G{Oir=i7Cm~Nlo`jqo!PyaPLN+0rkWI)YWD~LpE8c`f z6rCDHa7{ng^w&~u;e{ZFa8PcUHWxw}6`clHCz|M3FVVV1VnESE_sw$89JynzCYl#$ zw0LxbX}&a=q%NXih)Z!%=i>e)<<@oNteO)1oa=UM{`#Q2Kvs&{(jN=)W-2EJ&v=T?6Y>KFo8{6lopKG)$w)iL zHnd@nRCInm9xl70({Z-OI!8Bg>ZFNdpJf?PY_hV{rq##pn3$qdD-XT3$v6Q;2ZttA zmP26nFgSPw%zXmpKLrb)fy2i@$8pe^23;pXcQ5Ga1HBp0cMA0PgMk4sm<2;;!SEm$ z83NmLVDu+2mIo8(!OjAhya09$gQ=!Hl06PEy$bAgf|+Kp-vwsZfP-!@*8=7}d!(XM zSigsg4iz0Hzyz286JP>NfC(@GCcp%k025#WOn?b60VYuS1gPjx(V?P4MTd$G@0iXe zWD~Lp*@SFDHX)mkO~@u>6S4`}glxhpYr-Om4j1LqPNAkHXs0NG7s9l<5Xz|N)N^vq zgr){vt)$dViLz6IqcZ`wrW6ZjG5j0L7>^}*m4Z&y2|8`#2|ANALnkNQ99z>V2s%9< zbe3Jv8OjzHbmB+$%^6eBX*w`!hK|K%RhvGXh@gW*lOoGYU||zDycu-(L1!!I3V`k% zpeG1=cY?kU=-&kf!eFos3`M~38(^dzZ0`W0Q84xnn23R$aWI(xyFLI@onX&FFr5T@ zKL#^hV1GB5O@V_)!CViRKMoes;P6S%@fGO&8iJ1N8wxsx$}hgC_zhb5126$5zyz28 z6JP>NfC(@GCcp%k025#WOn?bgJOK(iT!O?UNL+$cWlNBFt7_h=y5hI0=1IttkS8H0 zM{sfkn~+V&CS((`3E6~f!iqOx5kbeSh-tMfecFt`pfW;bgx97b=uR20??Nb}pu?d# zr406=>(I_V3!L_tpR)7V7dkD^?;f6 zV80j4ZUhH?U~UVT_k)F2a5w-uUICr2g09y<_v@f%H|X8dCjA`W-Zl=egTKoN|NdYg$CLUmpUW%P6AOYcMz2OBlf_LwF% zuFyo&Q_|U4WzaZrEecFG)Af5(! z>??&Bc`U;7u?&@RDLYz8j?QR-W|U7;#(%DWnlnoiv&KIB$#}H^flu9$Ui_{GVYSvu z5l5(szBQRY3Fo4z$&+M3k``oC+`+x2a&I#1s!>TG>$QE%0^GvvP&i@|14vj+W% zQEkH~jW(ldl3SD98ub@Uf4*S4^nyvP(O)pDCui!PFsl~*6J}M@U#L+lG(g8A-5gtM z(_Sg)Y}}GByP$Ke0Xj{Ym^!AQ)4FTa0G(M7bdF9$(7~Zek>w50*ADtSz(5oXz5|A0 zU^os&5@7oWV6+pA9Rw3eu=8Uu*#&lWgQ*nQa}-SXfW61TOd9Mz31)l2!9FmT0rRK8 zLO(b>06NZq&U2vad(iy@==l-!{tWtl0sX&%f#1O3?+|oCe^Aij039a41egF5U;<2l z2`~XBzyz286JP>NfC(@GCQt*M;bx+; zQJoZ+M(mucpVUv9ud`T~N*JOeb(txBG*Hl?pd*BRwxxN2-DU|$H^R_ueqh zFx<(^zVQ5de&=}3ndP`I&fv_sV3;{GUNEyeUO2*xXBc(%?4d@C5TKNR5dzj4r9#9~ z1B4nNlmGz&gc6`&fKsE@8laY-5hB#{J_~j?l+dP^0ftS_68 zO~@u>6S4`}gk!7;^9VY0Q|P8p(ki2*r3hXKdb#8{fy}InRgwglm{0I^XNQg;)DKpqjYYM(6QVZsWVEW zbpEN&+9{XAcQBo&T>Z1$8f=RChQ77j*-lUw*o_vU|2NCmjdrk1r_#UI-2Wc=UTc}% zBn=m9r7pYNF?9k`(y_I!4@grdd2Pd}nL4f6!-JbTZR_tdb&P83bN5O)W@+lkfnoRV z>^F6AXwqnD149WgdaUhR2SIQ4W`pz&t)*v17>@{ zTn6m>33U7dI&XlkU%}d2p!+w_^HxC8`!-nr4(NLqY+M5Rmx4{p!N3YIxDpJl0>kft zk@vyq+JH25V(S7lb!h4^0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqPq&YmvAX ziEEL@*jgmsv6^?RF8Lj+c@pv@M z=21l3GDE+sbN3TD>}GHu<{`bEN?|BK)X5{~%(+XXSqPD)n2))mXy)|$KUGw8tcp}< z%ExLhYRyF;FGBse-6`XQi!7)lTXP~09;Etkmds{3;CnD_}TU#26&d!Uw zi>~OjK46$m@6C0p^{=`sq+%G)V^;Iu=gu&W77H-kL^FtZKJ2Ekkl*cSpFyFuq3(6tw=-3Pi4 zfS!Y(_b^z01oRyP8;^tj6JXOxFmM_So&iH&f#GvtQ|r025#WOn?b60Vco%m;e)C0!)AjFaajO1eid{6QH9*M~99M9UVG4 zymLC6kWI)YWD~Lp*@SFDHX)mkO~@u>6S4`%SQF-Pbf~9LPobw}rk*mM^c4OzW6PH9DtX(2-C6ppMRDsl}ArMrVx9mZ2P-e~-~AuA`HXdXBBLWvg^_ z)*m@tbVsM3AD-z5}}61#6dp?xmn-Ip|#h)~^J8tH8$h zK>zz-(^@dF4h(((hCT$tAAymN!RQ7s_9+Z-Yx`E|=%jaS zNp!5mQ(cvm2u0rA}q%pP%iMOlf z?W#+DyK0_#=+Gcr?jQc}*b^*XmnL+4iTO3@9S-gJILr}fy@DT5k1t!oEN(Sa$|xy*R)Xq~~Q z=-}9-$r1w7yTG0>m}v#G5iqwO?2Ceq!=UpB=sE`09tYhgK+j3gdm5}i1Ny!K8_$9M z3t-blFz_`P{00nt2Zp}~BR_!AOJM8@82=GWTm{>&fytl2j$gpk4Y2E1FntT``3=mx zwM{bn_BLtg1;waA)Am*$R=bHvI*IQY(h36n~+V&CLCi;n8(oJn*8yzCZC3d zBKX&&b63s_L0$+&G;}yNr?8Phm7?>d0cy^GFXxVosZ$}01=u+cui97^ieWq!;LWsi zH0S4RE{&a&kz$Um<}~b_4&OWfi}i7R*v%ty z*v-=utcDlthFNOeiv#vPZ#T@r-s$>#r{cX&SttH&+MBOUd%pVR*Q%@S%ItZs%$xuE zJazGv{B};;xyw%u&(5*ftZM6tVc9u2Hfgae26Ha3ZyD%V0XkQLu2o>|d!YM$(6biw zt^?~o0DT{VjUR#jkHMx5VBk|QxCsn>28KTeBU`}e7hr5F7~c*ic7W|)g2|m=$8IpS z2khDlruTt82f)lhFnbux9Rd4}fsQuNnSh<+Iz>CjKs$#CFaajO1egF5U;<2l2`~XB zzyz286JP>NfC-d70oplSf5i1iTz@pi)*taU)x1r0$!}B5laMDNPeM+O;M52cdg)n|Dgd*BG)1`Hyi9^+zXns`_YZhpt z=3mm-S!B>e+cJrk)e;TsCAt;4>ruI9irg_(6N_hPqW2k1oPAakt6|irtLq4YCJ(P6DoTM}42%R^iIb@J?WNwer znV>GP8!bZrZ@+ny;f`3hFcSc?a@_9JtfwG)a?19-A~- zHh|tw!TL?0?=!IRbI`v9Z2AHWYz2ec!O#vc{3RIK2}XB=u{~gXFPPW|wjThK2f>cR zVCo3ibqq`&2YXI{nUi4lG?+UB_I(99&V$Yl(A5dnra!`F*;0u2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKtqRXkhlhkYmmm+8YJGW zns=)%`Q55{67nSENyy<593H_YWD~Lp*@SFDHX)m^{ioP|df6=!_^$C4zy0bN_GxV>^)F<_A zxjIeXD(Yl?Yq_(XAh(K~SYuGr2K|XqZN_&RZAR53wMxl7{etPv3nsNpf5EJt zpQwL_S+(fjVOBN$g)+5&n9e`-Hh3f|17r#o1*@En9l8Ov4-jRq@`nP-V>Cn z&RbnaM@`jf%^n_H)oI&qwpQpyER(=TA)1{nMm4BZ05zk!jrf|Aj9|W6Pz(5EL z?g~m#Cln5HlnzJfFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPshoTNe9f~>> zb$Hu!HX)mkO~@u>6S4`}gls}KA)Am*$R=bHjO11hs6DCrV;VY-Njaw6Hadn*z^I!z z<+_Pul(*lV{2W$LmE1%6Ws0Gsb+bm5X3f)&S$9NmI@p*4{Wrm;J}^+TLo(<9LyN$$6O1edqplrN(TOeFK}DzZ%P(pS zqO||`3orpDzyz286JP>NfC(@GCcp%k025#WOn?cLJOL^?T!h3$NL++8#ug#*messv zb;)m8&6AKPAx}b1kl+LfHX)mkO~@u>6S4`}ge7mnJc>@4BDkiXYx>J6x9~zx1TTc~ zb0HK_(W!!UqKTH(5)JDmx)n{dJu3H1kvpbpV(|=(7LRT)O_v7KLSfQ`X>jjlZn1Tp zS%vvI+EAZ+zc}6h`Z@PZ#xdH=YUhdp<8lTUbZ~4^WLW{mR)XJ_OSrfju9CnGIm}Q!uv)?E4IKG=a`$&=mk{w}I{;=xG7HA+UZI z=nI35t)M>wHth!kQ80K248_3kQ7{q*qitX;0me^(iFUC4ECiiol7fzbf({d40!)Aj zFaajO1egF5U;<2l2`~XBzyz286DWNG6m%%)P|%^ELqUi4OJ@_Z3E6~fLN+0rkWI)Y zWD~Lp*@SFDHsKg+!aRZwm*kJ1CHWLA6v4kHl{>);L0$+&6m%*%H)mKggQTG|;88Iu z!cGAbbfhnH&!Ylgnn(089Y3FvPC=5+_&^w|C=oa zo6!!I=~Vjnn)}})-)k+io1`gYt< zpyMadnFU?f!P*??z6pBzKyOWpWW59QEdm>zpnoyg%sPVFxdch_`uXBU{_;{lyuS?TR26h^b0Ubev4NA0!)AjFaajO1egF5U;<2l z2`~XBzyz286JP=*Pk@pRmmzT(5|<&3v1LfSSv7A~UGke%^CaX+$diz>BRD&PO~@u> z6S4`}gls}KVab~?kECN(1XuNQRlkvX3oisk@In|r7eWyw9gfW@Y-CWS=zM8_oipIe zxg#U%R0u->g3e$r&fF0&Y37W&nj>YPDtxGQlpj^Xs@pX}RrIYv{foXes!!y8`c`{Rdxm+ zp@Un3z+!=%x(p9+rhpapd$o2cY&@jSlbG^BcNwL=#7H)hd^HpY&;73 z<6u)87)XG@Q(&ka44(xfNicdIjCFwVPB4)I+rI^qU0_Eym`a0Pm%(%o*wYJUGGO*6 z*g3f@?HrEKVFFBm2`~XBzyz286JP>NfC(@GCcp%k025#WV;}+AIkaO|(9z{TGblsglROP#F3jv+#W2M4h|$6ie9A3NmztG0J8pJQvHuKNS3sVSH{; z-b_iS%&N!{H)i$xL@5Xv6!;*AxY*J>q0p@-M`)+}bnlB}t4$!p-taXC!#h}LpdY6Iq zwVy zP3OTt2N>)GLn$!)Eg0znqupRE4aP5ni5{@M7ffcrj-S9(7VNqXrgLD=O)%33W@|!{ zIS1Ic2z0y)I+uX1rC{xH(7gimtPDv*$Ga*-L#OnsFKP^;wEy@EFaajO1egF5U;<2l z2`~XBzyz286JP>NfC-d50UA0qbZF?%(4nEjTc)!K*@SFDHX)mkO~@u>6S4`}gls}K zA)9cFHDMk@himf3&zgK17K-3slg?ckF9dlZ6w%O`$D^zeluL2+`)A$1Hp~sgTDjcpJy)SjufC^$9u3rd*D)nW1mX)F<_A zxjIeXD(Yl?Yq_(XAh(!>V^Gru{fSX+#&;TRM%5&@Cb>20FPQ%Qg6Yl+Cbe)$=b!qI zESKMbA6c&cS#AwBMI9MQ$1VLFTSE$x&c?IXMorRb%^n_H(y3`2An6Dx=`?(M-xwW> zT6fEEueoED19#e-CMmPXW0NwA2lTE1>%E|FJ=j@^Ce9j zs@6pFs~Sa~+aix5+LjsmHJ!Vk$U!%E3p)2de@JDgQiuW^or1)gLgbsf-%|+@M?B~u zf3XqLbSh#0Ja9BnC0@EmEgFR{<=`MGxs=<4s_0vT`WJm`ykqE$EmJ38)Qy~S^$oc- zN<(L$se}Cnn?--!V4I*Wup2Ex|8JK2jdrk1r_#UI-2Wc=UTc}%WLZ4JT4^;3VV^9Y z+HDq}lyq#(%{!&3vt<9$D(mC=i;u_`pPpbfykIxXQtMtEu=jbpVGj0A*WWu8?|sTT z@o&@Kd~MqE)i1wRU2Rup&wFLw{MYBHtq&Nd(|dEBYW=JJ{VKlyF{}CS^XLELKi^oh z$85FhuTR1+s#t{dpqeM#>*QEu9IY(_?45p52ZttwmTy6S7ueJd2GU^gG8pOs!@Xc6 z14e%WV_7hM9ZclF_M2d`5A3MfDVcJBU5mi96YNgP-q|qE2YbPEOM){UVH#->Q|r025#W zOn?b60Vco%m;e)C0!)AjFaajO1eid{6QHO=QHP=qMIDMdymLC6kWI)YWD~Lp*@SFD zHX)mkO~@u>6S4`%SQF+Eb+|Nt{4C9=xo3*pF;x?bXJ~YJbdPDe^q8ctqCri$Gg2q-lexu~c4if1={(S-d5~i>-2JcI zd*HKhm=6?bM(CxgKc=7axbE|e&Lo{JL-{!)Gf5{P)f`)EAN-v3mXC|>=d@fdvrf@9 zn+j+)lPukeP4<;qv-I>+!}D`6uShyUkLf?`PxxUL%iw+v4o!M20Wh=;38h? zd;@yF1M9yBeLsMWmq7m&u<1uIa1{(*14BQ9;a|YW4KVsE{G8Y=`Z*k=!vvTB6JP>N zfC(@GCcp%k025#WOn?b60Vco%#y|q}bGQPDE0DMXX^gEv;{B?5zv`0TubL+zPePuA z93R2)5o|&>A)Am*$R=bHvI$Gxgn9fNgCe+~p9}g;G+THfD1sNl__+{@=;u^&c+Ri} z2i-5VRNSkr+^_5u#n&ldjLu-r(EY#F!|$o0pi}SwozWC@noA?-M5LQzYdgA23OYyD z?JT;W)0@sO=(HZ&I%QBnr`Kn`SH>~h%&Js-?%g@Kp@UKCTMs%u1f3s&u8+am4WRo|(6b5jeg@Wm4*Ir$ zjbDKNtzgr3Ft7s*ehG$lg5lj@WDgkK3&!?=@dIGuAlQC*mo#*eM|N?74kzd^0Vco% zm;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oh7JuK8agy|c(-&mA)Am*$R=bHvI*IQ zY(h36n~+V&CS((iu_nx8=x|N`_*s)r!$J}KYtp$Z=Y=3Igd!R`9GO$txS&eW`O^J% z&fRIp&38R3BCNIF8S;l-o>qEFH5tLm!hG{L!?JU5Y|>(hgUL3qBLSvP zfnDuj`YhO!1T*KsYzLU@1p88;<2%s#J?Q!Yti1%fuYjH(LGM+t{u=1}8EpIo^xpuR zegy-!z~FCS=&jw7;kUuaJ7Dx(Ft!AYF9j3J!S)qkawXWY3QWBRcD=t_+Bxa9yJ_c? ze&t1tL6r6%e*q@I1egF5U;<2l2`~XBzyz286JP>NfC(^xk|#hrhwG2H{)p?3#@PBJ z-lm$jsV@0#s(BLfB;-lRsS%tS!6sxAvI*IQY(h36o3P|fn8(f$ir{j7F6S?!*TM@y z5xfw_&xKG#J7>DIPBd|-S`*E$YGTa-P1O8L8as;&nrK@l(Xv`XPMe^`)AneUXqmE5 znED^H@LUY?D3-9J4b#w>1$U-=q#v)~BWUQ1IYXyJ^K(*C&aw4&!_ZlNEnRd&=Rxyx zYEof!P(!ELGhlvBxy_<>ZZHmKeh!XJnk?(VbUoP90A_q(_7gDI2=;9R9iM^D&q3D~ zu=Weky%qFq2faJM`Y%D>POxz|=-&f2?F9q-z~BKebPx<51|vtn=rJ&M9E_g;6DPs; z(_r!p*zpyZItO-L0Mi%2p0B~oH(>TV7&^J{Y3LYe=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fzl^HLx+YA4ILUfG<0~cbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa z%wyVCsM(gs)@xjG|~HvCeA*qiPq;dvGD~> z)V-*Q>$5dlJi5VDAq}Q}J?H)>l_MtX%n^nHX6QUn$*CMMm8Npk4ISwORpCReqx`56 zR(%YPP!)Y^Q2(NDjd#Z9jIA*`LrOYt$Y0(dO_c@uc7nRVZnOyfzgg}#+QBlNO8;JS z|9j+ntz~wTW$_GarPU~eeX@LNw^=gM&#~3)2un$)r^E5TS|8V6d_=zZ^aQKn1-oIE zTKD3Bz0cbXbFg>1{@$s0?^D)^f1CE^Ytx>ue)+ZPYP&Lf-YfIwzdldxzLbBAPQ&I! zgOAavs~aHcSZr3cH9agz2gfF5mNb~V4EFVaj;o;a8tD2Nto;Ra-vB+og5FzT{coV} zt*~U{+o1m)u<2beumlV)1w+fh@Cq=p5{#|_WAB0S_rb(kuzekv`~d9u5KMgpc6|({ zH-J5#f|*TV_A@Z|IoP)abOb=>wy>0RT){BM=#+i|M#=Bd%3pv9FaajO1egF5U;<2l z2`~XBzyz286JP>NpyUZq(%~{BE<@rnq%pP(iFd2!-KtA|w`!h*JPCOca(D!XN3aRm zgls}KA)Am*$R;d#6Xub0Op4&Dey-{_Q*Ysgpa@5y5G~ed(YgWIy>^?3h;Ca8fx=O@g$d7kYx(`&?3H#l1`abkrItrJwH)8Hb(ix zBDW^F)$|w2)P70lpZXih)iV7JNfC(@GCcp%k025#W zOn?b60Vco%N}m8F9ZEWsbSUXi(&1gx*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL4YV zkEFw8`Qv94kbX??&To!*=4RO?^$K|Cr3@jPZV|9$@afBa_xPwUzNj*ft%v-ZA0I##vw%z&L% zIdZ4XX_79BJT~dFybF4lfb~m3-*T{V1?XQ1Hmw2!?}5Sh!O&VTybg?f07gFqV;_O> zkHN$Su>Dgoxe4s}3`~6vc5MOEUw}PZ!OV6ry93O93HI#-9bwSf3c4a-?S9Z51wDsA zZw#zI3i{$;V;dYDe}ayVq4eu7N`9AC{sK&Z2`~XBzyz286JP>NfC(@GCcp%k025#W zB~O5k4p$*@6%tn=jj>foyk9l%S6%Y^Rr4g|Nyw9s<0Cjef=$RKWD~Lp*@SFDHetz| zFpr~CrU)+T=c4{{+AX{g7&T!WAm|MEa&EDGommA5IuGO7-1%MwvW_VHF%xetf}^7hbF|M) zctxZAkTR1rUdnAkRrIYv{foXes!!iMIM^8Sx$rgGhowK zVBj1Wya0wSg5j^h$Twj0J23V=82Cn>QUDMfwY(h36 zn~+V&CS((`3E6~fLN+0rkWDzonlO*0!*%%-RA_1m3Mz`=g)puzgd$oxl~O&?MBR&; zxISAGoiAzPP_-tSU)98#1)8Y&m(+F^Nq2`T&mE0PYB~cR6{BM96fj3;7_rfbaQO2 zKSI#SEKe0((CJO*7j#;WZJjcxpi{HQY`sU&G1|;(+y9%_#mT!~k=_1x$Pawr>TK+rf?^lfLVxaRV z=!%22ZJ;{=dQO4fcCh{|=u3i)=Rto5*whIIQef~~Fw_NxyTM2rj9!MI6YHU%!!bHc zfC(@GCcp%k025#WOn?b60Vco%m;e)C0!&~GBtSujOOUt(iA#{i*b*e(t(teMF8ST6 zc@pv@LsA zcOeu}(Ba&i!e$0lN_pvSJLUf8y02mwFjD94FMr>7oeCiekaGqTY3~13hr4GwR2=hw zM+NrK&yj<5h900Zntsmaq5PbG56~&DpA(U4j;(F?Ug_tYym6!Ge$L`6`Td->bC;hS z($9HJAE5Im4A8My%(im1`9niq7bovdDYD38lOoGiFm?@${|qL60o!kY$zQ>aTVU!p zuo_D~^yI^(+m|F_=Ee9QSpwkVyR)e)3(7guqctP)au)ZGjHGqvi(Ekb8 z)CdMPf6JP>N zfC(@GCcp%k025#WOn?b60Vco%m_W%BprAuRhk_0T9SS6S4`}gls}K zA)Am*$R=bHvI)mn6Xp?gxc>cju79UGpa@1szs%_lJd-Jtv&sV?v zT6MKunLY27dGlYNr#@(cPE9JT4r=H$Y&PHP-IT-7k)w<5(RQphtJ>x?<=rVw7I|#a zWZ45I_ktb!z|;Y->mZmu4E7uWGsnQ}aWHoR>^liM+Ck@8(3J#h&x7s`(9;QeQ(*nK zpsx#T><0a5u<0@w=mCShU?>BIe*z;}FnS$~<-qt&FwqCL*F+?f4zObpm~w($izCv| zNxLF6bZF=>0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3> zshW4HF8Q6Rc@pv@L#cwRYT!p;0f_GuYS!`kTR~02tT?27_Rz1q_G4$SyD%24k&YJOU>6 zgY8i;c?j%?fvKZlR~$^YfjtQ@a|+D1!_di{rJ-Xe{pyR7-=LMh025#WOn?b60Vco% zm;e)C0!)AjFaajO1eid{6QH3(Lx+YA4ILUfyk9z-kWI)YWD~Lp*@SFDHX)mkO~@u> z6S4`%SQF+kbSS4#PNAh$MoUW(yb#9Kg-}F8XQs4HG*R;}O>{3ZXrgVIM9XT4hV>HN zirn?6+(U~;8cZ{#!L(4AZa-wL3@~&Cd^xwM!jAm7!X+JPKQ0uO(Rs|o2U$n?Q8ntH zC=XD6sP5wkRnfNw^)LF?IJP~Vf68CdD96T>$*oCl|17r#o1(s86ol5^+bN_qfd#z=5lazC;mHGgjJDyHPDmu2Bjr*jhlijs<)I6Qm?BT&Z zowoG@JROT1np5-3usj_coAg=Efw>D{-$l@o0-fK2t}d{)8+4~Z&t=ft1J?I~z6{v- z6X?%^P1nId4h-G|Lw#VlW}jrl0Y(>rF((*b3?^J)`!Xu$tk+W_FXgUMFd`N4j zQWy%*bn>VcQ+jzj7>wz@1FoiiK1 zD!QZ7`hZzFy*JmX*1zhqkSe{QrysML|2}{IKkUlvd9Tcy|N1<&u5N&%W3^3EJ9nD$ zj?KxtQ@Skj*rdzi2VI-N+9uH540-~fcN8z|ej$90enX zz-SDN9R=fYFwq9KC&1(>u%jJJodvs+VER1R(*b5W!E6f5eGB$=fsP+Q=Oxf}1+4uM zbYBHM*Wl=Qf2O0uSvpLB2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKu3p;4jmmj zI&^e+-*h%1n~+V&CS((`3E6~fLN+0rkWI)YWD}0DCd}jLP*0(rLQl&~J!L%UDg0~7 zzh)5~ohpvcDRgwubg}V;g+fjk%CG74D?4+G?d!}cNYEL2p3QV9HIs_?sK8Gf`g4o> z9?sDzvntZSF{|e%N)^W_pIGG9B)6LWLYcZ{C`V^x=IFSko?~m+zh62!Evv5<-O*{e zTxOl3t2h->#j#JabSpMF@k^~)dits1IXXf*It_8dKpkh2+8oF)?nswK9-DMou7lnj zSbr1r^?{8w`z8GjuxSw(aDu_bV8{iAmw}O5Fj@!3++ch)nDBt@Yrv!z>{t(`>cOrC zFzo|-J^?e0V0I&z^MiexLB|%*`330O3f681-8(?fm!NkiSic+e?ExG2?w5{^f8T!2 z(JB2sQfmyNwEy@EFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d50XjNdg~U}zT!l2o zRw40r)x2GG$!}N9laMDNPeM+Q;PePKA)Am*$R=bHvI*IQC2zt!j*dYQR5Yk)n5by* zLQn)RgzR!~u_1T)}d`T0Bsx{I4swUPf&_vC@G||0CYAJGlj3x?| zS+sHdGo_?6;88Iu&Q1YSbcSc&EP!7l9_FFp+lOc9EO=;3NGp*RBrTY7n@|;fYf%59 zZ;k2``qp%3YgT9IUze#*>f3U4n!Z)k$@E1-lwb+|2FN-*QPyR{qk$o)&CiW)0@t3=(HZ&I%QBpr**B_ zYS(}MB&a=#MaXoUpT1YnnS{wYryeBe;LxPV5(WK-z@``&I0^>iV5kiYC&0)lFxn2r z&VunIm^cr%cYw)GupjnEVpyOxI`3vZ}0oMKsx^ID= z-$3tM2PEs?27T{NfC(^x(kDPchk_0T9SS-Wba=ybHX)mkO~@u>6S4`}gls}KA)Am*$R=bH zj z`>ZBfpBvAz&2WyrQp!sM%$xyV&K((1r$QJDkaHg1v@tFe!+0#fn}=rvJ^cHt{+M#k zjC&0x9->}duw0WiA_%mu-|7SOR1bnXUS zd%)VgpnD(aIRJVOg7t?%-x09!80bF^Hk|+iC&A!pFmwhCe+5R)fzb<)b7B`M=NL-A z@S^0mX5}xy1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$MODCcnb5tkov`Oz3#e#Dzp z^Cs0LzezPuLY{;?2{|)@Gb7l9Y(h36n~+V&CS((qyb1HjIYJR!&Ck{RWz<@DAt-_u z!uYumiYVty=fs?Xrv+6>f$46AWflyTfx~p}-cu}JM=MCs8HQVP_fh11E|s8#!aU?- zI^IJ^M-J8*dVtPoIyyrR(0M}&Lk62gy*)^0g1W$NvNI$a<%s} z!@WBPpP++dlQK&RjC~8nyTC*@*q#QHm%)x6Fx3loWx(`LU{4mzTnDo`Fn1H|>jNEc zMJ1hYgRXbL+IK6S4`}gls}KA)Am*$R=bHvI*IQW2_1DNIG1WKYo_wQ?gJ5 z|C-e9jJy!!g-}FEN8#k0f~N*mh&2oHDlNBnR*smnGe?L5rsxdq)C?)zn>1OP!DImJ*aoJ8U{?#64uL(pz)Tp-wt~3`*tZ{a90Z+*LDvzm z_890s4th?2-jiVcY0!5DZ2SuJp97mNfPsr(@M|#i4H*6ojC>D9e*j~b!1xs~@gvxN z6--_OJAMXJzkppgVCbZOrJ=(yI!u5GFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zLx*dSxCV)9kjB^=B;Ku>cdIV>-Ku#K@+9O*$l(zj9>FGL6S4`}gls}KA)Bz|O_;~f zF)4ye`njavOt*y>f+Bb!jGqgkh=$HgX`N`I=3koVUS!Zj+cJrk)e;TsCAt;4>ruI9 ziri7K$)b(ZpE*F4+T%? z>5N@Z=buszGRmAhgjJ}`3t%pN=_J)PX)gPf#O`n#mo7({9R@fTnMOn?b60Vco%m;e)C0!)Aj zFaajO1egF5D0u?(bm-~O)1jwBPltC+XA`ms*@SFDHX)mkO~@u>6S4`}gls}K;TUVe zJf03$=8vD1`SdIl!M`TWI|DBSc_9?h)2Wc|i6+iItBKa7G?`h-3s$8=|F zR%hs6m#I(c+j4c9zE#xGoTU>m$`3V2CkHN0K1XL{X6a<4o@1+7dq_Guecqb?-TJuI zD1`kH%i+{ss2h7sxy}3@c{#E~e72p4u)%^GQ^Z)UmZ|wEm=jd3~*0)W=addEO(q%ac z=Hg&q8|XL*I!}YHGhppkp!*!?xd3`Eg7sg6zHh+B??C_eVABs^;1U?T0)~DB!&kw` zH8A=!82bf`-vAT8g6+4!NfC(@GCcp%k025#WOn?b6fzl^HM~ADBxC)7@kjB_5 zB;K-`x2!JtEvtDF@+9O*$O#ghAi*YN6S4`}gls}KA)Bz|O_;~gDN_U&^>a~wIqeo+ z2#Vl^Fn%tCB04%%9G_F@=%DFRVOl6mnlRmM=iL8%Zn1rxS%nEY8jKn1s67_ovp7U& z%ni}mQW`ABX-S-f5E;`o~UC?Q{TxOl3dpH%+!?91YbSpMF?n|v%dits12|8BW zB(>qtuml|(n-p2rfvyk0+7ChZN1*3p(7OSw{}l9X0vkUA{hxzPTfo2)x4DCkhop`b%Shc`@T6S4`}gls}KA)Am*$R=bHvI*IQY(h5S7;C~jf)1DDkDn#^ z6f6|Mzb2JC!3#lN2t^chDmgf3SVM!Pp)=r7F)G4N0V8wHx(u~dGmVS<{Vf$ez>9l6H6kXEkP3M<%T90j=GN`0e zS2tjaj)0`|w&C6>I(c_O=jjcbq|73ZP0B1kfZj`B{T0ynBiMKq^j`y;eg*@-fWaGJ z=vOd&3yk~*M&CLt8G9RyzXK-T1>2W^$)#Y&axk?5>{{>9l4($5? zbToiYAL#l7tZf9{8$pjB^lk?0n?PSP*cdo0B_02^!O|(9zQR2}ho=VaA(f~VWz?XAJM%Jkih5`hg!KT^Fh14R7`G3gIv=WHIz&be*n6x+`6NioOP{B}&*IUjW#EV`Yu_)2~|r|sP3 zCx^6i?7BuY2^x`N5jLaR`;4_h-!~Ea%H=+xihF&XNm9+}JSEi}9GFyCc7Xmb!KR&H zU^f`t1BUj3;eBA_02n<8#twt=BVghf*nS*Lo&Y;ef~nJB*BLPV71(nQ%v=Do7s1@u zVBa^OqYHF)gRV4Kdl_{1fSz8^n*r;80)1Jq@jB?wflW8TKpz;aiAgml#F_<~sQH)Fbru<5#RZ?GCC=C6NS$IBhoX*{Qie>9) z1?f4%uy1Cx2TWl+X?Dqt0ITg z7}T^ue_~XdXUJEK(j76$tx0Z;`U|FizhJuaf=MmYUoflZC+go}RxSE>m{m=Gp-e5& z{G5Q4b8M|AVCXnv7mIG_JZOGSO)9JoYUnh3%+`Ah9iz>xw*7yF8#*{JX|lWnhTa9k zOTfrdFuEL!tpMXI!Ne-C{XH=GKG?AqOsxaEJ^<4nf;}IBnUBHj1~B(2*tZFEYzCc8 zpsN|I4S?=#peG1=Tfq7d=-UN0hCzQT*c1T+`@vuo3>^Z)F)(rzjK*Q;#M)@+7-;A) z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3>qnh`qF8MvG zc@pv@l_MtX%n`-{2IxGDabsR6hW)6*w+urID$1Lw=RCZgGcp5o zBGSyUwS9O*>N($R|E1`9PWPq!19TcTFB*J+PU~9py_0dwwsN((#Zb7PgCmn3%SkYH z8jPO-6JLSt=fLCzu;U__`Wo!|226hk_IwX!egLzVz}ywE??=#)0i8dAt}Ix49dzeF z&rQ(V2iDgdk@Pvh#zmmt2{tVT11>PQ3=GwR;W{wl2BWLNm@PgeiORalxz~1NWhB??fU4QRX zy!R>V#J^2@^R;QuSHJvPb+uiYJ@1uy^IxB*K1k7N%^n_H(Wz-1py=T5PiVluU zsw^LY$&bN~4Pfe1uxk^T{tWE-9L#J1vtNL@tzh4F(9r@qL!fIHSQ`f2t)M3YdiR6% zQP6h?Y>a{aqhM1U477p41Q1Aks5wfup=>XWX4a@|= zY|Bw8>EuF3IY)NfC(@GCcp%kKvyx-I;kI z$P1x}l1>%86HT!S)Ny+2s^zWQo zY++|sL59u)J(>skHN)Ni3Va-E|6>-OQ_mT5dd`-i^qi5Ip_7qjj;+RZOzJt#8{wks zIW3pVtW$Igrvf_ABulqqlYOPuEIs|y@bnz3)N|ULro#0c9GTQuc7wS+VBcQQu^)6s zLDwO$HU_$nf}S|&Z3F8Qpzjpe*be&7f=x*oB|`?g3&H8)(ytfVB#{^ z-UB9k!Hx`=`U&jHg6ZpEPY%r71hai$uI89zp96Hf13KS5CiNWGl4I0!O271?#vn@j zkG}vDU;<2l2`~XBzyz286JP>NfC(@GCcp%kK*S7=7`NfrGZ^$9tNrrc(_vo)(T^smd*C-rT) zI!)gy>STRuQ6JN{6Xe&5oKa&?(+2&CQEkR|8f`|^B)2BHHR>;z{{4dK&I=~BOn&~+TF zJpsB;f}YdxbG&Eh=NRbcFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPahkg$I z9Qrx*b9k$CHX)mkO~@u>6S4`}gls}KA)Am*$R=bHjbM*sP#R(fQK-ZqD6%=8lY~Qz1+RxH*#+icx)3;o~?vXUw@dn@i*7 zxTTh3YiNv1H|J1WPto0+#aHsXIc?`IKRKkEW7mi7Ov2C|#UiAe)9^p61NIr!&gZR< z>F@g!-eOLml zA52gwmad}>)6bbTD(1{9_a6JKq1>c}!uarySMWndRYQ^=CruT0_(d)_Pa=D$8qebDTjnp9XF)X!=5nD2LS zn*Zm$`!Kag{vP@L92}YSST=%wKiISx3^akkW-t^0!`r|}5RA5fu@D&F1t!8^dn=fX zfF1k6R21ww1g2wP&rvWF2eWNpE&=wP0v%^S=U1TX99Vk+bYBELUxVIn!20h%-}hkS z51{`N*mMO9{0IiG!p{j^qo2dsIZS{FFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zKZkw}{T%u^^mBNhbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa%;V>9MgI6%kx#!u z5&Ub?xHIxXkQYJ`{T$`L*tZ$3Yg8fqrG7c*{wI|qrt8cRh63j2JkS)X95HjIvKY$E zaHX9pDmqrhVz8Oj^An}^W0X%Ua%+-XO@E$PYwmxKe6O|4Zjw5UwNf{0CS&4^%@UAyj;%FvTq-)QjrB!WbhtHG=X%&h_YyrAO)(D@;|KIj!Q=;w)Z$^=x~M(6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e(feFAiJxC)7@khltIjIBcAt*Uve>XP58nkOMoLY{=29Kp#EY(h36n~+V& zCS((`2}|CDc^n;+BDkoZi~7y9TX-QTf)~R0xe$uz=*;BYoPuWt&6G2P77D}dhn&lG zzov8dp1DO;cI3wupy?Deq(&b@V_qnV@R)@kQqYGM@oj%lM`yCsValZzGpb{B{wY7y zXe(FCf3U4n!Z)k(NuH-Lygh7JwB({icUh>IkwIYC^|=fJ63c>r}Y72bb4>DQ>}m1 z=lQ5|p3l>dS7g5K?5{SMIgCD^zV^zR0n z_JD!CU~nH8Isk?bf|0{u^avO`2F8zri4$P^NicaD>^K9az5=_>LD5NHprT`-qQeB3 z025#WOn?b60Vco%m;e)C0!)AjFaajO1WKO(6&)@@;vyt2LKka)Lh-mSXicdO<} z$diyKA%{nBcm$h}O~@u>6S4`}glxi+H(?${r%VxC)6X^i<&;}^A;>8llv~Elg-}FA zrwZ1ICR$cYG_05CRy5J}sN6F}?wG2H#WOToJi5U&T^dYs8jOayG5JNE+xzDhTi2OY zke~A~n$4Z>Rp9Da3V+PRo2lrGIYnp7P>RkQ(i<{JPvy3%GeKQoH(G@L-z@hV?O>Ts zrGKxv|2^`()-tM+(y-He$mpW1Dfl(chfy_-)+Md$NN-xXcaX}MfxouYqU1wL<* zrCYJd%2I2Vo_=b0ijKu*Rh#3(QFL%$8tl6aI(`J5S3%b` zu=Z!r{R`;10eXK0>u-U+-@wMVPDuLS2Akdi1Mh;tC17YN7+wxWR)Eo!U~Cl_e-BK& z54Nublk32a55UxiVAn@r`eU$X1DN>~%x*d%6`kB?Cpbl?^vf?wev4NA0!)AjFaajO z1egF5U;<2l2`~XBzyz286JP=*Pk@RJ6&)%%RCK85@Q&$hLN+0rkWI)YWD~Lp*@SFD zHX)mkO~@u3V@;Sx(cz+e+9}ku1nm^L@_yXdDg0~7zh)5?ok~v58P?RG`<0Z6QBigZ zn4vQax26yaXAb-u>mZN0c$I?Am=koGhZ1x~W`<5ix;eI*Um@t!p1DY1S6-x=ov8f6&ODUCN6;O7s2G$V8=IL>N~LO zdocY2*mDWYTmiE`g1M_;-!;&Y1)bL+=(utebT~tY2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?cDfdnY%a0wEZAaM!O7+ZqGTUGN`)g`}GHBUmGgggm3If9cT*o16CHX)mk zO~@u>6PCOQ^9VX-MbOWnpJAk*!3#kVyb#9Eg-}F6heLA;8yHk6I$s)GN0~b^qE3Yn z1;{yri8O;aHU+*k9V*VG0gnpop`SD6{G820`8gvqK*u4~99!Mtg!FTMjJ)@MS|8V^ z-8>?v-8?iQqz4_X-=c`|St-9K-%%1nk zy!o%sQx{*!@8`6gyZq$v{2U?uoSMjeevVmf_A%RIFrvp|nIzxXdCDX`7I|pW zW4Q&oegkXYN=Ulj20ib9-gm+JC7^F9*ti_@uK=4?f`L_F@I5f}J{VpLM%IDR55U-m zVEiL6@iExG0Ze`hc5DJupMhPUgXt|`&lg~3E12C5=5~O6UxJQZpfe1*TEW@~=-v-{ zq6z8ecn>8wK!*c#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz2;=@X!zLqCUp4*eYZ zIlNgqn~+V&CS((`3E6~fLN+0rkWI)YWD~Lp$5<2Q@pHH$fBdY-r(dB6{xxabnRp?{ z3!#X9&UERWXyQ<{CYoQ>#F_<~sQH(4b`}{l(Y8#YWwk`ZdWmjD?s`=2nId;g)x_c% zn&^E-6K9{*MC)_ff59N0DtYV+h3J3G!t=olm0~G7T0xG^K!Ij}PgBG%7eLJs^0ylN z@QeHE0z_@rgt>?ZQs9rO5|gauQp6FeqHhiAU-Yd}eL~-w?rhEK4E^ge^+|nOu1?dp ziaJ@}TGYq&?F9L?VlmhZYTBSbF{;h@PNU7Jn&j3bw?_R1)4yLZ-Fd;Jmgz5;)$;6hWAfOL8qN zfC(@GCcp%k025#WOn?b6fs!XcKZh%jxB`hQkjB^wB;KZ)x2Z1qZK`<^@+9O*$f*&W z8o?%H6S4`}gls}KA)Bz|O_;~eDOUs+^m9SKfo2OY1V!*d7(W+65&azHzu31Ku4_~w z{iS|6=l&;^L-{JdTi2N*Oa;u)`Hwhs|L-;ud#Z4}(v}(3N7WzG&v{HbH03tS9ZzRm z`Z+$S=GdCQgP-$-_dwD8obF5c2k10xUNrasonD{WYS;aoN$_(N>E{^L-oM}L;tVe6 z;LxPVvKI8O1Dieo10RCHkHFB!V0Z%<`4o(90%M78KDZZNY4%>dKtYEIFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d70SY=4bSUUh(4nBi zo29b}*@SFDHX)mkO~@u>6S4`}gls}KA)9cFHDMk>hfDIu&ysu!7K-3slgeEgF9dlZ z6j9Kb$)Py~4-A?q2L>$^rrQsBA`Wi7%q^;~BR{TiM@LSXD(>)_L4MjyeXb2PopRgI zV{~qh&lzP+=b!qVopL#N2a|cq)j!Lv!KSEh=v#}rK;KSK7uby!q5n6_*Nt|tOsCSn z*WCXe`Ce<8-6WM4Yo%_#Ooshpvjn7}V{6@dN@_Z8=Y>(zbXv2A2iJ7k)|>BDa*T3j zjvNto@6N$J9UPnVS-uBDKY-y&VB`uI{Sk~^1>@Ji#Lr;+FJSTp*zqfvx&?Oq2BzOS zCE4>fn0W`xz6<7-fPG6rM=j{A16^*gb~WhsfSxs=*9+FK2YvNmV*}{-flZ%)fkrU6 z5e)gk@MbX51V)=rNlzyhIK@FarC)?rwxDtsgk+>3RjIBiC{i=Dt>XP5DnkOMoLY{;iAHnevY(h36n~+V&CS((` z2}|CDc|0AXBDk=h3;P8P7hVX8;Ds=LE`%a_Iu+7A(ZtzjHPQNn@H_x;XtA(QNR_hC(~nurf1f}9A9iK-yjSMUe|?@>S2w`XG0RArPYlb^!Ldn~Wjh$# z0mi=s6Fb57-C%ML*s&K(?E||Gfa!x^&tWig1k4@-bH~BH6QJW1=xhgFXTjPe=spj6 zIzVqHSf2uY--3-@puZbzN`ryRV6X=a^@8CH82Jf|X2IBXFrEVwH^KHkI6BFi({yy` z=r92$zyz286JP>NfC(@GCcp%k025#WOn?b6fzl^HM~99M9UVG4baZ&%bT%QIkWI)Y zWD~Lp*@SFDHX)mkO~@u>6OOSa%;V^ARsQ%{l~2b)&WE6EMA?WJg1iul=;%~IJJCeT zYKeyR65Wa>+8&jArpO&rHL-YxMvX^Tn5IjGX`zr2<=1rjm7Tf8_H||zB(R0FO;cUhH`X9W{yrm>N&Q~ z<8X9NU0C*iSs&NEnMdHwOtAj{?41vM6K8(rNB$?Hu*r*)ER(Ri(6p4c+iu#lo0k5` zwN;vJuO+=+n{w?n*QU2zdQFq;rI+q*_qGuN95BQv0vs^JC<-`Wh!G+fV~7z7IKl)+ zC}6;V5rPmDf(hYwxw)_Q0ecJ~!#%8BabpW5!U`rty5SZM^cbaagx!Y-zVuP-MxD z`3LVc*3m&%%2ichWE~y6H|ny?0z2k_oiBp1d0^Lku;(SPcOjT~8SGmG_AdbkUICre zVEGELq6Vz=fNn2XwF<1R1#8xT-a4>$Jy=%{*84!eA8dFFY-#|T8^K@`*s>jLZ3f!{ zU?d2(?*cnoz|IgDYX!UBJ4hX!o;L32Sn@yrBJUSzWdZB}JHQUG1MC1hzz(nj>;OB! z4zL6406V}AumgE_fIB)o3yEhT@hqfKHVcU_SIw8J&im!6c_ZYFkT*hJJ%U${;3VWE z9ii}+*%5Ns(V4(&=NMWyh)Oz| z+qj$3*~wxR9b@gA=h3ezHFLA$$GM?HtK$^PUsYDgDUzos_r>x(%6*AENx8SmH**yo z-}M?gH)a)`7?pEG{MB}9=xpAzD(4NIcu(erPGH}r@wyG2z%tD$I;FxU$2$x)bnxD& z$+92p`2g%a2qq4JeMi9lqu{`C&>02GV_?N;u(AtucY{?uVD)*hrWf?a!P*2^_X$|v z2l_9A4gFx#6|i{#3_9A0El#j?HrQ4UM&^R;6=25#u(J}3xxg+r*s~bytpXEE!M^Ht zYUuQ@Xy;XQxS_)iumkJ>JHQUG1MC1hzz(nj>;OB!4zL6406V}A(3 z;f4-hF`bi;laP~;laP~;laP~;laP~;laP~;laP~elqF$?h7Qlkzjfy1bHll8HPsx=w@=)drp`F2Ad7a6`KpcW1&0s?SYzl(SyTD)z*b)L; zTfw&Xz(^a|9tJxiVCMla)(&1Z!0jBKf5h{Tc>d8Sn}5WYspiX6 z=lwF(ybJ`FT2jG51<{N3cq~ zBiuSWLJr$Glc{xLmqIh_lJ9A|wDftqthk6^0lRO#>mf?6_iTK|3&n8K^a)A(3*O*DD@h8 zt*yAMuypPe+XP#IB$eGwUzUkdC-rlLtLYGxbS`-g+dK0lGf{tNOs|%V14E=)D5g4uExz zL&SO~=${QXl!Hxk!R87uxBzUa1Y2ETn;VQQ2HUH^j-_B{H5gj~cGZAA9;OB!4zL6406V}AumkKs-W}kQ4wrPeq{Ag0F6r>)(m4q^2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2{{QzSrTR_>F~7tTW4B6mn^Ij|7&vXu7Gz0+7WVC(y{&&mu`#}j>b`m zDb>(P|6;-o>+DRIq%4-vxqheSx}}?pFL(k?94Y-qs%~6E=LvLwO44uB@!*Ymrz8Ez zQ>C#~YAF$tY*p?p^4FF70{N@Tec|AJkvv8DzF59Txi66?Dfd?SZsopo@O}*4Td4+P zk$Wu47X`8pKUp9OA}lI_Y9#qxC< zI@g*xxo+sVshlG`a}QHP=kS8_IdAB6pUb?Aj%WRB{bh6E#SX!DY=-{nUk!3qr-3-F8;7>j{jr@@{su(umb^niWm!Tw%wAPzb&g5{rr z6_>!u&p`L*VAWNy`U|jT=3%0D7FavyFcoy_UOdbN9WLmw1MC1hzz(nj>;OB!4zL64 z06V}AumkJ>JHQUG1NnD=3pzXniKig(6r@o$1&J?K%@?cA`^BnxBjk;cH$q-Jg4d4V zB;+LIB;+LIB;+LIB+PpfW+>q)j;nvv^a#+y0pI6Swdexx& zY1N=qPlYliy`!UkOhEn$FhCv;NR_m(qB-ofrCDYHzIU5{!WeYngr9gj{{UON%5z1Mcz1CxIE+@yzRJo(&=8D-YTo_Xq-XTSf9 zJW@?3(6>jwrW4+*sp+5$MSth8Mvi_>2XBq)EEQng0OTm_E zuyqC4Rs%*nV7nLWSOs?0g0VGVR~^{19_+0L6F#ud5B9$W4m5zytzh{!u;Oj7atG+% z30A!WR__LD-UYpTz}mfF-9E7XebB!jZ1?~*ou-3a)3M}#3P#>9(#itZ0d{~LU7TytPN62AKXB<^e?9$-}?NZ<& zyR_yJyX1P*E?s`iE=3==OQ9Kd$@jFKt321LJXR^Zz>=}O96HI*wmS+AsXGb=j&0;P&mqEvATsG6g#=-ilZ(urSLF5AALRD5JqeD1UredpO{zdO|JiF=EN*KrC&k?dNgSv5z$ql5QGU6#XO zQwP|53=DRHEm5#F2DY6BBVAy7H`vhwcAf`gy`8#VpMZ%zuw zbXs3O!b|CJM~59?2iO62fE{25*a3Ec9bgC80d{~LUcGf)u)QAa@PVCvF!mPM)d2Q1g1t>(VmsK^4E6`WfgtGI4VJ$P zR_p;Q_k!+yVAcCz^?tDC1JHXAtUUzQ9Rcf)g8t)R!wInIB-nfk41NfJHQUG1MC1hzz(nj>;OB!4zL6406V}Ai~=m3OYO`|JIq3&jkys z#Q&OHxs!NDpdBHH1)T}JaE_rxgQ%gSxsAIig`F&x(9w2t#@+Y^&E%Wj(4j&QmvoA4 z)=~>C1as^*S}X@E!j#f|A>G@RCyHg?^-4NFP*&L~m9MR)Ge&;Cte{j<{x70Gxu6Uz zR%lHBM_7mvjRAHjURU z>9|~)Rdh;3sT}qe3}0GDe-$0PH!8Dy0(SI)otME_KiG8z>=^)i9Ua7k6YQG}_LqYL zb3x~Pu>2*kVj)=hGU#3eRxJUmUjb{Df!^g{?W+A?QEa~vF zIoT~4G(n0!u4(6JzJ21xly$~QmMjW7daKzyfhJ99Zg11^2kQN$H{w}t=Ulg)v;KPR zoNKL|T(@&#RLl|a%A?fIdHao1IdA98y^y(`6Fz$W9%JpCQel&QXAHG-@ZPA!5(Im8 zfxRtYA_Vreg8lD-18tylKUn?&SaA@nJOsLrfK^Aq>f>O|3DA2ItUU$ReF)Z{0sUve zhI3%k1+e)eF!(XpauIC(6l}W$Mm_`EKLJHQUG1MC1hzz(nj>;OB!4zL6HcYxbD+|J>44!3i- zox_()=OpAL{4ikUGhC`mzF+nmmEK)#?EYuT?#KCG%hB1RuH=XqN&H*?$U_;+abC{RPRAx=q8^+7)%wkL0SbW6!FgW1xU@Kgu;W z^EWI05H9JImWV?6tF$Uj2`ytYMY%7Q?@{hc%esg+`Xp!F!`Jiwo>`g9D2}=PO|OGO%JfSotdGUI|vc23Efg)~p7- zZ-BLH!Mb%|{hOeF1K6+;Y}y1iZw7-~z?Q9G>o&0MZ7{L}Y~KlXyaRUb24nAnU389VqEk9OIIXCI1sJ@_v0*7Qha$1MC1hzz(nj>;OB! z4zL6406V}AumkJ>JCJt=xTM3=ka!vrPeU4I(~$T|)qJJuykDuBH$vVBc_ZY7BY5Em zPC`yXPC`yXPC`yXPQtt=VTO`UiB;lR{XDDR!u1y35v&sL2)E9Tki(LW^{2RWW3+HI zj!I0ahEDny6K+^%XS!67#WFf(N;hL)kPP3Z;zzil^OM)mxiQP=IH{Z?TZZDlX<-RdC%Oho$X|n54&7%*OS*GLB$;xXd;;q7r@$&z`BpY`ir3dQ?TI@ z*z_6L{5crB3buR!w$408Y?}o}=78-lf*td~&iP>MC9rED*z+>jy9i7y0sCG7`4BhZ{QF(BVs_a}shAauRY9auRY9auRY9auRY9auRY9auSZR zB+Ssz;W_!Y&YXO1SXd?g*W}J!G4BYpBjm85b3ZSgll8Je_tUaLFGvN~Zq$2BFX^Zs zc_OEU9s29CDCrn(*<>u{OhffX%I5YIUY47Z&NTcvS*-;06ke8_cbfg5Lv&kt>;PN@6bQzr=k;np3LJRL;Wu6lGYjkfBR{6%1bllX>5uVRb(uwxe-L#TU zpl^?UNykxJWGhqpIAhVrv6f1-NKv$Gct@uc9i7IJIy!i3)Ma@Stlj|DYy`cVz}n4V z-4?KZE9l<_HoOfs?Ess1g28vdmfc|MyI|WMFtQhH-v@TQ4|eVcV;_KB2f?00VDAwy zaTM%34)&h_2Tp>{(_ncQSkVnu_JHp5U{xZ-l&d1g{;zNytgaNytgaNytgaNtpK}%+S#(uu43spC|Q8+-~6=!7A~NaO>;{ zIqc|+quz;KI{ctr3Or<&);waDT#wqN%a7Tm=;L-NG{Y|Wp0;z3=UR`)DuowVQY{_z z7qrfex}TH&waLcL1WC%GqLWpxCY$b!`Ze@dtKX^S>j`+{SygLt__C=#nJt~WlvYov ze7ABxhVHFYn6bz`7Uhcq**ArrDWLvHA>9|!eSz{sVe*N>!6yo-bAul(l8@b{{6vvl zs{BNeY*(Hrmi-0vfQ4E)KTuwf-qOM27OJIOYwL`WpD!yYm6ZRB=<$Lwuvnor^=MJ* zHS}6raakcXUThPT^2^;Qzlc&V6?BB}-Q(2K+1Vb&(mNmW5T~e@*V)74eQhJ3%ZmV4i-$9Rzrw1n5rMXk=T;x$oOp~d^MD-QBZfk`~EKlURu`?~Jiq7B)KbjUz z4!?a0@2ejz-lK`?R%Y(D~a90fa%gRv7}*GaJF6xjPAm^cIWodx^PfddynXB;d~fEAyB zm3^T5GFa6QR$l>Y20*W)lUVBn>t=)X<)D8q*iZpBEdZM;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRq96eOO4#8Z$)*%TzcS~Xv- zI`3Dj=8cdyLf!~@@d#c#f|HPwkdu&;kdu&;kdrX)NtmIaW3fs+qn~H=7jm_QcLb}% zJHoBABjm84Gl7@RF|=%udV!g1Uyr*fg`F%G(HZK_q)Ie5^oyz7(2>7Ni_v7$(7Cp9 z&dpgt$CsamPLRquB76!BomusXoHumhJ((Lifqk3C>o#=a-lE}UoDy1G$VW>T4Xf>x z(uzWX#)9Dwjum6&=z&6Nve0{@Cd(^e>oTxyIT(2rY+ng>yasl@4#rl4U2lLrYr)=i zVB$@%Zv)uB5ggbAIvc?9MzEp@tlSQ|o5898SRDjwc7fg&ur>tNwSx8Uf&MnIAq+M} zz~%#BupMkU47PTFZO6b!C)gfEL#HFg4INAVXJ6#~`m8K~9bgC80d{~LU70a|gq(z&gq(z&gq(z&gq(z&gq(z& zgq(zWq^LvuNj(ydd4w+f@8@ynfE8Yv-)bPdg_<#T*gcc!JtFrs6z}}C+#6__0Q?UOM zIPe+hyaJXFfEA7t#7ZaVo()!&gVl4vnhMan0IaP9>s(;H8}u&*8>+yjrC@V47+e9i z)PSuXu+0lbR)OucV8;OB!4zL6406V}A zumkJ>JHQUG1MC1hkbeibox}5wc>WR3KN@B8kN7gxe3|OJU#6NjLf!~@Bji;hc-06_ zLQXdcY>QnAFCa87CU{m5x~=rk9rVa}dSIemnmffV#lK~j4nJs@0uR}L z3fAG7O7HyzsW5e$hOZ@^TdoQ_c7qL_X*Z=i^VChR=TISND*n>3H~BVI8mp|6Bgt0f z-Xecpxi8@Lb0n$kZYtK4iP9LAb42_LG<0^)@#egtGt&Awj##s-+tBe(7)yX%pMX7mVDDux z(GT`rK|`m1fEzlzehxdp4zL6406V}AumkJ>JHQUG1MC1hzz(nj>;OA33LM~u4mWhT zp~DRwZs_o}(m4q^2{{Ql2{{Ql2{{Ql2{{Ql2{{Ql2{{QzSrTSw=uW&22W;?yO{>7>S}?cJHQUG1MC1hzz(nj>_FZf;F1ncL*i*jJPm1-O+(_VRrA%V^M18z-UxXk zS1AnT9`yOFE?` zqHu6Uog#S(^$X}y-i?9xsb8=nMaZ*1=xK^B`lFp0A_usUV zPM~j(eo4o(NK?|0sHEdKY0;K+Y_jX5W_g{l^vdXgLMpS+d!sT-7_5kZl?Om~J6LrX ztnL77j)C4zur>;OB!4zL6406V}AumkJ>JHQU)-vKV^a7l+tI$YA>k`7-ros*E0kdu&; zkdu&;kdu&;kdu&;kdu&;kdttfC1Hk=4o}O!b*ANW$-*k}zb4o2EW9Joj*!EW&NwQb z*rme{+NHolc4^HccFFaqUAp|3U5Y+#mqIh_lJ99d7kRD~d8|@+!C*%xRo1!o#dPV$ zR(2*x1zB`-zD)CGDtb1OiQAXy=qQWfWYs7dVf~*k^G?k)lgK5ix0DD;wkr1)`RmGk z!C*IMbaiz6*IP*E+5$Sc?&!Fwo+CUx=;(BZ&gQ(M6BuD3o%ogIvh5qnias(e?sMOr zwxaJm`|NkitdBkO)HBb1{~0;3tjJcTJU^D6FSC|P)PV9#9A43}37Z@)*DS7+s_5XY zQI(|ztoDF4UeLP=tgQv>)`0bOpnpBsP!Bfwz-B)fd<$%809zZuwk9yL9c*s~I|5*5 z5RB~tyIR1W5ZK!aCf)=4+Q9xWI1mAy2f^|~V8s!z@+jy&4pyB2t51S8r$FzAVC@-H zbn4D>MaPo==@)swNGl6q2iO62fE{25*a3Ec9bgC80d{~LUsloJuRA841H2oaSw@>7{t}`vGevX-%&EU_{dSmnkv%lSs7n^C?e3_q}n!9Cy zvM=qJujt&RRC;dCGCCWsSJC+aiLr=WS9HA8&Jn)G7*%xkyfyQWYC2k$jj(J2r`vg5l(Dmr*;RAo62*7bt*anPRt8$JP>`oQMPV6Y!-xdOHh zfNhQ#G2#T;pm!Zu`zBbo0j%E$`Zs|Mn`2bbY1$IwRdl$b!w#?m>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz*c!0j}t9MTaXoT+!i*4qq{ylaP~;laP~;laP~;laP~;laP~; zlaP~;lW>$JVTOthPs-?j&u#wijda&(Pl_=Eb3=-|Ck zlcfo4+7321gTVmU5(Ha!fo&~dBm}m%f*tRHoo!$&40c7po&#WSJD4~O_H}^$$H0M3 z(0LLpKLu8N2v(i}-Dkn7b71uau;wGs`!QI15v=f&0m1QnWu;? zvrbV%r*+OLZs_EH_C?;W(8>bX0d{~LU1Rr5y38zFCmymSOF9l=S+NytgaNytgaNytf<_aw~F z&?&M?x757!TV@Rm?u>9}g!fHav4eMnFSjG)u%W}N=47{G&;%*^czTOP{m2tHrl~Vd zlCo&$=rw6{N;X+OH5px;63uNK9^rycX^AM5zpAW|QzTDO?u+Gnl=~8Sl5%gA?^f<_ z&I&r~uUF8yF)Qc;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRopL5B-ET+rcy z4qq;vlaP~;laP~;laP~;laP~;laP~;laP~;lW>$JVTOVZ&wnrYa`WH0I$)L9I_|d> z@{TY%cZ3`kbS6{r#4d$q*d^c7c4_JJcFFN$D(%d+*ro6SLgQkBX9c0#N)O#ZkBp}W zCfcRBQ|wawTXyO2gLWzKko~7%DW0kH-d~WU)NLBR)?2tHSDhVuRwW&6gGN)R$>F!p zqnlG|mrA*zqcm?uS3@UHOXx(XoFk&%)6~#8e<_snhR#S!=s04{vTj4ivtF}=PAM8X zjRi)Q(7}77CQCcmaTx6E0At6%u1>Hg3iigp#A&dv3+(R(2YNu~1+e@hu;OE|@*?Q| z6s)=gR(}T8d=7fAg0)|Obu&*B>t})fIbg$!VADLXc|I6?32a#iw!RFuEdnD;!1h-mlNf0@wj|fE{25*a3Ec9bgC80d{~LU6IY^^y4iaCfny*xy_bXNNM#vi>Z-l&X1TP%HNytgaNytga zNytgaNtpK}%+Sy&;WcNh61NO2qqAkeQ5vIij)=$6(Ajq?n)8NE_qoh#=y=x8 z)?Y&>?oDgx*kqrh&`3iE?~R%)tH7RGuy+lZr~~`fgZ=g3fDd$T0?Rjp6uE_VISD^KG?h;41NH%90Xerfo(^?$WgHUIM{Il z>^up^PJvw?f<0%z-m_rh9N2dO4W0gvxS_*q=&%Fq06V}AumkJ>JHQUG1MC1hzz(nj z>;OB!4zL5GzyWUPa6^Y1I^59Vh7MmZos*E0kdu&;kdu&;kdu&;kdu&;kdu&;kdttf zC1Hk!4wqB7oWd=wVs2?!CEgKksU0DQ4W0XW-JGn~47#7z40=H-ymq5lNiXTBA9*6D zg&q3qvM=daZ(?`me#yow>5S%;bbdtQ3Pg!qO!tL!|24X|2&?=9<-SyYUb!D5KVMc* zDk=XL(enjmV6j4L>d~UqYv{GM;*)mg_UQL?!YedA9h(>{yLKDu>EOLlpCtkIe*zBlfzHps^3TDFt6=39pnK+r#Hv|f z^&GI~MbJABtep?my#&@T1pP0A4U530C1CR_U~n1OvK(xE6>M7xMqUHkUk5u@gPm`H zv9(~=IF`QA>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz(nj>_Gk<;GPc8MBO5jHB*} zT{`@rT?#y8m)1ODmt2q9rOS`mrRd{!DKx__`JT3Om*-lS$0~&vSW-0&IHMlMNKC|V`e&PG-Gej?FszJ8UE56`ryMSKflC8?UMgF>S zUoa}y((xB4-J24ni<7CNbFG_`>yD0->N&#oQWte}7H)GrV!KNz@Z3%Xp8LkwERU2~ zrpc~HHIF`AW|@vhCo8X=h}Yh0yX}EVKYVV|!!w?IZpMr<>toM6^~|&1e?|_Bu$E5z z%5vHE4dv%${QRA^qVGKW?01KL(B;x}bR_ENII1k#wRFbH(QTUI&RBY7^gtnXS?Ik{ zmt`kd@eWwI8+5-5R_y_+_kuP1K=1ot?S8QC1F-%e=syHD908k-g3ZUl;0dthB-nZi zZ2J(5oB`X#7Ae2~u>f{}9bgC80d{~LU>@_%3RY)Rd$}pbzf&%Rt25w7jY(|QX}Q?HV%K<^*cI~bKR;Lcf(poBXxA> z@t>@k&W7vNbdpQyT)WTJQaWyG=m^hI)O6ytYjR%GX*^$S8?Ur+#$oB6v8CNsL8&D> z<{!NGx;33LgefOX5j z`sJYiRj^?t*z_9M{5lw14Ys@iwyp)+)`5{X!S)Sc$40Po6ByeJc5MNBwt~Iez{J~N z-wv>UCphp9=xhPYLtsTKSot34ZUd{rV08qnIRJXw!P>)U>C|;_OUJ@39d>{nUQ1*`dj)p@^QHE)Ex z5%NaJ>qqeV5uAjagq(z&gq(z&gq(zVPr?i>9Scu_|8kSyx!c3to-fgz;LV0No13%Q zI_c#X361`Zv#`VN+n{X2*|TzZ;a>*HBj`w|`(Zf13G% z&Rxp7JEij7%KaF+x0YIjMeeaEUlho`DfCPMRYwZxzL4$lJiv%sM(=>gEXF+A~zp+3ybIyr2{B$z0G0?AtV6x1bYP zR%9Di&?yi_a^QOzyEqviq#_HwH7c?k2kTCN^(R69DX`%~u;~ogd=?Cz16wYDtsjAH zAA^yLVEdJHQUG1MC1hzz(nj>_FZf;DQbpbhx0y1syKv@FmkZ2{{Ql2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2}fBHW+>=N#J*>N%yQMWRIZY_?=9;$(b~S}gS5sKv4lY_{;XK&X3pU5WU;=FU1nr#GK5pmm z@;U4PJHQUG1MC1hzz(nj>;OB!4zL6406V}AumkMCC~$z=IXwS}=O6L>qfs{hh_6!3 zSE{56Ep>Z+6vx3lVrHAgIN5<0w z6YbL6DRwFTExUC1LAw-q$o^BX49`@0@9OntreXaU?P@yeM{-rzp@nL)Xy_Oy*Ic)W zGY;*WlIz|k<278;DJ>C&@>gjUn-W@?1?%9Hh+_F3<-SCoq}*EvSI!wFE9m@yL|H_s zd~GG2G4k_e1*MYme-S-iPzDw&w5A>{O1*|&Yb!1*q{54Bg3^7t8{HRC8l--X2=DHu zlFp)@k8)np8EFL_N32=aFX{M;hF5P&=&d{Y4V858-l)v-8QA(c*mf0+d;zx4>?U^1 z0z2n`u@}Lvd0@|cu=gb}u@LNg8SGyK4lDtkOTqGLuwn&RSp&K~V3ilFUIo_Fg5EV? zZ5>#*9;~ki{XVe44>r98HaCF5MzEy`Y~2pFHG`1=*dFYrl1|62ZeBu%m(XDc*a3Ec z9bgC80d{~LU-?81L)OT*D`p$i0Y?epLEYoDyqnbw_F0)L>qmz}_PQ+{PwcYl> zq#r&v>ERhqJ~v}Vnf0+}o_gll?>{4VpUb?Aj%WRB{bh6~12J7dLI zIeMTlq4!2jmUqF9Jz(cvFt!iudLQiB5B7clCJusqhrs?L;J{JP*$I|M!HO7I zc^Y(gfmPjLbq`o`9`yEtwQ;a60oH#4`uo6!%V1MK*n9;H4uCC=v&2>>*ftxCl!NVa z!Hx>Ba{(Bu1iM^ksiD*3KFbZA{Lj8{Sfu>^#{$>^c7PpV2iO62fE{25*a3Ec9bgC8 z0d{~LUahQ!-mfNymU_1%Ld&~%Lct5 zN!M<4aMd()G~Ygv)4~q@by<{jGW2NDn>FSixgXse+qIh&UvN)n^!9XqL@VYLh!VM& z?hEPuYsv~bR{6%Pq!Xixj)Z2YU{Ky&YiU7}(c| zo=$(1dpZ{G>97Or06V}AumkJ>JHQUG1MC1hzz(nj>;OB!4zL6HcYu33+|%Kn4)=7p zr^8oG=OpAL%(wax?lIu~sbonv66n)$-g=W|#-_v&P@?7ikSf%g+ zORA}({({!J(bRM_-*>mlBX!&7t8l3 z_a*Wq<=!gat=yLm-jAVsYpF$8ApaDqA>YH;ouX6)VRTq z7RkqMQ+}dIE>(V_NVY3a6wCesdcZ=R9CUFqb#!jbS~>};=Lm`N-+ZTf!Rbb~*FjftAtpIy!z+MlS@Pd7-!2VirU=8S82bRCtLmi!p4L!V;PX6a# zeZ-l%N@E#!VDdqVs1`wbAp=_H?29r ze-SU1N2xGPrV7&wQlVWc zRJU`|zkVXueVu996?E)3w=whlOHjxiu{<)3&ztdiGd^z?9X`!w__QNj45~n~ zJUs0$ADjNub{->#A8;moi3<;;kLkzJQkKb)Mf%c_<1&p(M|v>Jq2$jiuN&%m-=}*To6^=RsQWyDC|PsH>0jxbh?_Y~X4;ZGb}+HF;aiGh8a#@j|N zaNmX(my}90!k26#e^WoGcJR-3KdSuMq01+KHlJ_kvb$_~`k2bo7peUMuM> zkfRUl2=)yMF2n1;QtDk{5lVyp$iCk(r#RhVN32;ML9wr@$o3T|D5ZbkA~`V6lD%Xa zbc;Y^fgEbu`j_S;cT>ZE_(tJbKYJvTU9ObmBHG|RZ3ZL<8w&(2(kX|QuYM}ml9eSx zGjGS9MJdT8v`;z~73jFUad5-96aOr?mqyA9Mf3nkzT%o?F{c?9QW$QtatVdZngh#H znoDVE)94`snqvnF1zmhW4jsQzZBldNd97y0{Db$7ur;eR+vtO2U`3&h&;H>)C%i?% z@iA#`?xtsMUdB0gqu?1VRXjVYEJ@q55bZx&A_8lzawzzUyTIHQoi17@?ZPWkq7l&14kMz2Ypo)DiA29X ztInioLvg1%svx2RD+>%b9l2x`(cPtTX!b`tO^P;@cPi1yXwPXQqGP*DMSRg1IrL%2 z5tE|R_d}gQohGCmYjwjzvm+)JjS-F*iEe56*raGffu|C!q;#{&=_1+n;^g>WX&zY{D}@qUYAUW>R!|dgu)A zREHLfNPT~$D|*xLgSdCVCVcPRCWp?>+nRIHdXYz~!4Kwi(p$~9!;Xj=)&BQx6M^cn za;Wvwuu0X1GEa4MQAW~dEZR4rr#e_YR)o7r^}3VYCRG~>J(X%@FulIOi0Vl9SP^ab ziX2+7&ry`?eNiX%G*KOm8&Vx>_=D>81Nv$G+8x645s7wPIgxYGI>G0tO^HTY$NxcBv?lkwAKf8*Zwoo}@tI2|MH|XJ z)kiMmUU+qC`^aduMz#NKAp-Nq$)U&x3yaMqh@sq5sZPoqJjNuy)lKX9<3#ufsou2X z4U?+V_eGuDQ>Aq&#?_6f+{;FF|*sXE>3>~arj?FgpQIvcMOpOy*7t~=#WyYnNHstx6y>KLP>^%~XQSStHgXwExn z$JV><6fXB&a%f9$d5O8bF7WOXYZR}Y`l)NepN)5 zj+a9#<_(xso$j?x?%~Yd;3i>%>he_XWurQ_bi9ank?O0TEV1S~L3BRos*W+r*t$($ zwI)ImUE_sgvn+=?qi^P1wNCEgI4)f$tT)TTHR~QZwBx{ zJse_EqUi+nx09lin~QE(&$>r=_LFE&<8hOs4dtFnv@*2*p{{66?0NU!BYZ3Gl|wUE zeQHv4`mU%Gdz!EgH>YARqtzPK{+0KNzy(qrTli9`xn0pv?5R}al!9+oD)zEb9lUU_ z2ydGphu-;ol}XjuiC z?%@!V$~{tDHYEDmh4A%q&sqC*;rfI`zgstZjJciBQ0@&AO=qqDASpU&{4mj)1aW`z zb>RtqLk_K}T47Rj`o5@>d#VpH%E-Drm3!H!_6ENpe6ERdXrA*eld28no=UYcwBBn- zwcj;S1UgA|&xO4vRi}HclY6Sudr0f}!c^{M<8`oeq6n{_B!~7MIb~9{q1;m)W0axw z0Yj=I>nDk5KdD~7`}3Tu*2z7b-%Aax$I8BCsocxx^;^S#9TV%HB;ui;kwY(STx2sB zA%=2Kb&SC&1jSnuZ*!J=VA>`xk-e6MmS!%PY(62SZ7kUq2z|MG3BxsOj*9LN?-QO)KP!iPpLCei zY$)|qAH0--wOey)U01Ys)6WXu%%77(FGmw5MW-8`UFa$2tPdFx?VtH`BJe(mPVB8P zIe*ho=BbV^%D7s4E>G9#;QK!(!mm!2LmzDM=3KN+A*fv^HJ@p9MMqwpETZQ~ z^johtnG|g(@l;0_WmLUdGxKe@Xw9H0c5bqWZ~b{Wv}9q#q-aBVrxLB?^8VDM3QSjs z-<=%04kn8xJ`!7hUO47Xkwbm`XHBY3-xGBr4=FupCuGtLz2W8#|4t*?Id_V1wUg*W zT{Fj;J0cp&Je6o?k z*O*joDEOdy&`zYaY`y3kI=7en zJ^A3L#jIrePyLDrZ2XoSa?YK5ySe?*Q1+>gGs@t4bLxabR2G1-wXz9pjj?w3QmPHZtLJAIGTNk2_y$D3296tZ$Vw(ousU;e9dX!YLr zP0BVDe=6C^5PO?ym!#jN(+NM_V@+cDuL?&G$u8e=#-!|Yw{`Llr}$K<9hjUM~tHRazYjWs}_Y0G%4dtKeSfdQG|5;zPHnkgnO?b*5kVB!$SMD&EBk8Jj;!kyo z52@|gklI8to2`-UEq_4x4v_4%{q-hg8;U>G!3Oacy=Y9f|G)zx;Qe(uRMFL9QnsP^ zQ^`(FP8dAcWaW0y`|BbcC)wxrcbb%)zC-H7pDMMJzwk05I}-nO5pDhrIrQG!eI{ia zioap9X{5b3mD-uz)(rn*&A%by3%)If-dtNL%dGaxCRL{giOz3!aja2=w&Q`M>g0y5AKFL1EnI6KltUjmnscsNC;n6u z`IMpdzv-*iq@R23gTix(REN(WG^yH9`l&vQDMRg{<9cSZHL|^z9u&UazbS_{9=>2w zwxRS>$wp{<9I3%yRUoq zS53+`lzuANIM0A;@nC3Y<#y!cZ;9xJsdDJR^35h?8%jUbX}{zz8FgiA;xD#gs)%1D z*_&V7XHvGI_`|U#6@L`ksWB@)=J*`ib37W=iK|nEfez=mpZSQRBb5!R3B@Ukv3f!VQ>@4O18WDcZ8>#WFKx_ zTxKpu48@;HwldTX8I$ep{vF|K_+2@)wPC$U+37o^PW)+7JGvkhe_6ThZ}?pinEiWl zsM@>Bq-;a+r;?3bFhIXvGP02bXaAlEM@aVPFC8-}JKb%a!JlecA5z;>l}hcb+>S(k zPeeTr$)RJPT{J1%Q2eP5HaOGZ*`ShrZJOWkB+>k0EarJg#Cu8h+aJ!q)7&nZE?Y1E z=;Yt9X%>TxxBrtIs@ea#NzsPlZ`h%RPHO*Ya>LH7S{vGL|0m&^ z|NC<2i|yM?sy38X5Sha&p2T@IZ-{8X8pRHkRO zMz#CO!@|@0M{=mU+Z#|N8)jGkan&hVp zvT3xc=e1@86?yABBI^8OIdp#W$0k)9%01OlM#*P7rmJGmGcg(7$z4^mJI0)UEaG93 zz3;VoUp2Qo8cIIMR_RQe*r&fcCc=L#94r4s4%NT(nn~5^yQ5C@4a@5yIsCWD_b`~= z8r7Oj#JTcMgzEySUiev~N!5m;PxUbjrx<)IwZGxjm+ijrC&IJsKg*%`>99%J>1OL> zpX!`n=HJZd%l2;j&%!tFPvy}5aJNa>hQd#Epiu_d{?rbc)kfl<_opIoh-CY>JI0&a zA=72+q@QZCpEAg%+HPu`r~F1q-|gU`KNaECencxB1ydQ`YNN%kv; zE}4{_?zT<CZ3zlny zYQELz_OAbP;p->eN9y0mxo*7(9Gdi({AHL%sODRZZh!xui$LhV(8aH-cbL>|C<9eT z9A&sYD|MnF86Wz(gQ5Q-!c~7kSHLbmWKy@G5LD?_2HYWox+7J8A)=>A_mz)(P3lhH zFLhE-8{R{|X;OagK6ElLX}>nTPydC8`~NFl3j28ZJ?8dHLoo>5>M{4XlDlPcc%|7) zG_n)^|0*0a|B|kTy{E>c>~yh`|+SHgFWbe~+% zWm31H6jU8`kb-f#>hj?BoSonPbAKfQTOXrKV_&{%Qn#TLROwFsGS86i;MT`P_{G1b zt7D&Da<92GN#8AXQc#uN$@AIzx;1eUdGW7B^dRY;9e&fKZbK;u-Ki9$UrPSg&}Kqg zJFUj2pMtT2e=Xv#Pp3;{Z`*BBcDmi!rJ!|`ENzE<-gN4v+K{B)Sd3PP7LCtV5YP3nnPz&wdR-;Yx(aYzWDFxn%T={f6d%p zX($DU9doG0{+(oak04tc-i!ZEI8Kr5wmB5J?9DG zI6$)7Pj#4-Z72m*AJCMc_p($9W~bYE;0fXK{v%yR`~8GT-G)+7rCS+#A2+Dm?fpmL ziIeW7+bX`1>%CGZ1vTj%_NMmA?ELn||55mw|0i8cyVh$`x1khN=~jl`XAJ80H~&u& zSnyrCoVL5kq;5kgsG1gt*mJE^gx8N#vqf6)cCcP=w2+E4TdzS4nsoGHb4O4yX1fT)c-jh!X--dspYiiFq zX;O8%*E%CWRrV<-yDw&*VVC@O&}K)(kSEY#>_M9lBzWy z-h)^FNrd5t6Q2C-+pR0+oQKnWL&099rcgDO+Q9WbgkL(PjTk7u61} zF)7R3+@x$n*{70?^9_Lu#$>x^ z|BLWMNcNgbFaC_V9Wvc*o$yml3{0M6HR856@-M>Yd77@QeQKpi+37i=^9y0skwzJ3 zr`3NozdqA!B!17+BG5~+d)l_-T((a9sZIha!|dH7%&OON(VkiMs zpT+QL%R{mT{mo9d^NsHb*QccW{j>EZb*I~{6M?ETfJkl!tw3%_xBJuY3C}zKhpw}I zu*IZqLm8+#;vfT~v}Up)-QIWpkMOxJOZ)W8u=4QqBw&Gzko`8g3d zLAn#``%LOKl!B^bjxyrjpE^e}!uG96!QhGKMEK2rr3-E^t^8SY`=y~2gzi)d(yu2| zZD!3G!I8rI&3_fq&q?-MvujPtPTwtcQcx4#>bWJ?)@#*lB^tM5pZ}|f@A~)rVkg7FfhrBUJOP zMz?e6^TO3dx(~g3!K7|OA*ed&D8p{LYOY48=39+!ch~d6^VYx7wYN9T`8jhtWx8&i z5LC?!M0k7l8r1E5>)(XW`F*NXUDs)G(f(BVpjV0PQL-}!wJ2$Syqip?f< z8wx>{Zp|6(5p-+LJOsnv7vY*8&=t7*-ZQB?eZSO6L47j zCz^gJ9P|F2F2cQa@#I{GiO#2C)#o#1|Pzb6HI?BlVkU`zv)jtxxi=_Ltt4B@hHWY#? z-3V{hx=q>n?Z5aV5!m_v=u+HgFPhYyzF+Eupenp^{$bdK_J&<9Fq!3=7!2nDHdw z2(q>5ef-BFzV1KhTHNcmnUtMAq|!-2)#o#1D(!0ca*wA zF9=VS!$Mc&{>VAS+-_+o235k9vG>qbc}5VvWl;Ft5H3o+BwF}RL;2dH)h3mv@0mJ5 zs0nb#+Uq5XM!BCvi-3P7DX)0fZ&JCTC{!s|B7Ep%m2xg+XtrQ73zHkmpyxpnEy4~b zDL?0LHK{z^bDcEA=|NS9hwn&^tw#79p%ES-%_75$iwjJ%omDF ze)`ntPLO2bI0oHS2f`+G8%jf!?&Nqx?-nI z-JA>8$w8bkRK+(<7|eL0B5AlrcZ4L1sBZ!3KGt!-@SKJ4=)4^!g{M2NlZC38gvrw=MudAw zv+!+MNWyo1+HO*~p)gb(e3W5$+X$2LlA)s6#{HyO1m?I%xc6jl&V}oQAx<0)TCAk_ zbusLAngbLs=z2$pM< zCrGq#G`mT8t*gzXazlBjQjQOkzFDb{_t`0Tl4#*tu!xkuc;!Qr%F{j92}ISrLL|8> zWKg-AL<>&`ly{%GYErqOL{uGvkcjaGsS^{~dG00A!nbBIDc|$Kl3y{GEa}SiB5~-{ zq2?UQ2+Fl#PNGGi56bIyylGOop-jY4D3yuyJI=D9Or*KALnjhP$iyIN7U5k>NcjA9 zyG;r=l!?QH7s%m1ND5DWa@Nnp2x%5kcNGagUv=E1@bvvtClgg4f|N0Nc`6gLQywGH zA|8eEea=r!DmRpgD&@+koR%5Z%mdQ*JVBy`W5X+?yuNqAx6JLI>7MIkqUyw9^3 zZ&@nMvs3OR(ZaWE87bet@qkI?hB8rg9KrxsHD3iAk7`NcWGHGr@B2x#2%Ld(|EuRs zDmN60P_B38O|yS$jzU3_EW(?sN%x|8v+vLK{;89RsxTjZh4rMv8sVB@VuUn{=&a== z{F6_Xn-p#+5>+4Kl##hR6^Yp?kCA8*-w)*nqnk`B|FV&&oFhMEP@VE7yrc)x<+(XnxY5ayN+UquZmKeBMqubPV%Ly@Rbu7vr}a*_Ba$Z(NDO(^+#vk2F&B;m`?1WgJz6pAY0 z2u06dTt~Pz%}KL}egffFA{{1$r#r4QOjMmq)O>Cm!E%lA7>O3~pof$nZ%&w0ZYUE~ z$022Cema%r*(pzuXyJJIHB#POSMh7+_Rn`NhNuk9*Q7ErJLOIiEnFv{d}Wo_ zq;f-B*w=(OLB| z4w)G6rZO=*^U77k}EDQ}s($E0#YnW#Dr zDP!}^sZ7jHxsyZ-R~X9Q_#|plxuHx{DObkk{RWl0Nwo0Pyg|z6MlYLGp1y(|Xii=j+ZDObkk+fsX|=38Z`P8T)QOpNoBXc1^yL&{sX ztTCxP-E*B##0f=Jng=FlJnfP^nrdVRNwEmeT}!gpy&f?)1Gt@0-+}?sv8+NGHm_ zr_MxQYI&0pbo)uN2&{XPbVrt)F{#^74C1(>N^i#l$?t}rd>%n}kR*%nXVATA<`+5F zt&@YQX^6^D{NMC-Yd-r#NV15&TTi+dU#R-FxjZqHgQ`z$%1E4!^Mm$hHe4e-Mw&%@ z$p#Ys>EU{l!VLwXN;m@C;}~I9Tyjg%2v3k^;W!Q9?Ymk`3QylNb&_ycf?w+@8*$u8 znuW{nBjH5l7Uq3YaWa@cJ|xR*2w zUmJvf_G0CO=5|g)VTgmzpi*VjJvn(|0uke=N)5?J*v9>&Sp-)2NqEP_)j1cglZM0Q zA=3HrhmyiaICrZ#F%cxqBHRPv+fKY~Qn;ZwRDJAGM&UyjGR#hSghY#I<0ewRV(&qd z$_?eAO1Ux+52dz_?3Bkyw1}6#Mar)>TrjCTefQJ}L`{-M7o7MIEqG~20vfPutsL9}Ou#(ZD z*+bnVT6jK&a!>zTCY2k?MAcCUndsS&I;EmHlbBSl9g>r1;oH$b%4c@HmviNMp-86= zGkxaKoJUOR)(lwuBv}Mr+CsWx`%jzHZ736m9fb;1R|_0LxMoucl4cP;3gPd(eZ{13 zLy@TZ?52##7p5XHJLM4)Euw21N%^X^i+{^pu%z#vI+3Ue^S}Xv%3~y2#4kbloF(f` zDmN5~D&@+sd|4{Yv-3PbqJ^VnD=F`C>@ul5-E*BtR82%gm^(TRDtD4-;ac28%HKVI z)TDAlk*GQhDFbtFDiX8v+)bi|CkEwf4qwc*mzWnBaaY@UJ@;Q zzHOv@-mdvm&F!IvLJ^0d)X=;@_Kiu7aFb`{G{;YEnv-S`7=ZAe4X>LNZYUI0!iRst zA3?Y_%}KKezqg%)?_Ivlr113pQzsKuX|5SvWv4tsqD8d&ZBoAe#Scs>Hb2kTNvyF{s>0qJ?YD4pRPpYjw_*>tv#8CZaMl9~$6h=ee6i3(p5o?rZRyRBk8} zRmUME&4-Tt_$P=CT@!JnG$+x*_ga9IzwB)_soYQ~Lb)o<<98=_(4Yc>e8mglm!|NSa0X?VTk2gU>!QDcn#dsy@cybDM8vDicSr zT%$ZfqD6Fmkd(jm;mf~cE?Lqo*U3cH*~AfsSDKAQqdZ2UMf?bqFWJA=q;f-7A}!X}j)%0$(12$>j;8&n=6(IVdZ zE-8O?LwC-V>t*85OhnDVFgwo^Bw9F@g-H28wd429?VpA+QFRaqE^GM+?FZaEs+eo`y~`yt!edcvgabi;LqiK=Oc z$nx;N>dMyqk|IcoMR?_2l6`9PC6lrZ<)P{`oH8zN%sihjX}3mqgd~gT1?b-O+QNs- zWlFkkoitRPNlczP%=EjK5p>5$vWRbck94no=?#;*4P~L~_=7Br@63FiyAgCJNV0HL z>?7UfpEaA*Z72y8GmEixswKI>4;ymMcz-v8{ z4929!w2iSxY}2&nPVJG)Xxm(E+B7|rHi=2wq}tl1NqS{jPC;M+0Re?&<&^tUQPE}P z^5B$9E|*L08wCWR&-;$M`^>Z54f7fQz|H>h%=3KT=Y8jy*_n6VG3QhUcMu>8y7+S7 zF0C9=$Stc1GHy#1gxj7^TEF49shHbD_Dek=3mFrPJL!byQ_A*B+k7vq46=m;VdmkD zR!A>Co4M(!I3NoKV+C;gBt|LZwyh@)o=`!@OMWV@wlgoz4!9ARg-UHDfcFOg;?Ni_@tW7XW(YF??g)Wn75yd9v05&tG2&rj%3NG_`;B65CS4i72cxbd?v z@-cuG)GdrWDtJaAxoyrJ)I>BDkyUeTisdr%!pJoMEohUQfqZ^0=((F+S{(drM#mvu z&8;sDVy6NI>zW{TieeBz3%Y4WZt6R9)5#qaMRoxZRdefVt9EX#ki(E3n1zgz7631( zGb#j^RTR--h!sV{>*i5z2b|QzEMOK4A*}$eH#RB+msJzduQ+m&vM_QZKns-}jNC14 zOd+}L{^_75qH0cd$c2$t1GHeuyaeQ(;XYfG?Vqx0qJ$h~*rr=*;=;(=0a_TDXXHg5 zdWGb+IbT>!55zFBeO zOu!Df9+-uUqwN5GCdc)8Wjm;>qKLp*HCMm$8Q`><1G8Z8>;UjXx^RWyw)>}pnkcE} zOV^s<1wv$+4M{m6Hv+U!S;NSE_hc$0msJxHIX_7mlu2F<(1JOB6BkB42GD}~R2Pt^RZQJ70r}z7BMQl7)kJh0;^*d7mTGP%=loV8TuUF)}Oe{a|J?W>o)w&VRB zJ3rd7Hu2*XF3jyJRpM5GE#GcBQnh5oI=0E3E6i3_bNHpdikUA{F;?{pj=rfD386=F zo!8w4{{g317v5U$=e*JT{&lyjir*EWqF*)(f+`xOxWIm^dHaTmpbE`#CW>F=BBr=n zZQL}A1sHiqxz@v#d8_B)iX{SM3fXiYUdpyYHq8f?5v9sq#kTUW#wzRQ3i4A_y_|T? z&s)J13>xdGNd=K&L8T5Ikp#02EaRs$>{L&Z5R(-`^WD4r{?$mop ztV?^>6QzEJE68(M`+dw0YP;dg2Bx5v9?SR=iJlquWfDDYjw`6`30+@d(NVkhpZ_XT zkmr!>`$=@~h>aw=#`S9?`gAu}ke~7LuQMCe@58wd2m_Mnm-l~z?UktQjcyOJy$QAb zT;GF?5ZM$|eUmUBJNhl6WJhme3Tk_@=R-s}IQS50MMTZR%m(Er;P@ji6kkuAt~+b&oR}r1a-MLHxAGKFJi+_FCU>GX-U}VvH+@r}*M#mQ@tJ(C{6m zpjampzDqKn>;FBbAkP_BxPqd`Hh-UZ)}MZgFoz#{nh*vDKEt*`{jQzj3d*Y2rDvH9 zYQ=ofb4)?e$5XZtoBrVESuE7jo@-n|t!QuC%4|^En@YDaKS-%f{{it_7WzY`Ai~1y zTtS2X(V1=x(l{wg;EL!fa5x{BwUyw({Kn z6Q-c(UT&|F5kBAhD&s-zx>EU5;(6rA&zORucSgNNY%Y2HoGB<9^#fc%gw+>bXEr|8 zY}6k61=*X<#{81(O$P5@F+vnQdzdRI)}gvzGaD2uHvcz7iHiL#Q&2C5`M$w;&^X*X z_6D;-o_#OA$rO|ySHtf}OQ++1Pn1!=x0r&);qU}kQ1srWx0wx6I*jiSKTS#7nS$D0 zp?#MrsHJBo-xUTW%%@x5V+x9HEcydeP^|ovKQaZiD>wL0OhKMAuWQ}&Vgh9AzOvU{Fg-E>-{NHP}}{7KV@54+U271tW!HlT)U_=`L0~D z#CK6m9@J2)aUVA>S+eG$8WRdH_FDCPfLk4{L>2dqV~hs(+Cm zmynfb3cS@BtdKMp1n*16L%~HgJ1t6<-f&S}HNiW6p*v7=CQ1USuYojpxhwcWnJ|R} z)9z@V6(s~w+X1i31oxKl=%T7Q8^R?7QIQGnNtBL>l6~+RKPi3DyeJ`vyvzHz70P{1 z8NAaCd?AR`oCq!>8Xy9E_Qi$kYQ9{1jXK&7)j ziB7^0I ^F$&VK(^|dxVCte8Y)Il#id70R9O4_sTcyJZ$y`EKE&i}FGWbfgTDe{o zr8ujkZ%O4=Nmi+(5LU{SLUbj=Qm!B&E1zIkBo!nCk#Y?r=rm+i-UbVMf|P7^KUNA0 zPr{f$5P=!lTtdDw!(hQdutJ$An-e8uHP{797G{p%l&q6j6x% zlcI#I3R;UPL`P8xmynfrYAKh{m`LAyhD!*dat0*mA=Dis?eMH2BnOQNzq9Z-oOBeX zXTpDJvn^bpM2feL{7F*gt< zl}PJE{S-*h+h|P8UV>>5AR&mz5)&O0$J5%lgfdYd0(X|BBWH3}lu)w|barqn)F8vT zPA;J_(Vo@CQ-~yw2)IWh&1$@BHwFC5&+i zbw`H(IJK(1B1#BivU!49?JvB-r8MhcDo(yiAub0^atZnJx(1STV8*pgaS2(Klupx# z1L@bO6c{$cC6tNWIZ;AmwYh7SLR>$8ouVGdnWGS=BIl{prcpaHdNfvd_lXjMNUd_^ zR!FKks^Jo9<9M{2GaF2(JG5TzTtZ_aX9y&Sh&HUN^`H=QCp>9HeXJLk(6~SC>&+$P zzGqC75QJyrZf8;f5(|8|gdj>1eVy4TM85h0_HYU1K44Om5JXz5AGboj%8UHDgdqA- z0;rYGUM-hU#MBv(AcaV(Y7gWR%KgNdAS#7r26G8P9No8pi*R7B9~AmPK6}Vd5CX_P-D*}a|u}$XQfc7Cp?u) zXiR8b)3}5na(YAwC8wz(opT}gvxhUdgt{XkB7yNN5qdu z#gE6tkH^K2C&Z5@H#n;n!QV3UXa3GU`QIj8)Q6iFtJuG5*uSf9Q#s$bK^kA0mht{( zxO5Qp;f9swTcmL7)GCR0NttvXx4Os9jZ#BYq08nwxcggGJ=hMCNEz8=wuC6N5|zxRO_I&Y$X-d3N=6x}L*f_}-6f<# zRzfN1|9ZdA;m*h5<|kMGb6wweUE+N>bzj&0e7v8p=i~8wJoJrJ=?DlYNf?O91BEw% z&sZpl2?z)n=)ji@l;lK+uw87Z*a@l);3a0vOSIr6J_<@gbp}E`wy<3|UfIb&xEH)) z2!2Kg^D{iq&u~yuNv-oZxo)S8-ANl82D079j~;h&JI*%JPqG<&D;wrp>7l=wn!J|` zk(@xtR`KjMt15Vr2lFBu^gfJ~Yc!mktbcoh)HR~udl)d^Lj}D7DLDaxfQ?{zw`B+k zuG?AX+8rbAmW=Uyy#21{V=d0{ke1?pcDv85(hcvf+nEU%G#2Y-C!L7aRD36Y%k!}w z!+6L)>HX|(wXM=ak=O0Er5iN5>*geVj?z|qCwIs5u@1+0$jhRBcBj5p>5i7`c4w>& z8o$+NCw-3AQhcXy+w-wL%XrA!yngoc?XA+&x3AkBJGti7Fm=&eX8VT<=yUqXKRaSV zZl57MLO?Lsc8i99fShRK&)2uP63Z@TbP;H3WQxbvzHnn#xWmsk$M^2n?IV+!?Equ z`_dl=lp8)i6h|FUir&`ub2KoFJP1J%KJWJ+!_Z`^9QYfE2neYN2zKfj@7}8o{*tJB zGz1$7d^F$xx^hZGV2?T#|AEl;%Kp0dN!JRn%d6OYyB|zPPe3q9MMXzOMnQIrfI!+P zd%cmfmfi+!z0JxSxs2-=zC;)sCrdR<%E{&Qf76S0UgJt%U z%M8j!y6y&inlbcr+G}d1(s-3gZ3)1C@2@okzbItDzoD^F|1(Or(SW>eeZxfU2C|kE zk9LpA_Z|c+n77E?=*gAJE4++Ew31}YO zq9i0>^J{wWWRxL7KnwFbUqFSNH>%QdBO5iT&)h$z?6#(UWKxZdE3T>M|6p-zBq-xx z9?>B#rueWn{~AfY3BwbCL5Ix}7`hXfVxx*PYb5xG_J2kN*;$lPcAGO@W9U9`_Z;(e zv)j3+uQwIFlDPizmcRMY=6klpN4FQU9Ohh~HHGIOfC>}cdzW+n9|nPZ0s>JTG4e}7Xue^!Z*25MH%3y(rYl@3*MuKVoGCGv=liJpuSr6BE+tss2{ zeag1j>O;MAMFY~xFJ8KO%?>+%o|J~_R$ZOsw}ji_>dg<}J&5ojgiF6mIcVK}^=%^b z0#66(E2x(S0_!j!0KZMp3z7JX6pJDO0U-c=cg=1i^lEsp=N0ATQi z@1uTjCMlVy5;gH6h4!oEtAz`_&1^GhPH8pj$OuaFxUz@d@rTQZ3pZLK^JAC=MY+b9 zR$qM6hOF*5ZDX;-{YN-|HVLes5~zAFhAU z1RS<&;~lCmQ0hRYsJ=mV`6NkK(@5L-6T3E0i@#1_7y2eAJw<-nb~zxy9%D;M^ph6b z1mr)tn6BJS4L1Q4FcKU}btY9`PW|V?Z?V2eshN8HqV-Sod;KGIGwFkStFqAfX5(M+QF#Sb`I*6y- zK64k`vH7VZX;^L+oi8wd--0RU?dq-@Cz%LD)>3;pcBSxoPSL24`Tr60b1MQmTM(=@0<70Q8kv1}B;@EkH-Dg4=fya{evCUE60Czn!CSD>)*s^N3s|MR235;(Eh}1 zsQ8Sde+vRCCTjm10+J(fKmaL%ob5s+(eio#1^@vK52=)u2`WA}{M z2J;e^zUGX*NeBQ6kDgu`9!2g*G8ps~E~3jM-a_!w&p^4=2kCFavxRhmcgT$y zVQ&dBV}QITN?q&dZQ<7c0)zudXI=+Vp{3P72VDQxg}L^77deMrUbOb-wExiKU;D(n z+Xj0j*K9}@rr#DBFfndef;uxI?Ayn^ov!z+G&m!$?02Qn-+xQ{iTM^Yn@&EBSA3tZ z9XzBt)vG3b+SGpchHu%i<*dFXT`9-6=B(}LXFk|xy0Haael+Egnk{==PheldVe~CW zx@WoVp+}yqP1^Bq>C1rzngi27GeQkCEqSv)F`}~-l>Sth58-Ooe#Ih>|BGky2pv_r%zNiaYkZh= zTDVRN$+OrZ(-v3N0ItNZE(#HDshuia(WweGblPXMuNAH5r9U_I@L_G_CH+w zg#-k*t&7psz4ZmNwd`hFT(Ok1J~iE;MD|_j^3umpr)@DU1Xb%>A{py_S92)G6bMw{ z+C6D&nQLeH$cI6>upT}I!1Dki$BTRimkI)K6VRER#dga_x08>%Puh7IO#plj(5%c_ zcN}vLcr5N=cko)qr5>LhyvU)hAB5!_gv@Ln?~mDCt-8l7{hiR-T~f|b{Ep$@8e}&K zY#&QU%1adGvX(u6`ie^Bjq_gFfmWIg_GaY^XO=S63AYiU$@x2wb@IoZ~+y0XUI>5<$&YkwCn>E`S9( z2<)IDv?8eJ>L@!{m*u5OeuCE*0Y-tv*l8ft2HZ=4uc=AJ>NMYuM^Vlaqj!7$VHyx| z5qZ_J*LYM3$^Ty8MBezl%DDKQpEz>G5U1o3PVq1(R7I4)iWT5&$5X3s5#Ct9|aH|21 z8A5fNE_Mu%tD)w-PLdz${cRpYSO84~h>4fh1YqhvH%p9?4EflHOi*fMaYkTHBOt8> z<^Z7n4c-(Z$}S{5I^baojR8RUt43I)ag&Wbxau6szSki>Gl)Q9X4akSp++DH zGy-M4deS#q?G0tmJa_!$Z{W7=ITzXKCmQAU+dp^3H(dX4S+bu7Q~q_E>?gm|4XSzl zbA4Fa>^Dweej_lNx(0FKGbsNDy@jh<{)-I)mqZU8${`Z+)g*pLfCZJBf-xRtZ^HpwJFu7h%poso0HLZ)Sd=7%m z;?^dj=2^Nvg%WjlY6l{o^Lr`1=Y4K=Bs5z(WjZQB?lP;)fu@ss@e+e>F7oL2nqQ^M zw~gLpdM$FsU@ZNcUTr|@28WBf6hDRW{1kw{gkUaA2!_C206RPjOa_D?>C!*|<^no% zvILDt*c&eRuVH5+z|sSlTfX2{-2a>p<_rX(X+U6`U!%&W$9|n8V**3l!UJ5tlg~~A zXrO6;y0opBc}*nC`c~&G=Dqy{Tqhi!8xKrokc+aES7FG(Gi@1e7jKJm4+ImJc0aQi z$?%UJe(XWPIuRV?>GtSz!r9KIL7p??7F827dsMoe_n$Kdq~2Dy4G5^sz|V6 zeh?r5?t%})0R%*VHw@dg01&{9zzh(K{uFsQ^we4ksVl^I)emvR!NGy-x62P9)$wse zB*mOTyH{3f$a;INXZTOvk2=qI?id>d9luJ-&S6*EXc(4Npkx@%B4`@push{+^p?%9 zUO2o=K6{B)l1owkbnol5qu01qJ+Fu)g?)(JF37*`l=>dc2{R!-#@ZeDJrKZ8S}+^t z^DtyUH$)3!W&@q|6$DFz0&e(Vk)6dVMza4=PkqH2Y}x0+Y5}7E0fD*Q0K^LRWzwaP zXHZpA^=hr5dyh=quEJVNcHW;Cs}Cq{30WJR->pTp=SZ6aNw%BD`Wv3XiVvB1Yf|^d zsZt59AMmCt9g}o+t?@BGVYRoco$5hr!RI$y|78VFO5{U3a+G&?gbo94&~|AcfDs7IE%~`$Wn;xd(>Me|hMJ|; zv7WhLTrtWubhJGf3b%`^dLu}uJ0 zAtYYfU6uc>kaeJSq^IDOyhCDA@UQn%3Sp;jPXHxr%_zA$}NiM^De zHlXU5rrJKe!UH|_J*QE_hRMeE1~SR+J8R>bDQt|zY)u;a?k4|;;1>5R88dD1QoSZ| z)@|$`F80&sXdHRcs%W?)r3+afl}FXiE+^+fGP!p@y|O5t9|!Q07R-lfK@1%L z(}M75V9B%~rt{B*-$aEcy|H5Ep^qB@*8DCEfLXyNKmC;6RH_u#Y+?GcNVlOe!xGKy z-=wJ>KDZ8{U)feg+HT7p5VJMiXjHvZe5jT1)w>tDKqROSi3DfP0qr)!M0Viu&n)a( z6Hwl%L9^k(+K+;q?c({Zo6y|Vf?iM3{@LNq+A*H(S^sGNq6dM(e@GD(Ib>|8s@Mpk z)#L^i1A!&+g6ISsAOJ}NI$tdp1fVWpb|i3VSa$OC!SoF5B}fL80kAxv4M4y}h#P!i z?|uyo1$T&x($|Zc85O0vtJcOeD_*9R+1}Kn+afM;yyzTyNb5<6q}-bwV`9BO)11`A z3~z*IhFb`$3g1?aQFe{wjC{@F!9w#&rl@w#%5hVPSZ}2Ie9|iD}8_b8f zK~k6-^o2D7Cjd9NEOk&0{1#g{Qg)JVS@aw*Zy3M?fj~*0N-s&j)|w6PqZQ6ZNk-63 zQ>$LRpQuoD7dev3PA66xjARLE-PjbYaQebT@tuCri78`Wrb{s!16Z{8wUHcjZHJV? zPOddU=AW%Ld^4<$goXhc6T#xAsf(UC*LHmFouKRRb5qRamOim(v4cQW4jBUd6<|P` zPwuK>z>4`nobn(@8&GkLXqk-wJQmzUMQ8}b0Ucx8BlX>Eu;Ks%Fb*tENbu~-h_&zp z%z#d;q7t6%p9~FFJWDsGV6@-x z{m$Kbcch=wY?f>-YArGP$MBT7q&5Xg78 z9ngI6C7L>dPO&AzGuOcev;n6FgDquWx6>%NR$nUbR{g~jE8{-Isd+LCNh&0NNIr_K z_!!d1>wU-FBmX2W&JQ+!B|~)9kh7tDhbvbV30BMx;zR;SDUjT=TqJ&Nf{+)u@4IsvVc03HAUV#O`q;?MP}FYQ_1`Z<|~ zl>^WMLlOeRlUk&;mpUA@6rUFria!`AIjvT(Hfg}0xU{R9(t@3{uls= z1_1Dr>}y^5;PVOO;tXMT4LM>Yl7bEO{#E;`0>EN3!6h+-3kn1-R#0a{`B-xXXa}Yrc4hfmyolO(v~cZ71+j^qD<(3Jc;7kYmI`|Czol-3$*a9 z4L@p3A32OF5n;slnIQghgn2MWSP#zxF`GjvfFl%I4h-aG5y<(BjfVMZv%4<}1GvWm z*b~8P4}r$leQ7S?XGZMUn(GH&r`&nw%lgdSpnZ3Jk!8An(4N;h#6?eT?AE{a@fu(H zz9L__B7>?b$!~>EJ4hPx-EWKU8VIKIs=UIHbVjy8XKUeJvz)`pivBtip&IvKZoqIn zF6cJVVn>3}ND9P-Zh!-iAAep|99S_wh!Y23rN9H6jldeVFuMf^Lik89(+GUqo^z9t z6I{W;d^-g&*G|kQzQLTF930@KbT9w3z~e&9O^puwnv3~Pt6tSAQS;Ypk&xgt9Y(p- zc0|4l_gdE|{7X|=sVgWoEJE6b{{1FqifGZ@?9W*}-?y6ONc4K89n^Z0_&wO4?DFOu z{pOC7d}CLpi1GVKfS>eWE=&(%_<(m1J%||(z&sz-QdWmCa-p0oZiY4NHA;vasf`2# z@IA|1%smxwui$?rL@-U*1eOQEso;)QOP+Ti^)}kaN@a*Mgl8El?M73c(b}*MlATwz zm%i6>;l&olJy$mw_lYGHS*G|a+$c68>kZVo4mJlifX#tk<_~BU3aZl1YDsu=z;Z8V z8?)1d_lC`{;mrZ7n{j|vQsmZ0}6KWGWSb072cr&kgAca69h z(hfos0=OjzJ*pi?Mncgpd>KAjD<1n{FD>J0b<1ZFw|&>#3xUJi~3P6aIi`B^?@ zD|hQ%wWGC3;f!Hxo44{g?K-Dh>7DA^Ehs3OAzmj{_tK=t17_Glut>BxTd2h4w#sZ_>NF*ypq`2i)faUCuVKK_*upzF z*+Tf4zg!TQ(+u>V<-^Vg04v}EaRl;ma&o-B2LF1CI8Xlazs67^B5EaOMDfYMD5Eta zPyN2eCHWg?Uv2JXL*Lhm4a=7(QFC!>;m@=Su(;Y`{+;dnM6X>=Z>01(H_eW~sZAt| zlvzyAM3N#cBKC*BF(}M%(Qj9dQ9BoL9=}fp_{kP#!&ILT%oYa1n}iTsXuGsV0HYV0 zTk&(hnxhw*VV;qzfscgCE&e%Lf!o7$Kq~Bs0#Tuiv|n%{$WQkasD@^CPNc0*KV zl(|S_c5yR**4N8*-u-Sz6mx#0Rt7vic=e~fVWjr@wM9poOZ4Jt9lg@O*0M#&@f}r} zh)TGhN4Yl=&#NC_sX|v<%X~Y_S#kjDDu$s7mxuxw;m`?N%N#q)w6$V3mc-ZzA#N!6 z{W!1-NYj9$5Y;u6J`7QCyex9B>vw8sfq`-(p^uknEv2-28Y``^%i z++*IIcyni%doOz?x{8x!B1G0}NU^$|+0C`)qWR}T8@{1`JrQ+DT1yteW}O)t6dJdy zmQ3=^hfa&)m*xHEgkSt-DKZ}os;*Ka(7gZ+u6OmWDjKYqD#VEfu%SR6(Q?sXj(qUP z*8o^P2(1w7=p2RoMI`78@6^5!imdfT)IkbDjC}BAI$tVFjl7Lx9gUc-n1sA*@1d4* z9sK|cMbFZ)zLuA1Z!hHX=U&d)E_hn~@`bCs4yT$~tp%(ztT)u|4=)vRQmvNIbL|-x zl{Yw(I-oll9UPw*ILiAAzdHduWD4_PrjQtB3acQd5E=~-^DnIpzyQE?_$|ov!o`Tm z76pLcnL@0k7}Yhu7nkAXfo;)lGYdo!p#JIX9c8_hwz7;lj-|Fy@9 zyZo`{BJY2#J*T~|F^b`?&_}i7N*(c2VPgT_fmIvSHr4FVW~j4GXYk{U{$e6-l(ut_ zay(Z?#@;B)BQi$fm5cq`4C8AJ#3doU@|pic4M}^(SxcZS!fy+?>2!on`RG!ejl%P0 z0Dq~%e3&YvhN(i#Hc={|3e}f^1DzO)QnOaIE^4!7;s7Kb9GuxCjIf&l0?1bgGy>9A z8SJv~CSg5hlkg3FEC7Ne;4OHlNqnxiU}gSRtheAkm>saaxfS*n%o-rZ`U;izZ)g17 zT5NtF(0!K%G2%W2VhHqJH*TF17IAeq^xuXrUKJ+mc#G*UWm-D z4+e0-ULz>qedN56wJ{EnwK7u=s)eaEo}#K8BeRUrDX8RPogl5Z-0w+E*o;o>%-9s2 zWr8$(p83kVU;Id3&V9kt8aId?G@A9T6Kcy0erd<(N3MM~Ttw~gCHZG$0_%OJ(d|s> zOg}~Tq~~dlHuKY|VIE8tVwQ#u&bf1Y(i zHGnWt!)0xxg5Y`eDrdH{&koCY^0JCHa?%a9~-)Y142QyQ`d27XiF8@vm-8xVT z3czd_yG~$ET68zQpb5KH%yOi-C0KlXDDeKmNC31$|001|o461(Nq~Uh$LvPY;y~ct zLW>AXp<_ec3`4Cd5UiLd1k$1TO9PNzAdGW>O8Agc8Eg}DHBfDB zeAl-t-|YU$_nDEqN@^kJaW_~@2(J_QB1S)crMbqiM>gH}4ZoezP*~ZfJ_;oJnig_xZ{3_Ac*h47qU$y6amCzPECz&puqfo0P-7k!TOC_)QKH~5XD2WjRLBEHBP!&`W4yk{Qs5qU5-j3HRqFoG0W>rJI4J*yR54hFn&;T|iQ+GNGof7m) z8&Mw8n$~Nfdv;Fdik2V?g{hE3bmnRQEpo3uIP6V78$`=2zo*YA%ge_?;`?3Ej*O_x zV^=5%qkou4w{jXhiQqOuM6idSW(uwKU}rwZ_Y1!v0AJ}sXIsl`JIh&yKr;p&6JY4V zB|-s)Ey&5*R7Z+zB-MC8eGGdFz>Nfp(}mk2fZy2Pr3(wo5r~RQ^lYf+mMbKyNf)e` zF2o51*ot3nKfnRm4b5y6@QiKGx+%R=00;=d#R>i5^o|M%38B0c930GmSusd2O_&s^ z;(pH5<-U>AM}hD>WhHf$Tpmlo{;G2{S>*W#+y~alDS7%1P*0KsuZ?ocK4id%zGj(@ z_Ec8zX;@n#H@4l91^vr=cU32uE6Fa^QRB+z6)$=1>q^LJ@LfoZhm2u1O!vXb12(fx z#Kh7|YX>mGp}7S=_p6!10Bkg*O!?blaq(x-5UlU;oo$6k4W!zS%D3pdUf)ir499yb zNE91!aaa@`$_OQMEJm|6Z2ig-@pVeTr$=qLC_U+jX*=^lS1xn<0e|7ru8QN|3idYW zY(Mkv{wa%==yI!+O$oOWv~|ty{)D!QCyxBk&0o}LsOl0u0^JU3|MtVIRjvKSMuSVD z3UNvW$RU4$VjNWA654=DNM6kJ?9?_Y_z=n#t{%}?6f5jBx z0`(7E1%58O{^!^V2#-x?-t^*~xw_Es)d@a!uCoSL`o4AGwKvk7f>nn#f_is;Mn+I& z?5#T<^|o(3xnJwFvYyspd{OWi9s6eF!9tG0^F2j96X^^|>gTSp$V-qFN!cr=U)K7E zo`hxaanZfLjVb69d+l2~wIN1V3hxgDzs7gtas1>6|4a;FC&sq)5)?y7wdg5e{!H+9 zju03P{bo4&H#oxXEC$5LH~@n2n3t;x1dBBSOIjtw$q_={&qEl1@Xt}gAj6{*Twm>lm3<$S*jP}$2NO_!jF9Yl-`GyLWBb`7`=Feh&ojmm^$x8eu198*v+yM!4<&0S0Cy zLuKK!+QGjh85$jDKwNml$cB31J-n(guwsS~r(_703S{FP3zD-HBrFtEG}8|#5nQku zVlK(*>yr!(+p02q?Nmw4PEMJ#%AtnJej3=z7glcaj? z3X!VSHNMzxLP72f)w(lc>L#Q`+%1yn)xM$hiJ2~;eNt&hRqn~XG4@>Fi)<3l%1!pv z($jKP^HgovbJwpO-%SVclOp^Rk%T}h07VjRGysu=v-Sel;d{=|J?ELD!pv-7-r*on zv(hf5)ej>an%flYf`vm4k99MyyEl9LaSRo8YnLl@H$x7jP)xa}AJ%H5KnHRl-PXoT z=2{b9-l;<)3kip?a)Bq13pvJV#Nza||8ODqe#D3fi(q0y-5c^JUCqJZVqmZ&o)82R z;+6}bMdr^1u|tE!Ujp5jE!v7f+ z%+&w9LqSycV$Oo-%80042Xl2#P} zR!kD&)CbgHvz2sCB%~V zMBht)8f@`eF=L@s#7O>j>HC-R$j|@ZO5yz=wdl&6+DQ*lC3SlXSBrBV z+ywZOP6(`ppmah@UXV^WBNtcz39uyLZ!7`og1+)=Ogh*bki|g-c)}j z_B_aB=1__3&+Sqcmux+eMw5Jzk?X{OguvTLZPMC+B&_>j$3TfIa335R@aHP+s=uS~;CljI|pP3CcS#x()p*7m(XKZKbUqS1(%`5`nY9Pm=5 z5au0LR@Ux{ObzK~9X$E!VV?o|vuajE!@&|0m!C~HGSda4H8Yk}Z?nZ{!&9 zg8!_NzuJeu?}s^dIY^~VdJhQ7iI2WuIws;G`l>E7@a+GQgt+kF|K|~fmN}S+LVplZ zIFnPp00_)$2bjXWXOqb+p7PI71zNz19HI)Gv)HLXCXpJ*ByvXwuLa`aqt}zVp>?9} zRJU2l5FAPrX@26+-ZO1fAC0$_eSZC0DpBkJaV%&Dz+N$wQ~qhuz2d@rW<zMk4upk`^ORT@5*uxakKg7v%k7vnTZL;1q?Hwcj zD$n6zhNVW=#Z>+pVPH^VVQpx=7vCEJ{G<&3Ohh3ldpM%-eSHN9hALP92<8n3nfkzR zz_%IBDfP}8MeX8Aff9?W!)%X#Bm~2O*ACdNLaNQj{&P48;)&qlpg6QdAUVJ+eOlM$ zk&NBaEr*)xSP-3YtZb-+py^e`fyJ7EB@u=oqHz8UfD*z1e;kD2;y3^^1&xq$sG{qG zVEt|P5*nZzg8f8*rHBsS`-~z5N~8Uz4XJNu4;(p?)@r@N#Y1)g+*2 z5i`Y^L)6EIlxn9W*-ZIKN-{MQ-v+IO^ zK|ItQ$BG#F03hJ!STxz+;s=&wJBm|01p5m#EE@vmE)sqI$%0uV>RbmbhHfBK))!GY zLg#>E?NF3?n-!S{2q+`Jxt^Imn`w_ovq-*CsHRX2s_gABw3{zBVB%{Bl(`rg< z@q}+m@mCuZ9k-P+L$~pIX?yo-kXqfIF&!sco{6==m%EBg*$9)sE*DXe+>ne#1ev-LR>5%HCw0o z{00F!C|vlusA6E|JaDkZA z@LCZtA5!XCM=uH8lEf;BX4t}c2ZF#hzuEDickF20ZwG=b!W)i4ukC+R3lO!Q$l#Y; zB(&TT8(I#tg;f@t42MIvCf%mnR`L_dEdHAN<3BQsz3;Fg!hG1-P!H>CRuuZvn2Ppd~OhJqk4g zl>2w(s9cY*;7?P|RNZrFV2e#@)F;DpN+hwt{9+y^?8awRp0S_)(0Z`hS#8=s^ML7q z0~dNoGk`I1gG*`imtEJ${5Ax4M4i#{PyeZT1r=DUi>ecGu+n+|e_@4xg)aPKVTIXn zSYaF#Rv3pJ4a_+uSk559e75lSu)@Y3$k3 z7d0euJ~=}FIsaxNBmMLMU`Q5%DhcCyn&5m>3eQuD@Nl1S1xNlJa z`29>UFRQQvIuc+~i*aWaCSRml<*dTgCU!)@H4Zk^#q{x2t^dV_gG*ux7gP(NAK+jL z7jFhQA*N8Ds?rh+2Rp8G{S?~%=70|7!gK^sTL4|+fGUg!Y~kb=FWM^%7Wa~U-)31V zbp@twsxj8KZ8bAEsx`Rzw3N1N>p#uT`jN!X^o!(;&t<&197Wx#_S%`!Q@ z7JMMXTo+`R`+*E|F%Oo>&6LL|KQYNM$yyr<zgy}<8KfH5PU!xu)%{$;+f@EQlAVh}(<_m#`!t2rcGF<*!i3LrD`0v{ZF;o?v* z=Unjc^G`s0IAeIcN$F1W-CbE!K_PzG^^QZ=d2x(jMB{m!fQuSB7$@m(&og4BPG#J<1tTxWLS>;Gl4zuA8mBhI36qk*zn$@CE zW#Q40r-bjV`=zWbDz2i4@9u;6$rk>Z#6qAOfD;RqK|0~g`oID}0FMWL%POow&~KwW zyb+sK7%+j9ie9 zw+lWHfDb@OhyX8{jD1P$5(($#(6$z?Ycw&-DKi{T{;m}->$s!nN7Dnr=&i23%^6m@QB#8f!@~5|to6y;(hsTG?0jyz zMU4yJSBvnMFkE?DAz%mKxI#Y=S6IB{!@>NHU|XcdU+xH+OV=VggE-kx=^x8h6$utw zD_Rn5h*LoX(S{=-IB3KF?g$!YVoN`cJP6cx`yanCVP7-Pz6?x|IUHO1)M?s-?`qVx zl+5Ihhz%8a75SBp>)KUH0tN^sDT1ZHv63^j-UPT&rY!D1!6i>xLf5;$cZXG{cbK+`&V;apZ*tNy*OGWYOTG}En~&iqZTKew3&FD=4lE=FfrZ#j z;|1CQ+~+Ec9Ol6#KBHczESE?HBoPKG_yU?!WqMn{UKLb@5r5Y>Zjt+ zmZLu`i02r9pQPc+(+WG`41!Kd{yK? zbprSe&s^(vf;(Wz5UdqU508i%avjxfW4uo_Dc+}m4DlxOlT*_VrqZ^Eijg_flAhWZ zU>oz{py^iAY5VsMZ^E>X$vDOrNf$Xa@g|>&KhA5zRmU$@V)EEz93|BAy~ntjSc-o? zYvn$Cw;jV*_t8HUR@iyMwiARECPQI`1X$(3N!!j`yUrAFYX%(tcSiW{b|GKMh-lzi zSV2_$d6z|d{q6eHpPfz)EsAGkQ@*NIMzbX3(9+PEo=6E@>qtP8&>M2tiFdMLuaxHY zFwBUmmzF~0Il*`8AAIt% z63Bl@;CU;6hh*U&h%3bUmqKxcjM#a=oYvrfs~>I#=8=UKb-;n}?~;WPV_b;jDsDDZ zc1*&mR{vs?!6lJ}I6a48mjQnqWZ~lF4>JTa{eW`47w89ovjApyAXDoAkPFSYk3xTH zf=&Ty2U2zj3hJs)=tySuXgo+R;)`ZmBa`2}KHcW>7W%v4stRosYzFdjBkkay9l;=qcTLYz1NTMRs1HV(|O7`QU_ zAnm4C6rcehUjZyNfQI}>v4jDyA$Y9=SPZz_O!2+N*CKpW&sC|x|8-aly~n1>1C$Ju zZcJHG1ESAF6bzD#Pox(n_wRfe!ENWccdXRlhx!`@nk1QXO=GIJH$3}QWUnRYayf(+ zsaW{kny!|;Hq%+SkQBd>06)3HKNDDJnF9wF78-)U!WqrLe|H4`BEd{xq5k{uzidWY z7{-IB=m1F2KlpT2kzlcoU`bpdPBT(SGca-m2Uoa+jsWZwz=4IrKq;JcKn^_lu?ix< z3%mlG1iB#5xDwhd0`r2fU&ABPzDKlG?IS^kIg6m9wqu=q(dk}v#%-N@+=_~x{f-6%u^mm)wAzERmj@2zScbap3r&K%Fx!NvG7#J8dmN6s16%7 zYdU=2Fvee|aOH`Gme+-OPGAxXw=T0GfHsUv`&34VJQ>`$Yk%J?!mJlI&!!c&+5cB= z=}cPT5VToTT(Gwu&Mi$i+u8J!r;KVCjwzcWTx={*)xm>6=dEKyJ#BxtsxYu(q7bKa z2(}t%T{aBB4Z-YcQTLCv=iabF3x$ZMEbyK&W~~q)z&@ZKg03paYM?$j+4r=dg1|@h z<2ySNr{ch*kOE8!jlrbQHmY2-vF}I9=#?XaIXiEwB#MoHk~xAdQoFR}Mrrz2d)egb zu+YQudcE7yo+i`_?iVlId~!<qGkY`+?@Xb%@UI00{DGe*85cSQ1wV0t@HQ0U=_b0EvSuTv|V12~tWPoFDy3 zGpiwlHV9rcLl$*>!EpjkEUbE&&PSA3wMC7zhD%#lyVT`&Z+qH3y@%U>a*5Gq z%cPFRi5cgXn8X{N-F2B{b8LEk_8pq2Y0}kx-0N-~rVQJ+kxaA9l8o{*vD%}HM57JVT>TBz&vjkH z-V$yhAh>P^j`q+p;7Sq;0Y3;Q7W#n1!V#?f@LUFQc1#HC2bC#*;|u`^pk3d=-9V+P zhwX2iA*>(V^Z%kB^!tC+4|acFhZsTevY{^Dilkgkj$p;4Ax_m01Ub2ktA2*F zhF=TQUPJ1q)hwhYDkWuYE1H;Pb}^4HTF2zN!ENnC`SG>Z_w1W9J!1DLzM}2jnEoa3 z_+d}oKDo4)-W@vM^ZH78?KFmx%J?URDJ7XKY|MH~@qMEJUunZX6!&?u_MaLB%OT^}vC7PXI?1(Um;zeAD9ugN_0 zM-R%lw|oPWLW%Q5T3Uh=@`=qZ&kVvcHfk6q9l9hFACjs|*%E#pzvqQ`NFDxx+`^eX zVoZwR+^^t`V-RqTBexJ4u(9L>uREb)(38xv>C0y)K8BMxo9ah$x7}IL3Ctlk9G(Sz;`^|{8Y(g6i&qSR^ZGh7@w}zu7 zw`AAL_)Ev=0L8$>dFPw2|1lhls~sm>tnR&I`4D06LESe7R8^zf~wMIS{ z@A(kPxB1yn1!J#PwfoWgAE2E2$u-}bIb>K*Bis;gL!#*z<&6ZkmD4D&LUap9~y5fRxd_N1|CsX)mVhb%X z%ZYwaY$4W!bao7Y0fiZez`aF4-t32_YK6K>^i+fkJ4juwx*Yv9!C|VkFI<6>=dB9bn$u z&t`w6E28{e)8H&ekhfSLkb0XRQE*X!4He!wx~e#^SRe5J&J;qL0rbn|;s91UROt+S z2BsUOHgxp~VU&Ypu;XArZH$986|e6;^6;3L7to3^a3R%a(cT;3_lo}+U%CF{3t(A* zp5{r9r%WRHkoqUN!59~_CJ)tn5{h4$I7P-pe+Tl7@Wu+v0#Kr1Pr0!jS}<5Ekw=%Q>8^=&v&oij68W z9FGgSO|%#e6kZfSR5St{7=0Q{wE?UlGgA^15D+lXf?Z@j3Q9tC20}e>cE<6_P6oog z;1xsgGeVf3;emdJgOW;WoyW;_J8kSv+So9V?LL0=xRcv)wiQ!_IJHBt%fN#1gbPgq z;ADbLRLa1N`14Td>)4S#3uq*$(rdV(+@-HM11u4ikCFORLL)-($GTr{N2q-3>Xdr7 zA&J@vFs~L+>T+8&8+vmcyEj^yNZhNzHb<-iDpisZ%pCGZf7He$_O% zWG#n1hm(V?{jD^VQJL_2O&*kcp~gvy$;kRZ0esIc#ZQv(Pec|1tpF5Rh&!I}zghu^ zBy?%QwgR;}f4LQ?Y7{`A69w5&E%#FY8U!qfBm`-N^GQP3V4!OuztM#tV2*n5;n?<6 zkYtXL3N}FsLg)f2Ld$e(YoHr6VS-R?>b`q+$mgnTwy0BuF!cVu5f`aVBTS1PFx`@6Idj!uqILdcoCWD zIFr|E#T7Ou3L-i`0R$*YD_K0-qnmN z?EWN(7`ZRRhKjz@ysA*JVqOp@6u_c@CZhQ@PXH*u5$5^#f1?Ix;tHRPO{aiv;NR5& z@Y!rr0J;LGDG=a=x`9G3{h%pjHMRGV{j8gvj_hbXtMwziBYabs_CAhw)srqKHG0^4 z;+gg_g^Cu6CK>djU9}ZrC#2=wT@qSqv%Yvs5}L=lP>CKe+HdDlux z5y-W$1bv-iQ0-Q8y6TXjeyODE`th{2DsMiUxN){}@rM>zj;-*Bv*F5l)rT7 zN6yF?r(bA`qI>1vw^uJt6OP^&LR<(GW_uzxcQ+}rR%u?=Ix?0kq zDJm4ad8#_sRFkXQu|3&%ze46>JM#z8^-}vj6qHnEUiiu7%}_t09q4TZB!GpTVp5hMJ|;$J@!c6{jmpNAm5SpAu?VG`rY4 zP#6*R0-(Ul4;QI%i~#@GaB#(BAx;#4$iig9g{pm_`%I7-u`*c|*d{tL+)~zv&RVOW zo3zbiw5JwYBNB>4)>@m0Ma#SNqEl1@Xu0HFG=IGXQjUibiu58zT+LT?WAk`lrlE~m zv)T-$l4a75X?pCe3CxY_-?W+HhH#Ni-_{Hr^_T2#PL?$G8KQ5e?^PQnIeX;0WZ%oM zhpCpTM|3>(KPXKI9H>hCKLb)+xy3*{Y;5C>_*C!A_Z(@ZnlD`)Vr1vz;lTo4+q+PNVq|cyF*%n)UsCTYtKwA05OINSiVdu}2 z(oo&1tCReea62p>{xflf;6V?^6?WHGV6)0*AN_M7V2Xf1u|XMvfSpYk2q@&BOfe`S z?*R}%dEa=*%x2^MHo6CX&eHdOTzIjIF-hi~a>n&G`$`oWCsV@yH=FX34mE}YeqfB^ zlzob?pZUn)Sco6k*Ul{KS`%AtnT4SVe-A4hn<5*oDz(^s6S;L!nP9{VVMKDe2pg)n z8M&$muvj0kB$^OsSm7E70TKuoD)oi#!+L?4$Z*BAX~Cf4D3~$OpGp6#-UE5aJXJ!R7)z%SM1X!$HRn%(KHmpV{DYU^NCtgb-~Q z2&RM3Y;b2=)J20}mrEf+{|MQk+(ZhQ+(b`AO^AGn`s%>abW$lkZbDUF(1ojsKjUlM zcJ{NvV^4FoW;7?%QRU`kmt?+*=~pKJr1>{qXWguR8CM4ga@CxIhokVi#xF zUynBUR6mmH*M#g+p$luVWm{zcu@J%F>@mOsdkok< zaMA^6g|@iX_yNnY4d0BatK8m|`7(OdXmSw`ZhBqlwyoqB6j0Wf`|&>l$_htC5ET`o zY^d)!wSNr)mP8POpu+hCA?zxUjbkRb_%vY7I`R2;KPYCC%Yl=CO1!R+F~}yFl@3;e zox;)W0U8uLu8Yv05vdpbZe(UKnpTv4N>A)WQ*Um|vt+MCZXw0BV}-qM((5zJ)cjO! z;)^8N6{X0mX==G6rBfP0zMtZ_FJs<(NKgFA6b;D%U`aA4y4~Pp)b8*kd>;zpCquaM zs6xx@@G2rLluJmvd=Nmpwy0H7^=hq|OA_XDnf{}<&;GGbT}rnWmC%uW4p1BLW3~<0 zi){nwN;#ofz-4F_P}WZMpmpPyH(NJ!J+jzj^hC*v`rZK7m7!mq=I$?V_scPUq+3iV zR8=90K!=L4q5g^PT2&BOF-M3~C73@#f4pkYA64yub#Z28c;VyDiQa$a9p6oka_{AEw7c8!fTjh&&yN0?p z`#7zsm9vfsJE-hH{9eLj!az}JaLu^1Og``<|3o+_?9ze$sWeO`#ygpzlvQb6`JsZ@ z+U#2@zUzkZl_mUBDTP2Q;A^5xSq!BVVsi)4SSeh(;YZBezh}?|v-q zVR!Ia#-$#g9lXe)t|x@$H-*+qy%T$y*~Kev+gDuce~DTtv`DR6;q9ZaZ|F$d{vvd) z#mJrw2R4wOt~P(;Y#>|KD!zU6*NM!OQ!kZ*d=hLdBS3I*sPfYnaz%Rw9dF)!+x2}> zdy(c)F+}HU00m_iCs!2;R?HFNgaX)AppbZ>!~a6}=WY>w`AH1B3KYQUge<&_DncwO z`yyJNcYr`ROQjH@h|SWNa(9!`Wss z>kt%YS6FbiX^<;8qZw? z@Q@;0c}ig?CZ+HTlv3!vj8*_}g8%^b+he)G10J>qdEokoxj`}vH+Z|q9B_l{A#U(E z;0BxLbAw*8zW*P4XC4n_|F(ZEvS)-0LdYJzW}lO}gcGKl6Hhuh(;T_tf|P^E#Z@c^v2QE`FWUw?0Qv zxkqy-fH5ET2D9aM!tcL6c>iylwv&AR!sP3PwsDfJ!|g9c5d*nmNKC?e6vKSl0;HCL zvoZ&Noe}UOItnaC-Z?_@{l87CY63=qx_9f1TD##M0X6EiKjwk%%D2Z=2Ou7jk`z7yqrwMWjpirMX7Bwe}i z55E8DrS%fnISwSSTcOu1WawJ3Ofi!pfYQO%aooLtCKsvjNsAZKN*baiDV$S29 z=XXy;2m9CfFW1}ljFJa$5&Bvl-Lv(dUdmkuP7FYw1;8TAXyICGSqRwW@M<6=_x^8I z1B1C@2+uJf1>c@!&#M&7*&zf{F!RlZasr<0Ggk(`@eS}6A;c>M1w*ffzAnTc6vNlU zpkfHEfdMx};Essj2T?ej!4)+szaf_TP!3IG7AB=g#VsI6DrU3Z6XDHSPTo@Dh0h~K zgr5qO7}a?O^5+IsCzv{FB~Xq;$?R;fQFlz*vf$9f6wsVMTx{}b1SC=l5^A1#qxyo8#b+z;y&TBQ)06Klq+dMGXBb8MX2D+X3Me zC1$;@!&1sN-!OW*0=+Ww><4Ue64D$Jmr!!B*>6d;jrl@LxtaDLO9}dNL!R#EQNB~C$ChEpUKPY=bbz|U?P_p?*C4=0{Blu%NpLnQS z2&@Vvz)7K*+EZq|^$f?LYoLB(Y({~R?*dkBa}G0Cy-4=WZ00*m9S!UK+!Q(b?2o07 z1#Dq^Zq1Ej`iQ95xx%>vZ>Hu3#R9?GH~%Q+be1Jw)!G2jqEH z_%|?xATR)93YP#(;lyPzaUJ-DXGo|XmF$e1Qbk_32(k<8?*;2^{g6&cC;QSK>L6Y` zc~X=={kwpwr2Q_qE@nHKv-@?)QL75yvOSqlbWj3B2f;u9qJ!Fx_x>h2m>MaLD0mHI zASHg_Ka+vkd4|L&g-bxcKta!+%fPfHA*aoIrugR zT*GO~wB=myI(bX@6JJhv1@@Fq{n_@l0?c|@5f6ov%&sIK(+y!&dOG|ZtC;*K>Xza- zS2l;qd!3OBjm-8mvX2^db#J3$LP3$S5GXQMZL!?qv#Vw!c{dD@r)4;IP$58r;&{Pkc``0YF}=^ zKzF&wtpB{qMGza5)YQ4s#ry`VMnhL(JsBHhBg6)UpmveEURLIjzO&s?pn&`}TrRRj zX@5KE0`kJw;)pv}B#;=dvuBsi=Z0|3W+Bng5Zo$ojYt9cZ-#(r-vf*tWuKi6LscU9 z=Lzf#JOlQ?3h2QAyMXufDX3#`TXY1q91APUVUZ>?x8z=p2I+@+GPmp7RUZglZizT; zH=4FrQ}k#~ec7gbL605g?~T>doTFtDhPm07#CQ}M6jF6;>sVTSRRn3$Wh|5VEW<`E zN9Y0H5V@y^$<;9YTdYEB$9>k|xDa9$wh|hMe~~97YzKZ|3cZ10IEg7V=AMcv+?X=D zc{-+WeLb>H<=2!#=btHs+8o=uf;x@&v=<&<+862mBKMHXv9A=QwgVMcBoNr|Kn`9H zCC#fG%-Jv`DhF`G!1$lb!Ni&n=8~|vxg-_^kY_8iN?mpV$&DKZ?&4wy1De+ zmpOr5lTsgkPvoH$y4G$;P)Mj>T~)O$+FW;B=wO98xqlImp z1_&Pm(*+=ikQm4*y&9t(qxre~752LOr=x29abAkzMW$RFR_Hw$=cx`AVd3}R^^uX3 zj@9rc0;SXzrj(v2=>MC(IIB$}s$8U|`@DkqYwPj04lo2pC89-R?7iAD@ZmEclHE zAwsAQJ+6N;bf|C=8gbF2ZX(fgohV?T;=6ISi1?|F;I!x=kRvH<@bUc?CvG#ZM}l? ziUG4hyH=Uv!pjqfMJ|(x;$ev+&Q%q8>#vBd`{*SpyJ5^BKKjVM{36r-Kid!v#;rkk z_5dyLyYY5jwSd%6Fe}RtunK2dhH%Azz)S;yusNLhAK6Tc1D<4)9m$?tW=!9e7#mCGEF06!AV@^^mVwurj6l{62|?qv%09_mvNX z-s(80v;=%Xc}e>+fuH|-^_XkSM5S_%<>oiCG)4{QcDbkRGHgy5_w%C)7SA!lwH)Ic z4k7nEG5OvV{smHDwh+(S$I_^v)sy`K)0c)5Z3Bc4gDJUzBDdw2IN7||?2K@hMvdw& zC(oYi;6GgfBw%#uWiG+h;JCQc{7M6mfGyy-_<|kJ+a8fOP{Dxrk9pvjGybwDm@p57 zkKP-F_JsV$>CyMs?CUQY{zNreQ3lS7<*s}qwHWZ~S%W|otwmzqrhcDS4Vbf8NHjhK z7YrQuV-1)#4#*8^CG`!1pZsORIOr#Tu?BStR62^kFKC0yNTG3njXLtl!jxS$QiX?) z$@}sQQflrB)Ym&DlV|q?b!MCBa~bhfwhY{35{5<7avro*v4cRM16VcYGevw-|75dO2 zjw39NQ2(In53^9d7ul&T+$}Jf9&k=zTgo% zyR#_xK`@0v|3-+4ijI#(N%L$*4kd~j@LZyNwqU*f5%qyBYXYo`{kGG8TjI62LR>@S zF8eC^bjl8a2*qsC>``nZmOJ@i>oFg9x_su7ipcogCw%J}?lG=Ow7yk7?)i05mgJbd zDN~}vck*5jkf&KVcTOR21Yk~~AK(-gkXC`2i2+CjUIf#@2ZoHDF+V2;p4t72#6ai2 zP7L@-AqpCS2;_|C&8rB^*)t>>9D+Q<)}cQa0r-PpVmGkWEzZcrwrH}092yIpjzN(j zsfi;4Cy*Q(@)^t!a`H7$xvnRXeo4z!BV6I(UJF^fIBbgU+lV~zVv*t--6`EoDbXn| z(aKS`l*Z-9M{_o;JgrlHwqmWOS>XPN)xwrx7U3=)RWRnis+2Eoh_Vwvf)PhxWC25 zo+s(cyq5&W$sBA*Re#6*emtGgeq(B-PmoBAW1&o8bjKx3GbVT7`u{9LVsYYZ3X1T3 z+uGjLt|@;M)25v*UFp4TzaMi>9X;o<1onLq3eqx-3eu%^yPftGVHd4(HdwHX?hQo% zTdxZ5spPZkjfsSiE1wwR2d6t==l!pXP*@z*9)3X#{ovy!`L#TjurUC$$%xm03tMEI^B(MPmJkn1#Q2`6t2x z_?x2PgfGRB-rj&x2#$(Yi;G{r2JVbyV%Ei6x0M3<;hOSt)fR9|z=s9r#p|k;2yoRH zh!&e$ps#heVsC28MJIZ=xev2rpJu&6EIs)L$! zu`d~16$b&jNMwV_$&?Pgri4KMOJ@fK$oyM?e67N{GYVVb&0p(8 zN&jye$EIe7lL*;i&Jx%dsL3lT5WCe&sW@p2T*!9OHn`EdD1>{KO(VDgX+-NS86>8* z?Bl!&z?@COUkCs_vf#m=3jmxJn7|Xi>L~eiJn4xAuMHUfnZZ9WqZkwp!Q4`BsB{3r z6IYztA8~~5*v5{kj?*GK!e8peGK7U9&&qM`G1QP~5Hd1m>#mCLtt z#U8hB%6jPekzXOWTOs8|M4zME>aLr7?uK0#NT5ZIy2;cIggmnwIHcU zU@i_JJ*X2P<_>^OK?c$O0SHqLeFfA_92NF=DD^go%c#k(?`@qZ8-uFGa6VepF$Ne= zNc?5YGI1!4_LV^nUspx7R{>Qh!%?1PRJRsk=y~P5+85Vwo9|10lv2ESFXc6c#v9mY zs*a487okJ2^4_is-qC(!h*vCPL~h(Y?p4M4glFJ1B3ER`bKzp^z^$6WO6@O#`&X z_k+(L%}d-<2W|@1c5&fv3R)}is?WekAQ$h<0b>DBgJ_@zRz*+KdOip~ zAz4N~7$l4Ed;!$pd}QXlssSmJAo-FHyE-5q^5BVwNQijIOuJb7@L`)Ds6&t)nrZ*T zRRdBp)qt=pnE4;z^8(X83wTDjz7#)Rs<>MGGU6PdmR|v9#&^}SGw>G$&fvD7(_?7} zU+N`ujem^8(i&IE8d-!6FJoUG!z%jBJkR8s>WJXA6+P&jR~aTDf{S7q?YnXN3_gn_ z>~s3ibt~Cgvn|$Yhg*M_V{(7!DP_(5f%y?Hg*{LlsE-lsDgNZX5Fk&h@UOrMvtd}_ z8WYN~$)bUuHU$$R@EfaeQ_FvB6?(mpMWAxlAu;`)xBfE~n3YusV1>k9YEU%b{KqN) ze;aJ1Td*5+54A9MiYN6wwA{3*@7S?pAV~Ob(i@;(0$Bx!!MCIIA!|f4 zK3?Fa*x#tWWbF>q5{b>U4^259 z<-Cr|(W$zjOeLSsLq9#TdZ79eiWOxhAQy48O_SVpfIO|jxx)&rbA`CJd!Y+Z!T67Y zUnszg$dE>{8|`yLHeah`CEap;*Us~dM?+7-S%E{n*yp<|y!X|bZ(==cQD9sd4zdD* z_@)u}K-31Z!kG}sP+G2mj) z$Zy7gu$*;>dV@pW!q8>(jnapmVH{9a; z&>!vDut#;HS)13f91>|UJC;1G(G)W%=Ak~`oP#v!R_(kXYGlfcvr5mhDXp%>jAOhC z9Laq*K(0RF-@+Aw$N+>ZwB`r6!imVhFSKChVd2?xSu3YBkxD<3I``50c81~Tz*nD^ zt(03Wj$|sxY_O~H1)R(Tb;wRE1Xun ztUTl2i_*&-->QUpudugHk&aRi2mno_&|%TV>&?|8B%26RN7o?=s^pNEE6?7~s~F7L zC?qNdP-Y<6lZa6`8!@;|;Fl*I8+#`KwGZ8mMU~8ypv3gBl3+io64twV=K*QSPM`VtFW>*x;mlj`gw}`UUQ& zQS_o_qhSTJV;RF*P2Ph-k%w{1KtoxkjYoA|_6WaFma}R?Sg~x5hEdwBO?j$T2`9t7 zmiR8$G~CX5Mf`tmVSr|2c~141tB8u0ot3lIDOYZa{~ukXa|acImtMe>lD*CsfC>qn zeI6qbgXzq?sO9JvFwQjY6g>K8r`CSiW zgD5q#dvDv5bHF1;Da^9DLV|E6aI5ju1>$GaBFSj z+Ipsj_&?|z!nLhOwC2krF^zH8=2Zgb>=qJ571Bcz(D>&P02>4o(BjL*Stan%zq7S+6o@Hbzb z&T@PenNI@b>lMzORM-mFjI|Oj2!5deaB%SdY;LGMUmh{=6$rpJ|A+G`0CP47L2hWK zH2{|j5IHD@+J_1}e~BxZHUeCE(|sMfCnh}9yc*CL0D}r+W$ba#JW&0e>y}P-Xpj$g zcuZJA=nm8_(`^lh)vs|q-g}@oC&u`l%KHcf?Vf#&iY6Pg%47T}4JBkG?T=?1()N+1^R#uJO5u?jg?-Lo3V@Du6}?iYw}63c5W;m_3n zct(Hp^byII?@UHcFq7U041SeIc-AT)F;PPu|CtKR${+-=LSibg5P3&=ra?%E70>(+ zro;p$rhqG@(TY;*+o@?LA0`@_fK)F(!n1^miuc`jKcj&0gCg}`vft|p6e~By4toai z?$^mvZ#jE1+G{N+5@H5LLYWauA|7T_=fs<_)}K<%x*(xnEa|pdP7f1LD8S5&z?2H|#r+)# z3CenG)p!M%Nh$;iu<7#;1)!Z&fHb;z%PUvJBzgno+{>dm_Q64+D_esa*J8OV#og~` zeBM~Z);Ds=?N-1DNo28CtpWlStBAymwtSga37E4*NVH`LZWZt*a!L3b4S<(|lN!*& z-PeAWf2ov|)G@ms58QBFy)PYEr2PJgj?XC24oxdE($(qqEIb>?8@YBfzr`vv zcY{R6Yaa>rW=FT>!?C!M^hZIrSiin8-(|X2^C82v>mB_f>ro1!%HcP|%Bo|JFp_&* zfD9eNxw8tbbA%`(y;!cA09N5MlEc7EgAly`6UT%Z8_-jZ3IB`g(bO(QM8Wk9NX*Ua zm*(~LC$$xvl|lH+gg|Q$5rgnIv%s|K(fykJudH;Shnkg@okN73U4FgH&r+Dc4OLB=ReC^uVkRcFRbinCggJqVsf8vr1%Nj#bH**W~ zk(}$goG4|!y#{Y0F0nT`aumyKM_pYdx$-o~5~ng$?Px+&ZzA`M0Que%&Ye~Wq5=?_ z@EC+9Tt6Ejn6ej`>Ja|t)uV;iHz4l#C?PSisdwj920&o9>Df84`A^DDUp4WYdt`Vd$XGDB$r%>Bg0*xbc{dD^r$_jANQJ-;;9j91jLsEGuMMq z%a8~Qs2s$GQzurd&jfx!2igTh|K}1C&kK zQJf(I|6hy9aX2MJYqByD^KxuxUS(j;ULn!E5abmm=M$9ydgMURFU>+&1~$^2Q3ZEI zeS6tUcfE%XhH9)&P!SqOD>L*DLn&gQ3h)^bi=Ji%R-aZThPBD_WvrU|eFn?b%{PXR z4L6&dG|E|IYh=FOY)tPOrz}sSt7*UCSH`i89CtT}_|b3(*vDt;`en+;cG_E|YX+G6 zIz@UZ#4KrAe)BpT)9$CA7m@ckK%Rc#-(VDir~u3;qyUUU!dWrU1^AB^fj^o66ih4# ziMNnX_=T`rpvG2Bb=r@pUsx49wQS_q0Q13r2{3O@Rz?i;0U;>sJjXnr>p@aNFgv^O z+F}uKdAtmNc}xc`kC!c>ra(|2DQ1odgfJ0Fw{W6rB&l=o>ow3d1V3mn70~kfT=#|8 zqc)PTIDtyUFNH8^a43YOV>t%2bOER1#&#-jH5%!Tg+z ze-KTK^ntpZ*QT%1>H~Xa4R4^I)eow~@xOoMRE!O0B7AKP=L`duHKa9X(f1UFBl~XVj|&>nJEL?zjF6 zkV3d<%zEN@U|M>}dg9fe`7Lm?0kDekq2Y2x*ZorxgF~KAoSE0f9!5uRlmk(k8#w0V8^ z=j;>yA~3WN>8bz6FF$M(P85#bFFe8XzO$p_t}jCEEFG^#vCW!^qEY&2D+PIZ7)0_h zRqqoct&Fdcl$Fh`>T%qtATrqEm&vdC?h z;WnFlo1+XX)wk5*j)pU!&YW8@*39=I;qV@vlU=&f$1UtOOP;tGeVNUtxGm3NeKx06 z4=c~u_oJC9j!#tryyI=H!}v4J&Z~k`LiE=6fKm3UZJcE7pjX*O1nQg$67#*ZWM0K! z&ORa0)-fnF&>BL-C;ZJQ0I5OUq#8)MzUl=m0>CQ=&~m&G7{k1BYF-!K4vEIj+h5)! z@tVs!P+L-7U9x3Sui*MaM-)+gs+o*Uyd7GoB$eH?Rq_$bjkl|MFkBXQa#^$f&L`@t z(yJv4KI}g2+rU*y_0nqKq&lKvbWuH{`Cw`y3r381Hk-Hj@p;I__NTD_U z9FY!!6w(n^0wkw_=}CbJr||NI{~A&lc1{J6+yn%mHM?P61%T9Ma8^#?FM|Si8vk4X ze%cIv{CYLgyVA$t1sMJXna|0porSsv(H#@uQhc!hYX)D?vhA#e7dsj;kE^Ba&9ij9 zGYu`!cFmcV{;3jGN_yiFONBXckF#D}=v$zq=2q7wn>8AyEH>`YbtB_TA}v!6mFWF+ zu4)~OLTEymqo7>4e48b?UlEb7PdIl(p>@vv;AN_&|3HYse`X^CzwinF^UYvtlM13B zOBIPJk9aq)GB9VKkf;p6l>#wDe8S(%0dQd8Hr)afiaQ@#l5B^*6|$*WLW{r*fFNk| z<8&wjzd&4~Wca)g#q;XXfQcI8RlN&qDVb^avpf{BGA&GXQhzUg{9~h%algh_X@)$R z;`(9Lb3&J2p+93kXh+AJw_7D|4ZUSsv%@?WWgBiOaG&>vR-*o?1hjUh>KUWm;e3Wi z_4jG)A$K7lPpk0nFbb_N3UPFMn;(N1g%g1RlJmfHA^5-3K3bTiinucZgdo+^n`J&X zgQVtxSviG(PdIZ600jm-TmD=KChiH1DO5~a_RHRqWaa=ILhuh@6b1zb1qDU01PAdE zP=%{jajZxMfI@1Im8`pA4^VKw-nSqx`Gn;x5+W8RrI|MBIfhM9$D(ONYPdW`&AGZ< z(#K;scx5HiwE(|dF|#hXm}aD}(MZ=FyNds-uGXeE=Pr50%Z+7R3TWm3pHuiBIfZj4 z6t)6F;ZZ!Hu;};Wz_gn}sWKMaYIfQ(756ZQvYB1vlT;!x*C2cY(UG zdND%Xn7^eP+xn9Ba;S^ErI&KofubSE9;hL75IcT6qa6b82)B>K|7J&6F`|mVR;nQ} z-o3H&ssg0^fmyYV5{(VP#R92BjKbfH1Jh6>du5$>^_oH~k~4Vj2m~Oxi!U9;D}z^H z;Hg{ED+Gow=V^>5_v((t1u{l%%;hi*@K(LQJVC*!50_!|bV0NBFh_%no?-ZN>7MN} z@eOwc9aW>$KZdP{e#z>}J$8`xWZ*g$stBKSe!UwJRT2H!I@vED#Ac(({hEM0y~4i% zDFntK3@KCxkiyo9;LvQx0X(EoVzFeh5YvW8ZRc@n%T+vI*!^T26&dXg$?eWymlL(D z3SYju@kkcy)}Cf2yT@j|V;R&U%gZWOSP1Xl&35uxh6_NIa`y%@m$W~G?h1qZ@43>j zeGH6I5LDPxY{PTH={r?_MG5!jkeKx(8HKn?HAHKuIug_IwD3O@f>{}bL>Yx}VxWV_ zI6(LdVpic|mQ7aBM{RSp$d-g%RvR=wFLcn51G>1A3`G{5<&X@=1hAB3DmC zKk#Y#BB>pPih4^UxJ49APGz>Mbw-i$3(9P@k5*t4xiIValqyQX&ppqSgre8aBF z>WExLGg3lfTddXfQ3$p4fXKC_qq$q%=4#UN-`G!c<}~%h+qC8 zm6A^O(l7Ys3GisJXD?JZ!UOZmKSBKR-T4 zO)w9A+qlvy^)9 zl{&&RPXmd`%x|ApC782yNK^^%4FZ3v1k+;#Qtq$bb%RwQVP&}WEP-DR{)OPPEC$0t zv=tmBPf7;qN>{3WoUZApOIG$MhMp}Nb zeOk|2?r@@zb7`+1qkCs*N2Oe%;#*nECeGy?mFy34?%uV1u@UMY&cFVsW1M9FuveZ2 z0`&#F`TlX^^ZMqK+7ZsmGb9=sApB7O(^r4`5OD4G_hp2i_Rx~>#|!{|2S$Rxz(87f zMff;05W5qnBz011QDKygzf7>*@`jU|=X~O2T}3E4X{7^iM07;-35yB8GBYvVZenTn z!uYA_>Ex8Op-=}UC8J^XF1au_-uHUr2|-jbl7%hD_J6Zp-UnLe@!%nNmMZXjNo{!Q;_2bi{mK~wLsN;0;-V5x!ba}H;}5N{chfgt86E8 zc6XM2DD$CnuhJ+}gq8wDU@0(KQ3jR*a#y~Q`pF;mMFWvsqlv_H`Gv90r$;bnv+x)1 z{}QBU+k!uR|6z+@GBZ@@p7ai~38N*T9RMg7=*F>vs{&vZ5=zKdT0%ehH3r4RQs|^u zb9@+tFNe`pKNUVFoZldvY45$9=5idm!Rep+YV1uzHvOkdn@x*dy4$*%(xS6ml9bs! zsLVT9=tWkT$)udns*WEHs}1b+!-?N4TTOEsb(*}t0?5-T{5zaN>l}aHT?p@Bh*QWg zn=gPVDp2ohltPX3#nLr~Q@2zrx{M2h)!-O2v>J4=J0#tvE6r{0%qlI8%-NmF>e5qG zuUz-otadDmT4ag%?xo@p1qF1sdJCc5U<e7Tfm42t&(W0)Lhp2(Rq6s>qOR|Mu8F>lVhcZ= zu`*>q%S7)(LRb(kzR)X>DpFvxk=xc`cazWRWs(fvWE^w__HkJ_+c#$gYmK9Vm|~a< zGi@GKZ^!r@*;=@SyrqDA+ehb4DzqkoB_z2F_(=+iyVbfVdvssA3WmJwPIDsQiQd0V zx%SY9?coXIrp>iz>vX?>EiZFkYvIesl0f|++D9I3Tu~wMsqjGyWgVCWAW9*ldwB>) zVyWD%=p#1yMKGTz8t{o6eqah|Nl8KB1uev#H$Vz9coaozp5^2*e~Hh@8Ldt)x6itUZs=wCVy^Fjsovty>Ibe zl$v6JPb2=(eXqlVSy7!RQfBNNP8s@&-1`T~(=7ZOm_m>lSZM3rcm%=}PP7b=-U&{T z0jBaSW2gH6GZR6@8!ZGDt&PN-t7-brWMEcyA>b6wObe<~b&1s+R?izmVy|7HMBq7WYSU*Y8sU^wKv@+p)(rf-(6z7`};yKwHH!d5(}a9{#d_zMk~Q9~NhelN?oxkz|;;m1{!MxCKkYDn)b zz5HIV)b5^lP0F4WN@m$o`>F&$C<~d~4I~kE1Dqu*pxr=><feC(jNGEgc4GU;JIMFSlY5_mNpl`%JT%1{oH)`YUw*#1p_qv4x zGJ~I4x>y*5*9mjsn$rK#U_95dcl^1zbGB4QPtXBJ)8*+BIk!^p#l~_TIF!R>x?j6l z$)2X0O6t5#BBmJSkx?D|Qqk3Da2HDli-+(xUCPj?Kr-JCkgH4hw^)TBB>=MuR{~by zL~;Pg0sLot(4P@9MU%_K>7t;U$)u4!g;=QQYP%~b1%GE)rln-iOg-%lGvSSl7qk z&TzGqpY+H3E5f7l9qlBazu4kYQc0Rv*zTo+7{CHic$gBfd_FgZbM^{>C=gE#46x4p zbG(Q6OHF{y!XI-1&&lrxu7Rr}n=1$o05GTKWme0E;)HP90QiN!8*bROeVM z-hGqruzJ6^$H!qM)amqAjW*v@u__lmZYh3TG`V{QWO!xx7l?(~cw(UsAQn!(^Aj}# zcmrV~E-=$FoI)(TF<48IShzsOPg(*XOXrVRIEd9jcqZv0F?qo$^C|$OR)w>&41W<8 zSd7G4{ILK`#0aecsO(l!C(ox-kE5&6G`x*C0_Xj4uZXST%zcj;tX4K2(zz^sO|tQ($>^p!Z#Sc!+jaw)ZT?!jIKE>H zbp=Xg9!jHYa8Hsw&Tn!pUM?bRvMnWB?`VR|xnVy8>V_yX9}39VHJm%K5TpiRVxbox z7H%V$1|)T0%8KAm`bcR3uB_;KkI2U_ramm^tO%ZgaxbprP7^Dmb5(%lhkcTtOe<^t zZ4seYk}d-E9_T>GSkb)dz?`i^q77tFFL^=If>L} z)4(Jy2!Q5tX_fJH`o))pUmBw`P8jujDr=7?3g5)`>-8j{QNvX1Ofg1Olq zd76iRhgt{*ewbSLruNQ~*~ozR8ieoe@EC)8Ea^pV%hOkas{(9dI#zw8mW!vIZw!;P zCOzgwnfgJ08l%_`$;Wb^*_RtIh+V#ry=cmk;75Q!9+CmhlJ$v|GIK0z16VIKb5CS zBN7XJSn@s=eYpOLVf51GlHs2Er&;LCok@XyqKgU_ z7W#MibezYW(+ROvyRePi?+VD*I-EPW5EPU{!9fDL^rukaPu~PnW`Uz4EV*$Kv{ouc zILk*JU{we@2nocZMHY$%C+dT!BJGmDiW(QV`fh5NyhFp#dK z{ee}1S!MX>z0qZLy{yb5eP_F)JX&$Dd0x_ux@}Q=_b`5y71Ug+w-mzu5{vB2f2ZDr7r^AC% zQuXj&t~#>l8nS1X-74%CRWZu1j9N?f=?eUXaa z8BwhY(F!N_fyBQFrf&yh4ER~VH-YhUO*YEO8^g&UQHe8jW0=X#DC!?5Bz_@#cLoR& zQM`Y>Q>iK*x;DoCxHdj7aRgusC$EjSoj80_vhA+bS~xs#1B3_q;PAlJo-x5EJS4*d z0s4qLPk|5=mU^7+VaJawM5=2FTMf zoIAU)6<=s70kI1uNPZ3wRRFAWgJwBhm+s>Hj*Ka$;Y@ZRGzrWvyRhP^J_4IzfW(An zUzk?~n6qg}G(H6P4V3({e|)B75KP0LqJ#`(r`Qwo`C3uCrHWx=~_~CR~yo`q-^Y-^IS`c%P>H+4gozF4lkHjJ(#m!NK_Bt5J4T$ z{QzheaQ&qoz$Zq8JmJSHrIn=(AWw*Ypuynzb;G!I6QhA3JQ_eV!J;PS-N{SR{cY~q z2zBi1cx%2oXJ`F>^;hCwKbi+;FE>+4xpyX3 zLC{5X%4-3-E!#h!7=Q@GL*2xQyY}dmb{R@vZnQIzJNfvKlX|vdV$XT+;@4?bGD)9a z$JJeY6P(w>a4x7V=(YU)^v)NGXf{wyeCyGXuOzF9+lLGg1LcND%;WF(=T!qp?FeUO z8U8XeP{8-gSN)fN0egmJ(0PHUN#EX)rOo?`S>Sj_dgHU&yI16wX+~p#n-(NIb)xAmPT8TMX<)1NII1Timg`zfZ-9g3ck7Z`@wFvCzP z6fg`g5XOOtSD#3BU|JbjZ09wUO*Nv6@_D$^yGXZA(GXPmx7Hg7hd0jevCkbU)WD(9 zQdT32H71pMVk%rN&;4h15OsEeJ>fs_o)GRJih(+aggyp_DlB=vx5&1i&_R^A`O$AW zhz83I5uT_`NX+$jujW+<=Ij{~6#}TE9JT+K@B1(R0v{frssJ5CNBj4V@Vf2l?yR=S z>B6R{`!D5EaDskgZUF*?)izLTv5}xr;YbcEc7YKMZ#kpp>K)3v6NG)KiXl0?G((n19K3GQXC-Ov|iU z@LlLTRA1QBaE3Or|27~`)9~+L41qfUV+_>*#t`%iO@H^N|9^xO(G=T|g)N_w=`J`h zKnsZso0c8B?_Z>&{7IVLo}dH<|3V3T|E&^up*A5C{a*$VDXvYfW1X&q zkAWOK$IuUO44=(L2c|fOQ&U8 zf(_4Dg`BJ6%3AdqI9~-1EtBYFl|F9Zbl0g`d7$eXV~ekgQZBdjQIb6*sh5lp1>-;l zn)0&#GZ~njZ+MNUZ%B>2Q$(bkbXMa4ND$!LNe4w*v@Jkt5Pm?S5O;90mGq$aWr$mF zR~391JYsU#s!(CpL6t7k&CRZU-6$o)PUWahH41mOkco9|7VTVsw!+1j1J1fFwxX;Z0t??|TjvR7k*es}#7Wz4dCw~Tn*aAPl9*~s8>ymC|X zfNB0G%=h~-esz|M1eP9)IHOs&ZK=GV_>Q+&t7|tdKX>+jN0MqO9VNG;4c8(oJ(1t z)ZUb4S9oIRTAQ~Wt10Pdxis|y7d^k+g~Zt!j-IvR9KRRQV(MU2wP^Du^NnWj^xktG z=Wg_XLFWvoWDIl#wAeZJ+lL9|tDFjOiZO`%CwQm2pawRA%!dIoG!5rYGHk`4DKbJN zL&jNr4E*p5)7KXN@C$bUmMP>H3T$LfewuxsrtGlbOWS*!pJP9SQC_@s0^9lD7 z$=i0Ft*r&%=ROW+p|9_=?X0XCHSh--GVxsmsx5B^g!!=Pd=;Xa3RSXt3^hFwal?gO zhZ?d}cln<(6uq9)m~t$nQF-%DoL_oK^#xQnXeXD>dg4_q#&O9WozZ=MAC(2Sy<_Uv z7%w{Ct$_|KuS7hIqcC2WBywL4lCNDjca&i(9%V=mp$zFK_JuR(hLgj=%*{akL$hp$ zoh)t(+?Tmu}6 zLm&n-uL+^9kpYRBb^zh-Xy#u`)q>l!ssLx$R=3xc{;lsOC{L{R5}^h`4DrpRZJFVl z6uP6twto9^tIe#U%qJ2XTN`^F zQ8$gnP#TTFNf}*Scv&AU(r{g`gyTJ@lO67gVb;?hK3ZcZL-ccZMZLf0Pk9G%0d}GNO`!y#0?I1?xy& z77P}eB0SHSAu+eA6X#V2NDT$EG7JIDaHbAG(E+UE%-4NZI`CshaJZH+7U~z7c>FU2 zq& z_Bo6$1~-n#z{Y-evN(2exJ(#3k#caRk4U zIfw;SrOV$dvJ|t72=|#urJvK@n&hycS!nnI&T&(>mz9`Xe)rXk{`04(Ox1Dc)3t5l zF@e>}4}y*?BX=DjPrLAM&gVQVUr!m2;&4B}4eklP#b|AfZL#~1JQD7F zSm9LHV4Ayu=m$NlobCsij1|YEI*QY;+Q>AmaP|ALfpUwM#x`jp1{gbIeohTc!j1nRH6Zq{Qv(U+h=O-O1loe$%&Q2@ z**GK`9)gU+fpQ|Lfj=AuPM`d~A1n&NlZ+A%uSQp+#eEP5?CtF@12++-9HU3&Hv}3h znlibC1S!XGG%Y&j*AZ+doB7^nshPqKmRjL>Gj#H^jK-{gbjM-?8OmISn;J%SZZm(C zG-t?m&T!KS=9J|>(450%CdVr)@Ksey>rHrXe0=mJ>U@*{dEXU~t8MtVFoqyLu(oDN z%p*#GG2BTo4Tx*OWG^96+fdqd-+yZx7QQn_+_||KiK+PXbza3_&bA>@F`xxvfb}C{ z8~#oVerh86KJYCOG!cbLz$C-tmTuh{oDM~8ew>qo!Ptkd^gIH?%K1L>#!ji945}k> zoVJXy8|^=2cKUDGXu+1gENr}Fwe>`my=BpR1mda+fbMb2;ht`pT> zhFdn+REq9h->^Jj^vg2!cFn_tR^p3rEAa<&m-t35o{`kCyZ(n2@?+(SbL5~_;wb&| zx(`NAL#@OnUoW(gE+DG7xfy}=--5)XbwqN^r%OO;J2)%LkZ5Lr9?%L}8~^3YU-yGx z#9Q`i?Wz?J#Ss35gJa>jz##Ah(sSdh2*9r`oweG`*B>|fT%~ZKY2nD@9JrC}jOgQ4 zP&w(-vACPPkd8%8xZhYwWvlgKEKSO1AFIV**)FW5Z&_ntJ$&XUZMdX?^o7nflGY=O zN*2FmbfC1-A8aT;{J_uH@6rPDo(0I)G@Luluodxz>HPlE7st940p)Jpe;ZIck z6(zqvF2wn7L9{*w8sLMyJg*usXQvQoz)Xh#ZW>rm%pm})f_x%h2?VEb;-Ux-F&pFW zck%H{g&sZ{@GwEYfG5-+pDLCCkH#pyHe0UHag*Vk6{fPOZfXxBp39dlAGWIBZwRUe zoEw|Wo~Do)NKOaa~|c)I|wOE%cuOVHbT1qW;bEA+y~qiK7$DODvxJ!JKYwJE;KmOGdG zTe`8WFWIvjS`TtUXGIe>!Ok3zD2@h+Vo*b7dE#kW&syfUBojpKkGCKO(zYTooh4QC zDg~s*fq!onLZN{I#hFq-4Yr1)&J2D!Dhlm8N{@g0X=wcY@Xr$Pn`EF3-u&m+tF@oy zPyf!TY!p$FDb#poIJ7nK1>cfTNLF z$UdsVld(MZtSApYp;e;>RvO=Yy)IPvq(-!hWw6`I)i{dwnJ+h2%9hA+#bu@TX?hq| z#@+L4j6&zfQ-2jA^O=A=&BDJyGX#+Vh-O$|3}}WEWdo#CK$Z4aW?{Jhe`^+c4Q@rC z?rcM1vcgCHGZmPXSxA&+2$v1q{TC_#&jXWC(+vTxarNoC;ev-^(e?mo8n=!~O?Vnleyl2?6CD@e`ejXWzWV5rAnD;{fMfwq5d_cTRa<-SD1 zh6Hy|HuOAehsNjvr9Ey-yCbpg8m@X(7ixlXX}Z~CcuW z&K+iGog>7x-3wh18ccE!nDIdn(cZH;yV;5N!~&&&){N4^DHnztlCM?z1n+y9?F5R4 z3QQ|Ei7oGGCJ`6dNr(%)gf9$j^DpG=P36x#`0(R8 zt$(vp7N9jCEP2l zFdNBxJxH!r;oo8zg4_VaEo_BYh7-8~(gVTN6T_VmUPVV?HQ?;;h`cOPxjKw$BX4(K z+h>}xe8HWz_qIo+wDF2b;Eskf2(eejT-Q1IHV(Or#|>wYu-xJ?T}zMcu$Qsi%f}3g z$1C;A+WEip?B`H$VTf3dFh6zQt{-^8l}9p6oO*scqTnTvgYKU6 zd6k1Xdxb>h016Hi)cp%NnA%GE=E{be4G$s8IW!93yGQ}%9IhFNAXJrlL#3oXw6-gq zd<`_N>uop4(^5_;-INxQ$zUODA!nb}{Verd%8t0cC~hvdeZxJt7xh`%yNqrtvr?IJ zn7SIY`J}Q}IqF441p0{CZG5%qoprX!uslk!bd<_UiZbmtQJu=jD_D{@M?!OeABCPJYT9%5_b<=vKKMe*Zf!pih4G z|Cn0#{_!Y#9!aus;mhraJGmA}OwE(hd42y$?FVON7ycqEK#xR)6R`__HxB&NIFR+} zTWuDr6m(a_zm&i*>xG&IywT_<;OG!MP>z^p$ZB&1E#?%8IgB%uE&O}BqgphZrCD*A z2uApealF}J20FP{+f`T7P}Jl=`s1AXRb`sZ>lhSd`syzkJ>Wg0g^t4>)ytBPp)1^C zlV9Bwb`W(VnzfCG+*``Y*Djnp#t=jXV2q(Rz!;t&`C&ka4Nch%Of4Wwtmgmuys)4? zh!dcTbm^8KC~624n|fY24+UfuxfTfQCm;nED;noj3g+w@5|skTHT0|?Dg_`X-Y7Y)ggO6iF>Af5ZDdZ+PT-G{8(ISX%e^j0gO zqQg(GIxgsuGN4tT&@Ln(xgF81P424!^7IY=4#yDW24Ie1YwaBh3T_IDi~GSu0RKs6 z?Hx)&pm4%9oLNKE-6O)HS}Y8f1iIzwuKTh=u$qJANxOBVZ@lw-vXq%y=JlqGizn*6 ze8F+i(X$_b)r3xR0r?A&eH+8Y>bd35*Ilh5dGH4bz@Cx*Uddoo+N);?^a;szwc zSzQ?7K3O1I@9jWhg7SLjRS8H91+y{@0m)E=f`WD#?IH>aie+@vXN8t6p{77kASq^g zhQG`e5_I6Fks$xmsamLnh!vEL$$)=$oCO~U6Qeq>EPaA6p6jg%{rbp`Jqf7*@Zt7g}*rM?tvn>6u+`&s^*l{Nf#LiteQ>evQ zVQqCsWtG+qzYHxA>eu-eu+YCkT61d*(+5LxR|4`i4gUhj5H2FKCL9#}lqsB0fSJ?4 z(S|In^J}$%g>K7Semf1sDC9ZKqK9zeq5uXN{z)Ba`@J2Afry<*%=;Lh74x|#oU>o} ziwqHH8_@ojB49*UZVlQ7T0%On_I{N(EC_z+gKlENv)dntf`A$ZE+`M*cpEg2w8h3Q zYdL%JoYz|4b(+k9T;rQ`lg#$1jf~Y&<<9TWo%S1{j8B7eO$7!#PqwCw;}_N z&4%iep(BEy)$2m`>E_&((u|Y2xqa|h2b-|4>9)*lK4!kWYBJvwlBZ?(S3rjNPE!pC zWVo2{RWLnU{A(Q`3Nrk{?kD3&1Z0?eUh)0pDDVSre#{9(3D?0><053ltH!CRL`L6&7M;ii_y@f9#!kI92=m{wqhO zWS4nPLQ)jUOp$pWN(oWMB+5)m#whcYd7esukz`&w=k6fP~2AB(~+=I%BW~R6`m-Cx-;B0shx3exm`dZ?q8sw{@s0 zv&QbMss^msGsKu3f~y7^{)Gm>Uj$-AYxPmXK!LqjO;m7WOf+~9I^mo;+~^x)@DFfk z#MDzaLe0REyZx4X@m4E|7hyvp0(j??HH6eZ$j$n?;&~_!7H51)nDt*zknwmos2=id zCt$bKwB2UumQCMr%+v+>3uREWIk1$4kQm~XJrf7(`Ae8OG0cl z4gVI(5X1){lwmW3GDMRKmi{zY=o$WBswEA&t%Hc419A}1UbCuluwu^;qa47E17R5U zfqys>RGfck*b9{s&MyVxrSO?3Xv~2UxNE@=)JiCBm9lk$hT4WZ%HVG0a6n^ zQ}#^^sutoBaZh+6*OS$nK8SzjGifGhK9HYOaVe3x+*pn?q+@%5`fT!vgDI35dqQ6) zvm{@vl&P8t3Jv@kjN8?N?M-A@=o+q^WvF)p1{YdE;6nao%mWKg4F50Hk|xgSAhMt7 zQdN#szFAcPSZY7GEYt9pp@GrXe=7ich}MEC!m*Buas9%qap+}7oCD85O1N&m^#GD( zzLKy+@6jCJOo-(LS(dgCjnbNcIF=YT^)1H|;}!Hn6KTqc1@{d!rCND$isVlY>`VS~ znc<|PQvcqRPgg~)ra8A4B}b=r`|R)>G&^;4@Qeu};0_DlHVf$s2OX7J!wB?Auw4MK z*D_o=#Sr)cYXsaT&OsEzyd`yD@se<%Wr(4X^nYs^<~-9yJc>F_Rf){~vZ_L`V#^St z5I_ax^>G+1!{1E;KUoEJpD$EHRsne6k`X$v4}g~fprx4U7(P=IIz7Obv7^hkDo$m0 zlBH5ZP&c{nKHhfZ3FUV<@44mttpyuDiu=+FNB7XrWr4TA!NE%&`^gKMb(bH2gapL+}Lva||T_$8hvd4Fjlw zh@A`+64nRf$5tH41$D;^RNOqR!})dU%WeD@22(FmGWg% zCGSbxk}p=SazSn~1b_Yy>`nUQDk?vHtN(uLOc>!FayyM6jSh85)sOp$72h*|+(4|% zdz)8Gy~o5Vp*grPNKWQ;WwOCY4|gEZjKmC&TAqWo;%l$)L;{q^-RH&n4 zAPw-3+`x~}!6LnR(_rzwaA8w3^mJ%kvKJ zV$gwgpnM=e>CbflvJPv%9UB(8Oj2>oJMjYahCuBk9^ivx!%^-Y3D?Q)yj0(S%z6as7aj(e+7mPwcL3zb2-a zqW{I+?NUGPZI2$}47VX1hqZ%5t!`77c9lHe_E6=pdsYpWCkU|9I$Sx+P!A3Z=%Mjt zU^?K+FMsW79V9omWDD;+16IhDoNGLq0?W-8H(|>Se(Y*xY#1 z$J^2Fv2_gqnU29A(;KnB0hx|_=p$U3z{~&QP3x+@{3|vMF&2@*g#!^7OvB$z0Y4QE z2#m=8!^Dh(Y9B%M3~bu8DKOF16EqNjzkx8og#%G?k`p{MZ+SUo_R1U!IumqzYvWck z#e0gip4wi8yKp%HqEOh!S#F)&rw}g9x7OSHh0~77*5MX$OgBxK z_uo&Rj1;6D98SN__C==FyN$>}Q_T)Z(z5w23zkm<*lQTBoMWhW1BMpTK^#N6C1n8P zMqoZjXxC9G^z&lye`^>fj-Ehd7oVi6oJvhxRUKHdVTe%&;JSgnf1v}gVfekBSg44^ z0ksokVlpr>fS#uM?+*gn8g}r|vCJr|d+$!`mp^-;${!h&Bley{FgJO#w6Bu6;5+Mq z`seXcvUXP(bR1hJ1{+=_6e@ej_qy!x&t)25=3bZc#6Y<+B1nkYLTT2ultq%m=Vi$0 z?d{c`m9{zSu{%$UordAx!5DU;Fox$KjNyECV5x~Hfnr#z3UC}tS)AuZ^B2GLA?q3r|3wmTnuog?2VCU|fPbSw9qExmM|vCabBa$Q9$ht{ zs%#C)UsWYov0Vr#hKqNAP;?+56N4-GyNLi&g4&1Z3{hO~$k?p^6{3UtowTwL}cU`UBKKIk-9Z{;AvP5?3-LxBv@_TVqJ1MsK856z_9MP$gG)Pzu1`5BSLUFF}=#zc=HuOj@3j z_W@Hk?%;7&k)1oVCUOU!r{0$^^b+NCE|~a`-kQ|v(!?}ylAF~)sVXv%KH$z4JIRj7 zX<8cds@DzYm1}T3q`4z9B#wV$KhAZeDgT0UB!r>%n)PA!DBaigS){xEpKK)x%G({Gr~(PKhHeMT;W){s%dUTsS8F3H}DP2RFB(*!gu zra`Huo<;(ZeWt3(hTezCwKPrD)6CrpDW=SkI@}_J>kYiPx3(R@_OSpvUBkb?GfYSE z4BY|GaQA#z;D<5-&|C64FGbkQ?5WHb?TZ4Zu%Q3r?z7s$(&}{#eF2@uSpQG>>P6CDj0$)Ncal6 zAK&_%O*xZ}LYOc+m4^zhZ#|i3Fk)~0<=giG$5I;5oo2_4y^n)!Z1a*gnLV zA%g6~N1Fdi1HLoP@thxmnE?H4xe3U|;>-qSAXZozR%O85qK^k19ph(ll5a{UJ4H4C z`bYO{H%HbbpT2UAzWSI(zv^w}6sAyVjkD>4YUbjDArk`BMBJK9y6SfFZuBj9G>oRU zsl6WiMBisw=Lg!Qbsp#nPNXcikQ3!*MrL7qeh6FL!@q?y1PKBdXSfdF40ob-hI*K# z;2)F}tlP7}qf0h8HGwM(`$zx91^Ud7d9Cb33dJ zWPo-*j425`*v8 zMwbe+#=xBsxF~q+0iec#oYW|eAr;si=sFI3p7V^`T)dGZLCjzS?(v)p2GYr!l|qnv zs>?I3#mfZ^vY#?c4^FudD}@^wW}=#E6KcD6oW}YzvMR+Y3TzrC`}f6v2|%47C4GvX zVHqHgAjUnGGil(&y)D`e+rVt z1iWk3Sk=`jk5wwvso+=dtevdl!fQ-$XmOb8bwI!F^D#jRc;kTiQCzUS|P9I>W$olH!MVlkB$WpQ&RKSM7-X$d1Q*k^YN7q z-bOiJ3Y#*`b}2sh{5KiH{OLlXgOVS-#fk?CPj&QP@5rvCsy(74z`E;oM?#T_h0{l8 z&v#4mi(Jc%5COr)RFw~J-d@#Lf2lR$vW!EF(E;>q|F2&C@81GUeS4H{2@a$N*M8@k zQ<0s6WB@ucn4Kx^QZHrq3avkM&dgxYfVSsurGUSo}G>StwB z$Peu!JDu?$t@hG18%>Oj-_XJf5<34#u7vZHzrh9BVi5J&mT`*et-4X=!D2t+Vgx zD05Rp*MUr)GH(rIPvQr)8+hbmsA*2cQ0wOIUPs)&cC%3G8gE=Hm&c<#>NNd!=oR70 zQHI~9{ogCVLZ4t^5!w8Wq5lklMC5}QyxenpD zt*Qd7*fRWu0>IG1f29E5zXT|q#ZLBuKH@S1Zu1%gZYEw(RV*WKwrkfebX95P7di$> z6P678A_Xar%pC%I&nb?d9(-=v9Dg?0m10+G?r{Ar`3g}F93QEW&ml_Nqma&0{sJQ5 z6Iu;U5^=knKTQa!%pN=?oX<>CvuwW}sxxhwTmJ*l8F39hjjW267?h zK>+;~u%Ml|!9zI#AL&s~)D(=*P|afp-x5rbf*vvxBE?FK14O?=aT~rzRiR5;J zc@3hq&()t!rQx%$--YfZEgs5&n5G@DVF;FjNuHD=z9TGjEgZL+_S7?!u{c$vyb{Z! z9R8T@Sm=Fs-BLo3*kz2!&N88@yyVffszR{TD6p&pLyV3glpPrTu`Jb zF(+?fL%-4+Hbq@tMU%;dDgQ9)jK{3fUxZ_PtamjeiI41XeivesULT($f0tE&;lZx) z^W(3Q_eN~Q^*kM3>XI0%;N4t}<<|t*dttb8l3^#R*HjxK8EP+QAMleBo(JLNNbm6k(KbM=xvy$`o#ypj6Cuc^GJFK%lxRW6!t z{5hwl1Gg5;0vt+1J|s>h{xq?Le8u6G23dJ(75kkwgb&e&pPB2_%>z)=G1eD^ODRE4 zmI>m~SD*ylqn}q*307Baogl(a!|-pg z3_)rDVi}G?EW`Q2fu;9?3v>Xo2&^0a&rAfRUriA8HKtURWo-dlxIn~+92GIz2qH#X z2?*CpZCCeVjM*6r zyE9Zv>Am~Oaxo^JSUcd)2y0PqaiHTZPMz=klnGea1~Gpvl;ogDEB<{7d* z%TpiNp_{9h;2C-9uO*+;IymGSOq$FNVGwG z$f-*g^b+ z8iPS4xjplCA^4{r6NnGxOd#c<^Dqf4lc6R_BpeKQ>bS2=x;;!${51O2y+adMQ_C_{kXP8G+(Yn8?pj_?9;G6W9--q^Tc+xH zc4eeR9=0uy(vjBur=Y9$O!IA8-63+u>AAe*605%bL2h)WdI>yX65if5F9h;5hs|%? zYp?ockN1Q^8_5nHY$bGb`alu@KbC`^&Wuu=2*8n1LIP8IWqSJl zgW?OGp3Af_@H7$J8A_x_@Ve`&6eSx;zK_h`^-*Y=-o&J_lBqFeERdB!QJMEsm0%-l z{8x{mfHn??O;db?CkRK#Sjfb-%d$JGg{jUaZcQ6kyuRgYLVj{;;$hWKp*Rg@*uE^p zLZ@)$I75&XfN_Sd0B1N4IRCMM;F|z^J2n0m761aUJ0Ha}O&hN@Vlbq5k9Sex)o@n@ zL6HUFaBioZjfs7OPpBXR{Qh%-_g_S;YKudI5c>TON4@`5XZW*eMTbH^1d+V9AC7)$ zZD_&Y_(r-uPPY+`6rQynHh9D;l(+oWU&9E80C0ue6Sg;*{ zt4SLrONjd6E#c6FYQBuQv`lKn1CQk`iL;%uKQDYDEme*Qr*bIH(<>pe3> zfct5x%8DzatEvPm_6;#A0n}3d=>4B70XzzPDN?G-TTS?Xv3p5~oxb7U zff<57?_qRd*V9MyIbzH*fSw23b|XCWgCi84kR|6iQY3HT{D-SPHo0EtV=X;n_tq)Y zcA!YY%!@!daxDJb^=8u&1!9hy*V1uv{V{|G1R!(?Z9(@Fbat0@kLjG3ILQL zPyx;7?|!2KLGGs!@kpQo>7TrqFf0Q9+KOvkRN}+Nn>)=5 ztSzmF8(0#|;?Bf^oni6>9g-vI(`8WsXQJs)4SIF{6ced2*QTI}$fuq|er>TqNtyT; zgDzsX2w<;exN?pm=qZP4$I!LJOMeh7m;=6-kRf%HdlF(q#VM<#XWhXe!7UsuA=Dwk zt!_mkqfB*HL4uYnQeHBdob11aG@(iRp5-%SRL@gcZzzy;@z z1>mQ`0p%hKdsPh$jU(W1^oFn=Y9syLPl_rWuzr}~ewKx2%R4DsncA4P%YG!|Cj>kS`8JX#)WLaA22W?E7)2dvIn6>;&c~Xj`rlYUo zMsVc3FtDgwE|4ay$Wfo!t8N!7F<$o>&2+%k1_}gYcxJkGSRZs5b1`P!0 zcJgXM$Uub00n{8&4b1_*U=C0cP0c-!Xig1FWs;XooW8*$^Jg} zS7~i7;}ScrX7DE%vbwal+5$mZVrqCpSiVeY&03oW6bW}e) zi6~m<_H7PMa98M55>8{(LHk;s|F-5;I6$}y1PH&t0m4kjV(*W5ODRC@C;RD zO@7^~D!_`ZLX5E?I4%%|p@tAL4dUG}{Kijaq23youHv{}>8aULKr3Mp$M;1LF}bS< z4P~PdW*fl+dxaoO%o3fK?*4_9i)l{cfLvF2;#F`S&@-`9mPMDQ+)&P1jw6IRL{s)g zaL1M;$-DdZ7Bmg0UfeGc%*q_;Zz_=T?CK;%4288$U=gda{IFjk&XhD^Et#&tfrD6| zCdO8?@NZ!ZL2dxT7*atPLo}&C?>hXWBhZhIpe17d&KR8@Kvk3G2TvSCoZXkQ&@lW~ zP5Lt3i=ZfrRon2%(boF009?wWXM>5^sLLYm$(-+tqT0;ph*aR%tJOqoVME)F81yU1 zKbbacpFw%PXYP%HMBgDCmW?^!yr`McNpUbQE|J-4T8Eg7ymIMb=~7mad2NWeV$ThSE(7{aoj2Dkhdy2`^^bHlJoAX zZxc7edoNnqi-*SeNo5On)kAV!;k1o#Hct*Qhp zwIN)VWBAL&z$44WZ~E^f;HUZ_O=mHEP(K7hLkplaYgw&#bi*Dzc;%Mfe@A(mlI0ALx; z|JlEU4*XC<7TJ3@z@Q4zkJ}n-{Vve)?GFEdlG)Dth%=ZB!~OJjtzL@@348n38Ec|S zhu}iO2&jjQZPF(YE*vogg(GkwVeFy0-xLy#H(MZF(k!VehrfnxT}`K8#eN~i9x|v^ zEdDNrkAU9^0URCperc#VHCqy1l}o^6AinGUf-vEhT{l7RNN$xuVO^JM>2a1&d>x+` zWKPNEGOq(4q?HC%vt8Gz%vbxOH0c)2_Ho_dLzB_eE-$I$G;=0FL2?R>yfyX0@fZSLZvx5UGl6KZ`8ys1^h;A9|f?}GWwGL>cNFdEOJgnY@U?H_7D;F`i3iq8G`TtOfI|x$c6lW@NE#Gz)bdkif$KcQXO(9)8~s9CHvWp%urEP>52vr;r*3o)@hU z!vOspU;_IA0bx0l`6An(BR ziLadISTZir-ZsF6Plt5RCZ`8gXdBo%HZX9G6|MV<1*wDX0e zHVW2{t*ZdbY!+sjU1#_W%Pg_%EFwGb993mtNymSt1IzOc?_x+1{SIfoczdv5Px$!z zw}J80>?4SC&H~cJFy9>drQ}jQ)ItPm2~AB+p+Z9U=d~RAc9jExv_X7ZF3OZ#izGeu zEL|vVHb{yi@{Gytl#j8tc!E;aJ_C-a6&@XYa>F5_>`sA`Od?EgGd}0OPMN%! z*f(=)i=Y6@IpXo^o+xfMS08L&6JV!4dmKmW!O3Y>SGoHv|aFj zm1S7^`W&MErWI9X<266#)m#&<*f9iBfXOl3K;nnh?D#R6)lK$=U(Hs9RFU+dy_8GKaWVXvd@mgQ;%Apu#I6`ZLz~jUQ$GDT~A= zz68Apkn5+RR?M-^Io3nf6WMdL=RSKj`%ryzf^2vyr=&=v1KB6`F?M>Hwtv@#kJ&wRfAs^3$^ zPWU-Jo=t^xk%{P7S5)ndx4yRqiLYGky81?-BDwdKST5yz^Q_`J%dcyf$`T=OS|K{W z02#=2PhC|RSZYPMEZ6Xt!GU=G#WH|?+kg2R3kQP9+rlq};eh~sL;&UjgXdrx08DdU zQzSSt1oww-P@WJ+0HGf*m0V$vYGB+$(GwnWll`^I?WA+c4{;8iyiT7HtMcaHvk)!g zQ}Tn_={M5tQi@Z%d|5r;+G!na72ST0OpJXjq*%r%xg*(%n=uFXL00-d#bj6>8o*x9 z@GsyD(aok1&QRf}yb!9XbpERV9tM8x8G1YR|EHc|_ZKU~Xq7cpW$0+ps;a<>JwuEt z0JjY||MF%3n2}QeG}e$ zO@=8Q_n~JIlM^u{roODL+K{ZK{(LKGka_TxN;{d4GM@|;-6gQSc?5eM!i}jQfWHcU7zln~8LHK3_v&*uGtN+yqOT4Arz}I)Yc`00w?GD7R=i(T z8CbDjh%rM1cMf=B@C$z@1Mp|T_w9hb1qEC(w2fgaL^5V#0{?*jEm=bY#W_ZR<$z-U z$qWLCz=5FNKoO=2w!JJ|TP{i+km-s}yxfh;vd2)qnskTSb?)n$O+8JwGC0z?1~eLM zB>W>|k2&t0=x9hw8Hv3>KY+)*p4;Cbrm8r=DwsK87~5$D*y$Di4U!@F@P|o;2LQ?N z+sQFz5kRj97n~eJ@gY-6P&yFLwn;eH%XS)HXB*=THB?njVq(sxpF+*ILkA24JDV)i z$TbJs_hiPNsN<59x!q6xwqBfosGzfqvk2T7jp=DdH-7HS0J&&7LB4@*^pNH76^Jz?^=W2Yc zq7DG@qJy*7CfuKf3|(~<@R0X8oWt7fi|wBU*l8B79AO9|18alUM^t$LgrViUQ-E0p z7H0+^8F=N3$_#MzhW?xx_%;jtL1uvCUuOn#Z4ue7c2t!SaWSi^0xR|mF=mG#&(I|R zgJ<}=Szx}4P-V@2)ehs2C-=z_qaWylU^9sWv>re`0jjDeR8>_S>MX7GukO{k7PK?` zv}CP&vh*prSmoKPt}dkll1$}9a$DuXL&`(4(q;PdQsw1d5{qy|51$mPDo-t^vIv@0L>lcaI@Sfvl_|H$hz;2rWTTR2ig)juc0T^M30}zJ7zuyxc zApC`C$l+l8?aI*U^9Avd8w8pWXTuCl>??h)ShpFJ$v&}6-L?7k6szZBuUKC7G>^j{ zUGF@1Fj6ug)yS8v(?t?-IABb zIj(kyNA*AprhV?Nsurx+G{mR{8-NycHmzI>3@ymE9oz;z;8!1?s~d>PeXm|6r1}0N zI(a5tRtfdF92sH~GN3`DaW{P*=p48s8NN}+Jl|$uxIRf*Ib4^a$a#8VfVSRZ|KR1^ zJ=E&l#s<%-bOY{e<72%jc|fi#nibc>%x7Q7Y4%sD57Gv=&?bsvdw>87EyI-)3_Ib7 zz_-8v<}m=39Qtk)ChxhWt9YH3k`<~X&M3T-#g$C-IVg^x-mK|^dxstAF!!4s=Kc0n zu8N0+4a#<$8-3rp+m z5cRS4RF$tLpRVegztm!ISw10%3oM&Rg03Z7{Oy1q4&b%mL!z~+(3e15FV}~Alu&30 z%`vAslh@@5z6vIgDe~c*I!x{}pD%d&Z>|-*>3*Z?Re*(zMb(iUA;NWppU6DO7`J=( zpG$U9$Rc$nt&YvGOd0n*MRH0s<3sAKmy{Tdaq{M1I>isX9LU0hR|t3A$h{uixaGd2 zZqy9t07Jwj1p*;#?;XKjqwp{A3x7bB{ZvObuLg^^0hsuOC~E)%zc51Zmi02S!}H$( zZ~#zLYIM-369?wO^N0vOj0%+0HlcJn_C{ z+&*XXX`A%sB(=C|iR)L>1ru6B201erOs$ZV2Dd85s{qbi!fltYUC)rIZG3N_E2)RE z4=34i_6MqtX+umzTQ6cep#(b}!oS2XMAeykzt|pii2(Hpu(S;P${|!A`EMP;ApgTTLc2%)IJBX0~2;j(l4>Z{9zt7ZO(W6)yr@EW%?zW<>cg~8Jo{7Qo3^Dc^ge#{P z>Y=U@cS7{Soy*aI1?IrQlfsTlp+&dF4OeMHZ-Evvic@)wJ{%v1sSYdO?gJ%cM<%|P zkiqQ2Y=~V5myjJUrtM?VZssr=yw-nt;7`VZ@q7n_i?<_H4lo_8i=s00W(4K^}UM!CfW2Z^@H|T{RCjik4vsD4T zaK2+;>3Lv*4*Xyeh8X|1CZVghBO>4>(1F(}ng5v%EXyRs$S#CC27)n|gv*)-zJC{F zIfp;r$N@NI+)Sk6_~zpHw}CPE?YLOqtJis>|JJ5V4fil+5oM`MPq!Xr$*E~d4^U4z zDx z?u;qC(YX$b_qbVS6bI8jwT3Vgh(@BfPJL5xu$xLV-lzj*)QsE`=ZC5UG} zd~)j4nLLDevb;sU^#D?+aQ=g!WIuFLEM6J+nnzf!|G*If3mGE?Tk$hev(c_Bhcq=_ zsW{8~itHt65>)-75W`_((d48uC!55lD3<%VL2zqmb9p~intaeOyQ5)$REkOL=13%N zwL|ii9%^ouoG|T(^K24Z(<`z3a{yc2!M{Z<1g|~SZ8wEZh*~(G6!?V{pjU(o?LqAj z*E|%ojMl#=;U;_8&LBK7rh(zEFY_g&VF_JK9Gu9-C8RMFTF;3(1`Q`%RruPT6ARgt z$rp7q&oLU)2-*>c@K>s?rw}+P&~q*Kbc}@4^NxF`UeVuLGCiD_;)KZl0JI?C>#J4O zf)(3?7_|V33OtI&U=RM`EO6i13%w(BE31BS4{Qd#_i{72qoP94P;j85Z*;r(dpSuH z!HD%O6apgWzVr~Ah$fd`F5P3!W_=CmAQh;5c7Ih6lHAa&nonaWy{)lN<~8Y8P7U95 z+Z0CyU)sH-r`j`O)cOvbl4=cp8a+fQfcwDW=q6qEIqZ%KV4*#@a%>^U2f)}uH-Igi zXPN&(0e&0?pb}DmEmT35kg7WvA7@OMFCk@MLzR$fno58Y(oCp?loQx~=7IKgTzj_Z zPNYU0K9!qJTRgZgpQ(vn*{0$^`Ux@3q|XU%Vz0i{3x~ zM15;XR$0ymC&Z(Y^Hi0t10PmZ0G3(~F3Tmvm=A!O$veF<3IHLMOF4!q0kcnN@nH~9 ziSPqCb@Fe?hwJ78bTMC><@``^2{HcW3$>_Rzu95$eyO(yCS}K>f=E)twEB`aN3Aoz zlFQLpEbc`^|IYdp@&O18v=>YaHrnl<+7wE6-7lj{%}HFu-&0gz?F033aY7@qQ%EbH zt=vR8p4}`ghm8*AV>_q_dws%{Qwu>(0HzkY0BRw+lN{&({38$Wqr9K8LqCcE)H>Gi zOg;UKHkspEr_FW@RXjhh2uG8t?LdB54j_vF$&_~Cig4pMP64zcJOY;uJ<%Oa@x+6I zgMe5B(#C%lA1*C9kEr)|rmB3?bcJ;_H-sxT41pM6Y8Y5Yg()}q%Mv2A7QpG@?>7V% zBDyj%GW@)}ywKm!^A9yO4GlFlH97b>{CwIVP2kj`QF=7+eyW~Cf_y{RiMTT`+kCr_ z+J(e!g1#6&dCCfdjz`9>cjvB;OXCbBH2eWUj| z2;$Nc68q%mFr84x_Vz*Sv<&|ex-cDuF6?>w=pXcg;Ku@@`QQLr1{OGm3( zA!$WFyPt~E<#FWdzmq%?d4cxS zaC)zblQMnyG#-ru??+RH`azjj0nwM+t`(Bv5`e6Lx*TVSQHTzEoou@d2e;i0-cXY3 zS5IG_#`aGF?6nJ5PA&wk zz5(16&yx#r)88mnWIOtZKmg-JWi-IpSI3c(zu<(SCfE=jpLC6(iG|LH`0r>cG!-zX z9{Ym@VRySTV)VuZs>+nif>nM0m%1WamQjeYa}4elz(gzhodCeuK`;l*3&61Wx2H4# zJSEUC2?LgJd;ubDL1%BuvU)VfK(i`#f7195+d35~71%%5qPyx?t zjsKYnEXyXuNG*hm1zIu01%9Uj2nbvF{cFHV8QcznNx;O!(quO@2?QWE!IJ z9@OQ0a_8hmLdZkJ(zj=vS39KoH8GX-W9q$xSqZnTU*)CPq)TV8{cZp|O~RF93-vN8 z_IGXBv=71-?pw}hz=EK_g1}G-hl6RFf4$^*Nu0h6&;1WBFKkKoMV?JU9uhV!5hu3O zI-Al#UpB}-Iiu!Ic12fzRD3X1)L#hm!6Or5J=UO8X<5#H?5ShGd(g;D?m5sXmESL ztH9Sak@(Ws-YbHwKH=X27J|3{1T5r-=!9q*!7n}tFxZ5A-&uvs4krK0D?%Z_3#Uc# z!pQ(Hd@~7tP~}BLe3;8xs>;d{SGLvM46fKG#Ha;uu|Uti&;o|FYsjHtK#$Btb+%$~ zAU82FaVs-)HGt>k2|xG*3!i}kWBy2PzsOXDxJ=TPwF1WHB*r;&eN3EF6ct|4zTe#X zdLSTZE`CeWsI02e*p(nxB%dKmFXN6KJ05C0*O*H;O>cY=nmVoH&=j18(?V7K_Q*Zf z9FgFd2Sc3J&(%8AG%x;tvrsa$B%^A|fqlP}iLR~g1qX&T>1);m8y)F-`*e3nT|}33 zI!~F~5o1r%jM@#f=VGXhLIinun*@0ZP4ume^6GK6*($8vO!Rh*H|5ZNrVH_x`BnCJ zL?lSGM|8Qx@{|$t7<=N_)|NFZtl0+_g{~Y}*ogubT0y`1gHQVN(IOv6+qx80tG;1_dqg`@hWmvWnihz;IiDpU&aMoGBCJ>%NPmB z7og1BmEeXk&@vjC3)PJVvqDn=$PHD)T#^H?=ID#}r>pqv-?sO~rYONz*L2*9x^t_J zmL^ds@8xPO+*kN+v|&7@#;&4{;i2%uFEg@JLY>E-%rQ|`Xh&K)JP8%h45z!r-)dQC zc(Rlyu_mu zcVgSHaNy#ap`<^r8OmH{&Cp^Y$N?IFc$9Mq2!T&U&Z-K*ik(7?LIB4GFyV<{UO^lN zh4^k2l0#S$OR7g*9sa$I=5r}2;W=R2;9pPyd_f|<+9UwLh1&wG0$-=zECkZALj zmW$^CEI)UjtWM*L<0(G>Ir78%8r729w`SSary15N)N*JaIjzx>dFOej)|DvpVe1F4 z-!!Y(7-)eBY_?5>A(}rQ6?7fo8P>@638i@2 zG@3}xw=h&p|3mZGZTPsz7s3=NL&rtYEgZKX%RnAj2A|+evFgWbKV*s*D?%v{14O-x zD^P@YIn}GG2uqCy%Q6iCxp1**2=UD8NfsX)|E_=p92=8A27LqibdS1%d3)o)JVa_oAs)Y=lsxU z`FK%HR=`6MrB4DpokIau%(eaxc6W7OmAB&LdXUpS=xKJ$_R|o3v)EZnBWFXalV3VV zs~V!-_y>h-6Jx>hZv*Ug4gUhSFdfA$bO+qRYv|Dca|6jQMuUVau9Wq`c^{-YS_YD= z1J92dQz4(eZ%w${?(ShEoKCkRHdimvGxV^F;4sGQKm#f}K#9%{@IcuCr6F=SJ78JU zF^1-q-H82xSGHIPkU#?vowbkQA7 zW90YDhMHQ;&t;T9-^iEWHf&|_sBDw+6XCe=Q%_#uGKQVk{a{{JNAKh`=}#BUUubFF z`}tN-P*1`*mX8J4X&wFzXrbP96tr+1fELc58ZW5=3j#z?DY=Rzs+3$rouvJzQu6Hj ze^E;Qc6p`bi&dZ-2moR<+6}0{O`qxiOa+$b9%AAa{t{32yLkX)3CSM=eyJri4{+7o zI5tW2Z7)!w5AO{jj3L}UQq}XiPe3>_FO@UW-fD8!Mhd!Cx?KXcpYNP-E*sd@XRF<2 z)%)lp^L?c@0jp1?MI4_jg!4X(mL##!axpw~81FSz`x?i-PyMuEb8Sv=pUtKA%}$1s zH|U!9Pg^!=1O?>X#P$ya>~s)UjxE&tflmfL6+oh>^XEoVjM%~*_&Q{TF0sk=@(zXz z%)@ssFE*0u^1eBca>KsSHG+mh>ulNr^H3CR9`3Up(CHlV3Dr5PuX(m>nlp&rh1J_;6I z8Sju0Cx^#V(<@trv~si zAV#$IEP5O$X!{xsKhWuMI4AYD77`pL_kfFW#`e4ocAh3DhFb)4oKN(X4u=*Ag)?e9 zo&E4EhU<&!N3q@fSI2hOCjH|dEG#?5`|Qq!x;urHW88H&)@5BOWFNa%>a`>8-FdUo zJLPfv-s$XfzUO!vZ)l>0IP~CS+U-+ov3p;Lg_njaXBX<-kd}1a!ifj53-Oi|0nFwh zLK`1yBfVXilZ6TsG0A4B`ja}oR>-TldLuXC4GYoXqhkTYl+t)|)#yHI;cCHTdTaNa%)T3kT%KeZ1NfeIk9pF>h` z`$5a9O2LZlLyS@Y*ACpp;1d3B9GL$&&;#SZEMraDv5au&-48YeHADl7D9;d%7Qnwi zUk9Go99TS$;M&R!g|*zsIexOeGrmecDrsYdh|+ZQ#wqUZch@XPizUW5nwsS0<(|*U8>^=!!ZWhiazhh;&veb(sZsfev=!fz+KFZ@S zhz=Wmd|ls?nVr&n>$X|8rt#|@CAR{KPmy%Zbb@pb=+MZSc^I7m>ZMo1uAY zd^hzF$pxY=qcblFZql=!=6xe&m~M}B4W+Twx~;-ki-BfJ4Xq9F3cArW)2>?R=77UV zGEFqP!D-U_7u95o1)&sZ0iu2k5`+%{-t4QnHe9iL2!chJV*~+oe@Ziey~A%TQ{nco zGW2-?^jV;lgNhgex~XYnsv?*SkWjqvpd$Oa1g%V=)A^^r46PrMrt=qwI4Ah#bjeiE zYqMGRnf7suigFS8@)Vr#y_$a7d$onmwx4_R)V!jIh(tvBmaxBr%^StihL}Uu(C)6T*Sa1FE}^SBxr{T!3D4SU>9t-dz}Lq*>cR>tiXWwqqa3 zoMzd@eI(`4=f_+HLy!pY6?8wo^*NhzCcUc)5&>qX@=(F`ttaygM(nM>K>Y&@hx{Y> zIY>ad70j6aX>mdl|^U-Tu5)m4Ov|h#1SsAP=!VW$mBp0K6|;Fb$k)jNB&xGR5-`Li{p3 z2?Xy0P;)RF_|6HPzoc>}AOGMU<06AQCkpiM8$RqCf7Iojh@?S zzF`iAHisjl{Us-#1+ys=bLke|D|GtMaCW?_$!fB<(BM$$+?4NJ>VFgUQ!L#LqoxQ89{^ss3~mrPzD=A@zHgToB3j^c)m>U z1Z3l{q{sn=V#fL}CrS?)MH?R7D{N3MDR?HQYa5QVz<7aVx6tdy0iWxGjRte;uj`H; z<}%~Lw<*(U&sOIvY~m0dpL?Sq(Rb_8X1oYPy&&dP3BLn1VFJ0D@6B)AYwwy~fHPV= z7K{TaK)AF)dJxuBzp8q$Viysk9-was{!|Z8$wItRaC4wPQZd-*OaInFV+SkLTn;ge zs-8YI;ASVM7qsrLhb)C_GqGh!vOF4zvO;gqlzzQxbH!I6YM(!auuZd8&ikA@CtjYk zcvf6j;#uaBw5x)NO-PBk_7nKCpFU6Nvty5Rb=K9_0v8Eq!)Aoes-Jawd+Y6N>9ntJ zLC~dZ%GjPT#zF&eW4-eH57M5JyK#*$LY zFrJ6Uwsm}v-{4JIWSSaEvun^ZE8q7FzD~eGDw-fxgyY=Gu*d+~C*w&rbg9WALt48oQ6Gd)LZYbdh_bsq}saQM(=US7a8 zVkXM_aW1Jx{Ysq=muSUWHghFnWBOVN&U|gHTaMa7Va|soHcFjlrBiCDy=#JhzaHCX z0qpe-SB^6Toup8Pa2X#2^CJKcWEMa;!}+Vj!XWgC(G;mCuayvqi8-JCPqyK;0jj=1 z_P0Y)gME<6JmaD)H9ScBCv zG{(f{(KHm04;iHl`d10%Li0NCjo z{vDuUC#vN13IsGHTJl@qpM~E7un;tOfLp^9q6ErxaBCQBH$0)5FC#k6w!!qR?Qngo zQE4!^F&f%oq_tbE>>$T*)7d2F=o@L*J}7Te+n`lSr?p*QpYnW*;mA5;&B7LPHrsNq zqE2aIt`6GQ^87iP+8HO~MLgvv{p9aowB%}gr~1*J)C_m24Phb>0Yvr_AD{#e!mq8W z5-c?j{CndNN)Wh|;{2%+%twjl^8-;~6c&(gsO8vr4Ei7tzlB!V_~VX#r7x>_PT~EXBu{*E*6C$rlIh^ zkOJ3wk9wjeaBe6Q`=C*JVCz(6-0tLvR}Uo2q6oMm$KrEtTr(|^Att?1=*n`#@FJpU&`+ir-Av9n>B_iQ*ummFS68L(*%ZG^lT@57LY(chB}JB5 z($C*0W|A)@1vx+k5RXEx04a!c%U)F}Sg~b@F-!zU1=80pb`hYHWEbwmmx$0aUiI>m{rW&@Wq|baCG()d^Ge4Y!?b*V*^)y5`NV5ioK^Ar&qqg%>q41dDxcQPbMEbk!6RWPwx4)A7S3k)@uF7mQQw>EUn$y> z73`W6%q)|+z4{*;luW?1ne{q{{r4H9mX5(4n*S(ftk3l%Y1%^ut)%4yrD`zY!#FU zIT$avJapJ^=0!mJEKhX+Y9L7Qj5+KAKKqf^)0#iXZ&KUo=pm4+Qd-LSIa?n3I1p|o zpKl7jes z?si7Utt|Boj~s7tr|aPN;V=Hc^R~`SovXI^!Dz3V+H6+pgTO9ZKPLxcwj?;pQim$tTmd6I6DjZWd{v!h(eGWd6buzgvGt;XTsLL7ouA3_}J zF#*Kkw;TaxIe_Ct2ME{g0dXSP;M530Hvl;Mc6>O}6Z%ZJW@jtOY}mf`vQMOr$H}!# zOCkpAN8`yU2lE=%*HhKrbNL{_c{Pv!Ei$Br=hn{VBRLO@%*rPHqMU~5@-7!o#7y=ynTD4D(~uv%FkEsPSkO}lSC{_WPu8Q1xu2{p11Hzt>LZ!e1rRPikhR;3LheX2 zE~BkLKA;-Ve!DQxe_J$G7*c`EInP}O_FVg+=+8y=UXZ#O4OM|YMahm59zNwDAA$=<*?`+p( zsFNLosQq~NRFvlNj`h#yBrezK7fXJo?p403Q?Bdk{A@>6Yql&q!R5Qy?i^sJiTHOA zhhPgxxdZp;Gr#JWvc`n>cxadZJiR zPt>b%8>%NdthGlK&x8>}mf)Q2r8^w^Y|DMoaFbFfOJF?d6A0ZHP~EZ9cASA^AnW?e z-^f4^Pyj^y03-vQ-Vaw*29_EKmgOJ<%Hd)c09{%B$1;FnO-T3u*gNldD*OMBE0u~k zC83NKDk4#wtUENV#$75SQB=3GvK5($5LsDCqR76D?3I1&Rn|Gi$zBmzzt44@qgy$o zTYi1d<8fbq)P3vk>*amDU$6J;ISIuEL407%woPC(5Qhwdzw)3Dg)8l!8JMI#nQxG? z-|oc{o3Acg^D0WN`Mgi!IwoGjinbdzv_guX4YB=Nx$fL!)Rd|WDC8yWC``3=qugZI zsBu%HZ8?uxxc4B^RD~wup4yJ9gX!n9^a`b)@7&RIHjl^$0^)TL7mqn?f`9ur;Q?oJ zd=<=E6mHZS+BjWNme`OhFvBTOAdAuS=zC621!%`!ptCrlMMR6l#=WJ480$DDWJ-{)mV;N1y@eLsd(v z0gJW{3Fe959s(5UztaG2^23M0rm#n4^bVJ$lm4~}2`+`VciI8?JPG$r06q;)fdfRr zansCV>YJV9G9QJ!NIy^?Q`3`m&=5508_9FwlMxXd8xY((#TAuQ*#A1MbGt@Esm8E5*MY($z7%&&<#0MUO(X}>M4#a_(ZwMX>dtu=gi$~O5NTFUKnI?q zdw!b^EXXkgti#!_I~*ZUm^~vvMnZ`_fqDYGWjGx#p0*67Bq%Azb?_&KHCLeX0&3CK zkKJ(3rc`X-b>%zk&uwc}LW+mYe71qD|GOmbKGpm&dcQUkg@{y)QW4rMfH8+m+QHiw znHX`e`FU}mfIDfJaQr6iiM&S-ytr;>*WJ@$yV9JgnqTGKZSRxxrMKgxHiqWEi$l)@ z+u3NNFcK2#t0T{_g_@a#o@g^4p3Jp;YSwG+p(sQ|edw984)J&&*Hx`?>MYzBP~qwC zU*(BDLLR`AX~ncPQe~~vg?(hlla$ET_LCmTy1#q?MZwsC%B+;sb81|MqZ=(Aci4o( z9j=0KhpR}o!ptPx9pfhGN1Je8318C#vl~fDi_QjbjoF%4E@aW^|@PH!-4=5Ob@BlY8c3vsS04jheDTk!MqR5(KDffhn_6|Xsh>#R4r*k?m zTMFhZG@h9rm~nLICc4@JAsQ?N7)Ma=0K_;dxet(?*zP=%^10=r zU>8k^RY>_f%H4#0a9QP^%qUNDueqtjWOzcCj&9~7vk!7TR)Tf?J29iavfN{2lp81= z83Gyn_)mNLUc6C7d(gVu)u*+K^~i}9>4(dS`}CMtt;1hNJq(7ahaM315YI9IP68Ap z_&!PWjcWkk9DZ!uedqK)wWW9Ox^X&FODFEamnxR*>ukPflvm70MX?C7^p!@yzi2#!J?f*f@%O45g7iw8o+}A9(&lM zGIYn<($aE7;`s67yfYp;dgwU;etP`)U4T7YAt%SXMxUV@DXgj8@+qs1U!e4Bw{UeXao63mLJ6t^TQ0X<>N>}rZ(We0O@B)5ONLU2ET@v7!hZnCO z#)&}AKz@qwN!mOzQ|q0{M}??9-sM4a@~GZOba3ama5Y=(;2*1QYrojLp?8$UI?Jws zxBTRzO}d$~^~3nf!!x+cLuN%?#RD35PHV*Hr|krBLIw*rjj0mFn7Dv1+?n$)2{7RG zNA&eV8j$@Wc1bm0zD41JoWnWO10^X0J^~gr5*(g+c68{@1Kp1$+xEDg%*NjygTImn z^4WW#!{g)kiqTr)j})}TWoip+IO8vAC%36)rZUC}^>ih4>hz{fn2{Z0vKBl9O#z;= z+!|b6EM1q8$B{`|Pqa+1JF%bAFQ$*W*f|%_vS)uOIvPEpc`bhOZJG1Cx5OSFAYSWm z@zg``yC0$+Mm+}9!L(g!Q`F>(;!(AzS{e?C^AbBYi-y~e5P6F^ zak_`UhI$BM1Tf+7Fd!UiP9GyLWGsNh;4!DKDD7U(6yytPWBH?<3dN)JAU^;Kf`l93>OjjB}jPu){(W^cBItQSy`Gr~x#|5sf~XlWQr zLZYHpv#qBvf&kf^g1ye0o^=GtGrn3^)d101sJXq;^NLs)Pp17DHEZijYSy%S?NJo8 zMrK(KyZ=y5XV}_r&bTX!Y9qay>e}TKlq9&t;o_NxU?sTjGqz*dYvhz`HY`dUShOW@8YR%BDZT4)w=Evvz znN&F2gmGXm8bk_ri{y;wkN6A*HT7DWvd9L>d+FRwM?Y+RTIn|AP;|fhaXoSa4Vvrm z2q|%I4T;l8{1wzg5G5clBdg6I1=PdoV`RcJ!EEmUWD8GS$EA$T^#@Sjdk0W=F+iS` znOkk{iNTyq7O<{7+cg@vdiSQ}=~jZ8Kz$G?V(NNGoroz?>Z$Uk!&>#Rf$W0OLQGw#2xN^k|VY<3zq3AXe}2S0N9Zv{ah_^6(jiJS4$~ z2?$USzq>@naSz|Rd=f`!9sGxT=-A@_m3s(!3t54IsOd_tc0(*jY?J)eY$;$KNNc~` zyPQ}aqdv129k=oEi>iDBg*wiX=wt$BuX7+l=Kb*^*0 z|8V|pV&)L(d;!A@paqDMELR{1b|KkIDhP}A59bhs<#Z*7XA1(p$#nKV%o!{KY@@$S zVFeJ)t$^95lmfUyV~)LhfpJipCB;yhH5TEg-PgLWc~|&k&Y{M&CVV5OTY~b!H>B>B z&i8)rjSQ8o4UVi;3dz&+VrLBd9AAm;aU$EnBx7)}wbh%CydiufMpeI{!C``w>=X%dl=35jl$Uh|5PXxCSSQGFMe}=q%L=1EeQaG|0U7(S$FKT#j*T#a|hVvM;@O5FCQPmM-vty1eGKkh&PFpM6 z^+3qF>R9f123A)V%J~P#s_BRy)++qBeUe=32HCJm2qWrMW(yp91A^ALt z*gp!0*Fao6_fQFUzQ_S_4>^893cfcG1%Eg95Cc>I(boh?L6P~;l1jm%4Mc)c02dJy z{0k{SKwV|v3V9iw89K$UwORpyk23?oy@5Cmd&D6}ML#OipB z^Y2-c+GTPaJ2<7yz@_;-H*bH?9`&jVo$3c_Z)>Gw26D=d)zN7hXC-q}1qpigo1}-Z zXjGfj^cm*F35ht4BO`ULe9mw1nII$MqHNs5ZJV;kl(@Hq#AzP>8tS3aE4Iz;7Jo*a z2GqmpAOYc#fZ(3+EA=qECGfXW4+nq(Ak4zufeeK8*mExBo^Za2U_sU)LFyr%VK{sG zU(iG_gL=5vCC}1nLTt6?rT1U+1E8AZ(j~Au^rV2!4K{RflfQFmki-Uy-pS%sV>b9S z0vQIr=ZZb~=QRw>fd(L)CLkejeE(ueghAtBj72bgTzsn@PJ zrlE0l1KGLTf8Ey5zfDIlPy|;IR1nAz{74999UT0-fx%YLO~L{9lR&4)5?>%9 zx=?^P^^O8n-#5!Yu;}G?okuD*i(9xzN&k(kM|?C3>Jmrre4o&`>(OTnf!23PrGYV4f{Y$q21cC1C$*M2l;Pmt815G?s`!% z3hRwlH0$_rk)}|N9>k1!fN~<#$N_AzGmkJV2G_7}yf~ zNI;y<;o_NxAV2^!4=n)mux~n5_%lL4ju!&OeD)aw!EX?P=RgAx5uTnv2udR>mQ)B9 z?I03N59Y2E~dU^QJ6AyWE$jsdU?X28(X)9>T^iC$#)A_bosNXhc{%Q4*H@6xWo z8l>et2vG@>N)T>-XZ}H==Z#@JI?n-XtjK6+jLEbrmQAT29c?PME9dLHeDKV{=#JFD zXdjiFCT(AKR^ye!dnq3>yk~Ib*L!kxrxwwdnG>sv_^YUgAW8sJ4@m&^kOJ=_hTwj* z4E%@zw?zE@BzX=O5fqrt?;?6%rBHjnz28Se`MAl)6FhG!uAk;hs=ReaELfnY!@=im zbD(11<0(g;bw0A|p77^xSK(JNE#$s2ddOz@dGf8wI01Q)`AozpAP0yvETjohK3z+y z35zxnfhPPc(q#59fma0h8(~IsVd8$sIxt!T8cSVVT+V73m3wi*g(ZPoAWH!hnxnO{ z&YfiF-`vHAydK>X(H_~Y!{2f}tKZUPcaRQhlQ-9B0;c+DV z7!qzK=wXa<-|*<~V3Idr80^Zwozd`U;)#Vor+tywiNoq)E&p9068J_4CLwWAy?|~j z+?QVLi9Qz0lWB25&6@fZieihl554M>Lp;0{hj@~%U%8_iUqZT5PjcA?@-`A%+O9vg z-Fx4-Zk zxPE=QhT!`3T>ylRN1@wR3zhDy)qLBmUOf8Pj@q48QQpuHTWJx;C7d)m-YR!)!nA6% zyy=pi*P+f^2PqB;sC$OK?YhwQzH9gJ7y~~8r9s<(N$%sE{s+{4<=kq^u+*HQEHt}= zh`p|mc)i2La}VceB$`hIpzXlSnli<_nI8qSYs%K1%JFfyGPKbcYAikChi@$1lK6>J zQ!hI|EIxugrE8_Y0m0GR-tL;J;U2+&XA;*%>c|9Fl0W}wwacHsf9`0mXCiGuiJWDXY3tgCI}92d}bEZ z6f_#B7{p|EKGD|@R`oWy&=#5!N`6ozmL>5f$DRzENMgS(AWq-#*KiM&aJP$A5cd$@ zS-PNUU^+i=#Ev|j{Nsrgy)8b!)i=Zd4M6mjLPAj46uqQEur$6Q6dx$5{~If+j*%ppZNsK3~%;j zv<$iF+7&U%9sFRd$Dl{mmC(hE?Bf^I>eA6}<8A87Fw6~q(8z5XHZBz(ujQ$&i#^ob zmVPVj1zY`pouvd@%3}W?4h|PjJyd#?bN|I*j5H-BNG=PN)+6#RpeZNdmipcyqcHAI?DM2_gF+%c!!o{GjM*dv>q`|G& zQVX_~GJb;WhLe~h*#&&z9N{8KuaJ`iRsCT%CYg8o$$9UR{@{HBB_KZ{ua~g4W;mX* zcIe{KQ0t?-mP}jv|FaGWtOY=0;gVX1zk+!P0t68AP!nPvPWKSZF%!&|0;q>z+B*F0 zj)6H)0fbWzBn9sA&A&|w7Gxa)+99D?AM_BQ|AiF5Uk5XW0(0D%;kfwpKmbhzU^G}w zLql@_%m(mK@NoyrQ#vLul~|KlwyqUj)`IpmUQx_C;Nk#nfI5-iZXRtv$U=*BXAfH& zR$b>-cPl=8B)(Om=(F|Zy{Y=+?IVfU%f(;rKceYvyTID?lT@hr&)D8NX-*pe#1qCG=`DI&OiAdrA(xUgY>JpF4eQ6PBS6^*)+ zV|YDFR{Jx|c%Ta7O;e3%>}pS|jP)Jaht$T}f}2J%Pvp6974K23Dw=v!khrQ>NSJr@ zl#?mCBVi+@twC)ZEp^(Z&HFL0i+jwHw~g(fe8&+KdZtPzmfYbyL-?a$ z+pB+N5xGY|te)Yo0v>`80R%j3f`Et9A%Z#d06!Ao{KB&KPvUgq@=wgkabE|wlU!tD zUs6OLXBxMBk!aJ)hQB(zXboyhBrxj&oF!rv;YzD&=&|ot)t7nrF0Dz zZ5R?11UN)cN^m4tOBaD6_z$b;(&CrHNc3-PgXh-Zd8UE$3ucrOIU!)%pdYFx^0Ple zovEUe?^Zl)?_*YvG~PBwPPv`Zogtecoc{#Bf{vSZ3VM69c0rCemp}^b`1`@QmUs6* zu=V&Pn6{=rvY}!Ux|n=%DxzN{I_&iu)(?C24xPM-(Q_sCTVx`=dW)0PsUIX=| z^0@4P?`7rVNgt(f*#Q$=cEGp8!QyRmfFiHoNS_q;+~85!A?qXWG~e}mtCS6NGrzYR zj+Jdxi=UiN3Zj4tAku0eDR?{fdP${VzLnvE3`CF}Ak;{(oGy)G_E!U*el+_Z@G}G4 zNB}MkNbmAv zDo#HylXEG5kM#5h6v_BqX1qr>-af$Ba@n!qZWu)kL(~Cf<3f~W{q9ctWF!y`H-<3& zOFBvDt2yny(L1nguFe~2T}6od;)r48~CpEdlk=1HUl*hI{mNK$yKH_=b8Y z&h)-L;Lc-{J~hJBL$gCYF;pS5nCJziD^X!J|1i;cpi2YkE}9*>6frrP1r7ZlC#f2& z^o2rvd9>!$feau4h?19nKnJ|A8B3}Ii}nx+>Hy>+K07`8b^nnL5Xcu>BBCxo!0ig) zcA`v5iBw1xz#}?&*Edt;f$qL+gj}@FRX(NOtb4g%3J+(YtpYrn+1b_v@uAM#YZ!IW zLZ`RVRyWj*YqNzW*c_V8Jg_%M0*m<|iF zl$?dgrGPk1#9zZb1VI9rdx$Aa!!yso2LoUlOdlS6RRa97UMM~gCg>+)_7CPtDQUA<^JOy{;tx}#d8mp zzU?I=tO4JC6MP>in&uwPI6M4~dpPslXbtY%C?Zm#ce$EuXxBqNh>L~>1 zQ(GhbtUCq5eK-mD@hpG=Z$F|h;~9{E`jmksm4HQ?hXnIPa0@{Tff&J$BmgcXnUN?K zPyoFHqIGuARY4YvLAQVk2;$LYfqV+%zG&{OdveMTs3XR%$V5-IZoSHQZc5Chh%SfP z`=s{4A}b1a9r*&C=-loNpZ5rc4!l4Ot_tU9h#e_G>$r2>O`#ltua3@wtE0b_jPp=W z>(?0z41Z=|a1|H#BYN5yan_ao3htrOEBJQtD8xNnHSb5kEZ<<`%(_yWfx_QfS2_SB z0AcpnALu~J>xbM+xh9-X2NvWU0_x%H*&nVU2qv%%T&NCc(WRjq99zp3q@_6(z;(d@ zAFiAz{*EntXawCEWC9Ypl)M1jQv=;$M#{~haW&VYYp;dmyjm7y<4v!lmoZ9D+a@os zeIvVdoouXpy-s)9u&oadUwB=D))Xp$OvW$Cp#MuzA{BG0$(oR<+m^|@#<(dlp=>n~ zPb{+}iN2hSIIY9QQxBCwa{fSXT~7%i4k>?L1xoN#6Z7A#0_H#f5KfJd3i!D_Us4rV zw0lS}N(6ThbpHz#nDs#bbr87bT@?j2g^v)Txtm1y?%mrBMUBA&u8T4QY<P5NAiTsxI)k}y zsN=Zm60|>~lF2?_7Bm>_D+q`fXFg`Qq1n(}_C&+WQ75#HgNEU~O(SPpj@vd12jzQL zcAc-G9{wwIorKRL8k#c)5ko{CDu4m!@0u))&%DP z3Och_1f*eZ%MiCBOv$8V2YW%dy!hq~+(XuWm(p?DC0Xt)^@z%=iP_R#^_0d6OrH$u zu(eL^I~pVnL|V(e!^wwb8A6eD+A+-|X?c&!dDLhp>-l?i(5P%P2`Ww-c|+|3nD=%E zR0oUBXAIl;JgE;qDV-$QO5{-@;2TEpU=)CUGywf*S{kOy zi)XJ0X3)@k1RvXck32-5;OY7E?L*g1g%y3zY4|#3kc}AIN<`O?CpKkrLde5NxR~fs za+Ab5D!f||R*|^lrNr?4W1mg?IFmY04|&)Q&dWcJ0y=<5%L@QvkWt&dq++mW+YpEW z;rM_V{p{7joTL-H7QmZADe`59NzBk~YNNVapqJl?@hb{l>OIatg#RQR1@VDq+mTjs znP9$7F0Ytzjp#0I?~MksEq6VsXoRaq{>19IaP3Xe8>wnF_G2c!=P<(4R%ki6%>AYO zx&-cKy{u#;mC7(J!!8gyy5S%jO3`^>OMzY{>cry>dFLN86L)}sI9heU_rH|1WU>w12OG60R!>S>IP5Oj)s zd`f>r(Et4z15y2&Pc_4}>k{a+km%|sjXIMYBZ-*Cry+7-qkv?vqe(UQ&4WZ^rYu3> zJ`Qy2W4_&EY?S*QPz+dadH!?W3driv+o9{c1mp^O(yd&cZ_f|fNaRK4#OWXY3fv(` z5Wu*@RRDK5eT7U|2WEd9%=8a`H|`Jv6adi|9SCHg)jRUH$-u(=LqfR2IlD=JBm?kB z04@(t(V3l|%;{*S0Nq6J(|_{#aeku9 ziOe>?9Vit}Re4tVq6F!Li@Vw%pol(5K%55R;&F!ng%09_O4BqW!U`~Nkf=n$@LG^# z$&mqEn6T5VijrE9xA`yto|c&D>p9$2HZ2KjY$fC|qMI7& z9awJm*s>UlmpSQKv)16*}lc5FV1^4i$ zXr0kE!iHkT(~lIjx`bGF((*m5^!i(M5M329aSe+-ajY(ki^%5yVzmx`74T3=M^!0A ztqB4iHceL*64rwMaqG|=2m!*W1X6>bPfbgz28*^13911UByjrjFVp}>KV7vPEDNs; zUJ$C^k9(q`_D4Za)yY?X&ZsB3^U-O%5p8GrzguR)!H9rqZBZmPC#WHH?{SvBqzEgy9$vLT{2 zsyQagvv;N3?& z^9=$5@StgANd;iO4dH^E!#T49eJKPC!XJ$T)87M18_iDls0`h)wmfzEF?cL8xuCBB z_=^CvlKk`B1@&YeY%EVNZ2O?5w_T4bRCzc&>7>EkW~uyipE&wnrljK0z7~~qv|WI6 zGanmG5Hso~Dpj6WJ~nyE6{Tyl#bSta%iW%J2K>r8g;kc_y9Idku%VlLW3nzA4G?*H zfOyTr#gh)3a3_m`5b02GUMZOE7QidQ>4Gw6+3$VBD*^O;dla%=HgQyF2GPjQ^bKjg z@eTWxWW8%>G=|N&4uqQUp(NrXWD6^}OV^jPjHLyPi5So}3a+Y#=qg<^Vj=Zg-%-^l4YFO5q(0tA@&X?-0aE{5AJ|6UECz0Kz z$HogrMA|S>H58?G+HGZ}@9a zhfO%t;r>$qbvT_OAn>DL7>KA10TJU#;9UXt+RS|vR_Ge%YuO5-(Mg;3$6Q_gX?gdF zAA;?KQ4Gl1!2D7U_tgF zU>?r44@nT9h=66ma07hYkf3?UZi!Gf1-FI*T@l~_8DCj?9I_Sd8g&eKwfr2eV_3RU z`Hv$_be?F9qg7BrTz=#!Pb#NEP;qM;*WLRQ$&Cah1zMn_Ks7o2s9n)PIfYo`ItER( z+zD<+RzY0@;pR&7dKTlqhWQxOk#$-zvS*w}p8aPY{;spbUqCzzW@F~A%`kZ0^T8Qk-;hCpBzw(fbR{7Zryc9 z;}BovVy+{{BR^8(Mgt`k_W%$n;*MtL1e-%BQX~UMifU#YATP9u0B8WB z@*uodZNxxy^c;Fky)t#gt?081di-UA?E`gvuXE%IY=r$78&{1}jPkT(Jt$W}NhA51 z$IZzlIKvWR&;t(a#$k*J_b~6Qykv)a83ZpRUr4PrseRXK>fW-C<%py>{qB3jJr@wC zbNKof01rWeVAa4%`V3J3JXFG`iRP1mQXm6h82I}{d1wY=z2I6>D}~j@_eXb{_qGUd z3D_&dFpsbEFRDW^Uiu(uP-;)+s*o*7e)Oit^_9WRM~&;`0MlR+Vj4(zh>^2!beVG{ zc0twwUm_+88VAOMapS-a<80SMtKO@`U+p!xGX|kgna(+S^0Uu(Xix?e0MS$h$$-6M z&2N)|1(}BgiHA^zz$|>(ztVvj1mrgiFQC%GKpIg~L_bpLdRwM?J>PIS;?=%e;F(4GPG$6<3SoZq1dsm_&Pf656wLuWM`R* z45FGwon`-OxIGflJm{-&Y*>atoLF6-4o9|anE*|rM(SGYz_XS#fT_j!Xu z7uYI*HWIXRa&uWXa~=6-qd$cjvMw!~PX;hR0T6vJUIQ8E{?faoGO%b1kzkYvvJe5* z@MmQJ&KHZ2FVn(fPZQyL<5T$>`*0V?`{)4L?2`l=`~d1{AKmjJe>#kM&EGX2qBL|O znokX?6-Sb?IeH_tCJO_}Wy9;@wZ_2F5tVPKqM*^>NP$c67m-JU9O;7_xyq6X2lnK2 zF84YTo3Ac=p}OW(^oOhA+4pQL!kiyi-X-={B4TwAe--Qy1PNf+;S93j97DmXncoJU zk}uaktg(^-D?`z~4NIZfOXv}gNDREKlh$(rk*ms`mFRsH+?K;Aiy!{yg~F zOwuV_U%8I24*EdgJMP{9rh+>#4c+pTdjlva;RK{Z?jvfh9vh4pTKUhbx7Kn-oyCY- zxtx=(MAj?CWqTepRyZ$A`$e8f^Hx?7xBkmIuYS#R7nb7cr1yP#IW|ImjwS;AgBgD8 zWcpQ9l$RNjk&{t05ff*X9|Zi@T1pUb@;}Zx{CY(IgU#l6{lE1L=3pI`iikZ`3}6DS z#h#k(^s+a^awIl|Mq##X;oPt3#*-?yvtdwz!j?Aq#!J`DdpoSd-#)s%S~urxz1Asx zlM1^wYSY=U!{mj*4$Z-9k8ny40q=fznctGW`}3^`7vvuj3=rUVen0!}!#l#6vw%15 zL1=5IBHF=b0J=NI%>{p~9{p@+eT;enW5@c=wMamYD)w?$QtgEH-mpnN>MoXA87+~b z01C2;lsXKphKfx`9=o$Bx*95=rCj558x9yWV#K^08HII6Lr1Ebv1TG-RXOS_)TL|I z*7`~LqvsWyc%IDVhHBQ7x@y)`MkortLT`G?8YEAs zrjDvd6SlB#M};T)a41iv?VoDa>k?2DJSTnVH{bN3SB%-WeBH-o8w6gG*sjzu%j)Af zyZNL3H1IU%(T2lkBn?h&Kkl}k>g0;N{Q=t!uTmu;Cn1?yK{B<5mxM&8xLBq?SF-rj z>U<8=a2^*~xxZnp;b^NNp?l*wL}2!QP^&y>x{ahKY1%j(rMBkP5umz$i|>2m(4(CW z{v-gILJLKFrcfB?6HvjQ1gcykXEO7Rn~m4d;>*eMgf-hIsd}u8gpfe+!K>%EWaUD=#ehQo*;yx2_ zK0#!`#nP--QF1Eb>GxbL4%#rHhWJ4vp%Z8%l#1Wq`o&w0yeG^cqOIReO+b;!K&NHG zQ-(IBVMGi2*(HEBhq-FU!4&m(>MWI}a%ybZVd;ub@48snJ6l@1ITLw|h&Y|YUjsXA z!W}N!K(IqHoG#3v1n_d;8>gVuO2(f&kldd(m~Fk_6SbaCr{5kI9$=%m(Wci^@?JH= z1zZ_*?NPn7?@XpRdryT{z+LZ)z*bzBWw z8YTFnWaVF}JQK2(7?r*u1RqFxbgEFJs%~^&mOLgeXU;=MOqJngUvM?0i|XonQICITgV z|0(n%$?t}SZ^MPqu5e}(0RiOUw@n1k8W^?ysyy2@8n}A*reu76z@#Qn|9&+4CTK2* zjD+$7=}>-v2N(uWX=xrPEydY}ubF<(TO0*s0Ff5*253QV9~ZA2ow>1=gs!d z{`p*l^qBzy8E}1!Ulhs(!*&7mM=KVJ4@_qWcEPg&SQVm6cQWW7ak-w~Z||})NCH*m ztr4HvobGFLjLFA98$IeOtKT3z(uW!J(ztwYQ{|M5r^KO78h}F&QTJ4@u8FL96Rj5A z*p!A=bP;n8*p$P>c1!v$>d?ux&Sb<6LMKk=@R#rogJIs`zSDqrINd`4+(hUA>H9YG zZ?^=H4*U%7aP6sHhxpj(o1=NY52kZOjMR+W3Q<&oZ>KEu(F%^Qb0ob*<{Jt!fCeB+ zMj;`HzCZQbgkV9&A>bVndf`D2fgYisFQ;>o{^3xd1cYFcxgbbx`u9ISe~ARJ4fkI( zDlhYDz;O&>TAmGo0`ncWWo&o&Dx0Pm<)(WSS$5lJB9BQp$a<^u*K5Z%n-r&#PC3d8 zYaP)UD*W55llC^Na{(582a}M&=0Qwer)52@@$NB=Dta>Jg9*;gHQ~<7 z|1%DcxTe3#8#Ubh$IvUls+3_SLvbFuBN3!lOA3#ws zcAzpVCH0&dmmMS_2~ks8Jnc~FH5=na^Oy*yNBHl81q#4V(GJbPFZQz6vq&azdAZ;} zBolooKO|qvuG-=8@j?63yG^Cd%t_?EM(2gpFyr$s2+F{_k7&vX0|M}*+wH(oS_F%> z3<(yL!R-Ty%YIq`;635DaI@#jac73F9GwvBA3JGYzm5Z)f&W;&=Zi5)lQ)k^pme9H zR)5<-!8y5!@Wi@LBZrzkIQ6RBlOEw|c0f%a9VvV{FYwh~h`JH^xAF&7wW6@ho0+}^ zlWt`$1_b28aQo&ovw6I1&kkr5_ib54RbME1qa9Z$P_JEtzOiG zIT;nxO(r2Ex8S}07cK2xYHnC&MSew7L%Du`&MCjUsLkGFp-<&4L*>-BbAC`)*?luZ z_r=IE;+_hK@b+-=v_p^`fXRmUA+jO<{CJM|PG z7wrc+4FBl$TBGmDE%8A&>1z6RcBUvaE2+~C0`6gVJ)*-Q=xy_ho{?bMBDP6`O2f*e z(j&>}J)|xS+zQ$hmr`K+(4ixTqI&)|0w4egvld7Nsg2kiJf|U-x55g5C9$jY@~s15kV=D3AjGwRbhD=bGApduJ~eH?S+W; zKFz!x?riL_g0a~1@<93s*#N{<9kMQ+BEV~ z$@ysFWP5J*XoQN?80?{cFnXTYIq1ZDd$@SkA&3yLoLNbavI49_e24jhlmOZgd{a+$ zv;kL7)*?fgby)X`G{#Y9XrsmlDxpBfoNgH_>`w&ea~OKRinV2d0=`+ zfLjNAddau<4MMqh1`Hh^Iu6Y4MG86u8*@0s?>43y=ArZr)j7yTN$$`f$Z0r?zLp)r z75-NPZ}-$2jDp=~PUiO}pGGRs(Z%kZmq}BG_$$4KkbLqK@e~Q0wHYHfM+#ig$e;AK zSlL-~QcP00mB_0KiPJ>u?=|W+Bh6s z%7x*gtwWF~n*FZBl>;oZodM_)(aeoNN5OYFg0JHMOh5;JG=P!dO+%g=xHiOx2;jet zgCqg}^VjDYN;$nY2QjJXqV95T5hORLZIQ|y+vK-hIBs;q;NO{Xmc0u1cHJ8u2^_7* zI-+~cOm`=`C2t$sLAjPPhe3+pS9`zO-%82K(QS7MY8?hGTpvC;#!lop0^)QIe+BKZ z2}e8p1EL+0;8O&+TSignN8h`L1XBdNcE#`dw0wA{If2Sj!5dHsL1Y;yEL4nH4l+e5 z7lg8XFF$lrE)^*Bx7x5tFWpALh?YN5`3lravc)*veKQl(OTr%wB1QVgB;C769ebvl z6bz~ZI=HE^RP%iqL;)2*q zv$he2qj|jW2r7Ap*B6L#bwO~kp63T8Bmt4}#=#Hpbl>#N> zG}MFoNM~n*v&4RIM4X3*zW{lNFFl2jhpc!D5uOi}rpX4=B?PlS3o45KI(%4j8&pNI zHu=x+y8dmP&UpSYxI6TUC;`{UVxXN!vLd2v?Wr7};Val8 z;r=k)?OFk*`l*E#mu3J3K$LVqGLYDqyreR)XbX{GI}zMLU_szF|BrM4ZYKhc0STQM zI>oQG8j2Qz2X}+awfBt^z@K|cvRWeGZv~ifbXY3esiY+s1rX>KXT73XnHhxRM%o6V zi?~|#ZA6$%6UPUUjm-drF2?yNUa&3SGUV(*HT9xe#fa#u`q^()H8LmN#snyb8JLiV zv`o+~`RxiUY_(#<&Nw1g1MycO4?&m!LLOd#kcW7h0q_)|K+zmxu$&tqUV5Ue{gXJI z_zdKsV~hXqMIM#`9Y8e2M*}hNE~s2mF<7*FNKg#mCW735AqMc0;Oo(0L6E?D7;z?= zo9s^k-5kZ^VCdF-3UBYUmt_MGvKYs$huel-e9S;3Xd8$GttVrqRB=dQVDo-aIVI^f zh~%V+ztnQn^8-y!rNIU42^Z{MMO}jr*P@GamiZksp&!)EkE3s-9lsFux2ka_%AsD! z_~C?w^OKJ*#C~!>gzn+unTJZR*jBolZ;U<-n1?m<4g}wR7JS!S`bIGN_coVez}t`L zvyK50FcI6mq!KXSns7nZ;hX`2CVK+a#Xp({X5AhlzTO(9aY46-vJm$SV9=q*)KATT zH?s3wX;x|Ntq8S<#+I~}h-_v%m0Qw!$jatVW;|jj2P7YQ+@4&AlQSU(Ghodw7uJ2D zFubP#a<#^3N*;_-}(5+rs~;=F$Nm0|>K9NDCUCTk$RBws6trAweyGdPvhs|AiL7 z-v%?rfyB3HDUI#$SwRwXdf4s^pCMasxgC!#1H*uX{I#qNT!krZTQl_R&w1-Q2ke%< z%J0)E?`yX*>`A<0+j-Z-15CUIbuAaY+!-4_88l+cox*qMv4R$J9aQ+rRN3*1gZc%j zjEQ8!4qbl8_=vkIt#oT_Q??MhV?dnN;jbYcf*1kQh86VLM*#6qbGn^q-VcKrN`NCC za<1Td1`ZHiX0cDN1pa)7vHHL6Fg6DwfN%=<8z@1lgU^yG!F*G}g3LoeJe<80gkl8G z+z9lU|7a?h)lLK#5_}1~{PqqQ51e@5j-qGqEiz)DcaXwyn`+#4>kD^d$-3-o7^9y? z`9{VmndVs?pyiW)uVq+VxuaTIK2@_hzepgCsqUT@wkVm4Dl>iRgeCU0Vz$8+teAQ6 ziIl(*&o&ursqFJ^w~C}46dcb9J2_n!aU$|tWa4!X7f(C{F#?!)XaR_a28$R91fA7> zZ=Ja_PyxiVen<+QeTi98DOj|HNH9?ZS%}ZV2};2nD8%nP#A*5|yfB3A0#=tb{v!Th zWl#%6i~|Fq+;RJ71}3Rb&o@ZfZ}7Sny9alV++^e;cdKVtkG^3|bDkNNDg0;>W3)(X zvpp9%z#YFqHfTmQ-zn}j6Cs=Q9no*=a@E;})sTO1AvJn^^r#a1m9aeO*E@IgNaPW@ z5)h|{_$!EqAVmNZ4@m&=aQYIN00eQ`J(vwW1DD7P_7Djcng6;&i8C^y3ir zGlk677Q^k!{lh{v)S1-%x*EE6P^JnxCQjMy3MOHhM6iM>Ci&}I+wUnV#E z0`oo}Og%K-t|hAdwiub6y-b$3+NN%Q9b;rn=(|uxKuRvY-6k$MQesQwIzXKE;V%Fl zep6Tu76vo%htu!=w~In3TRfwl$XsopM4H*(uEjzm(@FI`yvyy@2O0rAI)lW3E{3*l$VL`Z&S^G zLlwuyfzA6_DF-n(9R!`nqu})t%^KgCNnedU^sT@yhlmaOWrMBDs6Am8I z$W?pguqBFr$0fhNXVW=vv@iom0ixssqz9SF<4dXsi}nr)mX$$4f+j}--r!0aibsfk+7om zSf7duIDGbI;1|-?!;BYxCQl8MuZuU)9J)GDbTqxq=8bbVmjF$o?oeZSt%jyjUvQp^ z3)^1bOM)7M8Ew~WWGTZFleCasS}j_2*evXww2RMPu#TR}qatn%AYSKi@ytV|S8(yE zI>bCwpH~Y0$DKnAkO4&BXGjaITtk;s3l{Ah64U~?hQO16bNHiS0KPy@{|A6v<}YSL z$$03_$C7P(Tu&Z2|MfWvJ#7&-EmM@2w^m$u_{YcHUi2SUCbEnjsNOQ!9Z#?QS8?|P z0kWQVbpqNV=yY3~-Hs^1JcE1abEpsOHY<zNc%Szc;+K^dOsJ4gzCd54anP;X z#z$q!_4&4&T-ijv93W2X@Yg^Ol|pbhLkM~}ogrArP%wRge8i4Co&4j86}>G!KEk53 zd1R*6+W_5&^wI~Z3pd_90;t0m0CiY*11cY+2${2@2o#rkL5Q<0jgJO5aCXt~=t|!C zsjk8SpaBT8cL_iU+RU<+R0!r93KnD>0@mT|Z2%M@Fl+c1LI4j1k!#KW03`IDNTWl| z4|X^>IYC#3(1VZ>4G#AI`lP4l9`Bg4F5XbdPU{xjSZ5MH5!Ll{>SR<;INGVj{$B>xqUJB#=ecrxwJs99TIQnRMrYmcI!H8RU`*!_ocI>Xj}bH-g+ zR2%8tRM#$_pu|_1FCKT;1aOC1INV|UyfeYX zO6|9}hG9;SpTAjiBU^MP^mMwy7*|@#)b)@&5mTht7icX2DM1)s33$~OPCYdTB7ksu z@)jsTPS=MeRf0wPh6KYza1+6E0>0soMgllO0F>Yqo!RNhoQ`&T2|9XudVUy%&da-^ z-rxVQ=r>P?Kqj~Ns2q2eU<6IX-K%U@KW8asMe`J^D9Rgn_I2cUKH=l5qVMfVDy}uv zJf%6(65KM9eImQvEBfgm7416r%VC$o62F)R#CJz_L?e~{${l{-!<|UIm$ColrT>73 z1cU(e6_%@cobD(rIn`O%_uQ(~6D>;RRKvaUIEM%6PQ?gPtC9QLF@G%k3D7}ih2-65 zjmr+feTBaQcnAUnFz`?e01um{!$gEfg4rJjGiytk@^MGVI3e&MgHDd7g<#Xo`0|JB zLp^loWg@Ze+h1Jxv{vW3zonHz-xYW02H7BjT^c|Sp~J)TaBE==zO~R9v=&alt%ZLI zss5m~&>4sT;u!`~g5a9L-=+i$vJDCH4&ekr3&ELSHJ#H0fepe;xoW|kefrlVI00R%r0Pt`h3CVU?UP5p`yAOUgU0ep#0$7So z%SmPr1?y>L{LP!5myp-EtoF?ln=ur?TG01`UVo25_DOQpn+By1x6IT6%5SvbLJh<< zcn1vyWYAE6f}n?>kAP(?EnrNM*r+gTlytr$jGhDQZ z2*N~!kB}qQ5c+IbO_vr=KqBx;KyYPfiP-O{?)mqmvCL{hE}C&a=}r@ zqGfVoJ@yGA=u-P2omko~j#`Dt)V8!{&43rIrRRx#Wk{?h;xD5f2IHuQD~_dnMH#>W zgC=~9;a4^IUpEn>fD|CoCLulO2oG3NJy^7fNKg;vLKxs#h_i)a=48N)XQG2hQ8;{c z2$Rmhy`c|hGWb)1D;al`sQ6{d`tsJRyB8kfNzI`%iHDvMP9#=J}KwOyxa(6bqxRjbH*55R1U&`~(xYj(s+0N+;;xw%r{B-qag5EO z*jxTd%g$t??=mM|_b_eX6>C*eo5GpCOg~&z)fNa!H)nKJB6yo~>vQZg${}D}3lYt0?+LSM;2rSw*oI?aw zflqq^3xFSq0K6lV0-FI6GjyBUsO}cI8X{b$Y_ynI47@QXo+fjQAx6G+fNcnSX%@S-lI8_WZcBWjZMKM4 z`8iYt|A!`dm%&?{cP{isma6IGx?%D?Iw|8`*7@`+r3bLcR0%fr-OTwQ#Oz=q+~1v1 z%$}$}%*2qyZ=&-FJrc5$#WuW!$fHEWdvCaS;-OMV&ixmMF$@sikYQdSnC%vPQ$p}q z|M%V-4ge8An8l?4CFp4|KeUu9!bSUr1fxW72|>rdPy%=+m~m*Z*Ch}7Ch$buyg}A# zil-ZCYC5@{XLZTzgEEDa*LFW$!D%DwAXufLURP_P*^`$|E88GHrP*0fAOA&z#o$TH zjfdU50`!VkFSZ(}UmA$FW}6m~_~N!5-6`ug(ieMoGu}lVc^Z4-&|7JwQO5#^K_DhfO%(Vc#_Ha1IHW z^;Pgwz(bCT#D6M_*}thQ#-xYEN3b_^t-O3Qez2aO3!SgXeKtq@m}E8REM|f`i=#V` zKj zGoc^Q&5Gl0yn9b_6xm#D-Y78{I+}=XbckMt{VcyW!AMC+c`)~U;jpcbWqq`>y}>JI zB0n-BR=4n1;SNEB0LC3|1-QfM2mw$7=x0sa!Oy-93DXe*{8)f1Eq&|qNgVO`;uCXn z9PR9OlCys7ON!{@Oyia>j2{_!l#JcXIH&3zxQ>zPqUK{706lc(_P$}4rJrKEt>2%m zDCSJSk@~*#ai#10d*VX`^g0~ejna=zilBJ!C>}V4*=lExwx8h6dC}U*@ZN4X_FQ9+ z<^;*S83LdOhzS2QAPSG4=P#)!EZQ>!qChx9&__dPTv$t2Voe}$I5R^qv&bA*T^tyQ z>oC`=xqlftI)q!xES*gKuzlAtx;LNOHnu)1MwiCI}DNl~hLtBss*ToK~dv$HB`MZWaN<-%qrUV^a z7zuU-Cyo$%hJXld!(V_r48|c3Edlay-}ha`o1nGi*W|#r-v)Cb569`9KW!coeD=A> zGO^Gju=|i=l+t4ws#+5I_`Jul$CI$`bQ`#g$|czkT&h@iS@%QYwhBfW-JBEk{inH} z%9s!!9!`iEmp1&Eco+rVe?(d@Bmx7EFiR=|^X&^4WFO8E99ThDa_Q$qV0ti^2AfQ? z(0Ad&MAcV2pnd}QfjmTrYeJ_S>q@7rTiX*UprnyI=`mz!L2skuu!(^Sc~R?aTOryc zGuy3EV6tn}xT*1R!Bcj&uvPK`Izlb)ti0C8h1J<6m^PozC>H=VBvi3HdVXE4U8zr= zh|`{&B<@Th@p^}gCmwTE=K>ga|sVRWqO6F@Qctm`z8T6t&6Q=H$sU9IoRe#4QHId3^Y5kcS{hK)Yw< z#jLi%v{mzJ!K`h;MsRWTF{8|?Y2~M#=0O9wGa8Djm%U+Rt((3*WIxaqqwL=>UqO)- zo_P39zd6a0-8fs4k(!aasFsXLX!8HqI}doO_y3P;BI2ZktR_iF6elaR6=@h55z(+^ z3q^=RgzP9$WbZw*XTmX$z4spR|9sD(U*%9;`Q6`nJpSLuz3#n-+wH3F>wLcN*X#Xy z{vO(-9H;;cX`2D0Agb*J*IGIUE1d~`%R|ISJp9#S(Xvvo7%BP=hE9!wo1j1JsM&sW z&sCP>(Q+Xn7AEMLOU*3Vo_CnU`!Yf0Hje|fqydz=QpLKS1M!?%2F=}tDb2_X-}Xx9 zT1zgD{u}-E@jk=IbitfA^!9AtN&4<1vqcfeF{>6w;+w*U#B`=^&Zz8C*&yt3rNV;1 zAKPK**y|#$o_V-T7txO=f**5*|6`G&W^n(*1`&`5IG4R&QxRCTeTXqqglZy)Sbj3N ztO%fLiNM++&W@9!4a#FW0)m2~;DS)-s4HvVe(2h@Ye5f!MqlERIV(Y5O}?DeXifOBe-hsknofv0Hsc{Xs` zPHK0k7fzYmS`^TWGgPst9d)*gQg_Dki6M5Hhkt{3C>MmgPZyTQ58(}0o)0WN3V;;B zO`PdoC6_SnHxSgOo@keEa!$1IFW`UVN1;&%o|1}l(E$a}# z4wv53s2YNaf1m>B5CLumm=DO$+kXj=l66Zu>Q;D>&fgDROj0cb^^=_PzXie0obrM| zzF6}uH(m2No%ZRB@lS_tPZXa}7)UoMPIr3qxP^>Z$UDhEY-l#q*4_1{tfVYQ(fp~Q zciB>|da4cgc^^!+4n-HdFOjq#F|&0junyv_Vy#^lv@KC#-`29L`>h%ripsvt;Q z-Vs=;8{gZ9;4C1w!%VbaVeG+UbJQg$C}?3}4V?x|&w!5Y#7UvMA&K{{w>%c2ogZBLwyK# zsK25Z{NNidfgOgndi}ewLq{M4u-92xKn>zv4y~ygtlBrks0OGoK{3WjAt8BK#PTwM z8V`Utktj~uP(#jem#`}{s=OR57=n?2g&nL}7GfTvwhz-a0Pb*}W0rechuh$(VR^~0 z=?G@TwR+Q@k+4rrX39FUmPvO9d7A~D>{)$o7z{Z)H|fChjrTpB(%Klp9~}`qUSaKZ15$i?0T~N3sz4}?v7Vc#xqvqe#w1bCMZJq zR?H6o34nz;W&<5?GkePYZ;c6m%Q{>(KM=Ne>3csR0G%K!ejx+N;7|a1i5d#JyWjeO z%hJ+vGdf9RWjW}CE-OP7mK{1YZB(qmUdAKB&T4#ITxF9=C;}f*Ke)HK&E@MWb80L6 zvqpATg^4_sJgMO&@tm5H8anM#c_jp4NND8KVUazSXtlB)pCS=9z#Lq}yia)Molwl{Ym*4R0D;$@ANSx}Uq^jfqC$EN9Yj5FH_PO&m9xt$bNPwNj z;okusf)oJ?cqjmXhhdBL1HY>TXyBpDjwCbhQ-i_{OO(J(i93+$kav`vpQW?dZ^j4+ z{&9=|hybiKf(O2*bm{uTBL^wvIrM2oO(?YC;P~5>vG3 z7=Z)WM})4OAWwAXRs*x&(_utHBz4$`9EajSUX}G%GF(@x$h1o6$KGd$}~(&9YMBOizx7lAUwlwgheYF6%||7e{}TJ=Q@Dp zT7bRQ;p%~hAVh!y9$Et6Vf8;b6rinx{(l{KSPoPGhMa|@ASttSO{HMf4kE@x5#%72 z4*mlvSUeT{06bi93!$68^2Rj5bI3Gsfff_bLsHP(bAcWI2GuUAX!vn>pN7&B3IbASwZg?9dHGVpsAVl+kzahXcOKRpqEKrthF5gBxm%;cb2O60B3O9$u!L9?-H zS=Mue(1-byVtyMz^Ed1{bUhTuwN%>SQmq0{3rIt*$4hnyo6j)C)NpqVC*=gKQ@1iE zA2KK$e>uQra(FVnwaUGba;~=;5ox42kTv(joYCsSc>zBUM_TNT6=J7_xO(8>PwR%5 z1>naq;ZH%LQb7Zy0Q=I@!|6Wq=h$L<%qoet%d)rd1K{E3$8NiJ-p~yAE>6fRPsH>x zER5i|ZkG(Q*Tz{!Q(oK{L7;n2e;8UmER2zYyiMI>8HyR?LsX14!U*H~$U_E=i<_NtP)IIWRSAGu;_ zRrnAFn)5|(h(%J9@Pg)h@CN*hpA8)-R(qVhf95XH(cq5iu$LBDg^d5tJB-KBO%#EH z6R8+in%P+xrm)39;t1}!Bso2$xHRG&zb+vnB);WEA^i3Pn@X;Z61oGOprUa-yP|~q z`z@nfI0C44qQAvEL~1A@1C@~w?+}y|qjQ9q#b9Oku%y-P-*pck10{etbU<>@X!&YQ zDVN>k{)Q=4 zcb0dsBe#)lo|&pqVgeBtNf9UoDq^xh&d#5*KjQ38H^ zAoz)Q_&zD}JKNBHwYFi?;SIK);htI-Z=|4B4#&G^2&-sU@(mrq%@2DWod*;kZ8&sI z6=0mRi${Kc+9{|Y# z2W3=t;1NtfIyZDcc~T;y;n|Y|o=k~G<3xu=f{TMgdmB0N;wO-Ewz%pvCI&l>&C_do z)9Jf38@Q86%C_r+W}+8*2Y6kkt~41r`y89AfL9{lwqA4^vFY2%h_`?!D9zfs zRA%4*`-T_}1Ti=`+DiKA{e>Ej8r|fr;@C3R{{g#%knMsO#jbl^6tYoA)^F`xN5b_E z=jkSGv#de3tGhb&7RW}c#~mWkxI;n+cSyL9BEawxdb^Pu`YLWT&wzDH<`KdtP1TOK zvtG~PZPJ25n~3o!{{n5|2jl=2W}Xi;!M7o2O*LWF1|raeWx5EM z8E|m6kZjq2gM&jtfM-olvKbEthC`0Ctgra9XK-n;uDBPfE)G@!M}rxHaF{@#Kl*|W z1*|oX?%Xm|(W&l__^hCpHs>&-Kyyj|gaYZHRBSX7 zlH80EaF{YEyCT({EcAOeU0EUgSug2(xdYpMh*oeq9$`4E5)mns1i zB`DSYQzd{wOUQs@Ai6*}st28=;ITmMt%vZ5u6Iu0!&{I4pbbWOGq+pk>*JqNh8IUN zgfWFPMuuu#Y0)-kxs(z!RoP@@Py5h>XzEz=gazN#Im=lOZNt2HYg@*UgCU{u{_%-T z{Dmz{C4?-xRFYywm3A!~C`-n3nTH%nOoy>OW{kbo;p&Npav$hv!za0t;)MIaUMtY$pZO@7B-zVHvq2*CXhEA=S=BG3{& zv8E!hYU>bVUm2>5Aa32Cs=(s@@uG3KSV6LQEZ_v^g1Kn0a0I4?H^4QHDlg^Uaq00U zDcD!_8~^-A(SEjY{Ng5 z0@N0A`W7_UAe_721-*H+0Yr-Ss@A@{6*4~TU}U!ro3b$f-{C#SJuXHP=o=?>xr<_I$S;S5CjNN$iwgPhJU02{0k}& zl6U&wR)J<90IuTRt8BbwlDk7Fw`!B}whkfYTg>9p2c$!tAG|$} z_0>e!x`Ft&aEHhR+~FpGJCt7WRIvH`DS_L?ciU{LpNN25A&Q93(U8YDGksWQKke7; zw52n*pytxslE=6evrnH;i)h-v;VgVO@pLm+W0u2YsgqK7ROtuVK{1y5EOe}M};O1w`sD5kz?dc|YGFwg>+nN1N8 z1k0(EH5G(a+lUwiVPV1ep9;cB95ea;aKjTVyZ2dv)g#NzsE?L@et7spsCp9A2RqK@ zz=A#-eY^hTF z8%Bf(pOPitedoN3soOF%;kg=~PR`YO&C7bm6}Ii$O}}3~L|cf-P;2S0EyTy*#)mmn zLwA2iYvr2W{VNR#f6GF|7$R7R4*lufkBSha<7Ql@O4Cf`3rbZ3^MKYoCwdW(C00cB zkeoO0zf$IlZYb$2-7{rEL@P2|`jnpELooY8=8ed%h+|Y^oH3rG@aLSA8Z_Fhtw)>g zW$EM>y{e*6r#H@>;%>X29!t?EWKuXKIdngLz@1NM?)a=!cxqFcW}HH_@CbNV(;GJ>Vb!#$sFRGE-V=?BLPcigqoiNN(C0S3z8P`MpN4@*|oWd@2GOfzoFm& z0CXo{7-|H7C;eAl$pD+U^nTiU;-8n9~15NNsRAdls(loy4R+ z(flyaiX+Gdz|gCxOyPGqRY7fev4ItVc>%LHT@{bo;mR2m!fL9BW4DY7i!JIIjM&CI zs1Co3+Aodu*i-B@4gU`E5Q#<}K7x>k>(D6zIdl)|Q|Kl9pCU#-hKf+mVbRr20Zo)= zu#u(B`DgB-e(5tV=a5_eo7j#e-PshVced`v*bXBT1_7JX0}F#fT6D_ziHzngaTN%0 zvY0Z~^lzYMm5CtcPM9~>Ym%}5kaN+KZzbE%4+sG)Y!p(1_eqv~YdI!d=|u3)Z9^Q` zdh#%?rE0Jo7=7Ucff^I?1ODlPZ>S1i03P?sm%lylbo^3H)ZNeT&U>Kfhj7kwfPlIl z-`Vx#HK%B$&$sS#-iwT+(&K!}*{9WBJ;7;jP}6Zbqi5Z~6<(q#$DxN=EG0gyrrL9t z;y7E%Jqv_Fg5wgJYFv7Z)TlVQ!%sw}M8tjr8~)h}0jr~L64b|x>z2de$`9z`tG8CL68NaMQ4G?K^AE5_AMLFs;euK;9gg|ucof2 zMvgICh$SB{Iz@20dz`5P^2d=RXu@Ui7wNpKWGjlQo z&A%@W*y6J_<=WJ};g6s6Y-x2&YG&_gv}-i9X53EBK4VqHEcGx3xA$2!wYE`yG6gXi zx8$&Kc7yxZt+-M|dQ^RI-h&M1wXO|gd6Wn{ox{HYJp>7Y^#FPpb{Rkq7mEl$IRSWD zko$S-VBz_|93+Y4o!l;d+VOS=S?DJI^e3eRQS=O>SCX}Itm4k`4%0(PqQ24^wG06^ znrlR)A~%kRW?o?IuUJ=;6K`k(Pu;O6`?dHhL4pH^dmj@gQbB7+&!M%WBm_Z9#gUPQ zN!g62$+zCxJsKxhLvS(I&!;DAgerd-xfHzOi68^00IbxZ6i7jjQ_R061;1q+Vx%6T znh3J~TnbP}0zQIA{5WY%{R&eL)baMbTr4G%JCf8@?yn8CmR*7ZMpF|;mh>=rPLf9s z55*kE%s$@YPz;41<&I6-HBvbwpP^h5%fPp9u8=bvIqIl=pII#~sU_b#=xm5roJrFs zXAeC!p17~g#U6<_`?HJYkK`TAK9qT);CM-h+^LI`j@z)jh5$RA!__kn<=)d%OFoVY zd-42t&I12YfKv&ETedu5-jaOM$Pub8rlxJ)JC*Bm-+z@52J+?!891;ML1p%N<^n?U}EV21h$YcbWN=4W(Bmhqq1fU-X0Kgak0TBPv z^9une2lqY<*#HSZ;e64W3c#v8LyRFJRR6%!^?!U8Sh8pY-uTgoGg1LO-)&&iXyHTv zZ6NdaK+MA=KaxwvUgjiLo6p|1yCi&QHfHi_TWC@0u6gR>N_bVPu`L00WunR`f+`F( z1$arx9NSSRHK>v8bgU#Mscxt}ZDyZOaj#5us+LB}U~!pwOMv6E{VWFEC|lndN*v@x|h_ zlJso^JMat-Qv=%Ji*>m#8ktBMr|2xRti0K;20RfPdmL+sFys%U?C-WTPiXR&9eOcO ztGw%k-qx3#xssdd;I~e+kw>X`2^xKA7{50|9YVX(h@crr0cCG z81(?k5QG)~xgKCxJA$^3x96ibk@>yT<>EkpDT|kb zg_1rFhUCwj;Myw2RH=GgM%So!3_*Kt8%g>(8!QhMVxeWYdfXuh4jkHo>-Omhz#T49 zPM1-EA64ukt2W;O$ffL+zsH5GxC=7hiH94?a^AS8Ei+VIDR0@PE% zcPRA4gr)@w`jK<4Az%>^-`5q|IB2c#XlN=2Z3IFYNhCs1Iq%Y5l<21TdsVi`~39qiA_VY?Y(oI2E# z-sCjqQDpUorPkfc$MIF%8EnrLW3O|#dfXuh51?>|<^XrNP*(g~Lhx-+xQIJEMT1^3 zXcfVXJFI)Z@N@wA1^l7k`~r9aun#E{n=?Z6Bgu^4%eN;FC0lGa%05TEp^vpLLQfhP z21ZS?B_BFGBurO24Lk-K0CUJI2SPBA;e2u}4TDwNhZuzbvJcHNFbcs&^4Fp(`-gm> zm4qs-DTSpVC=tBtLl;a7-2LVLLav1Q>gWb@xg!96tPE-hZW@(k=|@i8X8bT$%c|Sl zOBd(ZYG{+pu6MMqWcXmtXKt1)y=3r%niB1*&U(khw(_!1RX1PtD3M7Sojif8br?`E zxi|@LDe)XTK*{UD7_&a6s@LMe-e$Fz*j{3eo&Mq9p&f!00g85rEKI|}p+U8sd_cd_ zT$uLrv~a0!(ACZG!1saZ;Ce&H$F9TqsE)FIrWp=OYAE6%_nl8(7sQABHW3^Pj%q$b z{aV2@DB#tI(6%dthF0*X%NZHW)d-Yta9lSVAP>Qs!4>qH!NUIW-C__X7(#^!g1mnj zCICVJd+ques6lDS$2C=hm5v3!Wgi0M;cw@LHiBiu*Pa-v3Kr}`lx^_67=Xbb(128M zHc*ReLcBVxzRH`JX`%ShXAY%a0|Ii#{`?=um-oritAK z*UTiBm!~ci5M%eS0DB$8zW_Y^Vci%kAVHSFVvG6m=HU;KB1iF+%|kXF=`M0Hk?;=J zy>BH(w`kw-wRj>kDDT<;5XkyrjKaM(97gG`yTqWaL+NTpy6*aIpAXEWy_xy8dOQV+ zQ*gFyBG~{8IUewLRA_&W6x!8?;rW649~RaIi9on}_L_>os?Eb?5=HCD2hT2j=l_)o zU>X=wQQ`Ssf~>)hh-Rg%q1uxym$*>Nj@C zr8POc*_NXqOy?;K_ZZ4->F~TsU=lhR**@T6%2iAo2h$kJ^>-+s4>={xr_HF;lA)AQ z>D>3yi!#y8(Q{JofGKus0Crl3e+7AnEreS9_?;Jmu{KK1Ry5Fo}QdMBa`_~lqEFWro+ue1;n9% ziLNwne>>&cq-=}0*9e{KgOHdw`LG71OvT7dmuy{EYL9vvBchpyV$8T3hp*xuOm%O%#3)JzRVyFz3qz&jkN{=wSv>09fgx zFF*!HhDO&^23Bn!VvG`@Ita@Cfeb7j2LSrCX^yraz+*8u1pO7<_h8YWvgeXIR6u~b zrg^M#NQ?rrNR+esEs_~#v?~dYGZIx&sd3h5CABD)bUljP*Fn!*EG;ViMLJQdUZdqx zk(rd9npK^F21j#)(+n%IEVr=!c>C@AaVxK9^(;p48NSLVj*hQ}Ph{el}Ro3OiMH{(ld8SPp~$hAe^9prZa6|5}aZLR+$6tJ03FQNR zh1p&O7D2dG3>+c{sPD32`@;binun`L9?Auw_=egL-;nbU&IA8zwWZDA=7$Y>RRRTw z&U?G23b4|M@VA`9W#a>bHUB^XQ2BxHsz~mA2K6JyRcoN$k;UueowL^Lcfrbm2i>Kb z$<$0Tyl93~yGLD$yj9cN9@W42>J~&`#bLK2M^BhNfVh~jL!@7dCP`mnxU)zh$B9^k zrzC|tRA1C}-gQstt|UACDn*Ks_t{kz*FALd-(?U_$4@(rgyk^d~SS_ zhW~8RZ~(ILd5@z9S1hR>@$s7~$=3~UnO?{82LtRh4*w4A5QGR&v_nBaI~-gL6|Oic z{I0GPO*>@X$nu(gTau&e634**$0g)?*5JjOLQ-^1Au^LW8PpUG$%%Vbld6wDW;v!} z%uy^97badj*5lmn_6ReCuTpSgC zn-RD)jK1SwN(NTJ-MddpttO;)!#<#f;lxUf7-nKq^zq#P4$iKOoT`{|AR|xfvh2w&Bm9vFHaEk?6_K@7yY^9!0nV0BWQhN9&Ly{WcNt z-t1m_s<%`j@O(}&&vs8$I7?i$rleMMM?!OZT8F#(S(D?laLta4uZ+Y4+=gLuw_1wY zT|dT5tLq`U816phx{U2|fSn%VUtt~wqM3&q0rPNi4><@GC0J-N1!{qAE?Ia&Sk6Pd z7W(gch<-o=U}2Sz5X4WK3#{e1a3vx5Jr6M&gAgoNJMb$ZSXx^O^$&n?!6kQSJiq`g zBwN$rE?!;pL^pRUB5Y3vI_NyPlogZD+@AF5jYWWt?As)@LFN_(Cl<$0rWA&h*eTyB zM3ZZio-3bAD$g98c|4vSsgYauss*2(O}N98Nisx2RYD!ni=e6YN;+sg+nbH$n+DkF zAFiHw_+y?JEFeKiqMvX7rN;oxu^or-P!RODrECUwRwN#->yG>UpXY-A)R#j+A_)X3 znQpdgvRr^AJp@57?j_w0XeM1Qe?T*d05?7??JWY_{qC(EYkK#u+B(EoPljqCaKrF+ z|Et@77lbhoAxPI)2p3DCdPff8;x1rLb*t0U<@;iz7OvKk4mh`ar#Fi`Ij_e$DdiYJ zUFP01#!15rN&}zxGn%~Kyllaxrw9-!)+)X!y<^ruyRBd>V!H8zR_rJ9&GsJL_eUZ! zJ!l4|lW2$Kv!*ScGk9pn>{JP@mIO$X#Ov685WrUV@NYp6<^=wSxX0$AxVBnUGlk!valt9B1D3IeK&p#C2S!p~U(sl|Sxb3*r_NFh`5 z?W3wk-5T&gyciq`y1Pn_6LfyqtEPi;{A7zk4#%3>OhmupN~*JV8UNWwaP^RfvDs z)dkDj2(ZvSTs`zq?)}-*k9RXuLeN9X1<&Ajjs*W}?qNB&`C-Vk8lV7mi3Mw_04q%j zf6F~wHa}46hhb9qt0Tb@#+i_iMJYHIK!0j!XlPW+x0zN$7<3H@kS3b-&V2Kfn)mef zj-o@iCNgCU=UmM#Zr&Z^tAOWgsAxFe<|#SzNc#Y@OZ=5K_q?JLdb{e>hXRW}GnW`8 z?Hbh0wPN+5H})DTYMyyaD+JbzR3!SU>m&6P=ilx>kbXK3%QpH~s?H*#30#pw{$3Kt)lxgte(*fiUEMAaJ zuw~E86O@x!)7l6<8*pxrV%O9Gi^h~x&p6rPG$T395EPxcFg~d@-zLcB{{0a3`0$o? zZz)_g+Ia2eem5hs{+p7i;hM@L>6K0a+gM~(QVm9K)4!ifygPohw}UfRTjH9dL)_a| zT~N_s00jvgf-wXMeswBXd@fidoqdF0 z=>7);=%B(vsG01=xGUr#qP7l3A4#lA(NkH>aU-aQ_v(pFup9g*fn!gZTj zEdEN1p}Ewh{uo5xU|`N?UY5;CG{b6zIruk~jC~?wTdQmuDd$wC5<40nfj$#uqEM=O z3DdA9hNn-buKj;;X*^~UvVwB;p8mqHlMQaZg7t1q=H6_XTiKK?H|Q1b>Y<$50QX)ID{r1>aTbvSUNE5>b@R$xf4Y+(QC-%;(BQDZRZFV;E%(~ zo>4l7G@mCByH|ZTTOgUsa{1a)g2d(`2!G54=|ISW4zMfJ>W@z14rC;HUoUavT)B(@ zNC3ytuB5FS!r7f>WShh zug@E?_U9NlAmttD8N34woSRf#h~c~)Q({Qv7kmdQd5-6+Osib4CaQja=&3a&*0-5s zr*Zff9)!@s5nzhxf+ z=pp7D0g`0tc>xc=5Yyt8mxCq80%+w3IvRkSp**Ak5c!mEFKTiKzw;Fu7I=&MJr$LU z%o20!JH&NNmr`0q%fD!hDAus9Z$vJ)!8$-e_M$#dOTDMPP)hJ*io9t-x_9P=v-EZ) zvx&p=pSp0Bjf;8VWFSYdPjRG3EDNk4Nu0k>7wWhXyPprR(?9$R*uy|H_RxahJlm1Q z1_D$&!4eTz8X|xyj5jS-7z-zBqbiKiF!ZGr#wV%uYOI!?3fejW3tgqxX7Ne)dX!u=L?UsQLNueXY=>a=^YK^;wqjO@{HCoM*58W^H!4KH493XvJ+FSFI zwstYK&kTX?RwE&obwQ*(4qu~5@#Xh2nqapKV5^7tx1fh0PQaSK*{x(ffF2%3KO%rV z1SmxG^O4|(>Oxe3X=gjb15u4V`s;X}QkJy3J$&c_Jt9~rCRowpj;6SW|tpY z{K~(Dj&k<~GI;sUrvt{@1)Bv$+~0o|2r^8UN>)kUU9nY9CzHQ^@Ji@p?tG4$WwFeL z14LRbkIvh@5{7SPlIkdxjgC#OF|q1dFA?wIbeI0yO)Y#X)p*)=ymz0*vDpbV$gKGLL-U*?%Z z;QdiQzP-rhdWIFX&xA48@JLqIlSsx6J!YEV)ar8x<4EW5uT4tDA6a4v`d@yT4=GOG zGO<4CgDbK3#}~Hsb1`hd(VHV-|o}92}lveu#zsq=}9VauDMT zNe|(nb`6FkWD!+oJNHf@OB~)-1$N*k575=!{&c;jbjWd&N3edb<@HY*@0dsirVdzU z**36$zVLjfPNsPM84intmiPUf@40xpPmeq$ZhRnnO#Q)CbqPg6<-@L2PA&uUrLs}6 zZ>Cu@$5y=iGr)}xD{X=9e)EvwHNE>+8WsMQfrv3euZ=ZKYnAMgx@Af7=e z7zpr#xPXAQI7{*<)L6(2@eIl-!RzUA4s)qZWcZQeGi@(_8>SU zn8$;%Hm#rL6#pc*HI@emu-83YJ@W9U+|cg`0Av0rsnay#tgdImJ2}!kT{P|b+`UL?uCW}U7OoIht5l+x zn)xVO-gTdcGLJV`Qm1zBuvQwQeU)Itq~|lvt(=sRTjF17P_&gh+;hkFl0xjX4gU`D zP%g-7&rO)QIfOS{OcY~Qf~Dg@XaV_u#WrjPDgYathom6;iQk$^!AfU=e{LH>34+0q ze;@^@`C$6y^8tw9*-VoSpdrKRL4Gw?uaGcO1B0WoN zcML?24}s`0Idfz1;T~dfyCT<*$`ek0dyJW|eZK&Emkw9YJ4B)jPo*K=q4bJFft9mF z-zdnl6w6VNzuPxlb^$r~A1@#e0}+6k6(fNXyc>#NQzckw&*-;|LyTD>R1-lQh5*5z z9ts>^EOZf}{$dS2UKWK|$X}sOA}}hHfH26{8m;bJ(fcz0E=Sd8Y#k+?hhWU%n6tgr zy>~1xk;q>i%ZXA z59-(hJx0gvE}XdKjrBF=*l8U84d5XN5J14gK?r!b*hR3+d~m50KwSjBo5&uMm149E zzeNak-i!Ne!}$Hv!7s~;mAYSbT6Z~urK7L3NdyCi*LakgXV)RcDz&^p2qO|@B#PvC#?}V&dmuW&)ap=v~urd85WLPcrhfh?K_d-K{{INt4Oh=RM5398M;xBx?1It*AYuqH5H%JVh~GAkf1DD6A;HV$ZiZ{HgoD5)IG-+L zaQnN}LxYp|t?)(MH3Q18C*4UG*vj}ZPISQTTY2#!2AwN$lvzVJ`%_3<7$YwVgFaZ# z$)%)x=zv`a0+EkFZG zEis%E7BpZnII!49g4#0zeZ}P-P%()=>WX%85yJzNBh>OSv}tsi(ocoK0zb)l3P8Bs{^fY`2EG-6m@O@ytE;a_7O2BMgUa(e;u@LPZY#Xv^>Of^|* z8DeA}PHuPXqA{T=ow=8>mn|ehWfx7vEf+fv6*EHV)^k$(Q${WkJ|*V4`%akPWA`TY zv52w9mpSUbY{k{d6iRO1w+iH;A5a2VSX?uZgS@Cmr`K|3xN6G~qZ~j90yD)wmjld> zp^yb?wcy@j`v7j6DQfBH#*J|(VXQU`yOcyW!vNmz*}(gKN7DSTFS5m6o*w*jf+zS!|ODn zu3vG|4`{{mts^XS3|G%Q1i1ke^YAfX9xjs5k(gEB#|Hz*H?+1@_60E_0LJ`1vB^u^SazZ|c-0 z?x~;_(aAYqKP1ZHE@F%UdH7Y(sI1}F$ioP5|HINOTYw0BcN#hO=QH?LxX0k3=DwH2tUk)dLX+?gk^Co{T&cxfq-C^KGsmbo#_PDX zdb0Wm^g`;ABj=IGY*ZL&V9vvDJ6v8!7~4^2*y|py9(xGl11RjFCBPn5FU$*nM++AB zjReqIfT^EM{>`5k7XI^jVFr)^SZM{M1-8$^*HjBuZ6adS0;tA3t?wUb!QunLVut7= zl}jaxUT|Zt1#H(IsJk>elZa@p4J8q~p=j~>K*qBDhcs0*ZfV`q6l)svmwr z!??9ItII9Y*70$xIJVcAW2c4qcc6zLM}UGJHWj8VW{WWk!BPhiEF5)al%oy>kM)L& zQNTkq+yn(YtXo_>WIWazb<4MbIXvljiqoS+;?01WBZ3Ur`%B6m%7V5Kv`Z#js7cewNcU|DhNmwc4|$2wX#?leJsY`^Y95>gI=c-sh5YsT^oWbAbiSI;~A$s@q50ZT@OKjeup zEFJzg^F+R-y}3?o^UgwX=-c5&7?T1yFk4)ru*K zC+aYg?u_JBRLE+7)rT*xB<-VpKd+vY_;|01K|)9Nqbm9auPObug6H*IM#SSjR7sr1 z+M}Y54wFIHJ{rJI{aMOy91WQi>AWL-l4*HScr<1^cA9FSg6aZ$H z+6H7Gq-646lY!r|4KdOVQ56K)|5OH83D(IkV9p>AI+}AVAd`?Thhm`f7t1*`jJh~w z4p7htRp za%3U~b@<*l{7wY8lDjUs0`$2hdYj+~IqLBMuO8|x&H^GJbZrkz=)sG=qg)aT~WgQ{gzQK9062Kz~5pWf&>A?IwXf!hvaCI0Z11?FX=-sE%cLo z%MmOb3(%FN2~MM@V4BDMkpc_rhM}!q-zrOG-_X#3bwdqj3gx?kIf6T-6|*lL_XG<> zHG7A=)Nh?L?mW--vEq&>N75J72ZDiI{oQt62`&DzgD>Xo*|vI#Z+*#`yH|lz!L*R| z?$imZald5MFR@(Gd@H4i9Dx|XUe`j3F!(NTO%-9)&LL0)%s~Pq83&RtsE&5DIx7Lg2%9kUW)H-@x}LYY$aX5oWX9gQzW_K*onKZ`R~;`t8^-NVr{ z&dm|aPX$=$8~z2{VIT^3$R-MKhp!fn1dGsuov2{Z;y*354LjQ<@9?7qAYiySh4%tk z9nwZ&??k3u~9+ptd7Ce4(vzpf1rTd2CYIr(1SL-#e=oweowr{7E z)q(a6Mb+c-)Apae=t^boqW+r;hkoGZhlRcG018m%n6{=0u+oU|w`{{@vjb@w%iieA zegfqh{?JnR!Uc7i++AofM^6jzKvf}3AUCx{Tc2=&))b@5AqgUstYG3P=yh zly!10FtOcWpLzW31)(#|b}qEoT|a=muHou=haf@#@eU�Pj%Z52WCmYv^kWMT_G` zF32Lf-aL3=Jg7`wUf5UT8(nYlM2S0)>X3JooS&t$nAwT`DB`6xg#~4RXHZ=Y@eEKk zg=|ngnQ$-ZJ{`iS9F2m8p@o{lU#=cT02P3xO+Zo*F;}&wQm|^*5MwdvZx~bBYBjRGa6ogrSNH| zI?O4p71gX?lxO*wNtwBWi+Nm{X40s&(Jg3Q6|*#Pyk?tH)Cz^Ot8u zjvocA!#0&v1Vi=A!S~iT4^3ftvKTvU!@q+(1OWmR^6(5m9)>L%hl>Y+m5v3F7JQF9 z+6ey~EWb4?%_ig*$5(aO&Vz(TwXI!PLGhPzN32Gr>10@TMH0=bGu$I695lU<=_hn@3{O&!*E zUTis=#VA~PI5t=}m{`T$(~FZ&TeE36|I2fkyaum_+3DtjMP4rxU5rHfmAUnYj&ziH z(HqxjjTlx+GpwE3sKEGnHv# zg8PWRKJrE0^oxL?@t_VtuTyff>p_MFwT+b2xp`N@+9E)u7& z>83T-exjv4Q#Ir*eZ6=dmX8Xs(?nc7@Nk)}f*+p>R*Vvrav3Opw{_TKR!Ov7mc51F zKS=-cW4B#9Z)gU57boOhh!Y;y?UF(E+BnN-%8MH#2z2l14?|mrg|e(*NAZ=~AEXI~ z!JQ8?1AJie+?bECYkK>y+CIb>B|^0jlr8TPE_?f<<^$YcxO`74dJhBbF)hc~qry>IKyjBjZ! z*jMs7VUEu=>5j&GeMIZ^+@784dmCa5$VO5!tt{+icms@k!i(d&Bl8eM)ug>vhjW7I zZd9D&{EY2!LTueT{9Cj`q?Qu$gEA7L9U^~gM6e9)5Hyv3vkv>4X7H0k>7;Lel63KH z+QZ0PuIaRuo+8sN=A3bd2RR9?VfjU`oj1M^G<4q{MQn z#ow~-49U7nojCZ62>3w{pnp&r@PqE9*g|l0w}IB~Q@541Zr>)sy{>j& zO~42XkzeNPD+GKbywg z)yws!H~mJ1b5wY^I6oMN;9tkZgGPd8LVx=DImp^ksTs5h_YPOxZrxk6e(y}oquldc zCtIUrLdiE#DEAp;>Fud7c6jaad76=t=oiybLNsF z>Pr-F-I9}&msg@87t1^;Nf;R`kDMja6v#ZenJ*WA>jsbYuiPj%({=5+(y&3`DGd7e z*W;`sz`;3n>FRayqc}MAxg6_oaPZQ?j`lRN^5#%VciIh;Cw~_QhY~07j3ADK zuprLlbwM1Yj37?#Z9$wcRY9CrI)XT64+U{LE!GL*ls*F=;$DIe;Fn|v%5I0sJP8SL z!2jz!F8E!v@GFV>91?>45pv@!{|~)TMQ1ux(RsRz&go+LIeuv#8|DZ5;>d>PO$)oC zAH1N_+HOfWVB*dGNvWJVQY@<8k4&t-bS#ZdzR`;{+ml|Y7(QB>ae*VbD@N~X0GUkJ zX?nvlSJ^TCs*Gf|oD6>T(H--XT=(N!dBgL^iO2IB?A+}*!c8j9;$+u9G`=S8hC4`i z`NGv2UjL6gAHP)IQ;oh6edD88(3hHSu^!!zx_vccw`)I2^%cbQ2@GSCRcowmFum*g>OrhOLecHuVrFoZ%y^KoYeG2(Zc=>G_ zWvy;{8w`7BYqcx2jdt<%`g%$Y@ojD&E!q5alR%D7GDFAC*(bPag?Wa$a+UT^=Q5&0 zeJEay9i5rx9UtR?bMWYbp7Hwm!XEXciRqpV>XV4tRMQzRzSK)3XG2QjpMQP2VVX8! z-LpK&th%Vd5UPwhw_5FK-iczaQ2*KXlZ~Dp)$09K^C#l$q?JdqXM<{jOk?IZ%_)D@ zX7afZmq6SgyS z&w9JdGi?(HogKl{K0;L!Vf9}u1sIw@0uzVWkejLJ#3hpVDh|=eBB+ly@*MhYe!K? zgxy2I^G}sO9xT{&A}W&h)ol*gE}DUo{_H5*i~S^};Yu8x8i=j1yJts7>)?(DN|#4|i%Rf^biaiT$po2}x2 z|Grq0$*KsJbkoY6eZ=i%niOT3?}V?OO0-FOsm|0}w)tknX_lzg(KaePwWOz|nNvbM zXKPQ(-oDBB`N@f>A-W^@_xTyOU`)@yPaOEcO6X1n9X;SM^sAkD?^t+y93hiTWr-Wvyf7FszL6r19eG_81C_279ZDq zBTSUCDT*mD#An;+tR$Dw)q>3FYhtbjip~!Qr(){XPe)1Ne=L0u{a6eC#*%4M z@@HNkU$Ech=@UC5y!%FKT$OgzrNO(A32$|pt77ilzu^-{`k2wI)Csqiv@Ar6rHzTq zi2sz5QoN{@7|yGGYWvs|ZjX*7zcOOz7Jf&dqo1V;LpZNd%XNfW zMHzK0zsmSYft+HQ_kp^}xkA*zEb&)zl!+~}D6aW1+#3cz-2xt7NvoMSQTh_XjjBa` z7HT72m#-D{kLcfURq{76A{H)=5$QP=Yu=w~UV0#BWO{7)0*g+JN%i!h*h}~4spYa@ zJiZ1W_STFELoXNXRK3(wO%3xjrT6W)PsZqX+tu*!$Fa*Ulsb3jyyRq0>5M!!JRRM8 z!#?@hc4e9OnPdYNxt8{@8$4}hel=sVEC(j`et~}GTWKFvTO{Ind72_dl)aQ9^_V6u zdsXE;Kf&9?e2(hmnE&S%wcJ8A`;yAB@q0acThnA-Undcm$Z9tT5ss__L#bDlq3=>; zqsH0gWzL^0N%kmpro0cnm##+d-;A|Xy4tErbTn)|v%&BAhkBbkRRzfvWz!p^qK*Yq z_FKy3@mLVKYo6K0lvvmDM&}av$p^O59I3rbP0JHusuW{n+@4#c`-UjCe#47ydy<6d z48qgVJ!7)@H+j7huAB|x+Y&x&gzu%P5fA>COjr(#lWcle7uYg2A>fDqMKFflFhn;7 zNtt#lv`kiYYtK6)evdI$0pbd(tJe}C5X)Fv1wHNEu(LBt4+%#U5BMn>vgkyqJ@B`4X(SFOy5?FX zJ1Fg?l1Q3(|y_{sy~jFWOTcQo%H2bPHvkTb;ON!yRckcPp97!?Q&UJ`m>1H z@ws&Z_9EUvxw^?s+b6Ck%sEut*hn#t(>e1Z$)Nh)>4=^&?;;WP#5wWGbi?@#a))R4 z`=);`vQa{`(a;8em1Hh`^u)&k`C4S#2|Ab2S$-}89$pEKcz&zR54RK+SqoQg74KEg z4;eSCYeV>Ddhl&}!9YxsaQl3kc2r~p4Q-HA-Rn13X)LNmPqDbCT-?S~TiJ4TtdLjt zPS53sqx|O%e`WV=Idhh@EW(4wyI?EgeY|Jf=Q&Y&SCd^)^2FN=V_G!ST^yO}Ike4G zg9=YvBR0YtRkc<-apcZPr7cE>N1lc1-ea3_544c*YHsTL)&?zCrC zdtRN=(JWdIzU#8~#9uy!RQR@L%DMaOiF%`w+;LCYpNIFF>y04AnC`GC3hQUnhkdzn zq7Qz!EoEdJsHW|2XE%K~OV`(M>h*}tJwy!+?FYfi&7Kc$3``&1-_9-2`SSSJ_QK$C z{Tm%R@6+4{YJ!}!Y1Z{|e_&B(xek8A^!dMy){TAWCsC0yVsDnxuWivUqhL4}>~(CS zI`bLUZjm~i_A$-u|goxPC!b_a~FxUGyya`^5P%F6- ztdT|3H^cpkfaX92he|b*+uebori2KoH-#?LvV#Xth3qzJd{1&ZM&+TseHC?obs3|* zfXZhwJef$nr+cMz({pfp-PJB}O6eMZCfm*0WouC!TpOJ0Qa?CU8q$B|i2?!NOojL8 z=~BDhTHSh!$~<$3yf4s>s`f*mOZ~)PA#(kq zsA56PlvapIRP|P_n&FLEhf;*^w7k7g;u8}Y?c;yH>EO9$(wkOU2NcDL>feIfa6)Kn zAMVwgNBN5qTEZ>I{X@-ThA1s|dfgk7?u#n1avHx{sXh|JvaRuxx*6}#d4bvq>1#mu zzfQjOs{N>>j;}IAMd@ZdcA4B^n2naJuH}Iq(i{3>Fm(Dx38tj z3VEZ+Ot%-NH|~9KOYZqzshBT0b6X;4S>olba7_vJiYneDk1rSr=nJmQxl((uTkIJ4 zmFtY%t;Bu0Gh`$H{DKtl?KC}e+NZ*lKx`!PYq~_8{uYv)7s(3*)23kr1Dd%g>F>6;24FnJ*keqZlNY_)nF>uWeVbqFY4vHj_rVOi7r zh>T)V`>f=y&2RU8T4~)uT#G0U?eHnAj@R?IDH@5Fw@c-3tmH|!uZyOh+6eLv?gi!*5gfxt1bUBcQp;Du} z5s(g%?gkM-LO{Ah7>q_b)zL8=2#$Qte*T{~ytp~r?sI*w`*UAsFLw5<(4@LT*Rplx ztb}}?!Z7z_4^<8d?SKCc7T{40`0>#D4_w;ctK@OxA6nVfUab}@Zwp`lGgA( zQN8#1Y1<7YtTu@hEu*GD-HFAm2dYgT#6v;&-Q*tV44Fmu#=6P;lE@#!3_E{sWgO?< zG81tvDKcsJGcd5f z7u=}+me=Ybry6>OmYM2|R^A&u+S-~v8&oEET5QWzHFHdUNeT}QRD+M4l>zM9AO*Fo zYWFsLJT1r%>3BH*I5id^)vk=Rr-3JTafvE=M265}NIlGMYDdZ&ym(!6Jj2)QLHx0e zpu2d>>4{juRvOQB&0UR(otYx_d<1|mll>O>YkJGoWLfcZC_o8wXPM2JY0ev}q%x2x?cp@)uiF}!Gua%U4GZ}{1LYF$nom3-=MJ2!L4c&mFA~(d6S^&vmidi|0)4XgJt!{4s{np1eZd{YgB32cAl=|Gx4;PaHe~`!WFHBAJ^xX2j5s(FSNbQ2}w@QJ({GY>k8*5xmwlND3UhCq1i14-C7S;$4Xr3Erp4hBK zLHHWay4yZX@I}7+OWR_|2-N6p>4sagdn6%Lthwk)Tdxc$Josnv} ziepcC4v_d>^WuCv;|2TFTPgCNxk!*b&zECQl^n(6lFHBnqy|p{NEK7dvtTSJel7v8 zzgFGK)B6Ub+0+8FUaf+Y=_`({aA(REw-sEhlK(Q$`gm{psz{n2gF$%~N%u@| zv^*y^UI~8r4Nbxz^x@l)&3+qouGqR7L-}8ZjP&OW+Bv!}ki0jsZD!N%y;Zu7l^LdI z43M3x*zxJs<8#5f+bF!{q*S_4&hjIcGze^2{3!HQdJ^&{yDD*cYJ$%D$HDJZG;DF&O&E5vB`GM6C1g7y;rAdQPB2Vj0RGFE_zS)QMsqT;g z2D2-XzKBi*IJP1bI1p~>zbThd1vIQL2O>LfM(-mSw>y7gS5<9Bx;`sk&(~FgllIZU zBIjoE@#b89OoH^Xfy=ffpUT-?&aFF1&mMX>OG}yq&c;L0=6N-FoR77U_U~M?3pIIG z0?4C7R7+M;>Fp65bOAq1x|VTD!cb=y4~+J-Y`5Lq3Ip8{V}}1k?o*l1TWQO9UMuJy zTv|Q;tD%Ut(_6~|ufB8m`x0m9O#VHb8cvHV;B4D$r>G%Cz=V9#SQh`>=J(tP?Ag(v zS)pBt>^r+=_Aq@Nj*|uvdC*z<%B0u-@;X@LM|@b`a`QBhC@wDiy`rnR8=ZSVax%dO zA3Ys``zK@qwH`qOvxs4d2Az|#rM;2eN;!+olX0+NMi8R|pis@bD@f@ZJlEA> zFqp(i_wXJ*7G;K>+(%w>Z)t9=fa{ik1>?w+-$tsa2(&GFdGB3j5bO6xbxIRFCS<5& z?__oFmm8~0KDU>rc=um;An)IFZ&_JH4Kqmm5#=|ysiq;y|MZis9@@v_4wrhRIzbSA3(+9 zqL0R<{=fB38TQhh_tB--#7c>2ikwnE{5b`u0pd4)4CQOS-I;tTa7@8O7gDJKYj7_r zGHrN-C`H$LM88gqq`YGdryPo`CVetwl^!bDW8d<9PiG-yh8C&k`(T1{B7jqdZea~8 z=a*#Vjm|8u?QXO`@Nvd{yC z$O6e)+f<-KE=1(-Fm>M>6O>CsDAKfp=PAfELaZ3<3D;B_}_2E1B;r2IGC-cL1iLsEt!BlXtB}o;-`-75S9GoqBc2j%F zL0(Qs_yc)?3=8D~|JP2qb;gf+Yqn4lF?(rEg}g~o{E)?!sM94NchkkDG1`=e?4;=v zWmB6rpt~mM@_9c=WbADB7D=k8BPq3=qz* zOL*4B5({WYAhK!9#@dtf@7n7uer2Hyo6LaGA>SA` zELfT7i#EYYo66mA@hs+A>qkAx!PLdx3FgC?AFG^ z7`OQ%=6oc*+0bPaQLz*L5KTQ&Djocp4x?@yt2iwR&0?hs`Sy*|*>L<5XSx{(Hd|l) zx;e@4Fi<88&QBrW3m*t|5K`vx)5K7`7rgi)&Vo-~AGb&3pF~<2RBO zg~gy5zw#;0T-ib9^<(bMbw8OwBPOP}UM;Y9Q}VL|<)FMKQmA6A1~gpz?nYF^+F=>z z>a&)^*i56x7=CAj)@5fwq{8jWV)3HxrBM`;cNm(2TzS!hc41)RT#lN2G(wrF7jmdd z+zTA=74vn$My&1HjF7TmpO)a?Z}2+6W%Ilfn2wQ5NkH1LJ1OWYRW5)u(E&ovo=>z{ z=)OLidJYq``I|1FtAHx@E&FtjL(Lr$+L1n}9)l04%fbR0S^U>A?8bVQb0)xsXoT`n z!lyCZg$jrG$&FuyIRPKVNRAw}9AkB2qKkw8GwhZYOSb*`Qe=99Pjl5f+Db^u3e5t% z`^$zYz;C}`KtLCQI&JO9B68An7gBJL38jENuOLY`?bJVB zilu4#r4rHiPJGL0SFCU=&5X^#3Ur1wvJ!Ox zWyw(xQgyI3DO2$-DhxNDW!j6n$Ygbo-)9QwLDl8SxZ1S#j@B&RkqovbnGyVpEiCR> zKz{E1P8r1ihuByuAZxw_DkqqctU%%{Ej;DkG{wj=x~5$+zihm65;j4W$Vvul_YP+P zMjkGX7|)c9fk)}_K(MHmO2C0QY?Z0|KAK2Cn1o@}V;AqP*TyPH$IwO@yW->M;An-B zddqB%)SyW;zq7cebD$u!HN_3>aQl82*l#pYm$K;grFb~MmO%4FjS7^vRRTpzrhmLG z&`HoTSjh^Ea+BM$A!3HPt!BxQo$&+N1EE*%E%}vs<-FSp^(J^$6^6CMd7j^BjKIMx zx!a(+=@Uu%GCd|U;OnQm9JhffsHnKTE;!ULotW8PxISrNgAoJxFp5&>PX%mx8WHf6 z?|$k}$Sc66GrWpeJjb}ru_bKb^#WtM8>mPQ)JnF#`+omL3i3V`_9Y|bG!}O$|4$8A z%Vo-@o6VvE<3{Ct8DCxsGEosA0rP_jCEu$oDs%Mu<*OZ~btSoNrIZag(UCwXs4Sz3 z#JgSAV{((G1lO*sc+W1o9>7@9sF01oEMPMG*sElHvEw{|v<$#-133B>7177c4)WcJ z0^0Hw2aD)_j{#m?Bxj-%iuvUqu!;brjo(l<9eNW*^ai1p;=IB?0nm*7wbm@&Y$i_I z27X@`SDd!$C|lT4$ymA@dl7DeyKUKr1_(p|KA;tUh$2>={DaoZyqAY4L=#}KD1^*U ztQuXD@Ce|8d!H#g)-QY&oAv|}&HMCO?7rc$1|>a}61M2lB5!zD1j-L9Q3V8lO`IHW zQnBr2=}BlZ6npk`2fv4_#iL*@zt<&}f~tYDMbR^|~2?2|%-&5<26Upr+3!0!lM zib0%I8L$&LF+$a2YL~Rzg&A;7;H}5aV==FJuCv?Qo@?ZefGAU(0KQoBTac}|et#Z@ zJEGiL=MQg9rf}@-y#E87T`n2)69Cl3?dazJfvYA~qa(l*0_dTInjK>QCvk))XTFXr z0w-LN6~6~Nj=ryBFi4tuXRD9VKy#*0ZY^AMmrTj%k?aD_^kq)Wd`?TsIn^Ddb-$GC z!P~%QQA#ZJbxK=9j^-7WJfM{oB5)Ti2w2GT1del8_piehJYM~Hbkn(#x#+<#QU8hK z+%EQ}w|>`;Wb8S>t$2W2Ps$WdI)=Oy8}@i>jzTPH4>*&UXZn}yKgI=e31;_=P8X7T zKN?KyJ-*PGAT%_bpC(2Y%%lv<*o=2 zitpDE_ zZXHXZl6^V~XgPuhCdi^OL|$UpiA!J)vcKNkOk3_sh7;}MhsN8rps=f&Hky#YQ-?y_ z#pmN>XJ?$?EJ?`&aHbI>%==HmG^5P-r)sW3q0a(I!I=S9l8Q~c%dfOood_ZqOhma< zBGv*oVEi@u6FlaSzK0jhLN2>CC^(V(D^_rtq;n$>9Ag!bR8ga-R}cluaec3+*ukm) zp^M&hs5mNO&YInM$?VX>`hH} z|66Zqi@G_3xS2De0T(R6k1sO8CJdOT&SM7x9UiM}#`f!gljpp=Aimu|irI7Ix{z21 z3;~e>Q!_%j)W4=awjS7+E*ja8K3uBmE)*(mK@9%T=9z>eEg};08p?z7TZBr|AfE#a zmyaIZJ}}fu21*(8!=q2CtAGLHCC3^9NcwNds=Qkm)Z)r-*9KCH;@(9JL zb*PKN94vL^@+}KUT5B!KdtWe)uN0dAJ^x_)&*_ z6EYLW3DBy$Tntu+`8+CEdP+Ua?^f~Of$U~6qyg&=+t`FQpRkWQePBJ+6Y2DF~ zX#SDufkQC>3gdrk!#6nB09GlJof!w@m?GZX_!R4Z{c5pm4|D0IAYrvVHb=??mGrpe z6=Fp^@46nS3+!+pB4Jzq<;0AAGRBsr55PX-z$~%@-)sNtwmuS&NLkD03X!`{gRQS; z;9Z9TYwnOA{3T#_d{aPI5tX0*iRzznDqs1rGkT8!?CMbwGH2j!gRUcv#p8F|V_*p4 zSn@~cCkL_d)Nz5FqGvU@aEOFDz2h?Vqfv_soJDN}+nc+K2VT0K{8`5L-8QFuAG8;I zRWj!0T{OL@S;y35Jhff@YLS(pjxw8<4M0Qs_H>#l5qJ$Ohu?;~_x(0X_ zu8V_jG;U$_@k}1*rz!D<=kO<%L=!L}b2LiB5f8uofGe@Pzjbh>*QTPR=)Zyt1rvzf zdZAmpNMf10NpnPSB$xeRC4uHVtt5r{xM$1PLiWCIgpr_ZTfln7VJH_1TmY~mZh@03_7Ua@b_^Nt5zEy_DMA}xa+Dd7kzlc))e6O}o z6_%9cfwhjv>f`FHVo$AZFJWJ{?I^Vt?#o%Uo5l#Ia&rk~jD1$NQ;d<1xXECrs2q48 zJWl-ZHg_kTesX3$ok3lua<+G!>_yJ7-}evbs~a2dCT6{#-A`JE$I0ve?n*4VWV^Eq zUU*~;N3cQG?G5=7`tI4QXip)9EqkVRg)~x|YFmvI{9Idg75t!oOp9N5##QUe;6uj! zmkAOaU-his{OwXr&PVyGKQD-Y1$AVtt>7m?MWe?4N<{p{Z)%78Rd`fxA+-TC$MF)a`#_(maXVjRop0OP8H^wzs<+)s(ouR+MLqeZN^L@HuwLdap)* zP+81zqxGvLn)K3ReI1%LHp52wDvx60lV%PGAw9-nqu~EDtMY$seW!v8m_s8C65H~QFnNGCRxU+?j9P5~@ zvF%Q6*9Bh4c10oaQEd3Iivl9~Sj6)~gleS3j4oTIYf_Fc3&{I6b%ndDNP=RZ(!G25 zR0rMXk7e7IKDXSCDmX}sq=t4+=|{gP?#$j0Ji|xaHn-UmPSA7`yp|d{yp;4~HZn}h z&~eG_`Uu&n^Nxab)AdI_fjqa;uzN7?uS8Bs(Ky(I2l?bK_axy?p7!=7zuQczXjtOe z_xiW#F0?|D#S^XO5s5=G{x>0pHZF9gX)4+IlGP$1RC+bPHGK7duH00&*)TNU-6osITZxOlG5>Aq#!Y(rGnfzAy z1Ss1tF^q?X!$J1LZTMF0=7C# zOl-jWJ3{cyk1&G(GMgisi=+ioo6UenT4wK5ncd}ulQj3jAk|g&>@&LGF|h$O*)D7r z-`K6tJ=uAsOMPeW`%G+JbMREP&62@FxZ))=E6Tg8KLpb3l2;{*+=lpYBXS`0$8KYI zR9Xlf5ao|I9f7%3Q(v85dywxztL$7_i?qvS-=X^qT)QJk$(@|D z3QL_|nXs26jdJPrBem4ms~Fp0FpB0y1sL7}2LD1K^9f;llCf38)SAD%DCH~KW&gu;X*W-bqwuHTvFVvm`X-0!jTuTdnS-hL0L+)C&lBgrw*`HR~4WH3WVUdq_*?cvH^ncUPeXdPUieRRP>P}_E<%)Dy zH$0@@UPC0sE#)QKd(e`Cg%r$FazB9+R0h3TayfYqT5{Ll?S5FQrQz9Ju1hZCuBuP& zFp#RHX`l6;GtzclKa*zaNq;f>CHH%&|B{y0uir*sT~RKpQ45>Zyt(d@;M`AcZ)l_P zlb7Md{`T_i930>|%h%G;vyb-9S^ko0bc8M=GqeM1Lfe%IwWHmLKU4VW&EQc5-Yc^s zVhPav>LoG95DCj2W7x}4o|FlJHI4oa3(Zwg`M3&>YT0ZjpCr9jCf+_@$qRjP#l#+R zEc8Q-pqYD0kaf;6945LE#%5M*Kt1BW^OI!db@ z8D+g=LT7TjQ&mKCSJQ)hct|=!sw|{c$WLsK!wfl~-u-$UgUw7x)fc{>yHty2mt3ja zWhy+9>Z*!YRJ|5vf3lsPcjfhs2GHIR=9F#gz`354)nds zH_w-Gp1$1fF`(rQbnl*q%~^@tgs%6NDCU}?$X(6sVl=P!n|0T0X6*yt{QfCJ5IOny zm$;xYz# literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc b/flink-connectors/flink-orc/src/test/resources/test-data-nested.orc similarity index 100% rename from flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc rename to flink-connectors/flink-orc/src/test/resources/test-data-nested.orc diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.listliststructlong.orc b/flink-connectors/flink-orc/src/test/resources/test-data-nestedlist.orc similarity index 100% rename from flink-connectors/flink-orc/src/test/resources/TestOrcFile.listliststructlong.orc rename to flink-connectors/flink-orc/src/test/resources/test-data-nestedlist.orc diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc b/flink-connectors/flink-orc/src/test/resources/test-data-timetypes.orc similarity index 100% rename from flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc rename to flink-connectors/flink-orc/src/test/resources/test-data-timetypes.orc diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java index 1d4b8a151e63c..24ccfb1722957 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java @@ -355,4 +355,21 @@ public TypeComparator createTypeComparator(ExecutionConfig config) { comparatorOrders); } } + + /** + * Creates a {@link RowTypeInfo} with projected fields. + * + * @param rowType The original RowTypeInfo whose fields are projected + * @param fieldMapping The field mapping of the projection + * @return A RowTypeInfo with projected fields. + */ + public static RowTypeInfo projectFields(RowTypeInfo rowType, int[] fieldMapping) { + TypeInformation[] fieldTypes = new TypeInformation[fieldMapping.length]; + String[] fieldNames = new String[fieldMapping.length]; + for (int i = 0; i < fieldMapping.length; i++) { + fieldTypes[i] = rowType.getTypeAt(fieldMapping[i]); + fieldNames[i] = rowType.getFieldNames()[fieldMapping[i]]; + } + return new RowTypeInfo(fieldTypes, fieldNames); + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala index 337025fa88f83..ca8e56921390c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala @@ -29,7 +29,7 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceTable} -import org.apache.flink.table.sources.{TableSource, TableSourceUtil} +import org.apache.flink.table.sources.{FilterableTableSource, TableSource, TableSourceUtil} import scala.collection.JavaConverters._ @@ -62,7 +62,19 @@ class FlinkLogicalTableSourceScan( override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { val rowCnt = metadata.getRowCount(this) - planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType)) + + val adjustedCnt: Double = tableSource match { + case f: FilterableTableSource[_] if f.isFilterPushedDown => + // ensure we prefer FilterableTableSources with pushed-down filters. + rowCnt - 1.0 + case _ => + rowCnt + } + + planner.getCostFactory.makeCost( + adjustedCnt, + adjustedCnt, + adjustedCnt * estimateRowSize(getRowType)) } override def explainTerms(pw: RelWriter): RelWriter = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index 9c06135e46806..a377a95f4a15f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -202,6 +202,18 @@ class RexNodeToExpressionConverter( // convert from BigDecimal to Double literal.getValueAs(classOf[java.lang.Double]) + case _@BasicTypeInfo.STRING_TYPE_INFO => + // convert from NlsString to String + literal.getValueAs(classOf[java.lang.String]) + + case _@BasicTypeInfo.BOOLEAN_TYPE_INFO => + // convert to Boolean + literal.getValueAs(classOf[java.lang.Boolean]) + + case _@BasicTypeInfo.BIG_DEC_TYPE_INFO => + // convert to BigDecimal + literal.getValueAs(classOf[java.math.BigDecimal]) + case _ => literal.getValue } From 8a052bf0948d92d6fccc4d1c6c4bd2aa459032c9 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Tue, 10 Oct 2017 07:31:17 +0200 Subject: [PATCH 101/367] [FLINK-6505] Proactively cleanup local FS for RocksDBKeyedStateBackend on startup --- .../state/RocksDBKeyedStateBackend.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index f67daab8f5ad2..9185ad02b8a33 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -235,20 +235,14 @@ public RocksDBKeyedStateBackend( this.instanceBasePath = Preconditions.checkNotNull(instanceBasePath); this.instanceRocksDBPath = new File(instanceBasePath, "db"); - if (!instanceBasePath.exists()) { - if (!instanceBasePath.mkdirs()) { - throw new IOException("Could not create RocksDB data directory."); - } + if (instanceBasePath.exists()) { + // Clear the base directory when the backend is created + // in case something crashed and the backend never reached dispose() + cleanInstanceBasePath(); } - // clean it, this will remove the last part of the path but RocksDB will recreate it - try { - if (instanceRocksDBPath.exists()) { - LOG.warn("Deleting already existing db directory {}.", instanceRocksDBPath); - FileUtils.deleteDirectory(instanceRocksDBPath); - } - } catch (IOException e) { - throw new IOException("Error cleaning RocksDB data directory.", e); + if (!instanceBasePath.mkdirs()) { + throw new IOException("Could not create RocksDB data directory."); } this.keyGroupPrefixBytes = getNumberOfKeyGroups() > (Byte.MAX_VALUE + 1) ? 2 : 1; @@ -312,10 +306,16 @@ public void dispose() { IOUtils.closeQuietly(dbOptions); IOUtils.closeQuietly(columnOptions); + cleanInstanceBasePath(); + } + + private void cleanInstanceBasePath() { + LOG.info("Deleting existing instance base directory {}.", instanceBasePath); + try { FileUtils.deleteDirectory(instanceBasePath); - } catch (IOException ioex) { - LOG.info("Could not delete instace base path for RocksDB: " + instanceBasePath, ioex); + } catch (IOException ex) { + LOG.warn("Could not delete instance base path for RocksDB: " + instanceBasePath, ex); } } From 736b9088dcee64a1d3b19575f29a80c377f94fb8 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 22 Nov 2017 11:37:48 +0100 Subject: [PATCH 102/367] [hotfix][kafka] Throw FlinkKafkaProducer011Exception with error codes instead of generic Exception --- .../kafka/FlinkKafka011ErrorCode.java | 26 ++++++++++++ .../kafka/FlinkKafka011Exception.java | 42 +++++++++++++++++++ .../kafka/FlinkKafkaProducer011.java | 22 ++++++---- 3 files changed, 81 insertions(+), 9 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011ErrorCode.java create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011Exception.java diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011ErrorCode.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011ErrorCode.java new file mode 100644 index 0000000000000..4f5de4f4dc5a0 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011ErrorCode.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.flink.streaming.connectors.kafka; + +/** + * Error codes used in {@link FlinkKafka011Exception}. + */ +public enum FlinkKafka011ErrorCode { + PRODUCERS_POOL_EMPTY, + EXTERNAL_ERROR +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011Exception.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011Exception.java new file mode 100644 index 0000000000000..6b16e53e2a3ae --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafka011Exception.java @@ -0,0 +1,42 @@ +/* + * 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.flink.streaming.connectors.kafka; + +import org.apache.flink.util.FlinkException; + +/** + * Exception used by {@link FlinkKafkaProducer011} and {@link FlinkKafkaConsumer011}. + */ +public class FlinkKafka011Exception extends FlinkException { + + private final FlinkKafka011ErrorCode errorCode; + + public FlinkKafka011Exception(FlinkKafka011ErrorCode errorCode, String message) { + super(message); + this.errorCode = errorCode; + } + + public FlinkKafka011Exception(FlinkKafka011ErrorCode errorCode, String message, Throwable cause) { + super(message, cause); + this.errorCode = errorCode; + } + + public FlinkKafka011ErrorCode getErrorCode() { + return errorCode; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 6b0136d5f3b23..0c741f5173335 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -551,7 +551,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } @Override - public void invoke(KafkaTransactionState transaction, IN next, Context context) throws Exception { + public void invoke(KafkaTransactionState transaction, IN next, Context context) throws FlinkKafka011Exception { checkErroneous(); byte[] serializedKey = schema.serializeKey(next); @@ -587,7 +587,7 @@ record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, seria } @Override - public void close() throws Exception { + public void close() throws FlinkKafka011Exception { final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction != null) { // to avoid exceptions on aborting transactions with some pending records @@ -612,7 +612,7 @@ public void close() throws Exception { // ------------------- Logic for handling checkpoint flushing -------------------------- // @Override - protected KafkaTransactionState beginTransaction() throws Exception { + protected KafkaTransactionState beginTransaction() throws FlinkKafka011Exception { switch (semantic) { case EXACTLY_ONCE: FlinkKafkaProducer producer = createOrGetProducerFromPool(); @@ -631,12 +631,13 @@ protected KafkaTransactionState beginTransaction() throws Exception { } } - private FlinkKafkaProducer createOrGetProducerFromPool() throws Exception { + private FlinkKafkaProducer createOrGetProducerFromPool() throws FlinkKafka011Exception { FlinkKafkaProducer producer = getProducersPool().poll(); if (producer == null) { String transactionalId = availableTransactionalIds.poll(); if (transactionalId == null) { - throw new Exception( + throw new FlinkKafka011Exception( + FlinkKafka011ErrorCode.PRODUCERS_POOL_EMPTY, "Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checkpoints."); } producer = initTransactionalProducer(transactionalId, true); @@ -646,7 +647,7 @@ private FlinkKafkaProducer createOrGetProducerFromPool() throws } @Override - protected void preCommit(KafkaTransactionState transaction) throws Exception { + protected void preCommit(KafkaTransactionState transaction) throws FlinkKafka011Exception { switch (semantic) { case EXACTLY_ONCE: case AT_LEAST_ONCE: @@ -740,7 +741,7 @@ private void acknowledgeMessage() { * Flush pending records. * @param transaction */ - private void flush(KafkaTransactionState transaction) throws Exception { + private void flush(KafkaTransactionState transaction) throws FlinkKafka011Exception { if (transaction.producer != null) { transaction.producer.flush(); } @@ -936,12 +937,15 @@ private FlinkKafkaProducer initProducer(boolean registerMetrics) return producer; } - private void checkErroneous() throws Exception { + private void checkErroneous() throws FlinkKafka011Exception { Exception e = asyncException; if (e != null) { // prevent double throwing asyncException = null; - throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e); + throw new FlinkKafka011Exception( + FlinkKafka011ErrorCode.EXTERNAL_ERROR, + "Failed to send data to Kafka: " + e.getMessage(), + e); } } From 27564c33955d8e53f0275a5b43d4b2415ba86547 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 22 Nov 2017 15:53:08 +0100 Subject: [PATCH 103/367] [FLINK-8132][kafka] Re-initialize transactional KafkaProducer on each checkpoint Previously faulty scenario with producer pool of 2. 1. started transaction 1 with producerA, written record 42 2. checkpoint 1 triggered, pre committing txn1, started txn2 with producerB, written record 43 3. checkpoint 1 completed, committing txn1, returning producerA to the pool 4. checkpoint 2 triggered , committing txn2, started txn3 with producerA, written record 44 5. crash.... 6. recover to checkpoint 1, txn1 from producerA found to "pendingCommitTransactions", attempting to recoverAndCommit(txn1) 7. unfortunately txn1 and txn3 from the same producers are identical from KafkaBroker perspective and thus txn3 is being committed result is that both records 42 and 44 are committed. With this fix, after re-initialization txn3 will have different producerId/epoch counters compared to txn1. --- .../kafka/FlinkKafkaProducer011.java | 85 ++++++------------- .../kafka/FlinkKafkaProducer011ITCase.java | 65 ++++++++++++++ 2 files changed, 91 insertions(+), 59 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 0c741f5173335..b14e4871287e5 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -66,7 +66,6 @@ import javax.annotation.Nullable; -import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -223,15 +222,10 @@ public enum Semantic { private final int kafkaProducersPoolSize; /** - * Available transactional ids. + * Pool of available transactional ids. */ private final BlockingDeque availableTransactionalIds = new LinkedBlockingDeque<>(); - /** - * Pool of KafkaProducers objects. - */ - private transient Optional producersPool = Optional.empty(); - /** * Flag controlling whether we are writing the Flink record's timestamp into Kafka. */ @@ -599,12 +593,6 @@ public void close() throws FlinkKafka011Exception { catch (Exception e) { asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException); } - try { - producersPool.ifPresent(pool -> pool.close()); - } - catch (Exception e) { - asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException); - } // make sure we propagate pending errors checkErroneous(); } @@ -615,7 +603,7 @@ public void close() throws FlinkKafka011Exception { protected KafkaTransactionState beginTransaction() throws FlinkKafka011Exception { switch (semantic) { case EXACTLY_ONCE: - FlinkKafkaProducer producer = createOrGetProducerFromPool(); + FlinkKafkaProducer producer = createTransactionalProducer(); producer.beginTransaction(); return new KafkaTransactionState(producer.getTransactionalId(), producer); case AT_LEAST_ONCE: @@ -631,21 +619,6 @@ protected KafkaTransactionState beginTransaction() throws FlinkKafka011Exception } } - private FlinkKafkaProducer createOrGetProducerFromPool() throws FlinkKafka011Exception { - FlinkKafkaProducer producer = getProducersPool().poll(); - if (producer == null) { - String transactionalId = availableTransactionalIds.poll(); - if (transactionalId == null) { - throw new FlinkKafka011Exception( - FlinkKafka011ErrorCode.PRODUCERS_POOL_EMPTY, - "Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checkpoints."); - } - producer = initTransactionalProducer(transactionalId, true); - producer.initTransactions(); - } - return producer; - } - @Override protected void preCommit(KafkaTransactionState transaction) throws FlinkKafka011Exception { switch (semantic) { @@ -666,7 +639,7 @@ protected void commit(KafkaTransactionState transaction) { switch (semantic) { case EXACTLY_ONCE: transaction.producer.commitTransaction(); - getProducersPool().add(transaction.producer); + recycleTransactionalProducer(transaction.producer); break; case AT_LEAST_ONCE: case NONE: @@ -706,7 +679,7 @@ protected void abort(KafkaTransactionState transaction) { switch (semantic) { case EXACTLY_ONCE: transaction.producer.abortTransaction(); - getProducersPool().add(transaction.producer); + recycleTransactionalProducer(transaction.producer); break; case AT_LEAST_ONCE: case NONE: @@ -796,10 +769,7 @@ public void initializeState(FunctionInitializationContext context) throws Except if (semantic != Semantic.EXACTLY_ONCE) { nextTransactionalIdHint = null; - producersPool = Optional.empty(); } else { - producersPool = Optional.of(new ProducersPool()); - ArrayList transactionalIdHints = Lists.newArrayList(nextTransactionalIdHintState.get()); if (transactionalIdHints.size() > 1) { throw new IllegalStateException( @@ -883,16 +853,33 @@ int getTransactionCoordinatorId() { return currentTransaction.producer.getTransactionCoordinatorId(); } + /** + * For each checkpoint we create new {@link FlinkKafkaProducer} so that new transactions will not clash + * with transactions created during previous checkpoints ({@code producer.initTransactions()} assures that we + * obtain new producerId and epoch counters). + */ + private FlinkKafkaProducer createTransactionalProducer() throws FlinkKafka011Exception { + String transactionalId = availableTransactionalIds.poll(); + if (transactionalId == null) { + throw new FlinkKafka011Exception( + FlinkKafka011ErrorCode.PRODUCERS_POOL_EMPTY, + "Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checkpoints."); + } + FlinkKafkaProducer producer = initTransactionalProducer(transactionalId, true); + producer.initTransactions(); + return producer; + } + + private void recycleTransactionalProducer(FlinkKafkaProducer producer) { + availableTransactionalIds.add(producer.getTransactionalId()); + producer.close(); + } + private FlinkKafkaProducer initTransactionalProducer(String transactionalId, boolean registerMetrics) { producerConfig.put("transactional.id", transactionalId); return initProducer(registerMetrics); } - private ProducersPool getProducersPool() { - checkState(producersPool.isPresent(), "Trying to access uninitialized producer pool"); - return producersPool.get(); - } - private FlinkKafkaProducer initProducer(boolean registerMetrics) { FlinkKafkaProducer producer = new FlinkKafkaProducer<>(this.producerConfig); @@ -951,7 +938,6 @@ private void checkErroneous() throws FlinkKafka011Exception { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - producersPool = Optional.empty(); } private static Properties getPropertiesFromBrokerList(String brokerList) { @@ -1264,25 +1250,6 @@ public boolean canEqual(Object obj) { } } - static class ProducersPool implements Closeable { - private final LinkedBlockingDeque> pool = new LinkedBlockingDeque<>(); - - public void add(FlinkKafkaProducer producer) { - pool.add(producer); - } - - public FlinkKafkaProducer poll() { - return pool.poll(); - } - - @Override - public void close() { - while (!pool.isEmpty()) { - pool.poll().close(); - } - } - } - /** * Keep information required to deduce next safe to use transactional id. */ diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index 07acd4f6c5dec..a32c7f88b4cdb 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -52,6 +52,7 @@ import static org.apache.flink.util.Preconditions.checkState; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; /** * IT cases for the {@link FlinkKafkaProducer011}. @@ -79,6 +80,49 @@ public void before() { extraProperties.put("isolation.level", "read_committed"); } + /** + * This test ensures that transactions reusing transactional.ids (after returning to the pool) will not clash + * with previous transactions using same transactional.ids. + */ + @Test(timeout = 120_000L) + public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exception { + String topic = "flink-kafka-producer-fail-before-notify"; + + try (OneInputStreamOperatorTestHarness testHarness1 = createTestHarness(topic)) { + testHarness1.setup(); + testHarness1.open(); + testHarness1.processElement(42, 0); + OperatorStateHandles snapshot = testHarness1.snapshot(0, 0); + testHarness1.processElement(43, 0); + testHarness1.notifyOfCompletedCheckpoint(0); + try { + for (int i = 0; i < FlinkKafkaProducer011.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE; i++) { + testHarness1.snapshot(i + 1, 0); + testHarness1.processElement(i, 0); + } + throw new IllegalStateException("This should not be reached."); + } + catch (Exception ex) { + assertIsCausedBy(FlinkKafka011ErrorCode.PRODUCERS_POOL_EMPTY, ex); + } + + // Resume transactions before testHrness1 is being closed (in case of failures close() might not be called) + try (OneInputStreamOperatorTestHarness testHarness2 = createTestHarness(topic)) { + testHarness2.setup(); + // restore from snapshot1, transactions with records 43 and 44 should be aborted + testHarness2.initializeState(snapshot); + testHarness2.open(); + } + + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42), 30_000L); + deleteTestTopic(topic); + } + catch (Exception ex) { + // testHarness1 will be fenced off after creating and closing testHarness2 + assertIsCausedBy(ProducerFencedException.class, ex); + } + } + @Test(timeout = 120_000L) public void testFlinkKafkaProducer011FailBeforeNotify() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; @@ -563,4 +607,25 @@ private Properties createProperties() { properties.put(FlinkKafkaProducer011.KEY_DISABLE_METRICS, "true"); return properties; } + + private void assertIsCausedBy(Class clazz, Throwable ex) { + for (int depth = 0; depth < 50 && ex != null; depth++) { + if (clazz.isInstance(ex)) { + return; + } + ex = ex.getCause(); + } + fail(String.format("Exception [%s] was not caused by [%s]", ex, clazz)); + } + + private void assertIsCausedBy(FlinkKafka011ErrorCode expectedErrorCode, Throwable ex) { + for (int depth = 0; depth < 50 && ex != null; depth++) { + if (ex instanceof FlinkKafka011Exception) { + assertEquals(expectedErrorCode, ((FlinkKafka011Exception) ex).getErrorCode()); + return; + } + ex = ex.getCause(); + } + fail(String.format("Exception [%s] was not caused by FlinkKafka011Exception[errorCode=%s]", ex, expectedErrorCode)); + } } From 62bf00189b136dd34c5af5d6181c33c1415f16b9 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 22 Nov 2017 15:55:20 +0100 Subject: [PATCH 104/367] [hotfix][kafka] Remove unused method in kafka tests --- .../kafka/FlinkKafkaProducer011ITCase.java | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index a32c7f88b4cdb..85735c80cc09e 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -29,12 +29,7 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; -import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; - import kafka.server.KafkaServer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.errors.ProducerFencedException; import org.junit.Before; import org.junit.Test; @@ -553,17 +548,6 @@ private void failBroker(int brokerId) { } } - private void assertRecord(String topicName, String expectedKey, String expectedValue) { - try (KafkaConsumer kafkaConsumer = new KafkaConsumer<>(extraProperties)) { - kafkaConsumer.subscribe(Collections.singletonList(topicName)); - ConsumerRecords records = kafkaConsumer.poll(10000); - - ConsumerRecord record = Iterables.getOnlyElement(records); - assertEquals(expectedKey, record.key()); - assertEquals(expectedValue, record.value()); - } - } - private void closeIgnoringProducerFenced(AutoCloseable autoCloseable) throws Exception { try { autoCloseable.close(); From e100861f84fe60ec6bb8172bb5a3cc453640fdb3 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 23 Nov 2017 13:08:43 +0100 Subject: [PATCH 105/367] [hotfix][docs] Improve Kafka exactly-once docs --- docs/dev/connectors/kafka.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index ad4cc2fe5894c..5376d5be2c8f2 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -538,7 +538,10 @@ chosen by passing appropriate `semantic` parameter to the `FlinkKafkaProducer011 be duplicated. * `Semantic.AT_LEAST_ONCE` (default setting): similar to `setFlushOnCheckpoint(true)` in `FlinkKafkaProducer010`. This guarantees that no records will be lost (although they can be duplicated). - * `Semantic.EXACTLY_ONCE`: uses Kafka transactions to provide exactly-once semantic. + * `Semantic.EXACTLY_ONCE`: uses Kafka transactions to provide exactly-once semantic. Whenever you write + to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_committed` + or `read_uncommitted` - the latter one is the default value) for any application consuming records + from Kafka.

Attention: Depending on your Kafka configuration, even after Kafka acknowledges From 828ef09b09f872107b412501774b42efaf6caa37 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 22 Nov 2017 17:52:35 +0100 Subject: [PATCH 106/367] [FLINK-5465] [streaming] Wait for pending timer threads to finish or to exceed a time limit in exceptional stream task shutdown. This closes #5058. (cherry picked from commit d86c6b6) --- .../configuration/TimerServiceOptions.java | 38 +++++++++++ .../runtime/tasks/ProcessingTimeService.java | 12 ++++ .../streaming/runtime/tasks/StreamTask.java | 18 ++++- .../tasks/SystemProcessingTimeService.java | 6 ++ .../tasks/TestProcessingTimeService.java | 6 ++ .../SystemProcessingTimeServiceTest.java | 65 +++++++++++++++++++ 6 files changed, 142 insertions(+), 3 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/configuration/TimerServiceOptions.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/configuration/TimerServiceOptions.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/configuration/TimerServiceOptions.java new file mode 100644 index 0000000000000..835adce4035c0 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/configuration/TimerServiceOptions.java @@ -0,0 +1,38 @@ +/* + * 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.flink.streaming.configuration; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * Timer service configuration options. + */ +@PublicEvolving +public class TimerServiceOptions { + + /** + * This configures how long we wait for the {@link org.apache.flink.streaming.runtime.tasks.ProcessingTimeService} + * to finish all pending timer threads when the stream task performs a failover shutdown. See FLINK-5465. + */ + public static final ConfigOption TIMER_SERVICE_TERMINATION_AWAIT_MS = ConfigOptions + .key("timerservice.exceptional.shutdown.timeout") + .defaultValue(7500L); +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java index b2382529e1c12..251629987b365 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; /** * Defines the current processing time and handles all related actions, @@ -93,4 +94,15 @@ public abstract class ProcessingTimeService { * will result in a hard exception. */ public abstract void shutdownService(); + + /** + * Shuts down and clean up the timer service provider hard and immediately. This does wait + * for all timers to complete or until the time limit is exceeded. Any call to + * {@link #registerTimer(long, ProcessingTimeCallback)} will result in a hard exception after calling this method. + * @param time time to wait for termination. + * @param timeUnit time unit of parameter time. + * @return {@code true} if this timer service and all pending timers are terminated and + * {@code false} if the timeout elapsed before this happened. + */ + public abstract boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 68f590e2875d1..6ae45c6d4d29a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -51,6 +51,7 @@ import org.apache.flink.streaming.api.operators.OperatorSnapshotResult; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.configuration.TimerServiceOptions; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; @@ -69,6 +70,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** @@ -212,7 +214,6 @@ public final void invoke() throws Exception { LOG.debug("Initializing {}.", getName()); asyncOperationsThreadPool = Executors.newCachedThreadPool(); - configuration = new StreamConfig(getTaskConfiguration()); stateBackend = createStateBackend(); @@ -305,9 +306,20 @@ public final void invoke() throws Exception { isRunning = false; // stop all timers and threads - if (timerService != null) { + if (timerService != null && !timerService.isTerminated()) { try { - timerService.shutdownService(); + + final long timeoutMs = getEnvironment().getTaskManagerInfo().getConfiguration(). + getLong(TimerServiceOptions.TIMER_SERVICE_TERMINATION_AWAIT_MS); + + // wait for a reasonable time for all pending timer threads to finish + boolean timerShutdownComplete = + timerService.shutdownAndAwaitPending(timeoutMs, TimeUnit.MILLISECONDS); + + if (!timerShutdownComplete) { + LOG.warn("Timer service shutdown exceeded time limit of {} ms while waiting for pending " + + "timers. Will continue with shutdown procedure.", timeoutMs); + } } catch (Throwable t) { // catch and log the exception to not replace the original exception diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java index 71bfdf6f7d5f6..be8b23c7ee861 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java @@ -191,6 +191,12 @@ public void shutdownService() { } } + @Override + public boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException { + shutdownService(); + return timerService.awaitTermination(time, timeUnit); + } + // safety net to destroy the thread pool @Override protected void finalize() throws Throwable { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java index 080eeb55ddc1d..2081f193cfcb1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java @@ -134,6 +134,12 @@ public void shutdownService() { this.isTerminated = true; } + @Override + public boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException { + shutdownService(); + return true; + } + public int getNumActiveTimers() { int count = 0; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java index 4c105d3c903a1..01fd7789b4682 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java @@ -22,11 +22,13 @@ import org.apache.flink.streaming.runtime.operators.TestProcessingTimeServiceTest.ReferenceSettingExceptionHandler; import org.apache.flink.util.TestLogger; +import org.junit.Assert; import org.junit.Test; import java.util.concurrent.CancellationException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; @@ -442,4 +444,67 @@ public void onProcessingTime(long timestamp) throws Exception { latch.await(); assertTrue(exceptionWasThrown.get()); } + + @Test + public void testShutdownAndWaitPending() { + + final Object lock = new Object(); + final OneShotLatch waitUntilTimerStarted = new OneShotLatch(); + final OneShotLatch blockUntilTerminationInterrupts = new OneShotLatch(); + final OneShotLatch blockUntilTriggered = new OneShotLatch(); + final AtomicBoolean check = new AtomicBoolean(true); + + final SystemProcessingTimeService timeService = new SystemProcessingTimeService( + (message, exception) -> { + }, + lock); + + timeService.scheduleAtFixedRate( + timestamp -> { + + waitUntilTimerStarted.trigger(); + + try { + blockUntilTerminationInterrupts.await(); + check.set(false); + } catch (InterruptedException ignore) { + } + + try { + blockUntilTriggered.await(); + } catch (InterruptedException ignore) { + check.set(false); + } + }, + 0L, + 10L); + + try { + waitUntilTimerStarted.await(); + } catch (InterruptedException e) { + Assert.fail(); + } + + Assert.assertFalse(timeService.isTerminated()); + + // Check that we wait for the timer to terminate. As the timer blocks on the second latch, this should time out. + try { + Assert.assertFalse(timeService.shutdownAndAwaitPending(1, TimeUnit.SECONDS)); + } catch (InterruptedException e) { + Assert.fail("Unexpected interruption."); + } + + // Let the timer proceed. + blockUntilTriggered.trigger(); + + // Now we should succeed in terminating the timer. + try { + Assert.assertTrue(timeService.shutdownAndAwaitPending(60, TimeUnit.SECONDS)); + } catch (InterruptedException e) { + Assert.fail("Unexpected interruption."); + } + + Assert.assertTrue(check.get()); + Assert.assertTrue(timeService.isTerminated()); + } } From 2fb24581a1775084e3be8c2575c129d250f39313 Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Thu, 23 Nov 2017 00:00:39 +0800 Subject: [PATCH 107/367] [FLINK-8118] [table] Allow to specify the offsets of KafkaTableSources This closes #5056. --- docs/dev/table/sourceSinks.md | 50 ++++++- .../kafka/Kafka010AvroTableSource.java | 2 +- .../kafka/Kafka010JsonTableSource.java | 2 +- .../connectors/kafka/Kafka010TableSource.java | 2 +- .../kafka/Kafka011AvroTableSource.java | 2 +- .../kafka/Kafka011JsonTableSource.java | 2 +- .../connectors/kafka/Kafka011TableSource.java | 2 +- .../kafka/Kafka08AvroTableSource.java | 2 +- .../kafka/Kafka08JsonTableSource.java | 2 +- .../connectors/kafka/Kafka08TableSource.java | 2 +- .../kafka/Kafka09AvroTableSource.java | 2 +- .../kafka/Kafka09JsonTableSource.java | 2 +- .../connectors/kafka/Kafka09TableSource.java | 2 +- .../kafka/KafkaAvroTableSource.java | 2 +- .../kafka/KafkaJsonTableSource.java | 2 +- .../connectors/kafka/KafkaTableSource.java | 126 +++++++++++++++++- .../kafka/KafkaTableSourceTestBase.java | 44 ++++++ 17 files changed, 230 insertions(+), 18 deletions(-) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 7387358beb931..aaf23bc8e5064 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -145,7 +145,7 @@ val source: TableSource[_] = Kafka010JsonTableSource.builder()
-* **Missing Field Handling** By default, a missing JSON field is set to `null`. You can enable strict JSON parsing that will cancel the source (and query) if a field is missing. +* **Missing Field Handling:** By default, a missing JSON field is set to `null`. You can enable strict JSON parsing that will cancel the source (and query) if a field is missing.
@@ -169,6 +169,30 @@ val source: TableSource[_] = Kafka010JsonTableSource.builder()
+* **Specify the start reading position:** By default, the table source will start reading data from the committed group offsets in Zookeeper or Kafka brokers. You can specify other start positions via the builder's methods, which correspond to the configurations in section [Kafka Consumers Start Position Configuration](../connectors/kafka.html#kafka-consumers-start-position-configuration). + +
+
+{% highlight java %} +TableSource source = Kafka010JsonTableSource.builder() + // ... + // start reading from the earliest offset + .startReadingFromEarliest() + .build(); +{% endhighlight %} +
+ +
+{% highlight scala %} +val source: TableSource[_] = Kafka010JsonTableSource.builder() + // ... + // start reading from the earliest offset + .startReadingFromEarliest() + .build() +{% endhighlight %} +
+
+ {% top %} ### KafkaAvroTableSource @@ -265,6 +289,30 @@ val source: TableSource[_] = Kafka010AvroTableSource.builder() +* **Specify the start reading position:** By default, the table source will start reading data from the committed group offsets in Zookeeper or Kafka brokers. You can specify other start positions via the builder's methods, which correspond to the configurations in section [Kafka Consumers Start Position Configuration](../connectors/kafka.html#kafka-consumers-start-position-configuration). + +
+
+{% highlight java %} +TableSource source = Kafka010JsonTableSource.builder() + // ... + // start reading from the earliest offset + .startReadingFromEarliest() + .build(); +{% endhighlight %} +
+ +
+{% highlight scala %} +val source: TableSource[_] = Kafka010JsonTableSource.builder() + // ... + // start reading from the earliest offset + .startReadingFromEarliest() + .build() +{% endhighlight %} +
+
+ {% top %} ### Configuring a Processing Time Attribute diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java index fbc58ea0bdd08..660162aff85f1 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java @@ -94,7 +94,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java index bbdb32ffa846e..5f9984ebc0732 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -93,7 +93,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java index bc675eb6cd124..379c562288968 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -61,7 +61,7 @@ public DeserializationSchema getDeserializationSchema() { } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties); } } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java index af3b5afc38248..a9a109ca0772f 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java @@ -94,7 +94,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer011<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java index 71158f6a9e40e..cee7c61a3f389 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java @@ -93,7 +93,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer011<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java index dbf980b7c2b2c..8c403181b4059 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java @@ -61,7 +61,7 @@ public DeserializationSchema getDeserializationSchema() { } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer011<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java index 8f45881ab7e8c..9105c73e9806e 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java @@ -94,7 +94,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java index b3b37c6c3a331..639093dca5a73 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java @@ -93,7 +93,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index 8270b789417af..3bb6a94387fcb 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -61,7 +61,7 @@ public DeserializationSchema getDeserializationSchema() { } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties); } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java index 808be012512f2..fb8496acd39a7 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java @@ -94,7 +94,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java index a699d65f56dd4..ded23b0fb22b4 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java @@ -93,7 +93,7 @@ public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttr } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties); } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index 1d2c02898cb55..df15452ccb0c7 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -61,7 +61,7 @@ public DeserializationSchema getDeserializationSchema() { } @Override - FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java index 8cea36ca5aa69..055b67924822c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java @@ -44,7 +44,7 @@ * A version-agnostic Kafka Avro {@link StreamTableSource}. * *

The version-specific Kafka consumers need to extend this class and - * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}. + * override {@link #createKafkaConsumer(String, Properties, DeserializationSchema)}}. */ public abstract class KafkaAvroTableSource extends KafkaTableSource implements DefinedFieldMapping { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java index 9a6525ce88f73..68066731f5606 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java @@ -32,7 +32,7 @@ * A version-agnostic Kafka JSON {@link StreamTableSource}. * *

The version-specific Kafka consumers need to extend this class and - * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}. + * override {@link #createKafkaConsumer(String, Properties, DeserializationSchema)}}. * *

The field names are used to parse the JSON file and so are the types. */ diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index 3291f7dc068ac..d0ee7de7d847d 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -22,6 +22,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; import org.apache.flink.table.api.ValidationException; @@ -37,6 +39,7 @@ import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Properties; import scala.Option; @@ -45,7 +48,7 @@ * A version-agnostic Kafka {@link StreamTableSource}. * *

The version-specific Kafka consumers need to extend this class and - * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}. + * override {@link #createKafkaConsumer(String, Properties, DeserializationSchema)}}. */ public abstract class KafkaTableSource implements StreamTableSource, DefinedProctimeAttribute, DefinedRowtimeAttributes { @@ -68,6 +71,12 @@ public abstract class KafkaTableSource /** Descriptor for a rowtime attribute. */ private List rowtimeAttributeDescriptors; + /** The startup mode for the contained consumer (default is {@link StartupMode#GROUP_OFFSETS}). */ + private StartupMode startupMode; + + /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ + private Map specificStartupOffsets; + /** * Creates a generic Kafka {@link StreamTableSource}. * @@ -121,6 +130,37 @@ public List getRowtimeAttributeDescriptors() { return rowtimeAttributeDescriptors; } + /** + * Returns a version-specific Kafka consumer with the start position configured. + * + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema to use for Kafka records. + * @return The version-specific Kafka consumer + */ + protected FlinkKafkaConsumerBase getKafkaConsumer( + String topic, + Properties properties, + DeserializationSchema deserializationSchema) { + FlinkKafkaConsumerBase kafkaConsumer = + createKafkaConsumer(topic, properties, deserializationSchema); + switch (startupMode) { + case EARLIEST: + kafkaConsumer.setStartFromEarliest(); + break; + case LATEST: + kafkaConsumer.setStartFromLatest(); + break; + case GROUP_OFFSETS: + kafkaConsumer.setStartFromGroupOffsets(); + break; + case SPECIFIC_OFFSETS: + kafkaConsumer.setStartFromSpecificOffsets(specificStartupOffsets); + break; + } + return kafkaConsumer; + } + //////// SETTERS FOR OPTIONAL PARAMETERS /** @@ -160,17 +200,35 @@ protected void setRowtimeAttributeDescriptors(List r this.rowtimeAttributeDescriptors = rowtimeAttributeDescriptors; } + /** + * Sets the startup mode of the TableSource. + * + * @param startupMode The startup mode. + */ + protected void setStartupMode(StartupMode startupMode) { + this.startupMode = startupMode; + } + + /** + * Sets the startup offsets of the TableSource; only relevant when the startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. + * + * @param specificStartupOffsets The startup offsets for different partitions. + */ + protected void setSpecificStartupOffsets(Map specificStartupOffsets) { + this.specificStartupOffsets = specificStartupOffsets; + } + //////// ABSTRACT METHODS FOR SUBCLASSES /** - * Returns the version-specific Kafka consumer. + * Creates a version-specific Kafka consumer. * * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema to use for Kafka records. * @return The version-specific Kafka consumer */ - abstract FlinkKafkaConsumerBase getKafkaConsumer( + protected abstract FlinkKafkaConsumerBase createKafkaConsumer( String topic, Properties properties, DeserializationSchema deserializationSchema); @@ -201,6 +259,13 @@ protected abstract static class Builder specificStartupOffsets = null; + + /** * Sets the topic from which the table is read. * @@ -309,6 +374,51 @@ public B withKafkaTimestampAsRowtimeAttribute( return builder(); } + /** + * Configures the TableSource to start reading from the earliest offset for all partitions. + * + * @see FlinkKafkaConsumerBase#setStartFromEarliest() + */ + public B fromEarliest() { + this.startupMode = StartupMode.EARLIEST; + this.specificStartupOffsets = null; + return builder(); + } + + /** + * Configures the TableSource to start reading from the latest offset for all partitions. + * + * @see FlinkKafkaConsumerBase#setStartFromLatest() + */ + public B fromLatest() { + this.startupMode = StartupMode.LATEST; + this.specificStartupOffsets = null; + return builder(); + } + + /** + * Configures the TableSource to start reading from any committed group offsets found in Zookeeper / Kafka brokers. + * + * @see FlinkKafkaConsumerBase#setStartFromGroupOffsets() + */ + public B fromGroupOffsets() { + this.startupMode = StartupMode.GROUP_OFFSETS; + this.specificStartupOffsets = null; + return builder(); + } + + /** + * Configures the TableSource to start reading partitions from specific offsets, set independently for each partition. + * + * @param specificStartupOffsets the specified offsets for partitions + * @see FlinkKafkaConsumerBase#setStartFromSpecificOffsets(Map) + */ + public B fromSpecificOffsets(Map specificStartupOffsets) { + this.startupMode = StartupMode.SPECIFIC_OFFSETS; + this.specificStartupOffsets = Preconditions.checkNotNull(specificStartupOffsets); + return builder(); + } + /** * Returns the configured topic. * @@ -357,6 +467,16 @@ protected void configureTableSource(T tableSource) { } else { tableSource.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); } + tableSource.setStartupMode(startupMode); + switch (startupMode) { + case EARLIEST: + case LATEST: + case GROUP_OFFSETS: + break; + case SPECIFIC_OFFSETS: + tableSource.setSpecificStartupOffsets(specificStartupOffsets); + break; + } } /** diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java index 7a882f48d5f5a..64dac06d821e9 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java @@ -32,6 +32,7 @@ import org.junit.Test; import java.util.List; +import java.util.Map; import java.util.Properties; import static org.junit.Assert.assertEquals; @@ -44,6 +45,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Abstract test base for all Kafka table sources. @@ -188,6 +190,48 @@ public void testKafkaTSRowtimeAttribute() { } } + @Test + public void testKafkaTSSetConsumeOffsets() { + KafkaTableSource.Builder b = getBuilder(); + configureBuilder(b); + + // test the default behavior + KafkaTableSource source = spy(b.build()); + when(source.createKafkaConsumer(TOPIC, PROPS, null)) + .thenReturn(mock(getFlinkKafkaConsumer())); + + verify(source.getKafkaConsumer(TOPIC, PROPS, null)).setStartFromGroupOffsets(); + + // test reading from earliest + b.fromEarliest(); + source = spy(b.build()); + when(source.createKafkaConsumer(TOPIC, PROPS, null)) + .thenReturn(mock(getFlinkKafkaConsumer())); + + verify(source.getKafkaConsumer(TOPIC, PROPS, null)).setStartFromEarliest(); + + // test reading from latest + b.fromLatest(); + source = spy(b.build()); + when(source.createKafkaConsumer(TOPIC, PROPS, null)) + .thenReturn(mock(getFlinkKafkaConsumer())); + verify(source.getKafkaConsumer(TOPIC, PROPS, null)).setStartFromLatest(); + + // test reading from group offsets + b.fromGroupOffsets(); + source = spy(b.build()); + when(source.createKafkaConsumer(TOPIC, PROPS, null)) + .thenReturn(mock(getFlinkKafkaConsumer())); + verify(source.getKafkaConsumer(TOPIC, PROPS, null)).setStartFromGroupOffsets(); + + // test reading from given offsets + b.fromSpecificOffsets(mock(Map.class)); + source = spy(b.build()); + when(source.createKafkaConsumer(TOPIC, PROPS, null)) + .thenReturn(mock(getFlinkKafkaConsumer())); + verify(source.getKafkaConsumer(TOPIC, PROPS, null)).setStartFromSpecificOffsets(any(Map.class)); + } + protected abstract KafkaTableSource.Builder getBuilder(); protected abstract Class> getDeserializationSchema(); From 13631b9617d32e46eba51c9125019ec5e77c39f3 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 23 Nov 2017 14:30:15 +0100 Subject: [PATCH 108/367] [FLINK-8118] [table] Improve KafkaTableSource documentation --- docs/dev/table/sourceSinks.md | 56 +++++++++---------- .../kafka/KafkaTableSourceTestBase.java | 5 +- 2 files changed, 31 insertions(+), 30 deletions(-) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index aaf23bc8e5064..2b1027835d43c 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -63,7 +63,7 @@ A `KafkaJsonTableSource` is created and configured using a builder. The followin

{% highlight java %} // create builder -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // set Kafka topic .forTopic("sensors") // set Kafka consumer properties @@ -80,7 +80,7 @@ TableSource source = Kafka010JsonTableSource.builder()
{% highlight scala %} // create builder -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // set Kafka topic .forTopic("sensors") // set Kafka consumer properties @@ -108,7 +108,7 @@ Map mapping = new HashMap<>(); mapping.put("sensorId", "id"); mapping.put("temperature", "temp"); -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // ... // set Table schema .withSchema(TableSchema.builder() @@ -126,7 +126,7 @@ TableSource source = Kafka010JsonTableSource.builder()
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // ... // set Table schema .withSchema(TableSchema.builder() @@ -150,7 +150,7 @@ val source: TableSource[_] = Kafka010JsonTableSource.builder()
{% highlight java %} -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // ... // configure missing field behavior .failOnMissingField(true) @@ -160,7 +160,7 @@ TableSource source = Kafka010JsonTableSource.builder()
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // ... // configure missing field behavior .failOnMissingField(true) @@ -174,20 +174,20 @@ val source: TableSource[_] = Kafka010JsonTableSource.builder()
{% highlight java %} -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // ... // start reading from the earliest offset - .startReadingFromEarliest() + .fromEarliest() .build(); {% endhighlight %}
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // ... // start reading from the earliest offset - .startReadingFromEarliest() + .fromEarliest() .build() {% endhighlight %}
@@ -205,7 +205,7 @@ A `KafkaAvroTableSource` is created and configured using a builder. The followin
{% highlight java %} // create builder -TableSource source = Kafka010AvroTableSource.builder() +KafkaTableSource source = Kafka010AvroTableSource.builder() // set Kafka topic .forTopic("sensors") // set Kafka consumer properties @@ -224,7 +224,7 @@ TableSource source = Kafka010AvroTableSource.builder()
{% highlight scala %} // create builder -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // set Kafka topic .forTopic("sensors") // set Kafka consumer properties @@ -256,7 +256,7 @@ Map mapping = new HashMap<>(); mapping.put("sensorId", "id"); mapping.put("temperature", "temp"); -TableSource source = Kafka010AvroTableSource.builder() +KafkaTableSource source = Kafka010AvroTableSource.builder() // ... // set Table schema .withSchema(TableSchema.builder() @@ -264,15 +264,15 @@ TableSource source = Kafka010AvroTableSource.builder() .field("temperature", Types.DOUBLE()).build()) // set class of Avro record with fields [id, temp] .forAvroRecordClass(SensorReading.class) - // set mapping from table fields to JSON fields - .withTableToJsonMapping(mapping) + // set mapping from table fields to Avro fields + .withTableToAvroMapping(mapping) .build(); {% endhighlight %}
{% highlight scala %} -val source: TableSource[_] = Kafka010AvroTableSource.builder() +val source: KafkaTableSource = Kafka010AvroTableSource.builder() // ... // set Table schema .withSchema(TableSchema.builder() @@ -280,8 +280,8 @@ val source: TableSource[_] = Kafka010AvroTableSource.builder() .field("temperature", Types.DOUBLE).build()) // set class of Avro record with fields [id, temp] .forAvroRecordClass(classOf[SensorReading]) - // set mapping from table fields to JSON fields - .withTableToJsonMapping(Map( + // set mapping from table fields to Avro fields + .withTableToAvroMapping(Map( "sensorId" -> "id", "temperature" -> "temp").asJava) .build() @@ -294,20 +294,20 @@ val source: TableSource[_] = Kafka010AvroTableSource.builder()
{% highlight java %} -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010AvroTableSource.builder() // ... // start reading from the earliest offset - .startReadingFromEarliest() + .fromEarliest() .build(); {% endhighlight %}
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010AvroTableSource.builder() // ... // start reading from the earliest offset - .startReadingFromEarliest() + .fromEarliest() .build() {% endhighlight %}
@@ -326,7 +326,7 @@ A table schema field of type `SQL_TIMESTAMP` can be declared as a processing tim
{% highlight java %} -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // ... .withSchema(TableSchema.builder() .field("sensorId", Types.LONG()) @@ -341,7 +341,7 @@ TableSource source = Kafka010JsonTableSource.builder()
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // ... .withSchema(TableSchema.builder() .field("sensorId", Types.LONG) @@ -372,7 +372,7 @@ The following example shows how to configure a rowtime attribute.
{% highlight java %} -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // ... .withSchema(TableSchema.builder() .field("sensorId", Types.LONG()) @@ -392,7 +392,7 @@ TableSource source = Kafka010JsonTableSource.builder()
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // ... .withSchema(TableSchema.builder() .field("sensorId", Types.LONG) @@ -418,7 +418,7 @@ Since Kafka 0.10, Kafka messages have a timestamp as metadata that specifies whe
{% highlight java %} -TableSource source = Kafka010JsonTableSource.builder() +KafkaTableSource source = Kafka010JsonTableSource.builder() // ... .withSchema(TableSchema.builder() .field("sensorId", Types.LONG()) @@ -437,7 +437,7 @@ TableSource source = Kafka010JsonTableSource.builder()
{% highlight scala %} -val source: TableSource[_] = Kafka010JsonTableSource.builder() +val source: KafkaTableSource = Kafka010JsonTableSource.builder() // ... .withSchema(TableSchema.builder() .field("sensorId", Types.LONG) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java index 64dac06d821e9..688fd7363ea63 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java @@ -159,7 +159,7 @@ public void testRowtimeAttribute() { } @Test - public void testKafkaTSRowtimeAttribute() { + public void testRowtimeAttribute2() { KafkaTableSource.Builder b = getBuilder(); configureBuilder(b); @@ -191,7 +191,8 @@ public void testKafkaTSRowtimeAttribute() { } @Test - public void testKafkaTSSetConsumeOffsets() { + @SuppressWarnings("unchecked") + public void testConsumerOffsets() { KafkaTableSource.Builder b = getBuilder(); configureBuilder(b); From 3b58038d65cabdfd9da16150d4328227ca6afba1 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 20 Nov 2017 18:06:11 +0100 Subject: [PATCH 109/367] [hotfix] Always explicitly set hadoop.version in create_binary_release Before, the "hadoop2" profile would create a binary release for whatever happens to be the default hadoop.version. --- tools/releasing/create_binary_release.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 42b5a8e908046..3b1508e8bff9a 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -78,7 +78,7 @@ cd .. if [ "$SCALA_VERSION" == "none" ] && [ "$HADOOP_VERSION" == "none" ]; then - make_binary_release "hadoop2" "" "2.11" + make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "2.11" make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "2.11" make_binary_release "hadoop27" "-Dhadoop.version=2.7.3" "2.11" make_binary_release "hadoop28" "-Dhadoop.version=2.8.0" "2.11" @@ -87,7 +87,7 @@ then make_binary_release "hadoop2" "-Dhadoop.version=$HADOOP_VERSION" "2.11" elif [ "$SCALA_VERSION" != none ] && [ "$HADOOP_VERSION" == "none" ] then - make_binary_release "hadoop2" "" "$SCALA_VERSION" + make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "$SCALA_VERSION" make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "$SCALA_VERSION" make_binary_release "hadoop27" "-Dhadoop.version=2.7.3" "$SCALA_VERSION" make_binary_release "hadoop28" "-Dhadoop.version=2.8.0" "$SCALA_VERSION" From b5e156f79ae7e9cd2f8d5008f0c350e10ad4a821 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 8 Nov 2017 20:14:34 +0100 Subject: [PATCH 110/367] [hotfix] [core] Fix lots of checkstyle errors in core.fs --- .../fs/AbstractMultiFSDataInputStream.java | 8 +- .../apache/flink/core/fs/BlockLocation.java | 6 +- .../flink/core/fs/CloseableRegistry.java | 8 +- .../core/fs/ClosingFSDataInputStream.java | 4 +- .../core/fs/ClosingFSDataOutputStream.java | 4 +- .../flink/core/fs/FSDataInputStream.java | 6 +- .../core/fs/FSDataInputStreamWrapper.java | 2 +- .../flink/core/fs/FSDataOutputStream.java | 40 +++-- .../core/fs/FSDataOutputStreamWrapper.java | 2 +- .../apache/flink/core/fs/FileInputSplit.java | 30 ++-- .../org/apache/flink/core/fs/FileStatus.java | 19 ++- .../org/apache/flink/core/fs/FileSystem.java | 137 +++++++++--------- .../flink/core/fs/FileSystemFactory.java | 2 +- .../java/org/apache/flink/core/fs/Path.java | 59 ++++---- .../core/fs/UnsupportedSchemeFactory.java | 4 +- .../flink/core/fs/WrappingProxyCloseable.java | 4 +- 16 files changed, 163 insertions(+), 172 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java index a161cebc59b0c..e01ac2e0cc212 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/AbstractMultiFSDataInputStream.java @@ -32,13 +32,13 @@ @Internal public abstract class AbstractMultiFSDataInputStream extends FSDataInputStream { - /** Inner stream for the currently accessed segment of the virtual global stream */ + /** Inner stream for the currently accessed segment of the virtual global stream. */ protected FSDataInputStream delegate; - /** Position in the virtual global stream */ + /** Position in the virtual global stream. */ protected long totalPos; - /** Total available bytes in the virtual global stream */ + /** Total available bytes in the virtual global stream. */ protected long totalAvailable; public AbstractMultiFSDataInputStream() { @@ -48,7 +48,7 @@ public AbstractMultiFSDataInputStream() { @Override public void seek(long desired) throws IOException { - if(desired == totalPos) { + if (desired == totalPos) { return; } diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java b/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java index dff0c3ee8dbaf..fcdf905d0a91a 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java @@ -30,7 +30,7 @@ public interface BlockLocation extends Comparable { /** * Get the list of hosts (hostname) hosting this block. - * + * * @return A list of hosts (hostname) hosting this block. * @throws IOException * thrown if the list of hosts could not be retrieved @@ -39,14 +39,14 @@ public interface BlockLocation extends Comparable { /** * Get the start offset of the file associated with this block. - * + * * @return The start offset of the file associated with this block. */ long getOffset(); /** * Get the length of the block. - * + * * @return the length of the block */ long getLength(); diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java index 87d33d23a3362..5f1c9fb72385a 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java @@ -29,10 +29,10 @@ /** * This class allows to register instances of {@link Closeable}, which are all closed if this registry is closed. - *

- * Registering to an already closed registry will throw an exception and close the provided {@link Closeable} - *

- * All methods in this class are thread-safe. + * + *

Registering to an already closed registry will throw an exception and close the provided {@link Closeable} + * + *

All methods in this class are thread-safe. */ @Internal public class CloseableRegistry extends AbstractCloseableRegistry { diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java index 173a890b90a9d..1a62f62714501 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataInputStream.java @@ -26,8 +26,8 @@ /** * This class is a {@link org.apache.flink.util.WrappingProxy} for {@link FSDataInputStream} that is used to * implement a safety net against unclosed streams. - *

- * See {@link SafetyNetCloseableRegistry} for more details on how this is utilized. + * + *

See {@link SafetyNetCloseableRegistry} for more details on how this is utilized. */ @Internal public class ClosingFSDataInputStream diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java index cb7de927b1976..0f252a4a666f1 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/ClosingFSDataOutputStream.java @@ -26,8 +26,8 @@ /** * This class is a {@link org.apache.flink.util.WrappingProxy} for {@link FSDataOutputStream} that is used to * implement a safety net against unclosed streams. - *

- * See {@link SafetyNetCloseableRegistry} for more details on how this is utilized. + * + *

See {@link SafetyNetCloseableRegistry} for more details on how this is utilized. */ @Internal public class ClosingFSDataOutputStream diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java index 44dbcb17174d5..fa931c6326749 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java @@ -25,7 +25,7 @@ /** * Interface for a data input stream to a file on a {@link FileSystem}. - * + * *

This extends the {@link java.io.InputStream} with methods for accessing * the stream's {@link #getPos() current position} and * {@link #seek(long) seeking} to a desired position. @@ -36,7 +36,7 @@ public abstract class FSDataInputStream extends InputStream { /** * Seek to the given offset from the start of the file. The next read() will be from that location. * Can't seek past the end of the file. - * + * * @param desired * the desired offset * @throws IOException Thrown if an error occurred while seeking inside the input stream. @@ -47,7 +47,7 @@ public abstract class FSDataInputStream extends InputStream { * Gets the current position in the input stream. * * @return current position in the input stream - * @throws IOException Thrown if an I/O error occurred in the underlying stream + * @throws IOException Thrown if an I/O error occurred in the underlying stream * implementation while accessing the stream's position. */ public abstract long getPos() throws IOException; diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java index d2eb9f27629f8..6a3874f1dd8b7 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStreamWrapper.java @@ -25,7 +25,7 @@ import java.io.IOException; /** - * Simple forwarding wrapper around {@link FSDataInputStream} + * Simple forwarding wrapper around {@link FSDataInputStream}. */ @Internal public class FSDataInputStreamWrapper extends FSDataInputStream implements WrappingProxy { diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java index a8df5c18c0d3f..4a6e01d91efa3 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java @@ -26,22 +26,22 @@ /** * An output stream to a file that is created via a {@link FileSystem}. * This class extends the base {@link java.io.OutputStream} with some additional important methods. - * + * *

Data Persistence Guarantees

- * - * These streams are used to persistently store data, both for results of streaming applications + * + *

These streams are used to persistently store data, both for results of streaming applications * and for fault tolerance and recovery. It is therefore crucial that the persistence semantics * of these streams are well defined. - * + * *

Please refer to the class-level docs of {@link FileSystem} for the definition of data persistence * via Flink's FileSystem abstraction and the {@code FSDataOutputStream}. - * + * *

Thread Safety

- * - * Implementations of the {@code FSDataOutputStream} are generally not assumed to be thread safe. + * + *

Implementations of the {@code FSDataOutputStream} are generally not assumed to be thread safe. * Instances of {@code FSDataOutputStream} should not be passed between threads, because there * are no guarantees about the order of visibility of operations across threads. - * + * * @see FileSystem * @see FSDataInputStream */ @@ -52,13 +52,13 @@ public abstract class FSDataOutputStream extends OutputStream { * Gets the position of the stream (non-negative), defined as the number of bytes * from the beginning of the file to the current writing position. The position * corresponds to the zero-based index of the next byte that will be written. - * + * *

This method must report accurately report the current position of the stream. * Various components of the high-availability and recovery logic rely on the accurate - * + * * @return The current position in the stream, defined as the number of bytes * from the beginning of the file to the current writing position. - * + * * @throws IOException Thrown if an I/O error occurs while obtaining the position from * the stream implementation. */ @@ -68,14 +68,14 @@ public abstract class FSDataOutputStream extends OutputStream { * Flushes the stream, writing any data currently buffered in stream implementation * to the proper output stream. After this method has been called, the stream implementation * must not hold onto any buffered data any more. - * + * *

A completed flush does not mean that the data is necessarily persistent. Data * persistence can is only assumed after calls to {@link #close()} or {@link #sync()}. - * + * *

Implementation note: This overrides the method defined in {@link OutputStream} * as abstract to force implementations of the {@code FSDataOutputStream} to implement * this method directly. - * + * * @throws IOException Thrown if an I/O error occurs while flushing the stream. */ public abstract void flush() throws IOException; @@ -84,9 +84,7 @@ public abstract class FSDataOutputStream extends OutputStream { * Flushes the data all the way to the persistent non-volatile storage (for example disks). * The method behaves similar to the fsync function, forcing all data to * be persistent on the devices. - * - *

- * + * * @throws IOException Thrown if an I/O error occurs */ public abstract void sync() throws IOException; @@ -95,20 +93,20 @@ public abstract class FSDataOutputStream extends OutputStream { * Closes the output stream. After this method returns, the implementation must guarantee * that all data written to the stream is persistent/visible, as defined in the * {@link FileSystem class-level docs}. - * + * *

The above implies that the method must block until persistence can be guaranteed. * For example for distributed replicated file systems, the method must block until the * replication quorum has been reached. If the calling thread is interrupted in the * process, it must fail with an {@code IOException} to indicate that persistence cannot * be guaranteed. - * + * *

If this method throws an exception, the data in the stream cannot be assumed to be * persistent. - * + * *

Implementation note: This overrides the method defined in {@link OutputStream} * as abstract to force implementations of the {@code FSDataOutputStream} to implement * this method directly. - * + * * @throws IOException Thrown, if an error occurred while closing the stream or guaranteeing * that the data is persistent. */ diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java index f015012049671..21e68ef2fe280 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStreamWrapper.java @@ -25,7 +25,7 @@ import java.io.IOException; /** - * Simple forwarding wrapper around {@link FSDataInputStream} + * Simple forwarding wrapper around {@link FSDataInputStream}. */ @Internal public class FSDataOutputStreamWrapper extends FSDataOutputStream implements WrappingProxy { diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java index 8af0a20a8cd0a..bef13faf22e48 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java @@ -23,7 +23,7 @@ /** * A file input split provides information on a particular part of a file, possibly - * hosted on a distributed file system and replicated among several hosts. + * hosted on a distributed file system and replicated among several hosts. */ @Public public class FileInputSplit extends LocatableInputSplit { @@ -34,16 +34,16 @@ public class FileInputSplit extends LocatableInputSplit { private final Path file; /** The position of the first byte in the file to process. */ - private long start; + private final long start; /** The number of bytes in the file to process. */ - private long length; + private final long length; // -------------------------------------------------------------------------------------------- - + /** * Constructs a split with host information. - * + * * @param num * the number of this input split * @param file @@ -57,17 +57,17 @@ public class FileInputSplit extends LocatableInputSplit { */ public FileInputSplit(int num, Path file, long start, long length, String[] hosts) { super(num, hosts); - + this.file = file; this.start = start; this.length = length; } // -------------------------------------------------------------------------------------------- - + /** * Returns the path of the file containing this split's data. - * + * * @return the path of the file containing this split's data. */ public Path getPath() { @@ -76,7 +76,7 @@ public Path getPath() { /** * Returns the position of the first byte in the file to process. - * + * * @return the position of the first byte in the file to process */ public long getStart() { @@ -85,20 +85,20 @@ public long getStart() { /** * Returns the number of bytes in the file to process. - * + * * @return the number of bytes in the file to process */ public long getLength() { return length; } - + // -------------------------------------------------------------------------------------------- - + @Override public int hashCode() { return getSplitNumber() ^ (file == null ? 0 : file.hashCode()); } - + @Override public boolean equals(Object obj) { if (obj == this) { @@ -106,7 +106,7 @@ public boolean equals(Object obj) { } else if (obj != null && obj instanceof FileInputSplit && super.equals(obj)) { FileInputSplit other = (FileInputSplit) obj; - + return this.start == other.start && this.length == other.length && (this.file == null ? other.file == null : (other.file != null && this.file.equals(other.file))); @@ -115,7 +115,7 @@ else if (obj != null && obj instanceof FileInputSplit && super.equals(obj)) { return false; } } - + @Override public String toString() { return "[" + getSplitNumber() + "] " + file + ":" + start + "+" + length; diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileStatus.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileStatus.java index 8b62659c19dbc..f9794e6be6379 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileStatus.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileStatus.java @@ -20,7 +20,7 @@ /** * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. + * additional information regarding copyright ownership. */ package org.apache.flink.core.fs; @@ -30,56 +30,55 @@ /** * Interface that represents the client side information for a file * independent of the file system. - * */ @Public public interface FileStatus { /** - * Return the length of this file - * + * Return the length of this file. + * * @return the length of this file */ long getLen(); /** *Get the block size of the file. - * + * * @return the number of bytes */ long getBlockSize(); /** * Get the replication factor of a file. - * + * * @return the replication factor of a file. */ short getReplication(); /** * Get the modification time of the file. - * + * * @return the modification time of file in milliseconds since January 1, 1970 UTC. */ long getModificationTime(); /** * Get the access time of the file. - * + * * @return the access time of file in milliseconds since January 1, 1970 UTC. */ long getAccessTime(); /** * Checks if this object represents a directory. - * + * * @return true if this is a directory, false otherwise */ boolean isDir(); /** * Returns the corresponding Path to the FileStatus. - * + * * @return the corresponding Path to the FileStatus */ Path getPath(); diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index 982e496d26253..7a8245a80bcaf 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -16,7 +16,6 @@ * limitations under the License. */ - /* * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for @@ -38,6 +37,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; + import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -55,40 +55,40 @@ * distributed file systems, or local file systems. The abstraction by this file system is very simple, * and the set of available operations quite limited, to support the common denominator of a wide * range of file systems. For example, appending to or mutating existing files is not supported. - * + * *

Flink implements and supports some file system types directly (for example the default * machine-local file system). Other file system types are accessed by an implementation that bridges * to the suite of file systems supported by Hadoop (such as for example HDFS). - * + * *

Scope and Purpose

- * - * The purpose of this abstraction is used to expose a common and well defined interface for + * + *

The purpose of this abstraction is used to expose a common and well defined interface for * access to files. This abstraction is used both by Flink's fault tolerance mechanism (storing * state and recovery data) and by reusable built-in connectors (file sources / sinks). - * + * *

The purpose of this abstraction is not to give user programs an abstraction with * extreme flexibility and control across all possible file systems. That mission would be a folly, * as the differences in characteristics of even the most common file systems are already quite * large. It is expected that user programs that need specialized functionality of certain file systems * in their functions, operations, sources, or sinks instantiate the specialized file system adapters * directly. - * + * *

Data Persistence Contract

- * - * The FileSystem's {@link FSDataOutputStream output streams} are used to persistently store data, + * + *

The FileSystem's {@link FSDataOutputStream output streams} are used to persistently store data, * both for results of streaming applications and for fault tolerance and recovery. It is therefore * crucial that the persistence semantics of these streams are well defined. - * + * *

Definition of Persistence Guarantees

- * - * Data written to an output stream is considered persistent, if two requirements are met: - * + * + *

Data written to an output stream is considered persistent, if two requirements are met: + * *

    *
  1. Visibility Requirement: It must be guaranteed that all other processes, machines, * virtual machines, containers, etc. that are able to access the file see the data consistently * when given the absolute file path. This requirement is similar to the close-to-open * semantics defined by POSIX, but restricted to the file itself (by its absolute path).
  2. - * + * *
  3. Durability Requirement: The file system's specific durability/persistence requirements * must be met. These are specific to the particular file system. For example the * {@link LocalFileSystem} does not provide any durability guarantees for crashes of both @@ -101,14 +101,14 @@ * listing the directory contents) are not required to be complete for the data in the file stream * to be considered persistent. This relaxation is important for file systems where updates to * directory contents are only eventually consistent. - * + * *

    The {@link FSDataOutputStream} has to guarantee data persistence for the written bytes * once the call to {@link FSDataOutputStream#close()} returns. * *

    Examples

    * *
      - *
    • For fault-tolerant distributed file systems, data is considered persistent once + *
    • For fault-tolerant distributed file systems, data is considered persistent once * it has been received and acknowledged by the file system, typically by having been replicated * to a quorum of machines (durability requirement). In addition the absolute file path * must be visible to all other machines that will potentially access the file (visibility @@ -125,12 +125,12 @@ *
    • A local file system must support the POSIX close-to-open semantics. * Because the local file system does not have any fault tolerance guarantees, no further * requirements exist. - * + * *

      The above implies specifically that data may still be in the OS cache when considered * persistent from the local file system's perspective. Crashes that cause the OS cache to loose * data are considered fatal to the local machine and are not covered by the local file system's * guarantees as defined by Flink. - * + * *

      That means that computed results, checkpoints, and savepoints that are written only to * the local filesystem are not guaranteed to be recoverable from the local machine's failure, * making local file systems unsuitable for production setups.

    • @@ -138,14 +138,14 @@ * *

      Updating File Contents

      * - * Many file systems either do not support overwriting contents of existing files at all, or do + *

      Many file systems either do not support overwriting contents of existing files at all, or do * not support consistent visibility of the updated contents in that case. For that reason, * Flink's FileSystem does not support appending to existing files, or seeking within output streams * so that previously written data could be overwritten. * *

      Overwriting Files

      * - * Overwriting files is in general possible. A file is overwritten by deleting it and creating + *

      Overwriting files is in general possible. A file is overwritten by deleting it and creating * a new file. However, certain filesystems cannot make that change synchronously visible * to all parties that have access to the file. * For example Amazon S3 guarantees only @@ -154,29 +154,29 @@ * *

      To avoid these consistency issues, the implementations of failure/recovery mechanisms in * Flink strictly avoid writing to the same file path more than once. - * + * *

      Thread Safety

      - * - * Implementations of {@code FileSystem} must be thread-safe: The same instance of FileSystem + * + *

      Implementations of {@code FileSystem} must be thread-safe: The same instance of FileSystem * is frequently shared across multiple threads in Flink and must be able to concurrently * create input/output streams and list file metadata. - * + * *

      The {@link FSDataOutputStream} and {@link FSDataOutputStream} implementations are strictly * not thread-safe. Instances of the streams should also not be passed between threads * in between read or write operations, because there are no guarantees about the visibility of * operations across threads (many operations do not create memory fences). - * + * *

      Streams Safety Net

      - * - * When application code obtains a FileSystem (via {@link FileSystem#get(URI)} or via + * + *

      When application code obtains a FileSystem (via {@link FileSystem#get(URI)} or via * {@link Path#getFileSystem()}), the FileSystem instantiates a safety net for that FileSystem. * The safety net ensures that all streams created from the FileSystem are closed when the * application task finishes (or is canceled or failed). That way, the task's threads do not * leak connections. - * + * *

      Internal runtime code can explicitly obtain a FileSystem that does not use the safety * net via {@link FileSystem#getUnguardedFileSystem(URI)}. - * + * * @see FSDataInputStream * @see FSDataOutputStream */ @@ -201,7 +201,7 @@ public enum WriteMode { // ------------------------------------------------------------------------ - /** Logger for all FileSystem work */ + /** Logger for all FileSystem work. */ private static final Logger LOG = LoggerFactory.getLogger(FileSystem.class); /** This lock guards the methods {@link #initOutPathLocalFS(Path, WriteMode, boolean)} and @@ -223,14 +223,13 @@ public enum WriteMode { /** The default filesystem scheme to be used, configured during process-wide initialization. * This value defaults to the local file systems scheme {@code 'file:///'} or {@code 'file:/'}. */ private static URI DEFAULT_SCHEME; - // ------------------------------------------------------------------------ // Initialization // ------------------------------------------------------------------------ /** - * Initializes the shared file system settings. + * Initializes the shared file system settings. * *

      The given configuration is passed to each file system factory to initialize the respective * file systems. Because the configuration of file systems may be different subsequent to the call @@ -351,7 +350,7 @@ public static FileSystem getUnguardedFileSystem(final URI fsUri) throws IOExcept } } - // print a helpful pointer for malformed local URIs (happens a lot to new users) + // print a helpful pointer for malformed local URIs (happens a lot to new users) if (uri.getScheme().equals("file") && uri.getAuthority() != null && !uri.getAuthority().isEmpty()) { String supposedUri = "file:///" + uri.getAuthority() + uri.getPath(); @@ -382,7 +381,7 @@ public static FileSystem getUnguardedFileSystem(final URI fsUri) throws IOExcept } catch (UnsupportedFileSystemSchemeException e) { throw new UnsupportedFileSystemSchemeException( - "Could not find a file system implementation for scheme '" + uri.getScheme() + + "Could not find a file system implementation for scheme '" + uri.getScheme() + "'. The scheme is not directly supported by Flink and no Hadoop file " + "system to support this scheme could be loaded.", e); } @@ -479,7 +478,7 @@ public static URI getDefaultFsUri() { * Return the number of bytes that large input files should be optimally be split into to minimize I/O time. * * @return the number of bytes that large input files should be optimally be split into to minimize I/O time - * + * * @deprecated This value is no longer used and is meaningless. */ @Deprecated @@ -539,7 +538,7 @@ public boolean exists(final Path f) throws IOException { /** * Opens an FSDataOutputStream at the indicated Path. - * + * *

      This method is deprecated, because most of its parameters are ignored by most file systems. * To control for example the replication factor and block size in the Hadoop Distributed File system, * make sure that the respective Hadoop configuration file is either linked from the Flink configuration, @@ -556,13 +555,13 @@ public boolean exists(final Path f) throws IOException { * required block replication for the file. * @param blockSize * the size of the file blocks - * + * * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because * a file already exists at that path and the write mode indicates to not * overwrite the file. - * + * * @deprecated Deprecated because not well supported across types of file systems. - * Control the behavior of specific file systems via configurations instead. + * Control the behavior of specific file systems via configurations instead. */ @Deprecated public FSDataOutputStream create( @@ -583,11 +582,11 @@ public FSDataOutputStream create( * @param overwrite * if a file with this name already exists, then if true, * the file will be overwritten, and if false an error will be thrown. - * + * * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because * a file already exists at that path and the write mode indicates to not * overwrite the file. - * + * * @deprecated Use {@link #create(Path, WriteMode)} instead. */ @Deprecated @@ -597,7 +596,7 @@ public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { /** * Opens an FSDataOutputStream to a new file at the given path. - * + * *

      If the file already exists, the behavior depends on the given {@code WriteMode}. * If the mode is set to {@link WriteMode#NO_OVERWRITE}, then this method fails with an * exception. @@ -605,7 +604,7 @@ public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { * @param f The file path to write to * @param overwriteMode The action to take if a file or directory already exists at the given path. * @return The stream to the new file at the target path. - * + * * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because * a file already exists at that path and the write mode indicates to not * overwrite the file. @@ -674,7 +673,7 @@ public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { *

    *
  4. * - * + * *

    Files contained in an existing directory are not deleted, because multiple instances of a * DataSinkTask might call this function at the same time and hence might perform concurrent * delete operations on the file system (possibly deleting output files of concurrently running tasks). @@ -684,7 +683,7 @@ public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { * @param outPath Output path that should be prepared. * @param writeMode Write mode to consider. * @param createDirectory True, to initialize a directory at the given path, false to prepare space for a file. - * + * * @return True, if the path was successfully prepared, false otherwise. * @throws IOException Thrown, if any of the file system access operations failed. */ @@ -708,7 +707,7 @@ public boolean initOutPathLocalFS(Path outPath, WriteMode writeMode, boolean cre // restore the interruption state Thread.currentThread().interrupt(); - // leave the method - we don't have the lock anyways + // leave the method - we don't have the lock anyways throw new IOException("The thread was interrupted while trying to initialize the output directory"); } @@ -733,7 +732,7 @@ public boolean initOutPathLocalFS(Path outPath, WriteMode writeMode, boolean cre } else { // file may not be overwritten throw new IOException("File or directory already exists. Existing files and directories " + - "are not overwritten in " + WriteMode.NO_OVERWRITE.name() + " mode. Use " + + "are not overwritten in " + WriteMode.NO_OVERWRITE.name() + " mode. Use " + WriteMode.OVERWRITE.name() + " mode to overwrite existing files and directories."); } @@ -748,7 +747,7 @@ public boolean initOutPathLocalFS(Path outPath, WriteMode writeMode, boolean cre delete(outPath, true); } catch (IOException e) { - throw new IOException("Could not remove existing directory '" + outPath + + throw new IOException("Could not remove existing directory '" + outPath + "' to allow overwrite by result file", e); } } @@ -798,27 +797,27 @@ public boolean initOutPathLocalFS(Path outPath, WriteMode writeMode, boolean cre /** * Initializes output directories on distributed file systems according to the given write mode. * - * WriteMode.NO_OVERWRITE & parallel output: + *

    WriteMode.NO_OVERWRITE & parallel output: * - A directory is created if the output path does not exist. * - An existing file or directory raises an exception. * - * WriteMode.NO_OVERWRITE & NONE parallel output: + *

    WriteMode.NO_OVERWRITE & NONE parallel output: * - An existing file or directory raises an exception. * - * WriteMode.OVERWRITE & parallel output: + *

    WriteMode.OVERWRITE & parallel output: * - A directory is created if the output path does not exist. * - An existing directory and its content is deleted and a new directory is created. * - An existing file is deleted and replaced by a new directory. * - * WriteMode.OVERWRITE & NONE parallel output: + *

    WriteMode.OVERWRITE & NONE parallel output: * - An existing file or directory is deleted and replaced by a new directory. * * @param outPath Output path that should be prepared. * @param writeMode Write mode to consider. * @param createDirectory True, to initialize a directory at the given path, false otherwise. - * + * * @return True, if the path was successfully prepared, false otherwise. - * + * * @throws IOException Thrown, if any of the file system access operations failed. */ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean createDirectory) throws IOException { @@ -841,7 +840,7 @@ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean crea // restore the interruption state Thread.currentThread().interrupt(); - // leave the method - we don't have the lock anyways + // leave the method - we don't have the lock anyways throw new IOException("The thread was interrupted while trying to initialize the output directory"); } @@ -850,13 +849,13 @@ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean crea if (exists(outPath)) { // path exists, check write mode switch(writeMode) { - + case NO_OVERWRITE: // file or directory may not be overwritten throw new IOException("File or directory already exists. Existing files and directories are not overwritten in " + WriteMode.NO_OVERWRITE.name() + " mode. Use " + WriteMode.OVERWRITE.name() + " mode to overwrite existing files and directories."); - + case OVERWRITE: // output path exists. We delete it and all contained files in case of a directory. try { @@ -867,12 +866,12 @@ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean crea // this will be handled later. } break; - + default: - throw new IllegalArgumentException("Invalid write mode: "+writeMode); + throw new IllegalArgumentException("Invalid write mode: " + writeMode); } } - + if (createDirectory) { // Output directory needs to be created try { @@ -881,10 +880,10 @@ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean crea } } catch (IOException ioe) { // Some other thread might already have created the directory. - // If - for some other reason - the directory could not be created + // If - for some other reason - the directory could not be created // and the path does not exist, this will be handled later. } - + // double check that the output directory exists return exists(outPath) && getFileStatus(outPath).isDir(); } @@ -906,7 +905,7 @@ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean crea * Aside from the {@link LocalFileSystem}, these file systems are loaded * via Java's service framework. * - * @return A map from the file system scheme to corresponding file system factory. + * @return A map from the file system scheme to corresponding file system factory. */ private static HashMap loadFileSystems() { final HashMap map = new HashMap<>(); @@ -935,7 +934,7 @@ private static HashMap loadFileSystems() { // catching Throwable here to handle various forms of class loading // and initialization errors ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - LOG.error("Failed to load a file systems via services", t); + LOG.error("Failed to load a file system via services", t); } } } @@ -948,7 +947,7 @@ private static HashMap loadFileSystems() { return map; } - + /** * Utility loader for the Hadoop file system factory. * We treat the Hadoop FS factory in a special way, because we use it as a catch @@ -963,10 +962,12 @@ private static FileSystemFactory loadHadoopFsFactory() { // first, see if the Flink runtime classes are available final Class factoryClass; try { - factoryClass = Class.forName("org.apache.flink.runtime.fs.hdfs.HadoopFsFactory", false, cl).asSubclass(FileSystemFactory.class); + factoryClass = Class + .forName("org.apache.flink.runtime.fs.hdfs.HadoopFsFactory", false, cl) + .asSubclass(FileSystemFactory.class); } catch (ClassNotFoundException e) { - LOG.info("No Flink runtime dependency present. " + + LOG.info("No Flink runtime dependency present. " + "The extended set of supported File Systems via Hadoop is not available."); return new UnsupportedSchemeFactory("Flink runtime classes missing in classpath/dependencies."); } @@ -1039,7 +1040,7 @@ else if (obj != null && obj.getClass() == FSKey.class) { @Override public int hashCode() { - return 31 * scheme.hashCode() + + return 31 * scheme.hashCode() + (authority == null ? 17 : authority.hashCode()); } diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java index 982da359759df..8a3547107ee17 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java @@ -58,4 +58,4 @@ public interface FileSystemFactory { * @throws IOException Thrown if the file system could not be instantiated. */ FileSystem create(URI fsUri) throws IOException; -} \ No newline at end of file +} diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/Path.java b/flink-core/src/main/java/org/apache/flink/core/fs/Path.java index 53290edb8adcd..6398aa84b40f8 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/Path.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/Path.java @@ -22,26 +22,26 @@ package org.apache.flink.core.fs; -import java.io.IOException; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; - import org.apache.flink.annotation.Public; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.util.StringUtils; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; + /** * Names a file or directory in a {@link FileSystem}. Path strings use slash as * the directory separator. A path string is absolute if it begins with a slash. * - * Tailing slashes are removed from the path. + *

    Tailing slashes are removed from the path. */ @Public public class Path implements IOReadableWritable, Serializable { - + private static final long serialVersionUID = 1L; /** @@ -71,7 +71,7 @@ public Path() {} /** * Constructs a path object from a given URI. - * + * * @param uri * the URI to construct the path object from */ @@ -81,7 +81,7 @@ public Path(URI uri) { /** * Resolve a child path against a parent path. - * + * * @param parent * the parent path * @param child @@ -93,7 +93,7 @@ public Path(String parent, String child) { /** * Resolve a child path against a parent path. - * + * * @param parent * the parent path * @param child @@ -105,7 +105,7 @@ public Path(Path parent, String child) { /** * Resolve a child path against a parent path. - * + * * @param parent * the parent path * @param child @@ -117,7 +117,7 @@ public Path(String parent, Path child) { /** * Resolve a child path against a parent path. - * + * * @param parent * the parent path * @param child @@ -168,7 +168,7 @@ private String checkAndTrimPathArg(String path) { /** * Construct a path from a String. Path strings are URIs, but with unescaped * elements and some additional normalization. - * + * * @param pathString * the string to construct a path from */ @@ -214,7 +214,7 @@ public Path(String pathString) { /** * Construct a Path from a scheme, an authority and a path string. - * + * * @param scheme * the scheme string * @param authority @@ -229,7 +229,7 @@ public Path(String scheme, String authority, String path) { /** * Initializes a path object given the scheme, authority and path string. - * + * * @param scheme * the scheme string. * @param authority @@ -247,7 +247,7 @@ private void initialize(String scheme, String authority, String path) { /** * Normalizes a path string. - * + * * @param path * the path string to normalize * @return the normalized path string @@ -262,10 +262,10 @@ private String normalizePath(String path) { path = path.replaceAll("/+", "/"); // remove tailing separator - if(!path.equals(SEPARATOR) && // UNIX root path + if (!path.equals(SEPARATOR) && // UNIX root path !path.matches("/\\p{Alpha}+:/") && // Windows root path - path.endsWith(SEPARATOR)) - { + path.endsWith(SEPARATOR)) { + // remove tailing slash path = path.substring(0, path.length() - SEPARATOR.length()); } @@ -275,7 +275,7 @@ private String normalizePath(String path) { /** * Converts the path object to a {@link URI}. - * + * * @return the {@link URI} object converted from the path object */ public URI toUri() { @@ -284,7 +284,7 @@ public URI toUri() { /** * Returns the FileSystem that owns this Path. - * + * * @return the FileSystem that owns this Path * @throws IOException * thrown if the file system could not be retrieved @@ -295,7 +295,7 @@ public FileSystem getFileSystem() throws IOException { /** * Checks if the directory of this path is absolute. - * + * * @return true if the directory of this path is absolute, false otherwise */ public boolean isAbsolute() { @@ -305,7 +305,7 @@ public boolean isAbsolute() { /** * Returns the final component of this path, i.e., everything that follows the last separator. - * + * * @return the final component of the path */ public String getName() { @@ -325,7 +325,7 @@ public String getPath() { /** * Returns the parent of a path, i.e., everything that precedes the last separator * or null if at root. - * + * * @return the parent of a path or null if at root. */ public Path getParent() { @@ -348,7 +348,7 @@ public Path getParent() { /** * Adds a suffix to the final name in the path. - * + * * @param suffix The suffix to be added * @return the new path including the suffix */ @@ -381,7 +381,6 @@ public String toString() { return buffer.toString(); } - @Override public boolean equals(Object o) { if (!(o instanceof Path)) { @@ -391,7 +390,6 @@ public boolean equals(Object o) { return this.uri.equals(that.uri); } - @Override public int hashCode() { return uri.hashCode(); @@ -404,7 +402,7 @@ public int compareTo(Object o) { /** * Returns the number of elements in this path. - * + * * @return the number of elements in this path */ public int depth() { @@ -420,7 +418,7 @@ public int depth() { /** * Returns a qualified path object. - * + * * @param fs * the FileSystem that should be used to obtain the current working directory * @return the qualified path object @@ -479,7 +477,6 @@ public void read(DataInputView in) throws IOException { } } - @Override public void write(DataOutputView out) throws IOException { if (uri == null) { @@ -516,7 +513,7 @@ public boolean hasWindowsDrive() { * the path to check * @param slashed * true to indicate the first character of the string is a slash, false otherwise - * + * * @return true if the path string contains a windows drive letter, false otherwise */ private boolean hasWindowsDrive(String path, boolean slashed) { diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java b/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java index 234b49f6db042..c2cb2d570d4ed 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java @@ -20,11 +20,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.FileSystemFactory; -import org.apache.flink.core.fs.UnsupportedFileSystemSchemeException; import javax.annotation.Nullable; + import java.io.IOException; import java.net.URI; diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java b/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java index 9f100efa14a06..e1c5a7d0ca4be 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/WrappingProxyCloseable.java @@ -27,6 +27,4 @@ * {@link WrappingProxy} for {@link Closeable} that is also closeable. */ @Internal -public interface WrappingProxyCloseable extends Closeable, WrappingProxy { - -} +public interface WrappingProxyCloseable extends Closeable, WrappingProxy {} From a11e2cf0b1f37d3ef22e1978e89928fa374960db Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 8 Nov 2017 23:57:04 +0100 Subject: [PATCH 111/367] [FLINK-8125] [core] Introduce limiting of outgoing file system connections --- docs/ops/filesystems.md | 126 ++ .../flink/configuration/ConfigConstants.java | 5 +- .../flink/configuration/CoreOptions.java | 57 + .../core/fs/ConnectionLimitingFactory.java | 95 ++ .../org/apache/flink/core/fs/FileSystem.java | 46 +- .../core/fs/LimitedConnectionsFileSystem.java | 1114 +++++++++++++++++ .../core/fs/local/LocalFileSystemFactory.java | 1 - .../util/function/SupplierWithException.java | 38 + .../FilesystemSchemeConfigTest.java | 6 +- .../LimitedConnectionsConfigurationTest.java | 84 ++ ...edConnectionsFileSystemDelegationTest.java | 241 ++++ .../fs/LimitedConnectionsFileSystemTest.java | 742 +++++++++++ .../runtime/fs/hdfs/HadoopFsFactory.java | 35 +- .../runtime/fs/hdfs/HadoopFsFactoryTest.java | 3 +- .../LimitedConnectionsConfigurationTest.java | 84 ++ 15 files changed, 2654 insertions(+), 23 deletions(-) create mode 100644 docs/ops/filesystems.md create mode 100644 flink-core/src/main/java/org/apache/flink/core/fs/ConnectionLimitingFactory.java create mode 100644 flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java create mode 100644 flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java create mode 100644 flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java create mode 100644 flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/LimitedConnectionsConfigurationTest.java diff --git a/docs/ops/filesystems.md b/docs/ops/filesystems.md new file mode 100644 index 0000000000000..5b2a1e78d138f --- /dev/null +++ b/docs/ops/filesystems.md @@ -0,0 +1,126 @@ +--- +title: "File Systems" +nav-parent_id: ops +nav-pos: 12 +--- + + +This page provides details on setting up and configuring distributed file systems for use with Flink. + +## Flink' File System support + +Flink uses file systems both as a source and sink in streaming/batch applications, and as a target for checkpointing. +These file systens can for example be *Unix/Windows file systems*, *HDFS*, or even object stores like *S3*. + +The file system used for a specific file is determined by the file URI's scheme. For example `file:///home/user/text.txt` refers to +a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` refers to a file in a specific HDFS cluster. + +File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify +files and objects in that file system. FileSystem instances are instantiates once per process and then cached / pooled, to +avoid configuration overhead per stream creation, and to enforce certain constraints, like connection/stream limits. + +### Built-in File Systems + +Flink directly implements the following file systems: + + - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, +including any NFS or SAN that is mounted into that local file system. + + - **S3**: Flink directly provides file systems to talk to Amazon S3, registered under the scheme *"s3://"*. +There are two alternative implementations, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`, based on code from the [Presto project](https://prestodb.io/) +and the [Hadoop Project](https://hadoop.apache.org/). Both implementations are self-contained with no dependency footprint. +To use those when using Flink as a library, add the resective maven dependency (`org.apache.flink:flink-s3-fs-presto:{{ site.version }}` or `org.apache.flink:flink-s3-fs-hadoop:{{ site.version }}`). +When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder. +See [AWS setup](deployment/aws.html) for details. + + - **MapR FS**: The MapR file system *"maprfs://"* is automatically available when the MapR libraries are in the classpath. + +### HDFS and Hadoop File System support + +For a scheme where Flink does not implemented a file system itself, Flink will try to use Hadoop to instantiate a file system for the respective scheme. +All Hadoop file systems are automatically available once `flink-runtime` and the relevant Hadoop libraries are in classpath. + +That way, Flink seamslessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS), for example: + + - **hdfs** + - **ftp** + - **s3n** and **s3a** + - **har** + - ... + + +## Common File System configurations + +The following configuration settings exist across different file systems + +#### Default File System + +If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used. + +~~~ +fs.default-scheme: +~~~ + +For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a a file path of +`/user/hugo/in.txt'` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt'` + +#### Connection limiting + +You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number +of concurrent reads / writes or open connections at the same time. + +For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint. + +To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by +its scheme. + +~~~ +fs..limit.total: (number, 0/-1 mean no limit) +fs..limit.input: (number, 0/-1 mean no limit) +fs..limit.output: (number, 0/-1 mean no limit) +fs..limit.timeout: (milliseconds, 0 means infinite) +fs..limit.stream-timeout: (milliseconds, 0 means infinite) +~~~ + +You can limit the number if input/output connections (streams) separately (`fs..limit.input` and `fs..limit.output`), as well as impose a limit on +the total number of concurrent streams (`fs..limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed. +If the opening of the stream takes longer than `fs..limit.timeout`, the stream opening will fail. + +To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams: +`fs..limit.stream-timeout`. If a stream does not read/write any bytes for at least that amout of time, it is forcibly closed. + +These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections. +In addition, the The limit are also enforced only per FileSystem instance. Because File Systems are created per scheme and authority, different +authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools. + + +## Adding new File System Implementations + +File system implementations are discovered by Flink through Java's service abstraction, making it easy to add additional file system implementations. + +In order to add a new File System, the following steps are needed: + + - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`. + - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`. + - Add a service entry. Create a file `META-INF/services/org.apache.flink.core.fs.FileSystemFactory` which contains the class name of your file system factory class. + +See the [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html) for more details on how service loaders work. + +{% top %} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index fcf73b8809d44..f80bd9bf7f156 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -614,7 +614,10 @@ public final class ConfigConstants { * Key to specify the default filesystem to be used by a job. In the case of * file:///, which is the default (see {@link ConfigConstants#DEFAULT_FILESYSTEM_SCHEME}), * the local filesystem is going to be used to resolve URIs without an explicit scheme. - * */ + * + * @deprecated Use {@link CoreOptions#DEFAULT_FILESYSTEM_SCHEME} instead. + */ + @Deprecated public static final String FILESYSTEM_SCHEME = "fs.default-scheme"; /** diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index e8ab8e474fa65..928e8102a2ac4 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -20,6 +20,9 @@ import org.apache.flink.annotation.PublicEvolving; +/** + * The set of configuration options for core parameters. + */ @PublicEvolving public class CoreOptions { @@ -83,4 +86,58 @@ public class CoreOptions { public static final ConfigOption CHECKPOINTS_DIRECTORY = ConfigOptions .key("state.checkpoints.dir") .noDefaultValue(); + + // ------------------------------------------------------------------------ + // file systems + // ------------------------------------------------------------------------ + + /** + * The default filesystem scheme, used for paths that do not declare a scheme explicitly. + */ + public static final ConfigOption DEFAULT_FILESYSTEM_SCHEME = ConfigOptions + .key("fs.default-scheme") + .noDefaultValue(); + + /** + * The total number of input plus output connections that a file system for the given scheme may open. + * Unlimited be default. + */ + public static ConfigOption fileSystemConnectionLimit(String scheme) { + return ConfigOptions.key("fs." + scheme + ".limit.total").defaultValue(-1); + } + + /** + * The total number of input connections that a file system for the given scheme may open. + * Unlimited be default. + */ + public static ConfigOption fileSystemConnectionLimitIn(String scheme) { + return ConfigOptions.key("fs." + scheme + ".limit.input").defaultValue(-1); + } + + /** + * The total number of output connections that a file system for the given scheme may open. + * Unlimited be default. + */ + public static ConfigOption fileSystemConnectionLimitOut(String scheme) { + return ConfigOptions.key("fs." + scheme + ".limit.output").defaultValue(-1); + } + + /** + * If any connection limit is configured, this option can be optionally set to define after + * which time (in milliseconds) stream opening fails with a timeout exception, if no stream + * connection becomes available. Unlimited timeout be default. + */ + public static ConfigOption fileSystemConnectionLimitTimeout(String scheme) { + return ConfigOptions.key("fs." + scheme + ".limit.timeout").defaultValue(0L); + } + + /** + * If any connection limit is configured, this option can be optionally set to define after + * which time (in milliseconds) inactive streams are reclaimed. This option can help to prevent + * that inactive streams make up the full pool of limited connections, and no further connections + * can be established. Unlimited timeout be default. + */ + public static ConfigOption fileSystemConnectionLimitStreamInactivityTimeout(String scheme) { + return ConfigOptions.key("fs." + scheme + ".limit.stream-timeout").defaultValue(0L); + } } diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/ConnectionLimitingFactory.java b/flink-core/src/main/java/org/apache/flink/core/fs/ConnectionLimitingFactory.java new file mode 100644 index 0000000000000..b85a7d6ec006f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/core/fs/ConnectionLimitingFactory.java @@ -0,0 +1,95 @@ +/* + * 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.flink.core.fs; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.LimitedConnectionsFileSystem.ConnectionLimitingSettings; + +import java.io.IOException; +import java.net.URI; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A wrapping factory that adds a {@link LimitedConnectionsFileSystem} to a file system. + */ +@Internal +public class ConnectionLimitingFactory implements FileSystemFactory { + + private final FileSystemFactory factory; + + private final ConnectionLimitingSettings settings; + + private ConnectionLimitingFactory( + FileSystemFactory factory, + ConnectionLimitingSettings settings) { + + this.factory = checkNotNull(factory); + this.settings = checkNotNull(settings); + } + + // ------------------------------------------------------------------------ + + @Override + public String getScheme() { + return factory.getScheme(); + } + + @Override + public void configure(Configuration config) { + factory.configure(config); + } + + @Override + public FileSystem create(URI fsUri) throws IOException { + FileSystem original = factory.create(fsUri); + return new LimitedConnectionsFileSystem(original, + settings.limitTotal, settings.limitOutput, settings.limitInput, + settings.streamOpenTimeout, settings.streamInactivityTimeout); + } + + // ------------------------------------------------------------------------ + + /** + * Decorates the given factory for a {@code ConnectionLimitingFactory}, if the given + * configuration configured connection limiting for the given file system scheme. + * Otherwise, it returns the given factory as is. + * + * @param factory The factory to potentially decorate. + * @param scheme The file scheme for which to check the configuration. + * @param config The configuration + * + * @return The decorated factors, if connection limiting is configured, the original factory otherwise. + */ + public static FileSystemFactory decorateIfLimited(FileSystemFactory factory, String scheme, Configuration config) { + checkNotNull(factory, "factory"); + + final ConnectionLimitingSettings settings = ConnectionLimitingSettings.fromConfig(config, scheme); + + // decorate only if any limit is configured + if (settings == null) { + // no limit configured + return factory; + } + else { + return new ConnectionLimitingFactory(factory, settings); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index 7a8245a80bcaf..18baaa5d5f95f 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -26,8 +26,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.fs.local.LocalFileSystemFactory; @@ -43,8 +43,11 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.ServiceLoader; import java.util.concurrent.locks.ReentrantLock; @@ -214,8 +217,12 @@ public enum WriteMode { /** Cache for file systems, by scheme + authority. */ private static final HashMap CACHE = new HashMap<>(); - /** Mapping of file system schemes to the corresponding implementation factories. */ - private static final HashMap FS_FACTORIES = loadFileSystems(); + /** All available file system factories. */ + private static final List RAW_FACTORIES = loadFileSystems(); + + /** Mapping of file system schemes to the corresponding factories, + * populated in {@link FileSystem#initialize(Configuration)}. */ + private static final HashMap FS_FACTORIES = new HashMap<>(); /** The default factory that is used when no scheme matches. */ private static final FileSystemFactory FALLBACK_FACTORY = loadHadoopFsFactory(); @@ -236,7 +243,7 @@ public enum WriteMode { * of this method, this method clears the file system instance cache. * *

    This method also reads the default file system URI from the configuration key - * {@link ConfigConstants#FILESYSTEM_SCHEME}. All calls to {@link FileSystem#get(URI)} where + * {@link CoreOptions#DEFAULT_FILESYSTEM_SCHEME}. All calls to {@link FileSystem#get(URI)} where * the URI has no scheme will be interpreted as relative to that URI. * As an example, assume the default file system URI is set to {@code 'hdfs://localhost:9000/'}. * A file path of {@code '/user/USERNAME/in.txt'} is interpreted as @@ -249,17 +256,22 @@ public static void initialize(Configuration config) throws IOException, IllegalC try { // make sure file systems are re-instantiated after re-configuration CACHE.clear(); + FS_FACTORIES.clear(); // configure all file system factories - for (FileSystemFactory factory : FS_FACTORIES.values()) { + for (FileSystemFactory factory : RAW_FACTORIES) { factory.configure(config); + String scheme = factory.getScheme(); + + FileSystemFactory fsf = ConnectionLimitingFactory.decorateIfLimited(factory, scheme, config); + FS_FACTORIES.put(scheme, fsf); } // configure the default (fallback) factory FALLBACK_FACTORY.configure(config); // also read the default file system scheme - final String stringifiedUri = config.getString(ConfigConstants.FILESYSTEM_SCHEME, null); + final String stringifiedUri = config.getString(CoreOptions.DEFAULT_FILESYSTEM_SCHEME, null); if (stringifiedUri == null) { DEFAULT_SCHEME = null; } @@ -269,7 +281,7 @@ public static void initialize(Configuration config) throws IOException, IllegalC } catch (URISyntaxException e) { throw new IllegalConfigurationException("The default file system scheme ('" + - ConfigConstants.FILESYSTEM_SCHEME + "') is invalid: " + stringifiedUri, e); + CoreOptions.DEFAULT_FILESYSTEM_SCHEME + "') is invalid: " + stringifiedUri, e); } } } @@ -368,6 +380,13 @@ public static FileSystem getUnguardedFileSystem(final URI fsUri) throws IOExcept } } + // this "default" initialization makes sure that the FileSystem class works + // even when not configured with an explicit Flink configuration, like on + // JobManager or TaskManager setup + if (FS_FACTORIES.isEmpty()) { + initialize(new Configuration()); + } + // Try to create a new file system final FileSystem fs; final FileSystemFactory factory = FS_FACTORIES.get(uri.getScheme()); @@ -907,11 +926,11 @@ public boolean initOutPathDistFS(Path outPath, WriteMode writeMode, boolean crea * * @return A map from the file system scheme to corresponding file system factory. */ - private static HashMap loadFileSystems() { - final HashMap map = new HashMap<>(); + private static List loadFileSystems() { + final ArrayList list = new ArrayList<>(); // by default, we always have the local file system factory - map.put("file", new LocalFileSystemFactory()); + list.add(new LocalFileSystemFactory()); LOG.debug("Loading extension file systems via services"); @@ -926,9 +945,8 @@ private static HashMap loadFileSystems() { while (iter.hasNext()) { try { FileSystemFactory factory = iter.next(); - String scheme = factory.getScheme(); - map.put(scheme, factory); - LOG.debug("Added file system {}:{}", scheme, factory.getClass().getName()); + list.add(factory); + LOG.debug("Added file system {}:{}", factory.getScheme(), factory.getClass().getName()); } catch (Throwable t) { // catching Throwable here to handle various forms of class loading @@ -945,7 +963,7 @@ private static HashMap loadFileSystems() { LOG.error("Failed to load additional file systems via services", t); } - return map; + return Collections.unmodifiableList(list); } /** diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java new file mode 100644 index 0000000000000..5353563494d90 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java @@ -0,0 +1,1114 @@ +/* + * 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.flink.core.fs; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.util.function.SupplierWithException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.io.Closeable; +import java.io.IOException; +import java.net.URI; +import java.util.HashSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A file system that limits the number of concurrently open input streams, + * output streams, and total streams for a target file system. + * + *

    This file system can wrap another existing file system in cases where + * the target file system cannot handle certain connection spikes and connections + * would fail in that case. This happens, for example, for very small HDFS clusters + * with few RPC handlers, when a large Flink job tries to build up many connections during + * a checkpoint. + * + *

    The filesystem may track the progress of streams and close streams that have been + * inactive for too long, to avoid locked streams of taking up the complete pool. + * Rather than having a dedicated reaper thread, the calls that try to open a new stream + * periodically check the currently open streams once the limit of open streams is reached. + */ +@Internal +public class LimitedConnectionsFileSystem extends FileSystem { + + private static final Logger LOG = LoggerFactory.getLogger(LimitedConnectionsFileSystem.class); + + /** The original file system to which connections are limited. */ + private final FileSystem originalFs; + + /** The lock that synchronizes connection bookkeeping. */ + private final ReentrantLock lock; + + /** Condition for threads that are blocking on the availability of new connections. */ + private final Condition available; + + /** The maximum number of concurrently open output streams. */ + private final int maxNumOpenOutputStreams; + + /** The maximum number of concurrently open input streams. */ + private final int maxNumOpenInputStreams; + + /** The maximum number of concurrently open streams (input + output). */ + private final int maxNumOpenStreamsTotal; + + /** The nanoseconds that a opening a stream may wait for availability. */ + private final long streamOpenTimeoutNanos; + + /** The nanoseconds that a stream may spend not writing any bytes before it is closed as inactive. */ + private final long streamInactivityTimeoutNanos; + + /** The set of currently open output streams. */ + @GuardedBy("lock") + private final HashSet openOutputStreams; + + /** The set of currently open input streams. */ + @GuardedBy("lock") + private final HashSet openInputStreams; + + /** The number of output streams reserved to be opened. */ + @GuardedBy("lock") + private int numReservedOutputStreams; + + /** The number of input streams reserved to be opened. */ + @GuardedBy("lock") + private int numReservedInputStreams; + + // ------------------------------------------------------------------------ + + /** + * Creates a new output connection limiting file system. + * + *

    If streams are inactive (meaning not writing bytes) for longer than the given timeout, + * then they are terminated as "inactive", to prevent that the limited number of connections gets + * stuck on only blocked threads. + * + * @param originalFs The original file system to which connections are limited. + * @param maxNumOpenStreamsTotal The maximum number of concurrent open streams (0 means no limit). + */ + public LimitedConnectionsFileSystem(FileSystem originalFs, int maxNumOpenStreamsTotal) { + this(originalFs, maxNumOpenStreamsTotal, 0, 0); + } + + /** + * Creates a new output connection limiting file system. + * + *

    If streams are inactive (meaning not writing bytes) for longer than the given timeout, + * then they are terminated as "inactive", to prevent that the limited number of connections gets + * stuck on only blocked threads. + * + * @param originalFs The original file system to which connections are limited. + * @param maxNumOpenStreamsTotal The maximum number of concurrent open streams (0 means no limit). + * @param streamOpenTimeout The maximum number of milliseconds that the file system will wait when + * no more connections are currently permitted. + * @param streamInactivityTimeout The milliseconds that a stream may spend not writing any + * bytes before it is closed as inactive. + */ + public LimitedConnectionsFileSystem( + FileSystem originalFs, + int maxNumOpenStreamsTotal, + long streamOpenTimeout, + long streamInactivityTimeout) { + this(originalFs, maxNumOpenStreamsTotal, 0, 0, streamOpenTimeout, streamInactivityTimeout); + } + + /** + * Creates a new output connection limiting file system, limiting input and output streams with + * potentially different quotas. + * + *

    If streams are inactive (meaning not writing bytes) for longer than the given timeout, + * then they are terminated as "inactive", to prevent that the limited number of connections gets + * stuck on only blocked threads. + * + * @param originalFs The original file system to which connections are limited. + * @param maxNumOpenStreamsTotal The maximum number of concurrent open streams (0 means no limit). + * @param maxNumOpenOutputStreams The maximum number of concurrent open output streams (0 means no limit). + * @param maxNumOpenInputStreams The maximum number of concurrent open input streams (0 means no limit). + * @param streamOpenTimeout The maximum number of milliseconds that the file system will wait when + * no more connections are currently permitted. + * @param streamInactivityTimeout The milliseconds that a stream may spend not writing any + * bytes before it is closed as inactive. + */ + public LimitedConnectionsFileSystem( + FileSystem originalFs, + int maxNumOpenStreamsTotal, + int maxNumOpenOutputStreams, + int maxNumOpenInputStreams, + long streamOpenTimeout, + long streamInactivityTimeout) { + + checkArgument(maxNumOpenStreamsTotal >= 0, "maxNumOpenStreamsTotal must be >= 0"); + checkArgument(maxNumOpenOutputStreams >= 0, "maxNumOpenOutputStreams must be >= 0"); + checkArgument(maxNumOpenInputStreams >= 0, "maxNumOpenInputStreams must be >= 0"); + checkArgument(streamOpenTimeout >= 0, "stream opening timeout must be >= 0 (0 means infinite timeout)"); + checkArgument(streamInactivityTimeout >= 0, "stream inactivity timeout must be >= 0 (0 means infinite timeout)"); + + this.originalFs = checkNotNull(originalFs, "originalFs"); + this.lock = new ReentrantLock(true); + this.available = lock.newCondition(); + this.openOutputStreams = new HashSet<>(); + this.openInputStreams = new HashSet<>(); + this.maxNumOpenStreamsTotal = maxNumOpenStreamsTotal; + this.maxNumOpenOutputStreams = maxNumOpenOutputStreams; + this.maxNumOpenInputStreams = maxNumOpenInputStreams; + + // assign nanos overflow aware + final long openTimeoutNanos = streamOpenTimeout * 1_000_000; + final long inactivityTimeoutNanos = streamInactivityTimeout * 1_000_000; + + this.streamOpenTimeoutNanos = + openTimeoutNanos >= streamOpenTimeout ? openTimeoutNanos : Long.MAX_VALUE; + + this.streamInactivityTimeoutNanos = + inactivityTimeoutNanos >= streamInactivityTimeout ? inactivityTimeoutNanos : Long.MAX_VALUE; + } + + // ------------------------------------------------------------------------ + + /** + * Gets the maximum number of concurrently open output streams. + */ + public int getMaxNumOpenOutputStreams() { + return maxNumOpenOutputStreams; + } + + /** + * Gets the maximum number of concurrently open input streams. + */ + public int getMaxNumOpenInputStreams() { + return maxNumOpenInputStreams; + } + + /** + * Gets the maximum number of concurrently open streams (input + output). + */ + public int getMaxNumOpenStreamsTotal() { + return maxNumOpenStreamsTotal; + } + + /** + * Gets the number of milliseconds that a opening a stream may wait for availability in the + * connection pool. + */ + public long getStreamOpenTimeout() { + return streamOpenTimeoutNanos / 1_000_000; + } + + /** + * Gets the milliseconds that a stream may spend not writing any bytes before it is closed as inactive. + */ + public long getStreamInactivityTimeout() { + return streamInactivityTimeoutNanos / 1_000_000; + } + + /** + * Gets the total number of open streams (input plus output). + */ + public int getTotalNumberOfOpenStreams() { + lock.lock(); + try { + return numReservedOutputStreams + numReservedInputStreams; + } finally { + lock.unlock(); + } + } + + /** + * Gets the number of currently open output streams. + */ + public int getNumberOfOpenOutputStreams() { + lock.lock(); + try { + return numReservedOutputStreams; + } + finally { + lock.unlock(); + } + } + + /** + * Gets the number of currently open input streams. + */ + public int getNumberOfOpenInputStreams() { + return numReservedInputStreams; + } + + // ------------------------------------------------------------------------ + // input & output stream opening methods + // ------------------------------------------------------------------------ + + @Override + public FSDataOutputStream create(Path f, WriteMode overwriteMode) throws IOException { + return createOutputStream(() -> originalFs.create(f, overwriteMode)); + } + + @Override + @Deprecated + @SuppressWarnings("deprecation") + public FSDataOutputStream create( + Path f, + boolean overwrite, + int bufferSize, + short replication, + long blockSize) throws IOException { + + return createOutputStream(() -> originalFs.create(f, overwrite, bufferSize, replication, blockSize)); + } + + @Override + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + return createInputStream(() -> originalFs.open(f, bufferSize)); + } + + @Override + public FSDataInputStream open(Path f) throws IOException { + return createInputStream(() -> originalFs.open(f)); + } + + private FSDataOutputStream createOutputStream( + final SupplierWithException streamOpener) throws IOException { + + final SupplierWithException wrappedStreamOpener = + () -> new OutStream(streamOpener.get(), this); + + return createStream(wrappedStreamOpener, openOutputStreams, true); + } + + private FSDataInputStream createInputStream( + final SupplierWithException streamOpener) throws IOException { + + final SupplierWithException wrappedStreamOpener = + () -> new InStream(streamOpener.get(), this); + + return createStream(wrappedStreamOpener, openInputStreams, false); + } + + // ------------------------------------------------------------------------ + // other delegating file system methods + // ------------------------------------------------------------------------ + + @Override + public FileSystemKind getKind() { + return originalFs.getKind(); + } + + @Override + public boolean isDistributedFS() { + return originalFs.isDistributedFS(); + } + + @Override + public Path getWorkingDirectory() { + return originalFs.getWorkingDirectory(); + } + + @Override + public Path getHomeDirectory() { + return originalFs.getHomeDirectory(); + } + + @Override + public URI getUri() { + return originalFs.getUri(); + } + + @Override + public FileStatus getFileStatus(Path f) throws IOException { + return originalFs.getFileStatus(f); + } + + @Override + public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException { + return originalFs.getFileBlockLocations(file, start, len); + } + + @Override + public FileStatus[] listStatus(Path f) throws IOException { + return originalFs.listStatus(f); + } + + @Override + public boolean delete(Path f, boolean recursive) throws IOException { + return originalFs.delete(f, recursive); + } + + @Override + public boolean mkdirs(Path f) throws IOException { + return originalFs.mkdirs(f); + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + return originalFs.rename(src, dst); + } + + @Override + public boolean exists(Path f) throws IOException { + return originalFs.exists(f); + } + + @Override + @Deprecated + @SuppressWarnings("deprecation") + public long getDefaultBlockSize() { + return originalFs.getDefaultBlockSize(); + } + + // ------------------------------------------------------------------------ + + private T createStream( + final SupplierWithException streamOpener, + final HashSet openStreams, + final boolean output) throws IOException { + + final int outputLimit = output && maxNumOpenInputStreams > 0 ? maxNumOpenOutputStreams : Integer.MAX_VALUE; + final int inputLimit = !output && maxNumOpenInputStreams > 0 ? maxNumOpenInputStreams : Integer.MAX_VALUE; + final int totalLimit = maxNumOpenStreamsTotal > 0 ? maxNumOpenStreamsTotal : Integer.MAX_VALUE; + final int outputCredit = output ? 1 : 0; + final int inputCredit = output ? 0 : 1; + + // because waiting for availability may take long, we need to be interruptible here + // and handle interrupted exceptions as I/O errors + // even though the code is written to make sure the lock is held for a short time only, + // making the lock acquisition interruptible helps to guard against the cases where + // a supposedly fast operation (like 'getPos()' on a stream) actually takes long. + try { + lock.lockInterruptibly(); + try { + // some integrity checks + assert openOutputStreams.size() <= numReservedOutputStreams; + assert openInputStreams.size() <= numReservedInputStreams; + + // wait until there are few enough streams so we can open another + waitForAvailability(totalLimit, outputLimit, inputLimit); + + // We do not open the stream here in the locked scope because opening a stream + // could take a while. Holding the lock during that operation would block all concurrent + // attempts to try and open a stream, effectively serializing all calls to open the streams. + numReservedOutputStreams += outputCredit; + numReservedInputStreams += inputCredit; + } + finally { + lock.unlock(); + } + } + catch (InterruptedException e) { + // restore interruption flag + Thread.currentThread().interrupt(); + throw new IOException("interrupted before opening stream"); + } + + // open the stream outside the lock. + boolean success = false; + try { + final T out = streamOpener.get(); + + // add the stream to the set, need to re-acquire the lock + lock.lock(); + try { + openStreams.add(out); + } finally { + lock.unlock(); + } + + // good, can now return cleanly + success = true; + return out; + } + finally { + if (!success) { + // remove the reserved credit + // we must open this non-interruptibly, because this must succeed! + lock.lock(); + try { + numReservedOutputStreams -= outputCredit; + numReservedInputStreams -= inputCredit; + available.signalAll(); + } finally { + lock.unlock(); + } + } + } + } + + @GuardedBy("lock") + private void waitForAvailability( + int totalLimit, + int outputLimit, + int inputLimit) throws InterruptedException, IOException { + + checkState(lock.isHeldByCurrentThread()); + + // compute the deadline of this operations + final long deadline; + if (streamOpenTimeoutNanos == 0) { + deadline = Long.MAX_VALUE; + } else { + long deadlineNanos = System.nanoTime() + streamOpenTimeoutNanos; + // check for overflow + deadline = deadlineNanos > 0 ? deadlineNanos : Long.MAX_VALUE; + } + + // wait for available connections + long timeLeft; + + if (streamInactivityTimeoutNanos == 0) { + // simple case: just wait + while ((timeLeft = (deadline - System.nanoTime())) > 0 && + !hasAvailability(totalLimit, outputLimit, inputLimit)) { + + available.await(timeLeft, TimeUnit.NANOSECONDS); + } + } + else { + // complex case: chase down inactive streams + final long checkIntervalNanos = (streamInactivityTimeoutNanos >>> 1) + 1; + + long now; + while ((timeLeft = (deadline - (now = System.nanoTime()))) > 0 && // while still within timeout + !hasAvailability(totalLimit, outputLimit, inputLimit)) { + + // check all streams whether there in one that has been inactive for too long + if (!(closeInactiveStream(openOutputStreams, now) || closeInactiveStream(openInputStreams, now))) { + // only wait if we did not manage to close any stream. + // otherwise eagerly check again if we have availability now (we should have!) + long timeToWait = Math.min(checkIntervalNanos, timeLeft); + available.await(timeToWait, TimeUnit.NANOSECONDS); + } + } + } + + // check for timeout + // we check availability again to catch cases where the timeout expired while waiting + // to re-acquire the lock + if (timeLeft <= 0 && !hasAvailability(totalLimit, outputLimit, inputLimit)) { + throw new IOException(String.format( + "Timeout while waiting for an available stream/connection. " + + "limits: total=%d, input=%d, output=%d ; Open: input=%d, output=%d ; timeout: %d ms", + maxNumOpenStreamsTotal, maxNumOpenInputStreams, maxNumOpenOutputStreams, + numReservedInputStreams, numReservedOutputStreams, getStreamOpenTimeout())); + } + } + + @GuardedBy("lock") + private boolean hasAvailability(int totalLimit, int outputLimit, int inputLimit) { + return numReservedOutputStreams < outputLimit && + numReservedInputStreams < inputLimit && + numReservedOutputStreams + numReservedInputStreams < totalLimit; + } + + @GuardedBy("lock") + private boolean closeInactiveStream(HashSet streams, long nowNanos) { + for (StreamWithTimeout stream : streams) { + try { + final StreamProgressTracker tracker = stream.getProgressTracker(); + + // If the stream is closed already, it will be removed anyways, so we + // do not classify it as inactive. We also skip the check if another check happened too recently. + if (stream.isClosed() || nowNanos < tracker.getLastCheckTimestampNanos() + streamInactivityTimeoutNanos) { + // interval since last check not yet over + return false; + } + else if (!tracker.checkNewBytesAndMark(nowNanos)) { + stream.closeDueToTimeout(); + return true; + } + } + catch (StreamTimeoutException ignored) { + // may happen due to races + } + catch (IOException e) { + // only log on debug level here, to avoid log spamming + LOG.debug("Could not check for stream progress to determine inactivity", e); + } + } + + return false; + } + + // ------------------------------------------------------------------------ + + /** + * Atomically removes the given output stream from the set of currently open output streams, + * and signals that new stream can now be opened. + */ + void unregisterOutputStream(OutStream stream) { + lock.lock(); + try { + // only decrement if we actually remove the stream + if (openOutputStreams.remove(stream)) { + numReservedOutputStreams--; + available.signalAll(); + } + } + finally { + lock.unlock(); + } + } + + /** + * Atomically removes the given input stream from the set of currently open input streams, + * and signals that new stream can now be opened. + */ + void unregisterInputStream(InStream stream) { + lock.lock(); + try { + // only decrement if we actually remove the stream + if (openInputStreams.remove(stream)) { + numReservedInputStreams--; + available.signalAll(); + } + } + finally { + lock.unlock(); + } + } + + // ------------------------------------------------------------------------ + + /** + * A special IOException, indicating a timeout in the data output stream. + */ + public static final class StreamTimeoutException extends IOException { + + private static final long serialVersionUID = -8790922066795901928L; + + public StreamTimeoutException() { + super("Stream closed due to inactivity timeout. " + + "This is done to prevent inactive streams from blocking the full " + + "pool of limited connections"); + } + + public StreamTimeoutException(StreamTimeoutException other) { + super(other.getMessage(), other); + } + } + + // ------------------------------------------------------------------------ + + /** + * Interface for streams that can be checked for inactivity. + */ + private interface StreamWithTimeout extends Closeable { + + /** + * Gets the progress tracker for this stream. + */ + StreamProgressTracker getProgressTracker(); + + /** + * Gets the current position in the stream, as in number of bytes read or written. + */ + long getPos() throws IOException; + + /** + * Closes the stream asynchronously with a special exception that indicates closing due + * to lack of progress. + */ + void closeDueToTimeout() throws IOException; + + /** + * Checks whether the stream was closed already. + */ + boolean isClosed(); + } + + // ------------------------------------------------------------------------ + + /** + * A tracker for stream progress. This records the number of bytes read / written together + * with a timestamp when the last check happened. + */ + private static final class StreamProgressTracker { + + /** The tracked stream. */ + private final StreamWithTimeout stream; + + /** The number of bytes written the last time that the {@link #checkNewBytesAndMark(long)} + * method was called. It is important to initialize this with {@code -1} so that the + * first check (0 bytes) always appears to have made progress. */ + private volatile long lastCheckBytes = -1; + + /** The timestamp when the last inactivity evaluation was made. */ + private volatile long lastCheckTimestampNanos; + + StreamProgressTracker(StreamWithTimeout stream) { + this.stream = stream; + } + + /** + * Gets the timestamp when the last inactivity evaluation was made. + */ + public long getLastCheckTimestampNanos() { + return lastCheckTimestampNanos; + } + + /** + * Checks whether there were new bytes since the last time this method was invoked. + * This also sets the given timestamp, to be read via {@link #getLastCheckTimestampNanos()}. + * + * @return True, if there were new bytes, false if not. + */ + public boolean checkNewBytesAndMark(long timestamp) throws IOException { + // remember the time when checked + lastCheckTimestampNanos = timestamp; + + final long bytesNow = stream.getPos(); + if (bytesNow > lastCheckBytes) { + lastCheckBytes = bytesNow; + return true; + } + else { + return false; + } + } + } + + // ------------------------------------------------------------------------ + + /** + * A data output stream that wraps a given data output stream and un-registers + * from a given connection-limiting file system + * (via {@link LimitedConnectionsFileSystem#unregisterOutputStream(OutStream)} + * upon closing. + */ + private static final class OutStream extends FSDataOutputStream implements StreamWithTimeout { + + /** The original data output stream to write to. */ + private final FSDataOutputStream originalStream; + + /** The connection-limiting file system to un-register from. */ + private final LimitedConnectionsFileSystem fs; + + /** The progress tracker for this stream. */ + private final StreamProgressTracker progressTracker; + + /** An exception with which the stream has been externally closed. */ + private volatile StreamTimeoutException timeoutException; + + /** Flag tracking whether the stream was already closed, for proper inactivity tracking. */ + private AtomicBoolean closed = new AtomicBoolean(); + + OutStream(FSDataOutputStream originalStream, LimitedConnectionsFileSystem fs) { + this.originalStream = checkNotNull(originalStream); + this.fs = checkNotNull(fs); + this.progressTracker = new StreamProgressTracker(this); + } + + // --- FSDataOutputStream API implementation + + @Override + public void write(int b) throws IOException { + try { + originalStream.write(b); + } + catch (IOException e) { + handleIOException(e); + } + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + try { + originalStream.write(b, off, len); + } + catch (IOException e) { + handleIOException(e); + } + } + + @Override + public long getPos() throws IOException { + try { + return originalStream.getPos(); + } + catch (IOException e) { + handleIOException(e); + return -1; // silence the compiler + } + } + + @Override + public void flush() throws IOException { + try { + originalStream.flush(); + } + catch (IOException e) { + handleIOException(e); + } + } + + @Override + public void sync() throws IOException { + try { + originalStream.sync(); + } + catch (IOException e) { + handleIOException(e); + } + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + try { + originalStream.close(); + } + catch (IOException e) { + handleIOException(e); + } + finally { + fs.unregisterOutputStream(this); + } + } + } + + @Override + public void closeDueToTimeout() throws IOException { + this.timeoutException = new StreamTimeoutException(); + close(); + } + + @Override + public boolean isClosed() { + return closed.get(); + } + + @Override + public StreamProgressTracker getProgressTracker() { + return progressTracker; + } + + private void handleIOException(IOException exception) throws IOException { + if (timeoutException == null) { + throw exception; + } else { + // throw a new exception to capture this call's stack trace + // the new exception is forwarded as a suppressed exception + StreamTimeoutException te = new StreamTimeoutException(timeoutException); + te.addSuppressed(exception); + throw te; + } + } + } + + /** + * A data input stream that wraps a given data input stream and un-registers + * from a given connection-limiting file system + * (via {@link LimitedConnectionsFileSystem#unregisterInputStream(InStream)} + * upon closing. + */ + private static final class InStream extends FSDataInputStream implements StreamWithTimeout { + + /** The original data input stream to read from. */ + private final FSDataInputStream originalStream; + + /** The connection-limiting file system to un-register from. */ + private final LimitedConnectionsFileSystem fs; + + /** An exception with which the stream has been externally closed. */ + private volatile StreamTimeoutException timeoutException; + + /** The progress tracker for this stream. */ + private final StreamProgressTracker progressTracker; + + /** Flag tracking whether the stream was already closed, for proper inactivity tracking. */ + private AtomicBoolean closed = new AtomicBoolean(); + + InStream(FSDataInputStream originalStream, LimitedConnectionsFileSystem fs) { + this.originalStream = checkNotNull(originalStream); + this.fs = checkNotNull(fs); + this.progressTracker = new StreamProgressTracker(this); + } + + // --- FSDataOutputStream API implementation + + @Override + public int read() throws IOException { + try { + return originalStream.read(); + } + catch (IOException e) { + handleIOException(e); + return 0; // silence the compiler + } + } + + @Override + public int read(byte[] b) throws IOException { + try { + return originalStream.read(b); + } + catch (IOException e) { + handleIOException(e); + return 0; // silence the compiler + } + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + try { + return originalStream.read(b, off, len); + } + catch (IOException e) { + handleIOException(e); + return 0; // silence the compiler + } + } + + @Override + public long skip(long n) throws IOException { + try { + return originalStream.skip(n); + } + catch (IOException e) { + handleIOException(e); + return 0L; // silence the compiler + } + } + + @Override + public int available() throws IOException { + try { + return originalStream.available(); + } + catch (IOException e) { + handleIOException(e); + return 0; // silence the compiler + } + } + + @Override + public void mark(int readlimit) { + originalStream.mark(readlimit); + } + + @Override + public void reset() throws IOException { + try { + originalStream.reset(); + } + catch (IOException e) { + handleIOException(e); + } + } + + @Override + public boolean markSupported() { + return originalStream.markSupported(); + } + + @Override + public void seek(long desired) throws IOException { + try { + originalStream.seek(desired); + } + catch (IOException e) { + handleIOException(e); + } + } + + @Override + public long getPos() throws IOException { + try { + return originalStream.getPos(); + } + catch (IOException e) { + handleIOException(e); + return 0; // silence the compiler + } + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + try { + originalStream.close(); + } + catch (IOException e) { + handleIOException(e); + } + finally { + fs.unregisterInputStream(this); + } + } + } + + @Override + public void closeDueToTimeout() throws IOException { + this.timeoutException = new StreamTimeoutException(); + close(); + } + + @Override + public boolean isClosed() { + return closed.get(); + } + + @Override + public StreamProgressTracker getProgressTracker() { + return progressTracker; + } + + private void handleIOException(IOException exception) throws IOException { + if (timeoutException == null) { + throw exception; + } else { + // throw a new exception to capture this call's stack trace + // the new exception is forwarded as a suppressed exception + StreamTimeoutException te = new StreamTimeoutException(timeoutException); + te.addSuppressed(exception); + throw te; + } + } + } + + // ------------------------------------------------------------------------ + + /** + * A simple configuration data object capturing the settings for limited connections. + */ + public static class ConnectionLimitingSettings { + + /** The limit for the total number of connections, or 0, if no limit. */ + public final int limitTotal; + + /** The limit for the number of input stream connections, or 0, if no limit. */ + public final int limitInput; + + /** The limit for the number of output stream connections, or 0, if no limit. */ + public final int limitOutput; + + /** The stream opening timeout for a stream, in milliseconds. */ + public final long streamOpenTimeout; + + /** The inactivity timeout for a stream, in milliseconds. */ + public final long streamInactivityTimeout; + + /** + * Creates a new ConnectionLimitingSettings with the given parameters. + * + * @param limitTotal The limit for the total number of connections, or 0, if no limit. + * @param limitInput The limit for the number of input stream connections, or 0, if no limit. + * @param limitOutput The limit for the number of output stream connections, or 0, if no limit. + * @param streamOpenTimeout The maximum number of milliseconds that the file system will wait when + * no more connections are currently permitted. + * @param streamInactivityTimeout The milliseconds that a stream may spend not writing any + * bytes before it is closed as inactive. + */ + public ConnectionLimitingSettings( + int limitTotal, + int limitInput, + int limitOutput, + long streamOpenTimeout, + long streamInactivityTimeout) { + checkArgument(limitTotal >= 0); + checkArgument(limitInput >= 0); + checkArgument(limitOutput >= 0); + checkArgument(streamOpenTimeout >= 0); + checkArgument(streamInactivityTimeout >= 0); + + this.limitTotal = limitTotal; + this.limitInput = limitInput; + this.limitOutput = limitOutput; + this.streamOpenTimeout = streamOpenTimeout; + this.streamInactivityTimeout = streamInactivityTimeout; + } + + // -------------------------------------------------------------------- + + /** + * Parses and returns the settings for connection limiting, for the file system with + * the given file system scheme. + * + * @param config The configuration to check. + * @param fsScheme The file system scheme. + * + * @return The parsed configuration, or null, if no connection limiting is configured. + */ + @Nullable + public static ConnectionLimitingSettings fromConfig(Configuration config, String fsScheme) { + checkNotNull(fsScheme, "fsScheme"); + checkNotNull(config, "config"); + + final ConfigOption totalLimitOption = CoreOptions.fileSystemConnectionLimit(fsScheme); + final ConfigOption limitInOption = CoreOptions.fileSystemConnectionLimitIn(fsScheme); + final ConfigOption limitOutOption = CoreOptions.fileSystemConnectionLimitOut(fsScheme); + + final int totalLimit = config.getInteger(totalLimitOption); + final int limitIn = config.getInteger(limitInOption); + final int limitOut = config.getInteger(limitOutOption); + + checkLimit(totalLimit, totalLimitOption); + checkLimit(limitIn, limitInOption); + checkLimit(limitOut, limitOutOption); + + // create the settings only, if at least one limit is configured + if (totalLimit <= 0 || limitIn <= 0 || limitOut <= 0) { + // no limit configured + return null; + } + else { + final ConfigOption openTimeoutOption = + CoreOptions.fileSystemConnectionLimitTimeout(fsScheme); + final ConfigOption inactivityTimeoutOption = + CoreOptions.fileSystemConnectionLimitStreamInactivityTimeout(fsScheme); + + final long openTimeout = config.getLong(openTimeoutOption); + final long inactivityTimeout = config.getLong(inactivityTimeoutOption); + + checkTimeout(openTimeout, openTimeoutOption); + checkTimeout(inactivityTimeout, inactivityTimeoutOption); + + return new ConnectionLimitingSettings( + totalLimit == -1 ? 0 : totalLimit, + limitIn == -1 ? 0 : limitIn, + limitOut == -1 ? 0 : limitOut, + openTimeout, + inactivityTimeout); + } + } + + private static void checkLimit(int value, ConfigOption option) { + if (value < -1) { + throw new IllegalConfigurationException("Invalid value for '" + option.key() + "': " + value); + } + } + + private static void checkTimeout(long timeout, ConfigOption option) { + if (timeout < 0) { + throw new IllegalConfigurationException("Invalid value for '" + option.key() + "': " + timeout); + } + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystemFactory.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystemFactory.java index 7cbc2bd70ead4..785391a2ad255 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystemFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystemFactory.java @@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystemFactory; -import org.apache.flink.core.fs.local.LocalFileSystem; import java.net.URI; diff --git a/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java b/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java new file mode 100644 index 0000000000000..63be9bf4a091c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/function/SupplierWithException.java @@ -0,0 +1,38 @@ +/* + * 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.flink.util.function; + +/** + * A functional interface for a {@link java.util.function.Supplier} that may + * throw exceptions. + * + * @param The type of the result of the supplier. + * @param The type of Exceptions thrown by this function. + */ +@FunctionalInterface +public interface SupplierWithException { + + /** + * Gets the result of this supplier. + * + * @return The result of thus supplier. + * @throws E This function may throw an exception. + */ + R get() throws E; +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java b/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java index 43c79c1c35c22..1cbc8f2045fc6 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/FilesystemSchemeConfigTest.java @@ -61,7 +61,7 @@ public void testDefaultsToLocal() throws Exception { @Test public void testExplicitlySetToLocal() throws Exception { final Configuration conf = new Configuration(); - conf.setString(ConfigConstants.FILESYSTEM_SCHEME, LocalFileSystem.getLocalFsURI().toString()); + conf.setString(CoreOptions.DEFAULT_FILESYSTEM_SCHEME, LocalFileSystem.getLocalFsURI().toString()); FileSystem.initialize(conf); URI justPath = new URI(tempFolder.newFile().toURI().getPath()); @@ -74,7 +74,7 @@ public void testExplicitlySetToLocal() throws Exception { @Test public void testExplicitlySetToOther() throws Exception { final Configuration conf = new Configuration(); - conf.setString(ConfigConstants.FILESYSTEM_SCHEME, "otherFS://localhost:1234/"); + conf.setString(CoreOptions.DEFAULT_FILESYSTEM_SCHEME, "otherFS://localhost:1234/"); FileSystem.initialize(conf); URI justPath = new URI(tempFolder.newFile().toURI().getPath()); @@ -92,7 +92,7 @@ public void testExplicitlySetToOther() throws Exception { @Test public void testExplicitlyPathTakesPrecedence() throws Exception { final Configuration conf = new Configuration(); - conf.setString(ConfigConstants.FILESYSTEM_SCHEME, "otherFS://localhost:1234/"); + conf.setString(CoreOptions.DEFAULT_FILESYSTEM_SCHEME, "otherFS://localhost:1234/"); FileSystem.initialize(conf); URI pathAndScheme = tempFolder.newFile().toURI(); diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java new file mode 100644 index 0000000000000..4742a7e063031 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java @@ -0,0 +1,84 @@ +/* + * 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.flink.core.fs; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.testutils.TestFileSystem; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.net.URI; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests that validate that the configuration for limited FS + * connections are properly picked up. + */ +public class LimitedConnectionsConfigurationTest { + + @Rule + public final TemporaryFolder tempDir = new TemporaryFolder(); + + @Test + public void testConfiguration() throws Exception { + final String fsScheme = TestFileSystem.SCHEME; + + // nothing configured, we should get a regular file system + FileSystem schemeFs = FileSystem.get(URI.create(fsScheme + ":///a/b/c")); + FileSystem localFs = FileSystem.get(tempDir.newFile().toURI()); + + assertFalse(schemeFs instanceof LimitedConnectionsFileSystem); + assertFalse(localFs instanceof LimitedConnectionsFileSystem); + + // configure some limits, which should cause "fsScheme" to be limited + + final Configuration config = new Configuration(); + config.setInteger("fs." + fsScheme + ".limit.total", 42); + config.setInteger("fs." + fsScheme + ".limit.input", 11); + config.setInteger("fs." + fsScheme + ".limit.output", 40); + config.setInteger("fs." + fsScheme + ".limit.timeout", 12345); + config.setInteger("fs." + fsScheme + ".limit.stream-timeout", 98765); + + try { + FileSystem.initialize(config); + + schemeFs = FileSystem.get(URI.create(fsScheme + ":///a/b/c")); + localFs = FileSystem.get(tempDir.newFile().toURI()); + + assertTrue(schemeFs instanceof LimitedConnectionsFileSystem); + assertFalse(localFs instanceof LimitedConnectionsFileSystem); + + LimitedConnectionsFileSystem limitedFs = (LimitedConnectionsFileSystem) schemeFs; + assertEquals(42, limitedFs.getMaxNumOpenStreamsTotal()); + assertEquals(11, limitedFs.getMaxNumOpenInputStreams()); + assertEquals(40, limitedFs.getMaxNumOpenOutputStreams()); + assertEquals(12345, limitedFs.getStreamOpenTimeout()); + assertEquals(98765, limitedFs.getStreamInactivityTimeout()); + } + finally { + // clear all settings + FileSystem.initialize(new Configuration()); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java new file mode 100644 index 0000000000000..b13367788b4a6 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java @@ -0,0 +1,241 @@ +/* + * 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.flink.core.fs; + +import org.apache.flink.core.fs.FileSystem.WriteMode; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.Random; + +import static org.junit.Assert.assertEquals; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyShort; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests that the method delegation works properly the {@link LimitedConnectionsFileSystem} + * and its created input and output streams. + */ +public class LimitedConnectionsFileSystemDelegationTest { + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + @SuppressWarnings("deprecation") + public void testDelegateFsMethods() throws IOException { + final FileSystem fs = mock(FileSystem.class); + when(fs.open(any(Path.class))).thenReturn(mock(FSDataInputStream.class)); + when(fs.open(any(Path.class), anyInt())).thenReturn(mock(FSDataInputStream.class)); + when(fs.create(any(Path.class), anyBoolean())).thenReturn(mock(FSDataOutputStream.class)); + when(fs.create(any(Path.class), any(WriteMode.class))).thenReturn(mock(FSDataOutputStream.class)); + when(fs.create(any(Path.class), anyBoolean(), anyInt(), anyShort(), anyLong())).thenReturn(mock(FSDataOutputStream.class)); + + final LimitedConnectionsFileSystem lfs = new LimitedConnectionsFileSystem(fs, 1000); + final Random rnd = new Random(); + + lfs.isDistributedFS(); + verify(fs).isDistributedFS(); + + lfs.getWorkingDirectory(); + verify(fs).isDistributedFS(); + + lfs.getHomeDirectory(); + verify(fs).getHomeDirectory(); + + lfs.getUri(); + verify(fs).getUri(); + + { + Path path = mock(Path.class); + lfs.getFileStatus(path); + verify(fs).getFileStatus(path); + } + + { + FileStatus path = mock(FileStatus.class); + int pos = rnd.nextInt(); + int len = rnd.nextInt(); + lfs.getFileBlockLocations(path, pos, len); + verify(fs).getFileBlockLocations(path, pos, len); + } + + { + Path path = mock(Path.class); + int bufferSize = rnd.nextInt(); + lfs.open(path, bufferSize); + verify(fs).open(path, bufferSize); + } + + { + Path path = mock(Path.class); + lfs.open(path); + verify(fs).open(path); + } + + lfs.getDefaultBlockSize(); + verify(fs).getDefaultBlockSize(); + + { + Path path = mock(Path.class); + lfs.listStatus(path); + verify(fs).listStatus(path); + } + + { + Path path = mock(Path.class); + lfs.exists(path); + verify(fs).exists(path); + } + + { + Path path = mock(Path.class); + boolean recursive = rnd.nextBoolean(); + lfs.delete(path, recursive); + verify(fs).delete(path, recursive); + } + + { + Path path = mock(Path.class); + lfs.mkdirs(path); + verify(fs).mkdirs(path); + } + + { + Path path = mock(Path.class); + boolean overwrite = rnd.nextBoolean(); + int bufferSize = rnd.nextInt(); + short replication = (short) rnd.nextInt(); + long blockSize = rnd.nextInt(); + + lfs.create(path, overwrite, bufferSize, replication, blockSize); + verify(fs).create(path, overwrite, bufferSize, replication, blockSize); + } + + { + Path path = mock(Path.class); + WriteMode mode = rnd.nextBoolean() ? WriteMode.OVERWRITE : WriteMode.NO_OVERWRITE; + lfs.create(path, mode); + verify(fs).create(path, mode); + } + + { + Path path1 = mock(Path.class); + Path path2 = mock(Path.class); + lfs.rename(path1, path2); + verify(fs).rename(path1, path2); + } + + { + FileSystemKind kind = rnd.nextBoolean() ? FileSystemKind.FILE_SYSTEM : FileSystemKind.OBJECT_STORE; + when(fs.getKind()).thenReturn(kind); + assertEquals(kind, lfs.getKind()); + verify(fs).getKind(); + } + } + + @Test + public void testDelegateOutStreamMethods() throws IOException { + + // mock the output stream + final FSDataOutputStream mockOut = mock(FSDataOutputStream.class); + final long outPos = 46651L; + when(mockOut.getPos()).thenReturn(outPos); + + final FileSystem fs = mock(FileSystem.class); + when(fs.create(any(Path.class), any(WriteMode.class))).thenReturn(mockOut); + + final LimitedConnectionsFileSystem lfs = new LimitedConnectionsFileSystem(fs, 100); + final FSDataOutputStream out = lfs.create(mock(Path.class), WriteMode.OVERWRITE); + + // validate the output stream + + out.write(77); + verify(mockOut).write(77); + + { + byte[] bytes = new byte[1786]; + out.write(bytes, 100, 111); + verify(mockOut).write(bytes, 100, 111); + } + + assertEquals(outPos, out.getPos()); + + out.flush(); + verify(mockOut).flush(); + + out.sync(); + verify(mockOut).sync(); + + out.close(); + verify(mockOut).close(); + } + + @Test + public void testDelegateInStreamMethods() throws IOException { + // mock the input stream + final FSDataInputStream mockIn = mock(FSDataInputStream.class); + final int value = 93; + final int bytesRead = 11; + final long inPos = 93; + final int available = 17; + final boolean markSupported = true; + when(mockIn.read()).thenReturn(value); + when(mockIn.read(any(byte[].class), anyInt(), anyInt())).thenReturn(11); + when(mockIn.getPos()).thenReturn(inPos); + when(mockIn.available()).thenReturn(available); + when(mockIn.markSupported()).thenReturn(markSupported); + + final FileSystem fs = mock(FileSystem.class); + when(fs.open(any(Path.class))).thenReturn(mockIn); + + final LimitedConnectionsFileSystem lfs = new LimitedConnectionsFileSystem(fs, 100); + final FSDataInputStream in = lfs.open(mock(Path.class)); + + // validate the input stream + + assertEquals(value, in.read()); + assertEquals(bytesRead, in.read(new byte[11], 2, 5)); + + assertEquals(inPos, in.getPos()); + + in.seek(17876); + verify(mockIn).seek(17876); + + assertEquals(available, in.available()); + + assertEquals(markSupported, in.markSupported()); + + in.mark(9876); + verify(mockIn).mark(9876); + + in.close(); + verify(mockIn).close(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java new file mode 100644 index 0000000000000..509b4aefd6539 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java @@ -0,0 +1,742 @@ +/* + * 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.flink.core.fs; + +import org.apache.flink.core.fs.FileSystem.WriteMode; +import org.apache.flink.core.fs.local.LocalFileSystem; +import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.OneShotLatch; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link LimitedConnectionsFileSystem}. + */ +public class LimitedConnectionsFileSystemTest { + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testConstructionNumericOverflow() { + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + Integer.MAX_VALUE, // unlimited total + Integer.MAX_VALUE, // limited outgoing + Integer.MAX_VALUE, // unlimited incoming + Long.MAX_VALUE - 1, // long timeout, close to overflow + Long.MAX_VALUE - 1); // long timeout, close to overflow + + assertEquals(Integer.MAX_VALUE, limitedFs.getMaxNumOpenStreamsTotal()); + assertEquals(Integer.MAX_VALUE, limitedFs.getMaxNumOpenOutputStreams()); + assertEquals(Integer.MAX_VALUE, limitedFs.getMaxNumOpenInputStreams()); + + assertTrue(limitedFs.getStreamOpenTimeout() > 0); + assertTrue(limitedFs.getStreamInactivityTimeout() > 0); + } + + @Test + public void testLimitingOutputStreams() throws Exception { + final int maxConcurrentOpen = 2; + final int numThreads = 61; + + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + Integer.MAX_VALUE, // unlimited total + maxConcurrentOpen, // limited outgoing + Integer.MAX_VALUE, // unlimited incoming + 0, + 0); + + final WriterThread[] threads = new WriterThread[numThreads]; + for (int i = 0; i < numThreads; i++) { + Path path = new Path(tempFolder.newFile().toURI()); + threads[i] = new WriterThread(limitedFs, path, maxConcurrentOpen, Integer.MAX_VALUE); + } + + for (WriterThread t : threads) { + t.start(); + } + + for (WriterThread t : threads) { + t.sync(); + } + } + + @Test + public void testLimitingInputStreams() throws Exception { + final int maxConcurrentOpen = 2; + final int numThreads = 61; + + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + Integer.MAX_VALUE, // unlimited total + Integer.MAX_VALUE, // unlimited outgoing + maxConcurrentOpen, // limited incoming + 0, + 0); + + final Random rnd = new Random(); + + final ReaderThread[] threads = new ReaderThread[numThreads]; + for (int i = 0; i < numThreads; i++) { + File file = tempFolder.newFile(); + createRandomContents(file, rnd); + Path path = new Path(file.toURI()); + threads[i] = new ReaderThread(limitedFs, path, maxConcurrentOpen, Integer.MAX_VALUE); + } + + for (ReaderThread t : threads) { + t.start(); + } + + for (ReaderThread t : threads) { + t.sync(); + } + } + + @Test + public void testLimitingMixedStreams() throws Exception { + final int maxConcurrentOpen = 2; + final int numThreads = 61; + + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + maxConcurrentOpen); // limited total + + final Random rnd = new Random(); + + final CheckedThread[] threads = new CheckedThread[numThreads]; + for (int i = 0; i < numThreads; i++) { + File file = tempFolder.newFile(); + Path path = new Path(file.toURI()); + + if (rnd.nextBoolean()) { + // reader thread + createRandomContents(file, rnd); + threads[i] = new ReaderThread(limitedFs, path, Integer.MAX_VALUE, maxConcurrentOpen); + } + else { + threads[i] = new WriterThread(limitedFs, path, Integer.MAX_VALUE, maxConcurrentOpen); + } + } + + for (CheckedThread t : threads) { + t.start(); + } + + for (CheckedThread t : threads) { + t.sync(); + } + } + + @Test + public void testOpenTimeoutOutputStreams() throws Exception { + final long openTimeout = 50L; + final int maxConcurrentOpen = 2; + + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + maxConcurrentOpen, // limited total + openTimeout, // small opening timeout + 0L); // infinite inactivity timeout + + // create the threads that block all streams + final BlockingWriterThread[] threads = new BlockingWriterThread[maxConcurrentOpen]; + for (int i = 0; i < maxConcurrentOpen; i++) { + Path path = new Path(tempFolder.newFile().toURI()); + threads[i] = new BlockingWriterThread(limitedFs, path, Integer.MAX_VALUE, maxConcurrentOpen); + threads[i].start(); + } + + // wait until all are open + while (limitedFs.getTotalNumberOfOpenStreams() < maxConcurrentOpen) { + Thread.sleep(1); + } + + // try to open another thread + try { + limitedFs.create(new Path(tempFolder.newFile().toURI()), WriteMode.OVERWRITE); + fail("this should have timed out"); + } + catch (IOException e) { + // expected + } + + // clean shutdown + for (BlockingWriterThread t : threads) { + t.wakeup(); + t.sync(); + } + } + + @Test + public void testOpenTimeoutInputStreams() throws Exception { + final long openTimeout = 50L; + final int maxConcurrentOpen = 2; + + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + maxConcurrentOpen, // limited total + openTimeout, // small opening timeout + 0L); // infinite inactivity timeout + + // create the threads that block all streams + final Random rnd = new Random(); + final BlockingReaderThread[] threads = new BlockingReaderThread[maxConcurrentOpen]; + for (int i = 0; i < maxConcurrentOpen; i++) { + File file = tempFolder.newFile(); + createRandomContents(file, rnd); + Path path = new Path(file.toURI()); + threads[i] = new BlockingReaderThread(limitedFs, path, maxConcurrentOpen, Integer.MAX_VALUE); + threads[i].start(); + } + + // wait until all are open + while (limitedFs.getTotalNumberOfOpenStreams() < maxConcurrentOpen) { + Thread.sleep(1); + } + + // try to open another thread + File file = tempFolder.newFile(); + createRandomContents(file, rnd); + try { + limitedFs.open(new Path(file.toURI())); + fail("this should have timed out"); + } + catch (IOException e) { + // expected + } + + // clean shutdown + for (BlockingReaderThread t : threads) { + t.wakeup(); + t.sync(); + } + } + + @Test + public void testTerminateStalledOutputStreams() throws Exception { + final int maxConcurrentOpen = 2; + final int numThreads = 20; + + // this testing file system has a 50 ms stream inactivity timeout + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + Integer.MAX_VALUE, // no limit on total streams + maxConcurrentOpen, // limit on output streams + Integer.MAX_VALUE, // no limit on input streams + 0, + 50); // timeout of 50 ms + + final WriterThread[] threads = new WriterThread[numThreads]; + final BlockingWriterThread[] blockers = new BlockingWriterThread[numThreads]; + + for (int i = 0; i < numThreads; i++) { + Path path1 = new Path(tempFolder.newFile().toURI()); + Path path2 = new Path(tempFolder.newFile().toURI()); + + threads[i] = new WriterThread(limitedFs, path1, maxConcurrentOpen, Integer.MAX_VALUE); + blockers[i] = new BlockingWriterThread(limitedFs, path2, maxConcurrentOpen, Integer.MAX_VALUE); + } + + // start normal and blocker threads + for (int i = 0; i < numThreads; i++) { + blockers[i].start(); + threads[i].start(); + } + + // all normal threads need to be able to finish because + // the blockers eventually time out + for (WriterThread t : threads) { + try { + t.sync(); + } catch (LimitedConnectionsFileSystem.StreamTimeoutException e) { + // also the regular threads may occasionally get a timeout on + // slower test machines because we set very aggressive timeouts + // to reduce the test time + } + } + + // unblock all the blocking threads + for (BlockingThread t : blockers) { + t.wakeup(); + } + for (BlockingThread t : blockers) { + try { + t.sync(); + } + catch (LimitedConnectionsFileSystem.StreamTimeoutException ignored) {} + } + } + + @Test + public void testTerminateStalledInputStreams() throws Exception { + final int maxConcurrentOpen = 2; + final int numThreads = 20; + + // this testing file system has a 50 ms stream inactivity timeout + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + Integer.MAX_VALUE, // no limit on total streams + Integer.MAX_VALUE, // limit on output streams + maxConcurrentOpen, // no limit on input streams + 0, + 50); // timeout of 50 ms + + final Random rnd = new Random(); + + final ReaderThread[] threads = new ReaderThread[numThreads]; + final BlockingReaderThread[] blockers = new BlockingReaderThread[numThreads]; + + for (int i = 0; i < numThreads; i++) { + File file1 = tempFolder.newFile(); + File file2 = tempFolder.newFile(); + + createRandomContents(file1, rnd); + createRandomContents(file2, rnd); + + Path path1 = new Path(file1.toURI()); + Path path2 = new Path(file2.toURI()); + + threads[i] = new ReaderThread(limitedFs, path1, maxConcurrentOpen, Integer.MAX_VALUE); + blockers[i] = new BlockingReaderThread(limitedFs, path2, maxConcurrentOpen, Integer.MAX_VALUE); + } + + // start normal and blocker threads + for (int i = 0; i < numThreads; i++) { + blockers[i].start(); + threads[i].start(); + } + + // all normal threads need to be able to finish because + // the blockers eventually time out + for (ReaderThread t : threads) { + try { + t.sync(); + } catch (LimitedConnectionsFileSystem.StreamTimeoutException e) { + // also the regular threads may occasionally get a timeout on + // slower test machines because we set very aggressive timeouts + // to reduce the test time + } + } + + // unblock all the blocking threads + for (BlockingThread t : blockers) { + t.wakeup(); + } + for (BlockingThread t : blockers) { + try { + t.sync(); + } + catch (LimitedConnectionsFileSystem.StreamTimeoutException ignored) {} + } + } + + @Test + public void testTerminateStalledMixedStreams() throws Exception { + final int maxConcurrentOpen = 2; + final int numThreads = 20; + + final LimitedConnectionsFileSystem limitedFs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), + maxConcurrentOpen, // limited total + 0L, // no opening timeout + 50L); // inactivity timeout of 50 ms + + final Random rnd = new Random(); + + final CheckedThread[] threads = new CheckedThread[numThreads]; + final BlockingThread[] blockers = new BlockingThread[numThreads]; + + for (int i = 0; i < numThreads; i++) { + File file1 = tempFolder.newFile(); + File file2 = tempFolder.newFile(); + Path path1 = new Path(file1.toURI()); + Path path2 = new Path(file2.toURI()); + + if (rnd.nextBoolean()) { + createRandomContents(file1, rnd); + createRandomContents(file2, rnd); + threads[i] = new ReaderThread(limitedFs, path1, maxConcurrentOpen, Integer.MAX_VALUE); + blockers[i] = new BlockingReaderThread(limitedFs, path2, maxConcurrentOpen, Integer.MAX_VALUE); + } + else { + threads[i] = new WriterThread(limitedFs, path1, maxConcurrentOpen, Integer.MAX_VALUE); + blockers[i] = new BlockingWriterThread(limitedFs, path2, maxConcurrentOpen, Integer.MAX_VALUE); + } + } + + // start normal and blocker threads + for (int i = 0; i < numThreads; i++) { + blockers[i].start(); + threads[i].start(); + } + + // all normal threads need to be able to finish because + // the blockers eventually time out + for (CheckedThread t : threads) { + try { + t.sync(); + } catch (LimitedConnectionsFileSystem.StreamTimeoutException e) { + // also the regular threads may occasionally get a timeout on + // slower test machines because we set very aggressive timeouts + // to reduce the test time + } + } + + // unblock all the blocking threads + for (BlockingThread t : blockers) { + t.wakeup(); + } + for (BlockingThread t : blockers) { + try { + t.sync(); + } + catch (LimitedConnectionsFileSystem.StreamTimeoutException ignored) {} + } + } + + @Test + public void testFailingStreamsUnregister() throws Exception { + final LimitedConnectionsFileSystem fs = new LimitedConnectionsFileSystem(new FailFs(), 1); + + assertEquals(0, fs.getNumberOfOpenInputStreams()); + assertEquals(0, fs.getNumberOfOpenOutputStreams()); + assertEquals(0, fs.getTotalNumberOfOpenStreams()); + + try { + fs.open(new Path(tempFolder.newFile().toURI())); + fail("this is expected to fail with an exception"); + } catch (IOException e) { + // expected + } + + try { + fs.create(new Path(tempFolder.newFile().toURI()), WriteMode.NO_OVERWRITE); + fail("this is expected to fail with an exception"); + } catch (IOException e) { + // expected + } + + assertEquals(0, fs.getNumberOfOpenInputStreams()); + assertEquals(0, fs.getNumberOfOpenOutputStreams()); + assertEquals(0, fs.getTotalNumberOfOpenStreams()); + } + + /** + * Tests that a slowly written output stream is not accidentally closed too aggressively, due to + * a wrong initialization of the timestamps or bytes written that mark when the last progress was checked. + */ + @Test + public void testSlowOutputStreamNotClosed() throws Exception { + final LimitedConnectionsFileSystem fs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), 1, 0L, 1000L); + + // some competing threads + final Random rnd = new Random(); + final ReaderThread[] threads = new ReaderThread[10]; + for (int i = 0; i < threads.length; i++) { + File file = tempFolder.newFile(); + createRandomContents(file, rnd); + Path path = new Path(file.toURI()); + threads[i] = new ReaderThread(fs, path, 1, Integer.MAX_VALUE); + } + + // open the stream we test + try (FSDataOutputStream out = fs.create(new Path(tempFolder.newFile().toURI()), WriteMode.OVERWRITE)) { + + // start the other threads that will try to shoot this stream down + for (ReaderThread t : threads) { + t.start(); + } + + // read the stream slowly. + Thread.sleep(5); + for (int bytesLeft = 50; bytesLeft > 0; bytesLeft--) { + out.write(bytesLeft); + Thread.sleep(5); + } + } + + // wait for clean shutdown + for (ReaderThread t : threads) { + t.sync(); + } + } + + /** + * Tests that a slowly read stream is not accidentally closed too aggressively, due to + * a wrong initialization of the timestamps or bytes written that mark when the last progress was checked. + */ + @Test + public void testSlowInputStreamNotClosed() throws Exception { + final File file = tempFolder.newFile(); + createRandomContents(file, new Random(), 50); + + final LimitedConnectionsFileSystem fs = new LimitedConnectionsFileSystem( + LocalFileSystem.getSharedInstance(), 1, 0L, 1000L); + + // some competing threads + final WriterThread[] threads = new WriterThread[10]; + for (int i = 0; i < threads.length; i++) { + Path path = new Path(tempFolder.newFile().toURI()); + threads[i] = new WriterThread(fs, path, 1, Integer.MAX_VALUE); + } + + // open the stream we test + try (FSDataInputStream in = fs.open(new Path(file.toURI()))) { + + // start the other threads that will try to shoot this stream down + for (WriterThread t : threads) { + t.start(); + } + + // read the stream slowly. + Thread.sleep(5); + while (in.read() != -1) { + Thread.sleep(5); + } + } + + // wait for clean shutdown + for (WriterThread t : threads) { + t.sync(); + } + } + + // ------------------------------------------------------------------------ + // Utils + // ------------------------------------------------------------------------ + + private void createRandomContents(File file, Random rnd) throws IOException { + createRandomContents(file, rnd, rnd.nextInt(10000) + 1); + } + + private void createRandomContents(File file, Random rnd, int size) throws IOException { + final byte[] data = new byte[size]; + rnd.nextBytes(data); + + try (FileOutputStream fos = new FileOutputStream(file)) { + fos.write(data); + } + } + + // ------------------------------------------------------------------------ + // Testing threads + // ------------------------------------------------------------------------ + + private static final class WriterThread extends CheckedThread { + + private final LimitedConnectionsFileSystem fs; + + private final Path path; + + private final int maxConcurrentOutputStreams; + + private final int maxConcurrentStreamsTotal; + + WriterThread( + LimitedConnectionsFileSystem fs, + Path path, + int maxConcurrentOutputStreams, + int maxConcurrentStreamsTotal) { + + this.fs = fs; + this.path = path; + this.maxConcurrentOutputStreams = maxConcurrentOutputStreams; + this.maxConcurrentStreamsTotal = maxConcurrentStreamsTotal; + } + + @Override + public void go() throws Exception { + + try (FSDataOutputStream stream = fs.create(path, WriteMode.OVERWRITE)) { + assertTrue(fs.getNumberOfOpenOutputStreams() <= maxConcurrentOutputStreams); + assertTrue(fs.getTotalNumberOfOpenStreams() <= maxConcurrentStreamsTotal); + + final Random rnd = new Random(); + final byte[] data = new byte[rnd.nextInt(10000) + 1]; + rnd.nextBytes(data); + stream.write(data); + } + } + } + + private static final class ReaderThread extends CheckedThread { + + private final LimitedConnectionsFileSystem fs; + + private final Path path; + + private final int maxConcurrentInputStreams; + + private final int maxConcurrentStreamsTotal; + + ReaderThread( + LimitedConnectionsFileSystem fs, + Path path, + int maxConcurrentInputStreams, + int maxConcurrentStreamsTotal) { + + this.fs = fs; + this.path = path; + this.maxConcurrentInputStreams = maxConcurrentInputStreams; + this.maxConcurrentStreamsTotal = maxConcurrentStreamsTotal; + } + + @Override + public void go() throws Exception { + + try (FSDataInputStream stream = fs.open(path)) { + assertTrue(fs.getNumberOfOpenInputStreams() <= maxConcurrentInputStreams); + assertTrue(fs.getTotalNumberOfOpenStreams() <= maxConcurrentStreamsTotal); + + final byte[] readBuffer = new byte[4096]; + + //noinspection StatementWithEmptyBody + while (stream.read(readBuffer) != -1) {} + } + } + } + + private static abstract class BlockingThread extends CheckedThread { + + private final OneShotLatch waiter = new OneShotLatch(); + + public void waitTillWokenUp() throws InterruptedException { + waiter.await(); + } + + public void wakeup() { + waiter.trigger(); + } + } + + private static final class BlockingWriterThread extends BlockingThread { + + private final LimitedConnectionsFileSystem fs; + + private final Path path; + + private final int maxConcurrentOutputStreams; + + private final int maxConcurrentStreamsTotal; + + BlockingWriterThread( + LimitedConnectionsFileSystem fs, + Path path, + int maxConcurrentOutputStreams, + int maxConcurrentStreamsTotal) { + + this.fs = fs; + this.path = path; + this.maxConcurrentOutputStreams = maxConcurrentOutputStreams; + this.maxConcurrentStreamsTotal = maxConcurrentStreamsTotal; + } + + @Override + public void go() throws Exception { + + try (FSDataOutputStream stream = fs.create(path, WriteMode.OVERWRITE)) { + assertTrue(fs.getNumberOfOpenOutputStreams() <= maxConcurrentOutputStreams); + assertTrue(fs.getTotalNumberOfOpenStreams() <= maxConcurrentStreamsTotal); + + final Random rnd = new Random(); + final byte[] data = new byte[rnd.nextInt(10000) + 1]; + rnd.nextBytes(data); + stream.write(data); + + waitTillWokenUp(); + + // try to write one more thing, which might/should fail with an I/O exception + stream.write(rnd.nextInt()); + } + } + } + + private static final class BlockingReaderThread extends BlockingThread { + + private final LimitedConnectionsFileSystem fs; + + private final Path path; + + private final int maxConcurrentInputStreams; + + private final int maxConcurrentStreamsTotal; + + BlockingReaderThread( + LimitedConnectionsFileSystem fs, + Path path, + int maxConcurrentInputStreams, + int maxConcurrentStreamsTotal) { + + this.fs = fs; + this.path = path; + this.maxConcurrentInputStreams = maxConcurrentInputStreams; + this.maxConcurrentStreamsTotal = maxConcurrentStreamsTotal; + } + + @Override + public void go() throws Exception { + + try (FSDataInputStream stream = fs.open(path)) { + assertTrue(fs.getNumberOfOpenInputStreams() <= maxConcurrentInputStreams); + assertTrue(fs.getTotalNumberOfOpenStreams() <= maxConcurrentStreamsTotal); + + final byte[] readBuffer = new byte[(int) fs.getFileStatus(path).getLen() - 1]; + assertTrue(stream.read(readBuffer) != -1); + + waitTillWokenUp(); + + // try to write one more thing, which might/should fail with an I/O exception + //noinspection ResultOfMethodCallIgnored + stream.read(); + } + } + } + + // ------------------------------------------------------------------------ + // failing file system + // ------------------------------------------------------------------------ + + private static class FailFs extends LocalFileSystem { + + @Override + public FSDataOutputStream create(Path filePath, WriteMode overwrite) throws IOException { + throw new IOException("test exception"); + } + + @Override + public FSDataInputStream open(Path f) throws IOException { + throw new IOException("test exception"); + } + } +} diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactory.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactory.java index 50e64e146cedb..2444c6534df1c 100644 --- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactory.java +++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactory.java @@ -19,7 +19,10 @@ package org.apache.flink.runtime.fs.hdfs; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystemFactory; +import org.apache.flink.core.fs.LimitedConnectionsFileSystem; +import org.apache.flink.core.fs.LimitedConnectionsFileSystem.ConnectionLimitingSettings; import org.apache.flink.core.fs.UnsupportedFileSystemSchemeException; import org.apache.flink.runtime.util.HadoopUtils; @@ -63,7 +66,7 @@ public void configure(Configuration config) { } @Override - public HadoopFileSystem create(URI fsUri) throws IOException { + public FileSystem create(URI fsUri) throws IOException { checkNotNull(fsUri, "fsUri"); final String scheme = fsUri.getScheme(); @@ -162,8 +165,15 @@ else if (flinkConfig != null) { throw new IOException(message, e); } - // all good, return the file system - return new HadoopFileSystem(hadoopFs); + HadoopFileSystem fs = new HadoopFileSystem(hadoopFs); + + // create the Flink file system, optionally limiting the open connections + if (flinkConfig != null) { + return limitIfConfigured(fs, scheme, flinkConfig); + } + else { + return fs; + } } catch (ReflectiveOperationException | LinkageError e) { throw new UnsupportedFileSystemSchemeException("Cannot support file system for '" + fsUri.getScheme() + @@ -183,4 +193,23 @@ private static String getMissingAuthorityErrorPrefix(URI fsURI) { "(like for example HDFS NameNode address/port or S3 host). " + "The attempt to use a configured default authority failed: "; } + + private static FileSystem limitIfConfigured(HadoopFileSystem fs, String scheme, Configuration config) { + final ConnectionLimitingSettings limitSettings = ConnectionLimitingSettings.fromConfig(config, scheme); + + // decorate only if any limit is configured + if (limitSettings == null) { + // no limit configured + return fs; + } + else { + return new LimitedConnectionsFileSystem( + fs, + limitSettings.limitTotal, + limitSettings.limitOutput, + limitSettings.limitInput, + limitSettings.streamOpenTimeout, + limitSettings.streamInactivityTimeout); + } + } } diff --git a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactoryTest.java b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactoryTest.java index 1f5c93219acd1..4b7592d1616a6 100644 --- a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactoryTest.java +++ b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopFsFactoryTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.fs.hdfs; +import org.apache.flink.core.fs.FileSystem; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -39,7 +40,7 @@ public void testCreateHadoopFsWithoutConfig() throws Exception { final URI uri = URI.create("hdfs://localhost:12345/"); HadoopFsFactory factory = new HadoopFsFactory(); - HadoopFileSystem fs = factory.create(uri); + FileSystem fs = factory.create(uri); assertEquals(uri.getScheme(), fs.getUri().getScheme()); assertEquals(uri.getAuthority(), fs.getUri().getAuthority()); diff --git a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/LimitedConnectionsConfigurationTest.java b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/LimitedConnectionsConfigurationTest.java new file mode 100644 index 0000000000000..8ab5419044ba8 --- /dev/null +++ b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/LimitedConnectionsConfigurationTest.java @@ -0,0 +1,84 @@ +/* + * 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.flink.runtime.fs.hdfs; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.LimitedConnectionsFileSystem; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.net.URI; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test that the Hadoop file system wrapper correctly picks up connection limiting + * settings for the correct file systems. + */ +public class LimitedConnectionsConfigurationTest { + + @Rule + public final TemporaryFolder tempDir = new TemporaryFolder(); + + @Test + public void testConfiguration() throws Exception { + + // nothing configured, we should get a regular file system + FileSystem hdfs = FileSystem.get(URI.create("hdfs://localhost:12345/a/b/c")); + FileSystem ftpfs = FileSystem.get(URI.create("ftp://localhost:12345/a/b/c")); + + assertFalse(hdfs instanceof LimitedConnectionsFileSystem); + assertFalse(ftpfs instanceof LimitedConnectionsFileSystem); + + // configure some limits, which should cause "fsScheme" to be limited + + final Configuration config = new Configuration(); + config.setInteger("fs.hdfs.limit.total", 40); + config.setInteger("fs.hdfs.limit.input", 39); + config.setInteger("fs.hdfs.limit.output", 38); + config.setInteger("fs.hdfs.limit.timeout", 23456); + config.setInteger("fs.hdfs.limit.stream-timeout", 34567); + + try { + FileSystem.initialize(config); + + hdfs = FileSystem.get(URI.create("hdfs://localhost:12345/a/b/c")); + ftpfs = FileSystem.get(URI.create("ftp://localhost:12345/a/b/c")); + + assertTrue(hdfs instanceof LimitedConnectionsFileSystem); + assertFalse(ftpfs instanceof LimitedConnectionsFileSystem); + + LimitedConnectionsFileSystem limitedFs = (LimitedConnectionsFileSystem) hdfs; + assertEquals(40, limitedFs.getMaxNumOpenStreamsTotal()); + assertEquals(39, limitedFs.getMaxNumOpenInputStreams()); + assertEquals(38, limitedFs.getMaxNumOpenOutputStreams()); + assertEquals(23456, limitedFs.getStreamOpenTimeout()); + assertEquals(34567, limitedFs.getStreamInactivityTimeout()); + } + finally { + // clear all settings + FileSystem.initialize(new Configuration()); + } + } +} From f636276dee991160a640442cbfcfdd58bfa57806 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 24 Nov 2017 14:59:36 +0100 Subject: [PATCH 112/367] [FLINK-6294] Fix potential NPE in BucketingSink.close() --- .../streaming/connectors/fs/bucketing/BucketingSink.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java index 55400c6ce19fa..0f13bbcfed76d 100644 --- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java +++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java @@ -416,8 +416,10 @@ private void initFileSystem() throws IOException { @Override public void close() throws Exception { - for (Map.Entry> entry : state.bucketStates.entrySet()) { - closeCurrentPartFile(entry.getValue()); + if (state != null) { + for (Map.Entry> entry : state.bucketStates.entrySet()) { + closeCurrentPartFile(entry.getValue()); + } } } From 4a46507e980b880a20bd821ee55d51bc787df124 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 23 Nov 2017 12:53:37 +0100 Subject: [PATCH 113/367] [FLINK-8022][kafka] Bump at-least-once timeout in tests Increasing timeout for reading the records from 30s to 60s seems to solve the issue for failing at-least-one tests. --- .../streaming/connectors/kafka/KafkaProducerTestBase.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 7ba3c95a404ff..8104d8fa0f783 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -68,6 +68,8 @@ @SuppressWarnings("serial") public abstract class KafkaProducerTestBase extends KafkaTestBase { + private static final long KAFKA_READ_TIMEOUT = 60_000L; + /** * This tests verifies that custom partitioning works correctly, with a default topic * and dynamic topic. The number of partitions for each topic is deliberately different. @@ -291,7 +293,7 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi topic, partition, Collections.unmodifiableSet(new HashSet<>(getIntegersSequence(BrokerRestartingMapper.numElementsBeforeSnapshot))), - 30000L); + KAFKA_READ_TIMEOUT); deleteTestTopic(topic); } @@ -367,7 +369,7 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi topic, partition, expectedElements, - 30000L); + KAFKA_READ_TIMEOUT); deleteTestTopic(topic); } From bd1e03374d0f240f5a0b406ace662e4391cce1d5 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 15 Nov 2017 13:14:18 +0100 Subject: [PATCH 114/367] [FLINK-8081][metrics] Annotate 'MetricRegistry#getReporters' with '@VisibleForTesting' This closes #5049. --- .../org/apache/flink/runtime/metrics/MetricRegistryImpl.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java index 3e4f56f765210..d36d09502cbf0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java @@ -214,6 +214,7 @@ public int getNumberReporters() { return reporters.size(); } + @VisibleForTesting public List getReporters() { return reporters; } From 728d2ffdd5215223405dceb7469d47466706b043 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Mon, 27 Nov 2017 17:34:01 +0800 Subject: [PATCH 115/367] [hotfix][docs] Fix missing period package declaration This closes #5077. --- docs/dev/packaging.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/dev/packaging.md b/docs/dev/packaging.md index e83d9ac35e12a..769a675b7bf8d 100644 --- a/docs/dev/packaging.md +++ b/docs/dev/packaging.md @@ -56,7 +56,7 @@ the program's data flow. To do that, the program must implement the strings passed to that method are the command line arguments. The program's plan can be created from the environment via the `ExecutionEnvironment#createProgramPlan()` method. When packaging the program's plan, the JAR manifest must point to the class implementing the -`org.apache.flinkapi.common.Program` interface, instead of the class with the main method. +`org.apache.flink.api.common.Program` interface, instead of the class with the main method. ### Summary @@ -68,10 +68,10 @@ attributes are found, the *program-class* attribute takes precedence over the *m attribute. Both the command line and the web interface support a parameter to pass the entry point class name manually for cases where the JAR manifest contains neither attribute. -2. If the entry point class implements the `org.apache.flinkapi.common.Program`, then the system +2. If the entry point class implements the `org.apache.flink.api.common.Program`, then the system calls the `getPlan(String...)` method to obtain the program plan to execute. -3. If the entry point class does not implement the `org.apache.flinkapi.common.Program` interface, +3. If the entry point class does not implement the `org.apache.flink.api.common.Program` interface, the system will invoke the main method of the class. {% top %} From 4103e5e37287d463603c021c403330a154d1d939 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 27 Nov 2017 16:27:29 +0100 Subject: [PATCH 116/367] [hotfix] Ensure pristine release in tools/releasing/create_source_release.sh --- tools/releasing/create_source_release.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tools/releasing/create_source_release.sh b/tools/releasing/create_source_release.sh index 2b29527f9e8d7..33b4e51d84701 100755 --- a/tools/releasing/create_source_release.sh +++ b/tools/releasing/create_source_release.sh @@ -49,10 +49,15 @@ cd .. echo "Creating source package" +# create a temporary git clone to ensure that we have a pristine source release +git clone . flink-tmp-clone +cd flink-tmp-clone + rsync -a \ --exclude ".git" --exclude ".gitignore" --exclude ".gitattributes" --exclude ".travis.yml" \ --exclude "deploysettings.xml" --exclude "CHANGELOG" --exclude ".github" --exclude "target" \ --exclude ".idea" --exclude "*.iml" --exclude ".DS_Store" --exclude "build-target" \ + --exclude "docs/content" --exclude ".rubydeps" \ . flink-$RELEASE_VERSION tar czf flink-${RELEASE_VERSION}-src.tgz flink-$RELEASE_VERSION @@ -60,4 +65,6 @@ gpg --armor --detach-sig flink-$RELEASE_VERSION-src.tgz $MD5SUM flink-$RELEASE_VERSION-src.tgz > flink-$RELEASE_VERSION-src.tgz.md5 $SHASUM flink-$RELEASE_VERSION-src.tgz > flink-$RELEASE_VERSION-src.tgz.sha -rm -rf flink-$RELEASE_VERSION +mv flink-$RELEASE_VERSION-src.* ../ +cd .. +rm -r flink-tmp-clone From 911aae4b532ae2d5acd5a7daf805d8d57a846adc Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 27 Nov 2017 16:37:56 +0100 Subject: [PATCH 117/367] [hotfix] Remove "Java 7" from README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 025dfb9e2dcbc..f6bf2ba44d770 100644 --- a/README.md +++ b/README.md @@ -70,7 +70,7 @@ Prerequisites for building Flink: * Unix-like environment (We use Linux, Mac OS X, Cygwin) * git * Maven (we recommend version 3.0.4) -* Java 7 or 8 +* Java 8 ``` git clone https://github.com/apache/flink.git From 4eae418b410c928b8e4b7893c1f5b9c48a5e3228 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Tue, 21 Nov 2017 14:05:53 -0500 Subject: [PATCH 118/367] [FLINK-8126] [build] Fix and update checkstyle Update to the latest checkstyle version and fix the errors not previously detected. This closes #5044. --- .../streaming/connectors/fs/RollingSink.java | 2 -- .../fs/bucketing/BucketingSink.java | 2 -- .../fs/bucketing/BucketingSinkTest.java | 1 - .../kafka/FlinkKafkaProducer08.java | 4 +-- .../internals/Kafka08PartitionDiscoverer.java | 1 - .../connectors/kafka/KafkaConsumer08Test.java | 1 - .../kafka/FlinkKafkaProducer09.java | 4 +-- .../kafka/FlinkKafkaConsumerBaseTest.java | 1 - .../kafka/KafkaAvroTableSourceTestBase.java | 1 - .../kafka/KafkaTestEnvironment.java | 1 - .../kinesis/FlinkKinesisProducer.java | 1 - .../kinesis/FlinkKinesisConsumerTest.java | 1 - .../kinesis/FlinkKinesisProducerTest.java | 2 -- .../kinesis/util/KinesisConfigUtilTest.java | 1 - .../benchmark/RocksDBPerformanceTest.java | 8 ++--- flink-core/pom.xml | 2 +- .../fs/hdfs/HadoopDataInputStreamTest.java | 1 - .../fs/s3hadoop/S3FileSystemFactory.java | 1 - .../fs/s3presto/S3FileSystemFactory.java | 1 - .../fs/s3presto/PrestoS3FileSystemTest.java | 2 -- .../typeutils/runtime/AvroSerializer.java | 2 -- .../apache/flink/api/java/io/CsvReader.java | 7 ++-- .../store/ZooKeeperMesosWorkerStore.java | 1 - flink-optimizer/pom.xml | 2 +- flink-runtime/pom.xml | 2 +- .../misc/KryoSerializerRegistrationsTest.java | 1 - .../networking/NetworkFailureHandler.java | 1 - .../test/checkpointing/SavepointITCase.java | 1 - .../TypeInfoTestCoverageTest.java | 1 - pom.xml | 2 +- tools/maven/checkstyle.xml | 32 +++++++++---------- 31 files changed, 31 insertions(+), 59 deletions(-) diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java index 9e547758dfaa1..4cd38a174e526 100644 --- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java +++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java @@ -33,12 +33,10 @@ import org.apache.flink.util.Preconditions; import org.apache.commons.lang3.time.StopWatch; - import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java index 0f13bbcfed76d..db0a5d859bed4 100644 --- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java +++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java @@ -42,12 +42,10 @@ import org.apache.flink.util.Preconditions; import org.apache.commons.lang3.time.StopWatch; - import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java index 695b6962abdc2..d6852efad2200 100644 --- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java +++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java @@ -52,7 +52,6 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; - import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java index 2fce9f9655557..d2f17d22c68aa 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java @@ -138,7 +138,7 @@ public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema seriali * * @deprecated This is a deprecated constructor that does not correctly handle partitioning when * producing to multiple topics. Use - * {@link FlinkKafkaProducer08(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * {@link #FlinkKafkaProducer08(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. */ @Deprecated public FlinkKafkaProducer08(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { @@ -155,7 +155,7 @@ public FlinkKafkaProducer08(String topicId, SerializationSchema serializatio * * @deprecated This is a deprecated constructor that does not correctly handle partitioning when * producing to multiple topics. Use - * {@link FlinkKafkaProducer08(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * {@link #FlinkKafkaProducer08(String, KeyedSerializationSchema, Properties, FlinkKafkaPartitioner)} instead. */ @Deprecated public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java index 9730114929f7b..5f7c370b3fe98 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java @@ -25,7 +25,6 @@ import kafka.javaapi.TopicMetadata; import kafka.javaapi.TopicMetadataRequest; import kafka.javaapi.consumer.SimpleConsumer; - import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.Node; import org.slf4j.Logger; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java index 8627ccbbcb957..3fd8beffa96fa 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java @@ -27,7 +27,6 @@ import org.apache.flink.util.NetUtils; import org.apache.kafka.clients.consumer.ConsumerConfig; - import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Matchers; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java index 946f7e95c7215..407bad5674341 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java @@ -142,7 +142,7 @@ public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema seriali * * @deprecated This is a deprecated constructor that does not correctly handle partitioning when * producing to multiple topics. Use - * {@link FlinkKafkaProducer09(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * {@link #FlinkKafkaProducer09(String, SerializationSchema, Properties, FlinkKafkaPartitioner)} instead. */ @Deprecated public FlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { @@ -160,7 +160,7 @@ public FlinkKafkaProducer09(String topicId, SerializationSchema serializatio * * @deprecated This is a deprecated constructor that does not correctly handle partitioning when * producing to multiple topics. Use - * {@link FlinkKafkaProducer09(String, org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema, Properties, FlinkKafkaPartitioner)} instead. + * {@link #FlinkKafkaProducer09(String, org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema, Properties, FlinkKafkaPartitioner)} instead. */ @Deprecated public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 59ce6666b1527..168cfd5c94af8 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -45,7 +45,6 @@ import org.apache.flink.util.SerializedValue; import org.apache.commons.collections.map.LinkedMap; - import org.junit.Assert; import org.junit.Test; import org.mockito.Matchers; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java index 871a6f62520ec..557a20f40eaef 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java @@ -25,7 +25,6 @@ import org.apache.avro.Schema; import org.apache.avro.specific.SpecificRecordBase; - import org.junit.Test; import java.sql.Timestamp; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index 68514741aa10f..8a287d3766fa2 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -28,7 +28,6 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import kafka.server.KafkaServer; - import org.apache.kafka.clients.consumer.ConsumerRecord; import java.util.ArrayList; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java index 28aa4b3dbd60a..225607396857d 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java @@ -36,7 +36,6 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java index 78ca160f2f184..2b1fcf4caa3fd 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java @@ -48,7 +48,6 @@ import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; import com.amazonaws.services.kinesis.model.Shard; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java index 2cd0c171f2452..702ab0b6226c6 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java @@ -34,9 +34,7 @@ import com.amazonaws.services.kinesis.producer.KinesisProducer; import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration; import com.amazonaws.services.kinesis.producer.UserRecordResult; - import com.google.common.util.concurrent.SettableFuture; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java index ddb300e20e44c..b52dce2cb7bf6 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java @@ -22,7 +22,6 @@ import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/benchmark/RocksDBPerformanceTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/benchmark/RocksDBPerformanceTest.java index 1667e55f31170..533196e6e7b02 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/benchmark/RocksDBPerformanceTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/benchmark/RocksDBPerformanceTest.java @@ -77,7 +77,7 @@ public void testRocksDbMergePerformance() throws Exception { .setCreateIfMissing(true) .setMergeOperatorName(RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME); - final WriteOptions write_options = new WriteOptions() + final WriteOptions writeOptions = new WriteOptions() .setSync(false) .setDisableWAL(true); @@ -88,7 +88,7 @@ public void testRocksDbMergePerformance() throws Exception { final long beginInsert = System.nanoTime(); for (int i = 0; i < num; i++) { - rocksDB.merge(write_options, keyBytes, valueBytes); + rocksDB.merge(writeOptions, keyBytes, valueBytes); } final long endInsert = System.nanoTime(); log.info("end insert - duration: {} ms", (endInsert - beginInsert) / 1_000_000); @@ -154,7 +154,7 @@ public void testRocksDbRangeGetPerformance() throws Exception { .setCreateIfMissing(true) .setMergeOperatorName(RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME); - final WriteOptions write_options = new WriteOptions() + final WriteOptions writeOptions = new WriteOptions() .setSync(false) .setDisableWAL(true); @@ -170,7 +170,7 @@ public void testRocksDbRangeGetPerformance() throws Exception { final long beginInsert = System.nanoTime(); for (int i = 0; i < num; i++) { unsafe.putInt(keyTemplate, offset, i); - rocksDB.put(write_options, keyTemplate, valueBytes); + rocksDB.put(writeOptions, keyTemplate, valueBytes); } final long endInsert = System.nanoTime(); log.info("end insert - duration: {} ms", (endInsert - beginInsert) / 1_000_000); diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 0ca742c73a12f..f82e1ed65fb21 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -168,7 +168,7 @@ under the License. com.puppycrawl.tools checkstyle - 6.19 + 8.4 diff --git a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStreamTest.java b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStreamTest.java index c6cf0ebd09f63..21c18bcc368df 100644 --- a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStreamTest.java +++ b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStreamTest.java @@ -23,7 +23,6 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.fs.Seekable; - import org.junit.Assert; import org.junit.Test; diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java index af0495214e99a..bd272e5701b73 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java @@ -25,7 +25,6 @@ import org.apache.flink.runtime.util.HadoopUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java index dabbfdabfe972..8847dc9cc2d45 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java +++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java @@ -25,7 +25,6 @@ import org.apache.flink.runtime.util.HadoopUtils; import com.facebook.presto.hive.PrestoS3FileSystem; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java index 8b17ccff56ba1..7e2d12aebb0a6 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java +++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java @@ -25,9 +25,7 @@ import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.services.s3.AmazonS3Client; - import com.facebook.presto.hive.PrestoS3FileSystem; - import org.junit.Test; import java.lang.reflect.Field; diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java index 228e67272f849..f3801991be6ae 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java @@ -32,12 +32,10 @@ import org.apache.flink.util.Preconditions; import com.esotericsoftware.kryo.Kryo; - import org.apache.avro.generic.GenericData; import org.apache.avro.reflect.ReflectDatumReader; import org.apache.avro.reflect.ReflectDatumWriter; import org.apache.avro.util.Utf8; - import org.objenesis.strategy.StdInstantiatorStrategy; import java.io.IOException; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java index 684911a76a0ea..df90d54be3fc8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java @@ -23,15 +23,16 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.operators.DataSource; -//CHECKSTYLE.OFF: AvoidStarImport|ImportOrder -import org.apache.flink.api.java.tuple.*; -//CHECKSTYLE.ON: AvoidStarImport|ImportOrder import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Preconditions; +//CHECKSTYLE.OFF: AvoidStarImport|ImportOrder +import org.apache.flink.api.java.tuple.*; +//CHECKSTYLE.ON: AvoidStarImport|ImportOrder + import java.util.ArrayList; import java.util.Arrays; diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java index 9f2fa44d21502..d9ca18c6c205c 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java @@ -28,7 +28,6 @@ import org.apache.flink.util.FlinkException; import org.apache.mesos.Protos; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml index ea4b5b75ea3dc..4e32a197a1956 100644 --- a/flink-optimizer/pom.xml +++ b/flink-optimizer/pom.xml @@ -87,7 +87,7 @@ under the License. com.puppycrawl.tools checkstyle - 6.19 + 8.4 diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 92a6f2f159950..7be858fa468e9 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -264,7 +264,7 @@ under the License. com.puppycrawl.tools checkstyle - 6.19 + 8.4 diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java index cbe9394a379f2..a654012478a9a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java @@ -23,7 +23,6 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Registration; - import org.junit.Test; import java.io.BufferedReader; diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java index 383961c829a5a..07d23412d0952 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java @@ -30,7 +30,6 @@ import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.SimpleChannelUpstreamHandler; import org.jboss.netty.channel.socket.ClientSocketChannelFactory; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 16e238b265b19..75f0aa463b741 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -80,7 +80,6 @@ import akka.actor.ActorRef; import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java index 92ab01341f443..b52884c6aa9c4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java @@ -23,7 +23,6 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; - import org.reflections.Reflections; import java.lang.reflect.Modifier; diff --git a/pom.xml b/pom.xml index cef9e643cc119..4dc9b96b6b4f3 100644 --- a/pom.xml +++ b/pom.xml @@ -1061,7 +1061,7 @@ under the License. com.puppycrawl.tools checkstyle - 6.19 + 8.4 diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index 514453ee5f403..a0168b001f917 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -64,12 +64,6 @@ This file is based on the checkstyle file of Apache Beam. - - - - - - @@ -90,6 +84,13 @@ This file is based on the checkstyle file of Apache Beam. + + + + + + + - + + @@ -484,15 +485,15 @@ This file is based on the checkstyle file of Apache Beam. --> - - - - + + + @@ -559,9 +560,6 @@ This file is based on the checkstyle file of Apache Beam. - - - From 7beb0b9ee26a7616c15bd0a976fea3bf5d5a2e2b Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Thu, 23 Nov 2017 07:54:09 -0500 Subject: [PATCH 119/367] [hotfix] [docs] Update checkstyle version in documentation This closes #5061 --- docs/internals/ide_setup.md | 2 +- pom.xml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/internals/ide_setup.md b/docs/internals/ide_setup.md index 02d54e7810497..ea80f56ed16e9 100644 --- a/docs/internals/ide_setup.md +++ b/docs/internals/ide_setup.md @@ -87,7 +87,7 @@ IntelliJ supports checkstyle within the IDE using the Checkstyle-IDEA plugin. 1. Install the "Checkstyle-IDEA" plugin from the IntelliJ plugin repository. 2. Configure the plugin by going to Settings -> Other Settings -> Checkstyle. 3. Set the "Scan Scope" to "Only Java sources (including tests)". -4. Select _6.19_ in the "Checkstyle Version" dropdown and click apply. **This step is important, +4. Select _8.4_ in the "Checkstyle Version" dropdown and click apply. **This step is important, don't skip it!** 5. In the "Configuration File" pane, add a new configuration using the plus icon: 1. Set the "Description" to "Flink". diff --git a/pom.xml b/pom.xml index 4dc9b96b6b4f3..63f2015619239 100644 --- a/pom.xml +++ b/pom.xml @@ -1061,6 +1061,7 @@ under the License. com.puppycrawl.tools checkstyle + 8.4 From 581bf80f3c1af5cc980c4cbe570bc64318fdbc5f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 29 Nov 2017 12:09:21 +0100 Subject: [PATCH 120/367] [FLINK-7300] Ignore more exceptions in end-to-end tests --- test-infra/end-to-end-test/common.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/test-infra/end-to-end-test/common.sh b/test-infra/end-to-end-test/common.sh index 5cbfb3f75a3cb..da8d22c607286 100644 --- a/test-infra/end-to-end-test/common.sh +++ b/test-infra/end-to-end-test/common.sh @@ -87,6 +87,8 @@ function stop_cluster { | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ | grep -v "jvm-exit-on-fatal-error" \ | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' \ + | grep -v "RejectedExecutionException" \ + | grep -v "An exception was thrown by an exception handler" \ | grep -iq "error"; then echo "Found error in log files:" cat $FLINK_DIR/log/* @@ -101,6 +103,8 @@ function stop_cluster { | grep -v "WARN akka.remote.transport.netty.NettyTransport" \ | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' \ + | grep -v "RejectedExecutionException" \ + | grep -v "An exception was thrown by an exception handler" \ | grep -iq "exception"; then echo "Found exception in log files:" cat $FLINK_DIR/log/* From cd4f048ed1cc410eac43151d0c6744fe9a75921f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 29 Nov 2017 12:34:31 +0100 Subject: [PATCH 121/367] [FLINK-8166] Create hadoop-free binary in release scripts --- tools/releasing/create_binary_release.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 3b1508e8bff9a..82b98a041ada0 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -55,7 +55,11 @@ make_binary_release() { SCALA_VERSION=$3 echo "Creating binary release name: $NAME, flags: $FLAGS, SCALA_VERSION: ${SCALA_VERSION}" - dir_name="flink-$RELEASE_VERSION-bin-$NAME-scala_${SCALA_VERSION}" + if [[ -z $NAME ]]; then + dir_name="flink-$RELEASE_VERSION-bin-scala_${SCALA_VERSION}" + else + dir_name="flink-$RELEASE_VERSION-bin-$NAME-scala_${SCALA_VERSION}" + fi # enable release profile here (to check for the maven version) $MVN clean package $FLAGS -DskipTests -Prelease,scala-${SCALA_VERSION} -Dgpg.skip @@ -78,6 +82,7 @@ cd .. if [ "$SCALA_VERSION" == "none" ] && [ "$HADOOP_VERSION" == "none" ]; then + make_binary_release "" "-DwithoutHadoop" "2.11" make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "2.11" make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "2.11" make_binary_release "hadoop27" "-Dhadoop.version=2.7.3" "2.11" @@ -87,6 +92,7 @@ then make_binary_release "hadoop2" "-Dhadoop.version=$HADOOP_VERSION" "2.11" elif [ "$SCALA_VERSION" != none ] && [ "$HADOOP_VERSION" == "none" ] then + make_binary_release "" "-DwithoutHadoop" "$SCALA_VERSION" make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "$SCALA_VERSION" make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "$SCALA_VERSION" make_binary_release "hadoop27" "-Dhadoop.version=2.7.3" "$SCALA_VERSION" From 9169ca3f15bb277e918aaf8dfd23b014d0ea67cc Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Wed, 29 Nov 2017 10:28:50 +0100 Subject: [PATCH 122/367] [FLINK-8165] ParameterTool serialization fix Closes #5096 --- .../flink/api/java/utils/ParameterTool.java | 4 ++-- .../java/utils/AbstractParameterToolTest.java | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java index e42a4b7c7c0e3..7518fa08faca1 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java @@ -618,7 +618,7 @@ public Map toMap() { private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - defaultData = Collections.emptyMap(); - unrequestedParameters = Collections.emptySet(); + defaultData = new ConcurrentHashMap<>(data.size()); + unrequestedParameters = Collections.newSetFromMap(new ConcurrentHashMap<>(data.size())); } } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java index f4afdd2d8b496..cd3712165860e 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java @@ -20,11 +20,14 @@ import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; import org.junit.Rule; import org.junit.rules.TemporaryFolder; +import static org.junit.Assert.fail; + import java.io.FileInputStream; import java.io.IOException; import java.util.Properties; @@ -39,6 +42,20 @@ public abstract class AbstractParameterToolTest { protected void validate(ParameterTool parameter) { ClosureCleaner.ensureSerializable(parameter); + validatePrivate(parameter); + + // -------- test behaviour after serialization ------------ + ParameterTool copy = null; + try { + byte[] b = InstantiationUtil.serializeObject(parameter); + copy = InstantiationUtil.deserializeObject(b, getClass().getClassLoader()); + } catch (Exception e) { + fail(); + } + validatePrivate(copy); + } + + private void validatePrivate(ParameterTool parameter) { Assert.assertEquals("myInput", parameter.getRequired("input")); Assert.assertEquals("myDefaultValue", parameter.get("output", "myDefaultValue")); Assert.assertEquals(null, parameter.get("whatever")); From 3d568f52070a7719803a2bea5346d0e6f2a72615 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 3 Nov 2017 21:19:34 +0100 Subject: [PATCH 123/367] [FLINK-7989][yarn] Do not upload the flink-dist jar twice We always add the dist.jar ourselves, but it could also be inside a shipped folder such as the "lib/" folder and was then distributed multiple times. This closes #4951. --- .../yarn/AbstractYarnClusterDescriptor.java | 106 ++++++++++++------ 1 file changed, 69 insertions(+), 37 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 5ac5c4ea4fde9..c0ffb54e02386 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -182,14 +182,17 @@ public void setLocalJarPath(Path localJarPath) { this.flinkJarPath = localJarPath; } + /** + * Adds the given files to the list of files to ship. + * + *

    Note that any file matching "flink-dist*.jar" will be excluded from the upload by + * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, StringBuilder)} + * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. + * + * @param shipFiles files to ship + */ public void addShipFiles(List shipFiles) { - for (File shipFile: shipFiles) { - // remove uberjar from ship list (by default everything in the lib/ folder is added to - // the list of files to ship, but we handle the uberjar separately. - if (!(shipFile.getName().startsWith("flink-dist") && shipFile.getName().endsWith("jar"))) { - this.shipFiles.add(shipFile); - } - } + this.shipFiles.addAll(shipFiles); } public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { @@ -1048,6 +1051,27 @@ private static Path setupSingleLocalResource( return resource.f0; } + /** + * Recursively uploads (and registers) any (user and system) files in shipFiles except + * for files matching "flink-dist*.jar" which should be uploaded separately. + * + * @param shipFiles + * files to upload + * @param fs + * file system to upload to + * @param targetHomeDir + * remote home directory to upload to + * @param appId + * application ID + * @param remotePaths + * paths of the remote resources (uploaded resources will be added) + * @param localResources + * map of resources (uploaded resources will be added) + * @param envShipFileList + * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} + * + * @return list of class paths with the the proper resource keys from the registration + */ static List uploadAndRegisterFiles( Collection shipFiles, FileSystem fs, @@ -1068,40 +1092,48 @@ static List uploadAndRegisterFiles( @Override public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) throws IOException { - java.nio.file.Path relativePath = parentPath.relativize(file); - - String key = relativePath.toString(); - try { - Path remotePath = setupSingleLocalResource( - key, - fs, - appId, - new Path(file.toUri()), - localResources, - targetHomeDir, - relativePath.getParent().toString()); - remotePaths.add(remotePath); - envShipFileList.append(key).append("=").append(remotePath).append(","); - - // add files to the classpath - classPaths.add(key); - - return FileVisitResult.CONTINUE; - } catch (URISyntaxException e) { - throw new IOException(e); + + if (!(file.getFileName().startsWith("flink-dist") && + file.getFileName().endsWith("jar"))) { + + java.nio.file.Path relativePath = parentPath.relativize(file); + + String key = relativePath.toString(); + try { + Path remotePath = setupSingleLocalResource( + key, + fs, + appId, + new Path(file.toUri()), + localResources, + targetHomeDir, + relativePath.getParent().toString()); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=") + .append(remotePath).append(","); + + // add files to the classpath + classPaths.add(key); + } catch (URISyntaxException e) { + throw new IOException(e); + } } + + return FileVisitResult.CONTINUE; } }); } else { - Path shipLocalPath = new Path("file://" + shipFile.getAbsolutePath()); - String key = shipFile.getName(); - Path remotePath = setupSingleLocalResource( - key, fs, appId, shipLocalPath, localResources, targetHomeDir, ""); - remotePaths.add(remotePath); - envShipFileList.append(key).append("=").append(remotePath).append(","); - - // add files to the classpath - classPaths.add(key); + if (!(shipFile.getName().startsWith("flink-dist") && shipFile.getName().endsWith("jar"))) { + Path shipLocalPath = new Path(shipFile.toURI()); + String key = shipFile.getName(); + Path remotePath = setupSingleLocalResource( + key, fs, appId, shipLocalPath, localResources, targetHomeDir, ""); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=").append(remotePath).append(","); + + // add files to the classpath + classPaths.add(key); + } } } From df58722f08b8fa8f22b48043618078acdb2a39b8 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 6 Sep 2017 15:38:20 +0200 Subject: [PATCH 124/367] [FLINK-7595] [Savepoints] Allow removing stateless operators --- .../checkpoint/savepoint/SavepointLoader.java | 12 +++- .../savepoint/SavepointLoaderTest.java | 16 +++-- .../AbstractOperatorRestoreTestBase.java | 11 +++- ...stractNonKeyedOperatorRestoreTestBase.java | 7 ++- .../ChainLengthStatelessDecreaseTest.java | 59 +++++++++++++++++++ 5 files changed, 96 insertions(+), 9 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthStatelessDecreaseTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java index 38db7c24b99b5..31d9124fad524 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoader.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointProperties; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -120,14 +121,19 @@ public static CompletedCheckpoint loadAndValidateSavepoint( } else if (allowNonRestoredState) { LOG.info("Skipping savepoint state for operator {}.", operatorState.getOperatorID()); } else { - String msg = String.format("Failed to rollback to savepoint %s. " + + for (OperatorSubtaskState operatorSubtaskState : operatorState.getStates()) { + if (operatorSubtaskState.hasState()) { + String msg = String.format("Failed to rollback to savepoint %s. " + "Cannot map savepoint state for operator %s to the new program, " + "because the operator is not available in the new program. If " + "you want to allow to skip this, you can set the --allowNonRestoredState " + "option on the CLI.", - savepointPath, operatorState.getOperatorID()); + savepointPath, operatorState.getOperatorID()); - throw new IllegalStateException(msg); + throw new IllegalStateException(msg); + } + } + LOG.info("Skipping empty savepoint state for operator {}.", operatorState.getOperatorID()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java index 331621d165f60..a461569f70724 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointLoaderTest.java @@ -22,9 +22,13 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -59,10 +63,14 @@ public void testLoadAndValidateSavepoint() throws Exception { JobVertexID jobVertexID = new JobVertexID(); OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); - OperatorState state = mock(OperatorState.class); - when(state.getParallelism()).thenReturn(parallelism); - when(state.getOperatorID()).thenReturn(operatorID); - when(state.getMaxParallelism()).thenReturn(parallelism); + OperatorSubtaskState subtaskState = new OperatorSubtaskState( + new OperatorStateHandle(Collections.emptyMap(), new ByteStreamStateHandle("testHandler", new byte[0])), + null, + null, + null); + + OperatorState state = new OperatorState(operatorID, parallelism, parallelism); + state.putState(0, subtaskState); Map taskStates = new HashMap<>(); taskStates.put(operatorID, state); diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index 7488b629fa375..c86f21fddc282 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -89,6 +89,15 @@ public abstract class AbstractOperatorRestoreTestBase extends TestLogger { private static ActorGateway taskManager = null; private static final FiniteDuration timeout = new FiniteDuration(30L, TimeUnit.SECONDS); + private final boolean allowNonRestoredState; + + protected AbstractOperatorRestoreTestBase() { + this(true); + } + + protected AbstractOperatorRestoreTestBase(boolean allowNonRestoredState) { + this.allowNonRestoredState = allowNonRestoredState; + } @BeforeClass public static void beforeClass() { @@ -238,7 +247,7 @@ private String migrateJob() throws Throwable { private void restoreJob(String savepointPath) throws Exception { JobGraph jobToRestore = createJobGraph(ExecutionMode.RESTORE); - jobToRestore.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, true)); + jobToRestore.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, allowNonRestoredState)); Object msg; Object result; diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java index f07bd4d53eee0..c100dc9fd24f7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java @@ -51,7 +51,12 @@ public static Collection parameters () { "nonKeyed-flink1.3"); } - public AbstractNonKeyedOperatorRestoreTestBase(String savepointPath) { + protected AbstractNonKeyedOperatorRestoreTestBase(String savepointPath) { + this.savepointPath = savepointPath; + } + + protected AbstractNonKeyedOperatorRestoreTestBase(String savepointPath, boolean allowNonRestoredState) { + super(allowNonRestoredState); this.savepointPath = savepointPath; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthStatelessDecreaseTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthStatelessDecreaseTest.java new file mode 100644 index 0000000000000..fb4dcf5bdabfc --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthStatelessDecreaseTest.java @@ -0,0 +1,59 @@ +/* + * 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.flink.test.state.operator.restore.unkeyed; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.state.operator.restore.ExecutionMode; + +import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createFirstStatefulMap; +import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createSecondStatefulMap; +import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createSource; +import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createThirdStatefulMap; + +/** + * Verifies that the state of all operators is restored if a topology change removes an operator from a chain. + * + *

    This test specifically checks that stateless operators can be removed even if all states from the previous job + * must be restored. + */ +public class ChainLengthStatelessDecreaseTest extends AbstractNonKeyedOperatorRestoreTestBase { + + public ChainLengthStatelessDecreaseTest(String savepointPath) { + super(savepointPath, false); + } + + @Override + public void createRestoredJob(StreamExecutionEnvironment env) { + /* + * Original job: Source -> StatefulMap1 -> CHAIN(StatefulMap2 -> Map -> StatefulMap3) + * Modified job: Source -> StatefulMap1 -> CHAIN(StatefulMap2 -> StatefulMap3) + */ + DataStream source = createSource(env, ExecutionMode.RESTORE); + + SingleOutputStreamOperator first = createFirstStatefulMap(ExecutionMode.RESTORE, source); + first.startNewChain(); + + SingleOutputStreamOperator second = createSecondStatefulMap(ExecutionMode.RESTORE, first); + second.startNewChain(); + + SingleOutputStreamOperator third = createThirdStatefulMap(ExecutionMode.RESTORE, second); + } +} From 6628f1954b2fcca517cb7bb13ead0b12fd886d1e Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Thu, 30 Nov 2017 15:25:48 -0500 Subject: [PATCH 125/367] [hotfix] [gelly] Fix Gelly CSV output Algorithm result types no longer extend Tuple but are now custom POJOs. This prevents the use of DataSet#writeAsCsv which accepts custom line and field delimiters. Anticipating this lose the result types override This patch uses DataSet#writeAsCsv for types extending Tuple and DataSet#writeAsText otherwise. Since most algorithm result types are now custom POJOs the custom delimiters are now largely ineffective. --- .../apache/flink/graph/drivers/output/CSV.java | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/output/CSV.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/output/CSV.java index 9b6aae671ec17..a24255c477a18 100644 --- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/output/CSV.java +++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/output/CSV.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.io.CsvOutputFormat; +import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.graph.drivers.parameter.StringParameter; import java.io.PrintStream; @@ -42,8 +43,17 @@ public class CSV @Override public void write(String executionName, PrintStream out, DataSet data) throws Exception { - data - .writeAsCsv(filename.getValue(), lineDelimiter.getValue(), fieldDelimiter.getValue()) - .name("CSV: " + filename.getValue()); + if (Tuple.class.isAssignableFrom(data.getType().getTypeClass())) { + data + .writeAsCsv(filename.getValue(), lineDelimiter.getValue(), fieldDelimiter.getValue()) + .name("CSV: " + filename.getValue()); + } else { + // line and field delimiters are ineffective when writing custom POJOs result types + data + .writeAsText(filename.getValue()) + .name("CSV: " + filename.getValue()); + } + + data.getExecutionEnvironment().execute(); } } From 6ca2be3ffbf3f8a92c21b0990e317ff3a60a3e82 Mon Sep 17 00:00:00 2001 From: twalthr Date: Fri, 1 Dec 2017 15:37:23 +0100 Subject: [PATCH 126/367] [FLINK-8173] [table] Fix input unboxing and support Avro Utf8 This closes #5111. --- flink-libraries/flink-table/pom.xml | 17 ++ .../flink/table/codegen/CodeGenerator.scala | 6 +- .../table/runtime/batch/AvroTypesITCase.scala | 188 ++++++++++++++++++ .../runtime/harness/JoinHarnessTest.scala | 1 - 4 files changed, 207 insertions(+), 5 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/AvroTypesITCase.scala diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml index c71dd5e7e60a6..d7253096a025d 100644 --- a/flink-libraries/flink-table/pom.xml +++ b/flink-libraries/flink-table/pom.xml @@ -157,6 +157,7 @@ under the License. ${project.version} test + org.apache.flink flink-streaming-java_${scala.binary.version} @@ -164,6 +165,7 @@ under the License. test-jar test + org.apache.flink flink-runtime_${scala.binary.version} @@ -171,6 +173,21 @@ under the License. test test-jar + + + org.apache.flink + flink-avro + ${project.version} + test + test-jar + + + + org.apache.flink + flink-avro + ${project.version} + test + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index 7660e2ba9b99b..61ae0ec636c4f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -1210,10 +1210,8 @@ abstract class CodeGenerator( fieldType: TypeInformation[_], fieldTerm: String) : GeneratedExpression = { - val tmpTerm = newName("tmp") val resultTerm = newName("result") val nullTerm = newName("isNull") - val tmpTypeTerm = boxedTypeTermForTypeInfo(fieldType) val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) val defaultValue = primitiveDefaultValue(fieldType) @@ -1244,12 +1242,12 @@ abstract class CodeGenerator( | $resultTerm = $defaultValue; |} |else { - | $resultTerm = $unboxedFieldCode; + | $resultTerm = ($resultTypeTerm) $unboxedFieldCode; |} |""".stripMargin } else { s""" - |$resultTypeTerm $resultTerm = $unboxedFieldCode; + |$resultTypeTerm $resultTerm = ($resultTypeTerm) $unboxedFieldCode; |""".stripMargin } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/AvroTypesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/AvroTypesITCase.scala new file mode 100644 index 0000000000000..198355423fdce --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/AvroTypesITCase.scala @@ -0,0 +1,188 @@ +/* + * 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.flink.table.runtime.batch + +import java.lang.{Boolean => JBoolean, Long => JLong} +import java.util +import java.util.Collections + +import org.apache.avro.util.Utf8 +import org.apache.flink.api.scala._ +import org.apache.flink.formats.avro.generated.{Address, Colors, Fixed16, User} +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase +import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode +import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.TestBaseUtils +import org.apache.flink.types.Row +import org.junit.Test +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * Tests for interoperability with Avro types. + */ +@RunWith(classOf[Parameterized]) +class AvroTypesITCase( + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsClusterTestBase(mode, configMode) { + + @Test + def testAvroToRow(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = testData(env).toTable(tEnv) + + val result = t.select('*) + + val results = result.toDataSet[Row].collect() + val expected = "black,null,Whatever,[true],[hello],true,0.0,GREEN," + + "[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0],42,{},null,null,null,null\n" + + "blue,null,Charlie,[],[],false,1.337,RED," + + "null,1337,{},{\"num\": 42, \"street\": \"Bakerstreet\", \"city\": \"Berlin\", " + + "\"state\": \"Berlin\", \"zip\": \"12049\"},null,null,null\n" + + "yellow,null,Terminator,[false],[world],false,0.0,GREEN," + + "[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0],1,{},null,null,null,null" + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testAvroStringAccess(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = testData(env).toTable(tEnv) + + val result = t.select('name) + val results = result.toDataSet[Utf8].collect() + val expected = "Charlie\n" + + "Terminator\n" + + "Whatever" + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testAvroObjectAccess(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = testData(env).toTable(tEnv) + + val result = t + .filter('type_nested.isNotNull) + .select('type_nested.flatten()).as('city, 'num, 'state, 'street, 'zip) + + val results = result.toDataSet[Address].collect() + val expected = AvroTypesITCase.USER_1.getTypeNested.toString + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testAvroToAvro(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = testData(env).toTable(tEnv) + + val result = t.select('*).toDataSet[User].collect() + val expected = AvroTypesITCase.USER_1 + "\n" + + AvroTypesITCase.USER_2 + "\n" + + AvroTypesITCase.USER_3 + TestBaseUtils.compareResultAsText(result.asJava, expected) + } + + private def testData(env: ExecutionEnvironment): DataSet[User] = { + + val data = new mutable.MutableList[User] + + data.+=(AvroTypesITCase.USER_1) + data.+=(AvroTypesITCase.USER_2) + data.+=(AvroTypesITCase.USER_3) + + env.fromCollection(data) + } +} + +object AvroTypesITCase { + + val USER_1: User = User.newBuilder() + .setName("Charlie") + .setFavoriteColor("blue") + .setFavoriteNumber(null) + .setTypeBoolTest(false) + .setTypeDoubleTest(1.337d) + .setTypeNullTest(null) + .setTypeLongTest(1337L) + .setTypeArrayString(new util.ArrayList[CharSequence]) + .setTypeArrayBoolean(new util.ArrayList[JBoolean]()) + .setTypeNullableArray(null) + .setTypeEnum(Colors.RED) + .setTypeMap(new util.HashMap[CharSequence, JLong]) + .setTypeFixed(null) + .setTypeUnion(null) + .setTypeNested( + Address.newBuilder + .setNum(42) + .setStreet("Bakerstreet") + .setCity("Berlin") + .setState("Berlin") + .setZip("12049") + .build) + .build + + val USER_2: User = User.newBuilder() + .setName("Whatever") + .setFavoriteNumber(null) + .setFavoriteColor("black") + .setTypeLongTest(42L) + .setTypeDoubleTest(0.0) + .setTypeNullTest(null) + .setTypeBoolTest(true) + .setTypeArrayString(Collections.singletonList("hello")) + .setTypeArrayBoolean(Collections.singletonList(true)) + .setTypeEnum(Colors.GREEN) + .setTypeMap(new util.HashMap[CharSequence, JLong]) + .setTypeFixed(new Fixed16()) + .setTypeUnion(null) + .setTypeNested(null) + .build() + + val USER_3: User = User.newBuilder() + .setName("Terminator") + .setFavoriteNumber(null) + .setFavoriteColor("yellow") + .setTypeLongTest(1L) + .setTypeDoubleTest(0.0) + .setTypeNullTest(null) + .setTypeBoolTest(false) + .setTypeArrayString(Collections.singletonList("world")) + .setTypeArrayBoolean(Collections.singletonList(false)) + .setTypeEnum(Colors.GREEN) + .setTypeMap(new util.HashMap[CharSequence, JLong]) + .setTypeFixed(new Fixed16()) + .setTypeUnion(null) + .setTypeNested(null) + .build() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala index 43397aea7c512..facdbd4e121b9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala @@ -334,7 +334,6 @@ class JoinHarnessTest extends HarnessTestBase { expectedOutput.add(new Watermark(41)) val result = testHarness.getOutput - println(result) verify( expectedOutput, result, From eceffaa5563fc856080a58b3f34b2ac486192246 Mon Sep 17 00:00:00 2001 From: nichuanlei Date: Fri, 1 Dec 2017 22:06:21 +0800 Subject: [PATCH 127/367] [hotfix][docs] Fix typo in checkpointing doc This closes #5110. --- docs/dev/stream/state/checkpointing.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/stream/state/checkpointing.md b/docs/dev/stream/state/checkpointing.md index 2a2edc98ae339..67d95d5c9988c 100644 --- a/docs/dev/stream/state/checkpointing.md +++ b/docs/dev/stream/state/checkpointing.md @@ -59,7 +59,7 @@ Other parameters for checkpointing include: started no sooner than 5 seconds after the previous checkpoint completed, regardless of the checkpoint duration and the checkpoint interval. Note that this implies that the checkpoint interval will never be smaller than this parameter. - It is often easier to configure applications by defining the "time between checkpoints" then the checkpoint interval, because the "time between checkpoints" + It is often easier to configure applications by defining the "time between checkpoints" than the checkpoint interval, because the "time between checkpoints" is not susceptible to the fact that checkpoints may sometimes take longer than on average (for example if the target storage system is temporarily slow). Note that this value also implies that the number of concurrent checkpoints is *one*. From e4c3eb64ad30fc4b7f8c459912c5d12a70aabaec Mon Sep 17 00:00:00 2001 From: yew1eb Date: Sun, 26 Nov 2017 17:44:38 +0800 Subject: [PATCH 128/367] [FLINK-7907][docs][metrics] Add scala examples This closes #7907. --- docs/monitoring/metrics.md | 212 ++++++++++++++++++++++++++++++++++--- 1 file changed, 198 insertions(+), 14 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 0bcab55704bcf..7963e1a015468 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -41,10 +41,12 @@ Flink supports `Counters`, `Gauges`, `Histograms` and `Meters`. A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(long n)` or `dec()/dec(long n)`. You can create and register a `Counter` by calling `counter(String name)` on a `MetricGroup`. +

    +
    {% highlight java %} public class MyMapper extends RichMapFunction { - private Counter counter; + private transient Counter counter; @Override public void open(Configuration config) { @@ -61,13 +63,39 @@ public class MyMapper extends RichMapFunction { } {% endhighlight %} +
    + +
    +{% highlight scala %} + +class MyMapper extends RichMapFunction[String,String] { + @transient private var counter: Counter + + override def open(parameters: Configuration): Unit = { + counter = getRuntimeContext() + .getMetricGroup() + .counter("myCounter") + } + + override def map(value: String): String = { + counter.inc() + value + } +} + +{% endhighlight %} +
    + +
    Alternatively you can also use your own `Counter` implementation: +
    +
    {% highlight java %} public class MyMapper extends RichMapFunction { - private Counter counter; + private transient Counter counter; @Override public void open(Configuration config) { @@ -83,7 +111,32 @@ public class MyMapper extends RichMapFunction { } } + +{% endhighlight %} +
    + +
    +{% highlight scala %} + +class MyMapper extends RichMapFunction[String,String] { + @transient private var counter: Counter + + override def open(parameters: Configuration): Unit = { + counter = getRuntimeContext() + .getMetricGroup() + .counter("myCustomCounter", new CustomCounter()) + } + + override def map(value: String): String = { + counter.inc() + value + } +} + {% endhighlight %} +
    + +
    #### Gauge @@ -96,7 +149,7 @@ You can register a gauge by calling `gauge(String name, Gauge gauge)` on a `Metr {% highlight java %} public class MyMapper extends RichMapFunction { - private int valueToExpose = 0; + private transient int valueToExpose = 0; @Override public void open(Configuration config) { @@ -123,8 +176,8 @@ public class MyMapper extends RichMapFunction {
    {% highlight scala %} -public class MyMapper extends RichMapFunction[String,String] { - val valueToExpose = 0 +new class MyMapper extends RichMapFunction[String,String] { + @transient private var valueToExpose = 0 override def open(parameters: Configuration): Unit = { getRuntimeContext() @@ -150,9 +203,11 @@ Note that reporters will turn the exposed object into a `String`, which means th A `Histogram` measures the distribution of long values. You can register one by calling `histogram(String name, Histogram histogram)` on a `MetricGroup`. +
    +
    {% highlight java %} public class MyMapper extends RichMapFunction { - private Histogram histogram; + private transient Histogram histogram; @Override public void open(Configuration config) { @@ -168,6 +223,30 @@ public class MyMapper extends RichMapFunction { } } {% endhighlight %} +
    + +
    +{% highlight scala %} + +class MyMapper extends RichMapFunction[Long,Long] { + @transient private var histogram: Histogram + + override def open(parameters: Configuration): Unit = { + histogram = getRuntimeContext() + .getMetricGroup() + .histogram("myHistogram", new MyHistogram()) + } + + override def map(value: Long): Long = { + histogram.update(value) + value + } +} + +{% endhighlight %} +
    + +
    Flink does not provide a default implementation for `Histogram`, but offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardHistogramWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard histograms. To use this wrapper add the following dependency in your `pom.xml`: @@ -181,30 +260,67 @@ To use this wrapper add the following dependency in your `pom.xml`: You can then register a Codahale/DropWizard histogram like this: +
    +
    {% highlight java %} -public class MyMapper extends RichMapFunction { - private Histogram histogram; +public class MyMapper extends RichMapFunction { + private transient Histogram histogram; @Override public void open(Configuration config) { - com.codahale.metrics.Histogram histogram = + com.codahale.metrics.Histogram dropwizardHistogram = new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500)); this.histogram = getRuntimeContext() .getMetricGroup() - .histogram("myHistogram", new DropwizardHistogramWrapper(histogram)); + .histogram("myHistogram", new DropwizardHistogramWrapper(dropwizardHistogram)); + } + + @Override + public Long map(Long value) throws Exception { + this.histogram.update(value); + return value; + } +} +{% endhighlight %} +
    + +
    +{% highlight scala %} + +class MyMapper extends RichMapFunction[Long, Long] { + @transient private var histogram: Histogram + + override def open(config: Configuration): Unit = { + com.codahale.metrics.Histogram dropwizardHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500)) + + histogram = getRuntimeContext() + .getMetricGroup() + .histogram("myHistogram", new DropwizardHistogramWrapper(dropwizardHistogram)) + } + + override def map(value: Long): Long = { + histogram.update(value) + value } } + {% endhighlight %} +
    + +
    #### Meter A `Meter` measures an average throughput. An occurrence of an event can be registered with the `markEvent()` method. Occurrence of multiple events at the same time can be registered with `markEvent(long n)` method. You can register a meter by calling `meter(String name, Meter meter)` on a `MetricGroup`. +
    +
    {% highlight java %} public class MyMapper extends RichMapFunction { - private Meter meter; + private transient Meter meter; @Override public void open(Configuration config) { @@ -220,6 +336,30 @@ public class MyMapper extends RichMapFunction { } } {% endhighlight %} +
    + +
    +{% highlight scala %} + +class MyMapper extends RichMapFunction[Long,Long] { + @transient private var meter: Meter + + override def open(config: Configuration): Unit = { + meter = getRuntimeContext() + .getMetricGroup() + .meter("myMeter", new MyMeter()) + } + + override def map(value: Long): Long = { + meter.markEvent() + value + } +} + +{% endhighlight %} +
    + +
    Flink offers a {% gh_link flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardMeterWrapper.java "Wrapper" %} that allows usage of Codahale/DropWizard meters. To use this wrapper add the following dependency in your `pom.xml`: @@ -233,17 +373,19 @@ To use this wrapper add the following dependency in your `pom.xml`: You can then register a Codahale/DropWizard meter like this: +
    +
    {% highlight java %} public class MyMapper extends RichMapFunction { - private Meter meter; + private transient Meter meter; @Override public void open(Configuration config) { - com.codahale.metrics.Meter meter = new com.codahale.metrics.Meter(); + com.codahale.metrics.Meter dropwizardMeter = new com.codahale.metrics.Meter(); this.meter = getRuntimeContext() .getMetricGroup() - .meter("myMeter", new DropwizardMeterWrapper(meter)); + .meter("myMeter", new DropwizardMeterWrapper(dropwizardMeter)); } @Override @@ -253,6 +395,32 @@ public class MyMapper extends RichMapFunction { } } {% endhighlight %} +
    + +
    +{% highlight scala %} + +class MyMapper extends RichMapFunction[Long,Long] { + @transient private var meter: Meter + + override def open(config: Configuration): Unit = { + com.codahale.metrics.Meter dropwizardMeter = new com.codahale.metrics.Meter() + + meter = getRuntimeContext() + .getMetricGroup() + .meter("myMeter", new DropwizardMeterWrapper(dropwizardMeter)) + } + + override def map(value: Long): Long = { + meter.markEvent() + value + } +} + +{% endhighlight %} +
    + +
    ## Scope @@ -265,6 +433,8 @@ You can configure which delimiter to use for the identifier (default: `.`) by se You can define a user scope by calling either `MetricGroup#addGroup(String name)` or `MetricGroup#addGroup(int name)`. +
    +
    {% highlight java %} counter = getRuntimeContext() @@ -273,6 +443,20 @@ counter = getRuntimeContext() .counter("myCounter"); {% endhighlight %} +
    + +
    +{% highlight scala %} + +counter = getRuntimeContext() + .getMetricGroup() + .addGroup("MyMetrics") + .counter("myCounter") + +{% endhighlight %} +
    + +
    ### System Scope From 4dca094bf88b12599add6ef37f02bf9797279f68 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 29 Nov 2017 16:28:18 +0100 Subject: [PATCH 129/367] [FLINK-8167] [connector-wikiedits] Harden WikipediaEditsSource - Minor eager sanity checks - Use UUID suffix for nickname. As reported in FLINK-8167, the current nickname suffix can result in nickname clashes which lead to test failures. --- .../WikipediaEditEventIrcStream.java | 44 +++++++++--------- .../wikiedits/WikipediaEditsSource.java | 46 ++++++++----------- 2 files changed, 41 insertions(+), 49 deletions(-) diff --git a/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditEventIrcStream.java b/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditEventIrcStream.java index 81dfa7bfe207e..84d89dd10ba2f 100644 --- a/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditEventIrcStream.java +++ b/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditEventIrcStream.java @@ -26,10 +26,12 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Objects; +import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -class WikipediaEditEventIrcStream { +class WikipediaEditEventIrcStream implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(WikipediaEditEventIrcStream.class); @@ -37,11 +39,11 @@ class WikipediaEditEventIrcStream { private final BlockingQueue edits = new ArrayBlockingQueue<>(128); - /** IRC connection (Thread). */ + /** IRC connection (NOTE: this is a separate Thread). */ private IRCConnection conn; WikipediaEditEventIrcStream(String host, int port) { - final String nick = "flink-bot-" + (int) (Math.random() * 1000); + final String nick = "flink-bot-" + UUID.randomUUID().toString(); this.conn = new IRCConnection(host, new int[] { port}, "", nick, nick, nick); conn.addIRCEventListener(new WikipediaIrcChannelListener(edits)); conn.setEncoding("UTF-8"); @@ -51,25 +53,18 @@ class WikipediaEditEventIrcStream { conn.setName("WikipediaEditEventIrcStreamThread"); } - void start() throws IOException { - if (!conn.isConnected()) { - conn.connect(); - } + BlockingQueue getEdits() { + return edits; } - void stop() throws InterruptedException { - if (conn.isConnected()) { + void connect() throws IOException { + if (!conn.isConnected()) { + conn.connect(); } - - conn.interrupt(); - conn.join(5 * 1000); - } - - BlockingQueue getEdits() { - return edits; } void join(String channel) { + Objects.requireNonNull(channel, "channel"); conn.send("JOIN " + channel); } @@ -77,6 +72,15 @@ void leave(String channel) { conn.send("PART " + channel); } + @Override + public void close() throws Exception { + if (conn != null && conn.isConnected()) { + conn.doQuit(); + conn.close(); + conn.join(5 * 1000); + } + } + // ------------------------------------------------------------------------ // IRC channel listener // ------------------------------------------------------------------------ @@ -85,12 +89,8 @@ private static class WikipediaIrcChannelListener implements IRCEventListener { private final BlockingQueue edits; - public WikipediaIrcChannelListener(BlockingQueue edits) { - if (edits == null) { - throw new NullPointerException(); - } - - this.edits = edits; + WikipediaIrcChannelListener(BlockingQueue edits) { + this.edits = Objects.requireNonNull(edits, "edits"); } @Override diff --git a/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSource.java b/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSource.java index a10abdb1afab3..4498beb1e090f 100644 --- a/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSource.java +++ b/flink-contrib/flink-connector-wikiedits/src/main/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.wikiedits; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import java.util.Objects; import java.util.concurrent.TimeUnit; /** @@ -44,8 +44,6 @@ public class WikipediaEditsSource extends RichSourceFunction private volatile boolean isRunning = true; - private WikipediaEditEventIrcStream ircStream; - /** * Creates a source reading {@link WikipediaEditEvent} instances from the * IRC channel #en.wikipedia. @@ -72,33 +70,27 @@ public WikipediaEditsSource() { public WikipediaEditsSource(String host, int port, String channel) { this.host = host; this.port = port; - this.channel = channel; + this.channel = Objects.requireNonNull(channel); } @Override - public void open(Configuration parameters) throws Exception { - ircStream = new WikipediaEditEventIrcStream(host, port); - ircStream.start(); - ircStream.join(channel); - } - - @Override - public void close() throws Exception { - if (ircStream != null) { - ircStream.leave(channel); - ircStream.stop(); - } - } - - @Override - public void run(SourceContext ctx) throws Exception { - while (isRunning) { - // Query for the next edit event - WikipediaEditEvent edit = ircStream.getEdits() - .poll(100, TimeUnit.MILLISECONDS); - - if (edit != null) { - ctx.collect(edit); + public void run(SourceContext ctx) throws Exception { + try (WikipediaEditEventIrcStream ircStream = new WikipediaEditEventIrcStream(host, port)) { + // Open connection and join channel + ircStream.connect(); + ircStream.join(channel); + + try { + while (isRunning) { + // Query for the next edit event + WikipediaEditEvent edit = ircStream.getEdits().poll(100, TimeUnit.MILLISECONDS); + + if (edit != null) { + ctx.collect(edit); + } + } + } finally { + ircStream.leave(channel); } } } From 0bc2cf9b479a4430ebc895c4c6db54d2a7f601dc Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 29 Nov 2017 16:36:29 +0100 Subject: [PATCH 130/367] [FLINK-7762] [connector-wikiedits] Make WikipediaEditsSourceTest proper test The WikipediaEditsSourceTest unnecessarily implements an integration test that starts a FlinkMiniCluster and executes a small Flink program. This simply creates a source and executes run in a separate thread until a single WikipediaEditEvent is received. This closes #5102. --- .../flink-connector-wikiedits/pom.xml | 7 +- .../wikiedits/WikipediaEditsSourceTest.java | 172 ++++++++++++------ 2 files changed, 125 insertions(+), 54 deletions(-) diff --git a/flink-contrib/flink-connector-wikiedits/pom.xml b/flink-contrib/flink-connector-wikiedits/pom.xml index 20b6db498dff2..08f491e90a2da 100644 --- a/flink-contrib/flink-connector-wikiedits/pom.xml +++ b/flink-contrib/flink-connector-wikiedits/pom.xml @@ -40,11 +40,16 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} - org.schwering irclib 1.10 + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + diff --git a/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java b/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java index 439aa36418def..f6fa8e0b53924 100644 --- a/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java +++ b/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java @@ -18,19 +18,29 @@ package org.apache.flink.streaming.connectors.wikiedits; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.testutils.junit.RetryOnFailure; +import org.apache.flink.testutils.junit.RetryRule; +import org.junit.Rule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; import java.net.Socket; +import java.util.Objects; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -40,71 +50,127 @@ public class WikipediaEditsSourceTest { private static final Logger LOG = LoggerFactory.getLogger(WikipediaEditsSourceTest.class); + @Rule + public RetryRule retryRule = new RetryRule(); + /** - * NOTE: if you are behind a firewall you may need to use a SOCKS Proxy for this test. - * - *

    We first check the connection to the IRC server. If it fails, this test - * is effectively ignored. - * - * @see Socks Proxy + * We first check the connection to the IRC server. If it fails, this test is ignored. */ - @Test(timeout = 120 * 1000) + @Test + @RetryOnFailure(times = 1) public void testWikipediaEditsSource() throws Exception { - final int numRetries = 5; - final int waitBetweenRetriesMillis = 2000; - final int connectTimeout = 1000; + if (canConnect(1, TimeUnit.SECONDS)) { + final Time testTimeout = Time.seconds(60); + final WikipediaEditsSource wikipediaEditsSource = new WikipediaEditsSource(); + + ExecutorService executorService = null; + try { + executorService = Executors.newSingleThreadExecutor(); + BlockingQueue collectedEvents = new ArrayBlockingQueue<>(1); + AtomicReference asyncError = new AtomicReference<>(); + + // Execute the source in a different thread and collect events into the queue. + // We do this in a separate thread in order to not block the main test thread + // indefinitely in case that somethign bad happens (like not receiving any + // events) + executorService.execute(() -> { + try { + wikipediaEditsSource.run(new CollectingSourceContext<>(collectedEvents)); + } catch (Exception e) { + boolean interrupted = e instanceof InterruptedException; + if (!interrupted) { + LOG.warn("Failure in WikipediaEditsSource", e); + } + + asyncError.compareAndSet(null, e); + } + }); + + long deadline = deadlineNanos(testTimeout); - boolean success = false; + Object event = null; + Exception error = null; - for (int i = 0; i < numRetries && !success; i++) { - // Check connection - boolean canConnect = false; + // Check event or error + while (event == null && error == null && System.nanoTime() < deadline) { + event = collectedEvents.poll(1, TimeUnit.SECONDS); + error = asyncError.get(); + } + + if (error != null) { + // We don't use assertNull, because we want to include the error message + fail("Failure in WikipediaEditsSource: " + error.getMessage()); + } - String host = WikipediaEditsSource.DEFAULT_HOST; - int port = WikipediaEditsSource.DEFAULT_PORT; + assertNotNull("Did not receive a WikipediaEditEvent within the desired timeout", event); + assertTrue("Received unexpected event " + event, event instanceof WikipediaEditEvent); + } finally { + wikipediaEditsSource.cancel(); - try (Socket s = new Socket()) { - s.connect(new InetSocketAddress(host, port), connectTimeout); - canConnect = s.isConnected(); - } catch (Throwable ignored) { + if (executorService != null) { + executorService.shutdownNow(); + executorService.awaitTermination(1, TimeUnit.SECONDS); + } } + } else { + LOG.info("Skipping test, because not able to connect to IRC server."); + } + } - if (canConnect) { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().disableSysoutLogging(); + private long deadlineNanos(Time testTimeout) { + return System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(testTimeout.toMilliseconds()); + } - DataStream edits = env.addSource(new WikipediaEditsSource()); + private static class CollectingSourceContext implements SourceFunction.SourceContext { - edits.addSink(new SinkFunction() { - @Override - public void invoke(WikipediaEditEvent value) throws Exception { - throw new Exception("Expected test exception"); - } - }); + private final BlockingQueue events; - try { - env.execute(); - fail("Did not throw expected Exception."); - } catch (Exception e) { - assertNotNull(e.getCause()); - assertEquals("Expected test exception", e.getCause().getMessage()); - } + private CollectingSourceContext(BlockingQueue events) { + this.events = Objects.requireNonNull(events, "events"); + } - success = true; - } else { - LOG.info("Failed to connect to IRC server ({}/{}). Retrying in {} ms.", - i + 1, - numRetries, - waitBetweenRetriesMillis); + @Override + public void collect(T element) { + events.offer(element); + } + + @Override + public void collectWithTimestamp(T element, long timestamp) { + throw new UnsupportedOperationException(); - Thread.sleep(waitBetweenRetriesMillis); - } } - if (success) { - LOG.info("Successfully ran test."); - } else { - LOG.info("Skipped test, because not able to connect to IRC server."); + @Override + public void emitWatermark(Watermark mark) { + throw new UnsupportedOperationException(); + } + + @Override + public void markAsTemporarilyIdle() { + throw new UnsupportedOperationException(); + } + + @Override + public Object getCheckpointLock() { + throw new UnsupportedOperationException(); } + + @Override + public void close() { + } + + } + + private static boolean canConnect(int timeout, TimeUnit unit) { + String host = WikipediaEditsSource.DEFAULT_HOST; + int port = WikipediaEditsSource.DEFAULT_PORT; + + try (Socket s = new Socket()) { + s.connect(new InetSocketAddress(host, port), (int) unit.toMillis(timeout)); + return s.isConnected(); + } catch (Throwable ignored) { + } + + return false; } } From cfd1b787b6e4fb3aee2a7fc76d920032f7d17fd5 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 29 Nov 2017 17:04:20 +0100 Subject: [PATCH 131/367] [FLINK-8177] Replace TestingContainer by mock in YARN UtilsTest The Container Interface was extended in Hadoop 2.9, meaning that the test would not run when compiling with Hadoop 2.9. Using a mock fixes this problem. --- .../java/org/apache/flink/yarn/UtilsTest.java | 97 +++---------------- 1 file changed, 11 insertions(+), 86 deletions(-) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java index 69c6e7d849698..33072910de107 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java @@ -43,9 +43,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; @@ -73,10 +70,12 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Tests for {@link Utils}. */ + public class UtilsTest extends TestLogger { private static ActorSystem system; @@ -118,7 +117,15 @@ public void testYarnFlinkResourceManagerJobManagerLostLeadership() throws Except final List containerList = new ArrayList<>(); for (int i = 0; i < numInitialTaskManagers; i++) { - containerList.add(new TestingContainer("container", 1234, i)); + Container mockContainer = mock(Container.class); + when(mockContainer.getId()).thenReturn( + ContainerId.newInstance( + ApplicationAttemptId.newInstance( + ApplicationId.newInstance(System.currentTimeMillis(), 1), + 1), + i)); + when(mockContainer.getNodeId()).thenReturn(NodeId.newInstance("container", 1234)); + containerList.add(mockContainer); } doAnswer(new Answer() { @@ -221,86 +228,4 @@ public Object answer(InvocationOnMock invocation) throws Throwable { } }}; } - - static class TestingContainer extends Container { - - private final NodeId nodeId; - private final ContainerId containerId; - - TestingContainer(String host, int port, int containerId) { - this.nodeId = NodeId.newInstance(host, port); - this.containerId = ContainerId.newInstance( - ApplicationAttemptId.newInstance( - ApplicationId.newInstance( - System.currentTimeMillis(), - 1), - 1), - containerId); - } - - @Override - public ContainerId getId() { - return containerId; - } - - @Override - public void setId(ContainerId containerId) { - - } - - @Override - public NodeId getNodeId() { - return nodeId; - } - - @Override - public void setNodeId(NodeId nodeId) { - - } - - @Override - public String getNodeHttpAddress() { - return null; - } - - @Override - public void setNodeHttpAddress(String s) { - - } - - @Override - public Resource getResource() { - return null; - } - - @Override - public void setResource(Resource resource) { - - } - - @Override - public Priority getPriority() { - return null; - } - - @Override - public void setPriority(Priority priority) { - - } - - @Override - public Token getContainerToken() { - return null; - } - - @Override - public void setContainerToken(Token token) { - - } - - @Override - public int compareTo(Container o) { - return 0; - } - } } From a86cb885ce0ed4255354cf0012c7e7e5d88777e6 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 24 Nov 2017 14:54:41 +0100 Subject: [PATCH 132/367] [FLINK-8148][yarn/s3] fix test instability in YarnFileStageTestS3ITCase Remove a check for a deleted directory since we may not see our own delete yet with S3. This closes #5066. --- .../test/java/org/apache/flink/yarn/YarnFileStageTest.java | 2 +- .../java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java index 4d3825311aef3..5cbe1be7eeafb 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java @@ -139,7 +139,7 @@ public void testCopyFromLocalRecursiveWithoutScheme() throws Exception { * @param addSchemeToLocalPath * whether add the file:// scheme to the local path to copy from */ - public static void testCopyFromLocalRecursive( + static void testCopyFromLocalRecursive( FileSystem targetFileSystem, Path targetDir, TemporaryFolder tempFolder, diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java index 74fb5963179d6..e1e95b1c379d1 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java @@ -42,7 +42,6 @@ import java.util.UUID; import static org.hamcrest.Matchers.greaterThan; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assume.assumeFalse; import static org.junit.Assume.assumeNoException; @@ -166,9 +165,6 @@ private void testRecursiveUploadForYarn(String scheme, String pathSuffix) throws YarnFileStageTest.testCopyFromLocalRecursive(fs.getHadoopFileSystem(), new org.apache.hadoop.fs.Path(directory.toUri()), tempFolder, true); - - // now directory must be gone - assertFalse(fs.exists(directory)); } finally { // clean up fs.delete(basePath, true); From d9551ae1dcfae76064786d37dd1a3cf90463ab41 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 4 Dec 2017 16:56:15 +0100 Subject: [PATCH 133/367] [FLINK-8194] [akka] Suppress Java serializer used warnings from Akka --- .../main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index ab559a13927f4..b80a0701ad033 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -224,6 +224,9 @@ object AkkaUtils { | | actor { | guardian-supervisor-strategy = $supervisorStrategy + | + | warn-about-java-serializer-usage = off + | | default-dispatcher { | throughput = $akkaThroughput | From 928daedfed27d031d30c8a631c522ada738fe83f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 4 Dec 2017 17:06:50 +0100 Subject: [PATCH 134/367] [hotfix] Remove empty line in UtilsTest --- flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java index 33072910de107..67783551410e9 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java @@ -75,7 +75,6 @@ /** * Tests for {@link Utils}. */ - public class UtilsTest extends TestLogger { private static ActorSystem system; From 810be6dcae3eb5af2d56a61bf6be5dff1cdf149a Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 4 Dec 2017 15:05:46 +0800 Subject: [PATCH 135/367] [FLINK-8190] [kafka] Add constructors to expose topic pattern-based subscription This closes #5117. --- docs/dev/connectors/kafka.md | 55 +++++++++++++++- .../kafka/FlinkKafkaConsumer010.java | 45 +++++++++++++ .../kafka/FlinkKafkaConsumer011.java | 45 +++++++++++++ .../kafka/FlinkKafkaConsumer08.java | 62 +++++++++++++++++- .../kafka/FlinkKafkaConsumer09.java | 64 ++++++++++++++++++- 5 files changed, 265 insertions(+), 6 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 5376d5be2c8f2..6c80370db4089 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -294,7 +294,9 @@ Flink on YARN supports automatic restart of lost YARN containers. If checkpointing is not enabled, the Kafka consumer will periodically commit the offsets to Zookeeper. -### Kafka Consumers Partition Discovery +### Kafka Consumers Topic and Partition Discovery + +#### Partition discovery The Flink Kafka Consumer supports discovering dynamically created Kafka partitions, and consumes them with exactly-once guarantees. All partitions discovered after the initial retrieval of partition metadata (i.e., when the @@ -309,6 +311,57 @@ prior to Flink 1.3.x, partition discovery cannot be enabled on the restore run. with an exception. In this case, in order to use partition discovery, please first take a savepoint in Flink 1.3.x and then restore again from that. +#### Topic discovery + +At a higher-level, the Flink Kafka Consumer is also capable of discovering topics, based on pattern matching on the +topic names using regular expressions. See the below for an example: + +
    +
    +{% highlight java %} +final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +Properties properties = new Properties(); +properties.setProperty("bootstrap.servers", "localhost:9092"); +properties.setProperty("group.id", "test"); + +FlinkKafkaConsumer011 myConsumer = new FlinkKafkaConsumer011<>( + java.util.regex.Pattern.compile("test-topic-[0-9]"), + new SimpleStringSchema(), + properties); + +DataStream stream = env.addSource(myConsumer); +... +{% endhighlight %} +
    +
    +{% highlight scala %} +val env = StreamExecutionEnvironment.getExecutionEnvironment() + +val properties = new Properties() +properties.setProperty("bootstrap.servers", "localhost:9092") +properties.setProperty("group.id", "test") + +val myConsumer = new FlinkKafkaConsumer08[String]( + java.util.regex.Pattern.compile("test-topic-[0-9]"), + new SimpleStringSchema, + properties) + +val stream = env.addSource(myConsumer) +... +{% endhighlight %} +
    +
    + +In the above example, all topics with names that match the specified regular expression +(starting with `test-topic-` and ending with a single digit) will be subscribed by the consumer +when the job starts running. + +To allow the consumer to discover dynamically created topics after the job started running, +set a non-negative value for `flink.partition-discovery.interval-millis`. This allows +the consumer to discover partitions of new topics with names that also match the specified +pattern. + ### Kafka Consumers Offset Committing Behaviour Configuration The Flink Kafka Consumer allows configuring the behaviour of how offsets diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java index f56947729aadf..6fb63e17b0852 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; @@ -39,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from @@ -126,6 +128,49 @@ public FlinkKafkaConsumer010(List topics, KeyedDeserializationSchema super(topics, deserializer, props); } + /** + * Creates a new Kafka streaming source consumer for Kafka 0.10.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer010#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param valueDeserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer010(Pattern subscriptionPattern, DeserializationSchema valueDeserializer, Properties props) { + this(subscriptionPattern, new KeyedDeserializationSchemaWrapper<>(valueDeserializer), props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.10.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer010#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + *

    This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + * pairs, offsets, and topic names from Kafka. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer010(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + } + @Override protected AbstractFetcher createFetcher( SourceContext sourceContext, diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java index 6f75828692e8f..c40463e422747 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; @@ -24,6 +25,7 @@ import java.util.Collections; import java.util.List; import java.util.Properties; +import java.util.regex.Pattern; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from @@ -110,4 +112,47 @@ public FlinkKafkaConsumer011(List topics, DeserializationSchema deser public FlinkKafkaConsumer011(List topics, KeyedDeserializationSchema deserializer, Properties props) { super(topics, deserializer, props); } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.11.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer011#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param valueDeserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer011(Pattern subscriptionPattern, DeserializationSchema valueDeserializer, Properties props) { + this(subscriptionPattern, new KeyedDeserializationSchemaWrapper<>(valueDeserializer), props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.11.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer011#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + *

    This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + * pairs, offsets, and topic names from Kafka. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer011(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + } } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index 0a70f61e2716e..f362046e25d68 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; @@ -39,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.PropertiesUtil.getLong; @@ -156,13 +158,67 @@ public FlinkKafkaConsumer08(List topics, DeserializationSchema deseri * The properties that are used to configure both the fetcher and the offset handler. */ public FlinkKafkaConsumer08(List topics, KeyedDeserializationSchema deserializer, Properties props) { + this(topics, null, deserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.8.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer08#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param valueDeserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer08(Pattern subscriptionPattern, DeserializationSchema valueDeserializer, Properties props) { + this(subscriptionPattern, new KeyedDeserializationSchemaWrapper<>(valueDeserializer), props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.8.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer08#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + *

    This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + * pairs, offsets, and topic names from Kafka. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer08(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + this(null, subscriptionPattern, deserializer, props); + } + + private FlinkKafkaConsumer08( + List topics, + Pattern subscriptionPattern, + KeyedDeserializationSchema deserializer, + Properties props) { + super( topics, - null, + subscriptionPattern, deserializer, - getLong(props, KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, PARTITION_DISCOVERY_DISABLED)); + getLong( + checkNotNull(props, "props"), + KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, PARTITION_DISCOVERY_DISABLED)); - this.kafkaProperties = checkNotNull(props, "props"); + this.kafkaProperties = props; // validate the zookeeper properties validateZooKeeperConfig(props); diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java index 65be712a35607..79be73ced73a7 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; @@ -42,6 +43,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.PropertiesUtil.getLong; @@ -147,9 +149,67 @@ public FlinkKafkaConsumer09(List topics, DeserializationSchema deseri * The properties that are used to configure both the fetcher and the offset handler. */ public FlinkKafkaConsumer09(List topics, KeyedDeserializationSchema deserializer, Properties props) { - super(topics, null, deserializer, getLong(props, KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, PARTITION_DISCOVERY_DISABLED)); + this(topics, null, deserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.9.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer09#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param valueDeserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer09(Pattern subscriptionPattern, DeserializationSchema valueDeserializer, Properties props) { + this(subscriptionPattern, new KeyedDeserializationSchemaWrapper<>(valueDeserializer), props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.9.x. Use this constructor to + * subscribe to multiple topics based on a regular expression pattern. + * + *

    If partition discovery is enabled (by setting a non-negative value for + * {@link FlinkKafkaConsumer09#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics + * with names matching the pattern will also be subscribed to as they are created on the fly. + * + *

    This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + * pairs, offsets, and topic names from Kafka. + * + * @param subscriptionPattern + * The regular expression for a pattern of topic names to subscribe to. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + @PublicEvolving + public FlinkKafkaConsumer09(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + this(null, subscriptionPattern, deserializer, props); + } + + private FlinkKafkaConsumer09( + List topics, + Pattern subscriptionPattern, + KeyedDeserializationSchema deserializer, + Properties props) { + + super( + topics, + subscriptionPattern, + deserializer, + getLong( + checkNotNull(props, "props"), + KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, PARTITION_DISCOVERY_DISABLED)); - this.properties = checkNotNull(props, "props"); + this.properties = props; setDeserializer(this.properties); // configure the polling timeout From f38d9403e9b11408d724fbe0cae4574b2fd2d2d0 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 5 Dec 2017 00:34:57 +0800 Subject: [PATCH 136/367] [hotfix] [kafka] Fix outdated Javadoc reference to non-existing restoreState method --- .../streaming/connectors/kafka/FlinkKafkaConsumerBase.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 7cd1ae1e241a4..865d66c3c1412 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -162,8 +162,7 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti /** * The offsets to restore to, if the consumer restores state from a checkpoint. * - *

    This map will be populated either by the legacy {@link #restoreState(HashMap)} method - * or {@link #initializeState(FunctionInitializationContext)}. + *

    This map will be populated by the {@link #initializeState(FunctionInitializationContext)} method. * *

    Using a sorted map as the ordering is important when using restored state * to seed the partition discoverer. From b53e3d65c980696d548d251a37eecf5c60934c8a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 4 Dec 2017 17:32:08 +0100 Subject: [PATCH 137/367] [FLINK-8196] [build] Remove 'javax.servlet' dependency exclusion. --- .../flink-shaded-hadoop2/pom.xml | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml index 66d8549bbfb3b..a6851aa2f7436 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml @@ -115,10 +115,6 @@ under the License. tomcat jasper-runtime - - javax.servlet - servlet-api - javax.servlet.jsp jsp-api @@ -243,10 +239,6 @@ under the License. tomcat jasper-runtime - - javax.servlet - servlet-api - javax.servlet.jsp jsp-api @@ -371,10 +363,6 @@ under the License. tomcat jasper-runtime - - javax.servlet - servlet-api - javax.servlet.jsp jsp-api @@ -499,10 +487,6 @@ under the License. tomcat jasper-runtime - - javax.servlet - servlet-api - javax.servlet.jsp jsp-api @@ -627,10 +611,6 @@ under the License. tomcat jasper-runtime - - javax.servlet - servlet-api - javax.servlet.jsp jsp-api From 95ac289ef41c04071fd93eac1ceba64cc8757806 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 24 Nov 2017 13:55:51 +0100 Subject: [PATCH 138/367] [hotfix] [core] Fix remaining checkstyle issues for 'core.fs' --- .../org/apache/flink/core/fs/FileSystem.java | 2 ++ .../flink/core/fs/FileSystemSafetyNet.java | 20 +++++++------- .../core/fs/SafetyNetCloseableRegistry.java | 26 +++++++++++-------- .../core/fs/SafetyNetWrapperFileSystem.java | 2 ++ .../core/fs/local/LocalBlockLocation.java | 4 +-- .../core/fs/local/LocalDataInputStream.java | 11 ++++---- .../core/fs/local/LocalDataOutputStream.java | 9 +++---- .../flink/core/fs/local/LocalFileStatus.java | 7 +++-- .../flink/core/fs/local/LocalFileSystem.java | 20 +++++++------- .../fs/AbstractCloseableRegistryTest.java | 14 +++++++--- .../flink/core/fs/CloseableRegistryTest.java | 3 +++ .../apache/flink/core/fs/FileSystemTest.java | 5 ++++ .../flink/core/fs/InitOutputPathTest.java | 12 +++++---- ...edConnectionsFileSystemDelegationTest.java | 1 - .../fs/LimitedConnectionsFileSystemTest.java | 2 +- .../org/apache/flink/core/fs/PathTest.java | 22 +++++++++++----- .../fs/SafetyNetCloseableRegistryTest.java | 7 ++--- .../core/fs/local/LocalFileSystemTest.java | 26 +++++++++---------- tools/maven/suppressions-core.xml | 8 ------ 19 files changed, 113 insertions(+), 88 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index 18baaa5d5f95f..07a1e76c0c3c2 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -229,7 +229,9 @@ public enum WriteMode { /** The default filesystem scheme to be used, configured during process-wide initialization. * This value defaults to the local file systems scheme {@code 'file:///'} or {@code 'file:/'}. */ + //CHECKSTYLE.OFF: StaticVariableName private static URI DEFAULT_SCHEME; + //CHECKSTYLE.ON: StaticVariableName // ------------------------------------------------------------------------ // Initialization diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java index c06ccacdf0666..d72aec0343959 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemSafetyNet.java @@ -30,22 +30,22 @@ * When activated for a thread, it tracks all streams that are opened by FileSystems that the thread * obtains. The safety net has a global cleanup hook that will close all streams that were * not properly closed. - * + * *

    The main thread of each Flink task, as well as the checkpointing thread are automatically guarded * by this safety net. - * + * *

    Important: This safety net works only for streams created by Flink's FileSystem abstraction, * i.e., for {@code FileSystem} instances obtained via {@link FileSystem#get(URI)} or through * {@link Path#getFileSystem()}. - * + * *

    Important: When a guarded thread obtains a {@code FileSystem} or a stream and passes them * to another thread, the safety net will close those resources once the former thread finishes. - * + * *

    The safety net can be used as follows: *

    {@code
    - * 
    + *
      * class GuardedThread extends Thread {
    - * 
    + *
      *     public void run() {
      *         FileSystemSafetyNet.initializeSafetyNetForThread();
      *         try {
    @@ -62,7 +62,7 @@
     @Internal
     public class FileSystemSafetyNet {
     
    -	/** The map from thread to the safety net registry for that thread */
    +	/** The map from thread to the safety net registry for that thread. */
     	private static final ThreadLocal REGISTRIES = new ThreadLocal<>();
     
     	// ------------------------------------------------------------------------
    @@ -73,9 +73,9 @@ public class FileSystemSafetyNet {
     	 * Activates the safety net for a thread. {@link FileSystem} instances obtained by the thread
     	 * that called this method will be guarded, meaning that their created streams are tracked and can
     	 * be closed via the safety net closing hook.
    -	 * 
    +	 *
     	 * 

    This method should be called at the beginning of a thread that should be guarded. - * + * * @throws IllegalStateException Thrown, if a safety net was already registered for the thread. */ @Internal @@ -94,7 +94,7 @@ public static void initializeSafetyNetForThread() { * Closes the safety net for a thread. This closes all remaining unclosed streams that were opened * by safety-net-guarded file systems. After this method was called, no streams can be opened any more * from any FileSystem instance that was obtained while the thread was guarded by the safety net. - * + * *

    This method should be called at the very end of a guarded thread. */ @Internal diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetCloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetCloseableRegistry.java index 9c4272f0acc68..ccf944eb2accc 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetCloseableRegistry.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetCloseableRegistry.java @@ -39,13 +39,13 @@ /** * This implementation of an {@link AbstractCloseableRegistry} registers {@link WrappingProxyCloseable}. When * the proxy becomes subject to GC, this registry takes care of closing unclosed {@link Closeable}s. - *

    - * Phantom references are used to track when {@link org.apache.flink.util.WrappingProxy}s of {@link Closeable} got + * + *

    Phantom references are used to track when {@link org.apache.flink.util.WrappingProxy}s of {@link Closeable} got * GC'ed. We ensure that the wrapped {@link Closeable} is properly closed to avoid resource leaks. - *

    - * Other than that, it works like a normal {@link CloseableRegistry}. - *

    - * All methods in this class are thread-safe. + * + *

    Other than that, it works like a normal {@link CloseableRegistry}. + * + *

    All methods in this class are thread-safe. */ @Internal public class SafetyNetCloseableRegistry extends @@ -54,15 +54,19 @@ public class SafetyNetCloseableRegistry extends private static final Logger LOG = LoggerFactory.getLogger(SafetyNetCloseableRegistry.class); - /** Lock for atomic modifications to reaper thread and registry count */ + /** Lock for atomic modifications to reaper thread and registry count. */ private static final Object REAPER_THREAD_LOCK = new Object(); - /** Singleton reaper thread takes care of all registries in VM */ + //CHECKSTYLE.OFF: StaticVariableName + + /** Singleton reaper thread takes care of all registries in VM. */ private static CloseableReaperThread REAPER_THREAD = null; - /** Global count of all instances of SafetyNetCloseableRegistry */ + /** Global count of all instances of SafetyNetCloseableRegistry. */ private static int GLOBAL_SAFETY_NET_REGISTRY_COUNT = 0; + //CHECKSTYLE.ON: StaticVariableName + SafetyNetCloseableRegistry() { super(new IdentityHashMap<>()); @@ -166,7 +170,7 @@ public void close() throws IOException { } /** - * Reaper runnable collects and closes leaking resources + * Reaper runnable collects and closes leaking resources. */ static final class CloseableReaperThread extends Thread { @@ -187,7 +191,7 @@ public void run() { try { while (running) { final PhantomDelegatingCloseableRef toClose = (PhantomDelegatingCloseableRef) referenceQueue.remove(); - + if (toClose != null) { try { LOG.warn("Closing unclosed resource via safety-net: {}", toClose.getDebugString()); diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java index e7f43a464d063..92b3a74c2ccb2 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java @@ -82,6 +82,7 @@ public FSDataInputStream open(Path f) throws IOException { } @Override + @SuppressWarnings("deprecation") public long getDefaultBlockSize() { return unsafeFileSystem.getDefaultBlockSize(); } @@ -107,6 +108,7 @@ public boolean mkdirs(Path f) throws IOException { } @Override + @SuppressWarnings("deprecation") public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException { diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalBlockLocation.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalBlockLocation.java index 9825781dfdfb3..25dd92d529fcc 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalBlockLocation.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalBlockLocation.java @@ -18,11 +18,11 @@ package org.apache.flink.core.fs.local; -import java.io.IOException; - import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.BlockLocation; +import java.io.IOException; + /** * Implementation of the {@link BlockLocation} interface for a local file system. */ diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java index 172da796f4f43..63017e385ad65 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java @@ -22,6 +22,7 @@ import org.apache.flink.core.fs.FSDataInputStream; import javax.annotation.Nonnull; + import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -40,9 +41,9 @@ public class LocalDataInputStream extends FSDataInputStream { /** * Constructs a new LocalDataInputStream object from a given {@link File} object. - * + * * @param file The File the data stream is read from - * + * * @throws IOException Thrown if the data input stream cannot be created. */ public LocalDataInputStream(File file) throws IOException { @@ -71,18 +72,18 @@ public int read() throws IOException { public int read(@Nonnull byte[] buffer, int offset, int length) throws IOException { return this.fis.read(buffer, offset, length); } - + @Override public void close() throws IOException { // Accoring to javadoc, this also closes the channel this.fis.close(); } - + @Override public int available() throws IOException { return this.fis.available(); } - + @Override public long skip(final long n) throws IOException { return this.fis.skip(n); diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java index 5cc011bc28454..aa64593acdc34 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java @@ -18,13 +18,13 @@ package org.apache.flink.core.fs.local; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.fs.FSDataOutputStream; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.fs.FSDataOutputStream; - /** * The LocalDataOutputStream class is a wrapper class for a data * output stream to the local file system. @@ -37,7 +37,7 @@ public class LocalDataOutputStream extends FSDataOutputStream { /** * Constructs a new LocalDataOutputStream object from a given {@link File} object. - * + * * @param file * the {@link File} object the data stream is read from * @throws IOException @@ -62,7 +62,6 @@ public void close() throws IOException { fos.close(); } - @Override public void flush() throws IOException { fos.flush(); diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileStatus.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileStatus.java index 63e999da3cbef..781e0d3014a88 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileStatus.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileStatus.java @@ -16,16 +16,15 @@ * limitations under the License. */ - package org.apache.flink.core.fs.local; -import java.io.File; - import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.FileStatus; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; +import java.io.File; + /** * The class LocalFileStatus provides an implementation of the {@link FileStatus} interface * for the local file system. @@ -45,7 +44,7 @@ public class LocalFileStatus implements FileStatus { /** * Creates a LocalFileStatus object from a given {@link File} object. - * + * * @param f * the {@link File} object this LocalFileStatus refers to * @param fs diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java index a96f2212d046c..c3e5a2fcc0ebd 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java @@ -16,10 +16,10 @@ * limitations under the License. */ - -/** - * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for +/* + * Parts of earlier versions of this file were based on source code from the + * Hadoop Project (http://hadoop.apache.org/), licensed by the Apache Software Foundation (ASF) + * under the Apache License, Version 2.0. See the NOTICE file distributed with this work for * additional information regarding copyright ownership. */ @@ -65,7 +65,7 @@ public class LocalFileSystem extends FileSystem { /** The URI representing the local file system. */ private static final URI LOCAL_URI = OperatingSystem.isWindows() ? URI.create("file:/") : URI.create("file:///"); - /** The shared instance of the local file system */ + /** The shared instance of the local file system. */ private static final LocalFileSystem INSTANCE = new LocalFileSystem(); /** Path pointing to the current working directory. @@ -73,10 +73,10 @@ public class LocalFileSystem extends FileSystem { private final String workingDir; /** Path pointing to the current working directory. - * Because Paths are not immutable, we cannot cache the proper path here */ + * Because Paths are not immutable, we cannot cache the proper path here. */ private final String homeDir; - /** The host name of this machine */ + /** The host name of this machine. */ private final String hostName; /** @@ -112,7 +112,7 @@ public FileStatus getFileStatus(Path f) throws IOException { } else { throw new FileNotFoundException("File " + f + " does not exist or the user running " - + "Flink ('"+System.getProperty("user.name")+"') has insufficient permissions to access it."); + + "Flink ('" + System.getProperty("user.name") + "') has insufficient permissions to access it."); } } @@ -149,7 +149,6 @@ private File pathToFile(Path path) { return new File(path.toUri().getPath()); } - @Override public FileStatus[] listStatus(final Path f) throws IOException { @@ -175,7 +174,6 @@ public FileStatus[] listStatus(final Path f) throws IOException { return results; } - @Override public boolean delete(final Path f, final boolean recursive) throws IOException { @@ -233,7 +231,7 @@ private boolean delete(final File f) throws IOException { public boolean mkdirs(final Path f) throws IOException { final File p2f = pathToFile(f); - if(p2f.isDirectory()) { + if (p2f.isDirectory()) { return true; } diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java index f9425f39a2021..eb07378493b0f 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java @@ -32,6 +32,9 @@ import static org.mockito.Mockito.verify; import static org.powermock.api.mockito.PowerMockito.spy; +/** + * Tests for the {@link AbstractCloseableRegistry}. + */ public abstract class AbstractCloseableRegistryTest { protected ProducerThread[] streamOpenThreads; @@ -140,8 +143,7 @@ public void testNonBlockingClose() throws Exception { try { closeableRegistry.registerCloseable(testCloseable); Assert.fail("Closed registry should not accept closeables!"); - }catch (IOException ignore) { - } + } catch (IOException ignored) {} blockCloseLatch.trigger(); closer.join(); @@ -151,7 +153,10 @@ public void testNonBlockingClose() throws Exception { Assert.assertEquals(0, closeableRegistry.getNumberOfRegisteredCloseables()); } - protected static abstract class ProducerThread extends Thread { + /** + * A testing producer. + */ + protected abstract static class ProducerThread extends Thread { protected final AbstractCloseableRegistry registry; protected final AtomicInteger refCount; @@ -188,6 +193,9 @@ public void run() { } } + /** + * Testing stream which adds itself to a reference counter while not closed. + */ protected static final class TestStream extends FSDataInputStream { protected AtomicInteger refCount; diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/CloseableRegistryTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/CloseableRegistryTest.java index c3bf6e677fc35..8a0fb9679b9d9 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/CloseableRegistryTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/CloseableRegistryTest.java @@ -24,6 +24,9 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; +/** + * Tests for the {@link CloseableRegistry}. + */ public class CloseableRegistryTest extends AbstractCloseableRegistryTest { @Override diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java index 1bde2fb5d06db..598b1e11a0378 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java @@ -15,10 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.core.fs; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.util.WrappingProxyUtil; + import org.junit.Test; import java.io.IOException; @@ -27,6 +29,9 @@ import static org.junit.Assert.assertTrue; +/** + * Tests for the {@link FileSystem} base class. + */ public class FileSystemTest { @Test diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/InitOutputPathTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/InitOutputPathTest.java index 9b67388dd5db9..d6de9b683bc1d 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/InitOutputPathTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/InitOutputPathTest.java @@ -28,7 +28,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; - import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -40,9 +39,12 @@ import java.lang.reflect.Field; import java.util.concurrent.locks.ReentrantLock; -import static org.powermock.api.mockito.PowerMockito.*; -import static org.junit.Assert.*; +import static org.junit.Assert.fail; +import static org.powermock.api.mockito.PowerMockito.whenNew; +/** + * A test validating that the initialization of local output paths is properly synchronized. + */ @RunWith(PowerMockRunner.class) @PrepareForTest(LocalFileSystem.class) public class InitOutputPathTest { @@ -79,7 +81,7 @@ public void testErrorOccursUnSynchronized() throws Exception { @Test public void testProperSynchronized() throws Exception { - // in the synchronized variant, we cannot use the "await latches" because not + // in the synchronized variant, we cannot use the "await latches" because not // both threads can make process interleaved (due to the synchronization) // the test uses sleeps (rather than latches) to produce the same interleaving. // while that is not guaranteed to produce the pathological interleaving, @@ -121,7 +123,7 @@ public LocalDataOutputStream answer(InvocationOnMock invocation) throws Throwabl }); final LocalFileSystem fs1 = new SyncedFileSystem( - deleteAwaitLatch1, mkdirsAwaitLatch1, + deleteAwaitLatch1, mkdirsAwaitLatch1, deleteTriggerLatch1, mkdirsTriggerLatch1); final LocalFileSystem fs2 = new SyncedFileSystem( diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java index b13367788b4a6..2e04648ce24b0 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemDelegationTest.java @@ -28,7 +28,6 @@ import java.util.Random; import static org.junit.Assert.assertEquals; - import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyInt; diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java index 509b4aefd6539..7391877d2e442 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsFileSystemTest.java @@ -627,7 +627,7 @@ public void go() throws Exception { } } - private static abstract class BlockingThread extends CheckedThread { + private abstract static class BlockingThread extends CheckedThread { private final OneShotLatch waiter = new OneShotLatch(); diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java index 66816ada3c2f1..b4da2dc00ba5a 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java @@ -15,13 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.core.fs; +import org.junit.Test; + import java.io.IOException; import java.net.URI; -import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link Path} class. + */ public class PathTest { @Test @@ -70,23 +80,23 @@ public void testPathFromString() { assertEquals("/C:/my/windows/path", p.toUri().getPath()); try { - new Path((String)null); + new Path((String) null); fail(); - } catch(Exception e) { + } catch (Exception e) { // exception expected } try { new Path(""); fail(); - } catch(Exception e) { + } catch (Exception e) { // exception expected } try { new Path(" "); fail(); - } catch(Exception e) { + } catch (Exception e) { // exception expected } diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java index 4ceda5033a5c0..5474f9905ae78 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/SafetyNetCloseableRegistryTest.java @@ -32,6 +32,9 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; +/** + * Tests for the {@link SafetyNetCloseableRegistry}. + */ public class SafetyNetCloseableRegistryTest extends AbstractCloseableRegistryTest, SafetyNetCloseableRegistry.PhantomDelegatingCloseableRef> { @@ -44,9 +47,7 @@ protected WrappingProxyCloseable createCloseable() { return new WrappingProxyCloseable() { @Override - public void close() throws IOException { - - } + public void close() throws IOException {} @Override public Closeable getWrappedDelegate() { diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java index 96c526944fa93..2352404d4ed7a 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java @@ -18,25 +18,13 @@ package org.apache.flink.core.fs.local; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.Arrays; -import java.util.UUID; - import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileStatus; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; -import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.util.FileUtils; import org.junit.Assume; @@ -44,6 +32,18 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.UUID; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + /** * This class tests the functionality of the {@link LocalFileSystem} class in its components. In particular, * file/directory access, creation, deletion, read, write is tested. diff --git a/tools/maven/suppressions-core.xml b/tools/maven/suppressions-core.xml index 066b30addf772..78341c9a76a13 100644 --- a/tools/maven/suppressions-core.xml +++ b/tools/maven/suppressions-core.xml @@ -111,14 +111,6 @@ under the License. files="(.*)test[/\\](.*)configuration[/\\](.*)" checks="AvoidStarImport"/> - - - - From 155149855ba39582ffe7507bd808259dbe33ce81 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 5 Dec 2017 12:21:08 +0100 Subject: [PATCH 139/367] [FLINK-8198] [core] Fix condition for parsing ConnectionLimitingSettings --- .../core/fs/LimitedConnectionsFileSystem.java | 2 +- .../LimitedConnectionsConfigurationTest.java | 56 +++++++++++++++++++ 2 files changed, 57 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java index 5353563494d90..fdf54e014b6f4 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/LimitedConnectionsFileSystem.java @@ -1074,7 +1074,7 @@ public static ConnectionLimitingSettings fromConfig(Configuration config, String checkLimit(limitOut, limitOutOption); // create the settings only, if at least one limit is configured - if (totalLimit <= 0 || limitIn <= 0 || limitOut <= 0) { + if (totalLimit <= 0 && limitIn <= 0 && limitOut <= 0) { // no limit configured return null; } diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java index 4742a7e063031..2c30ce85a40e6 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/LimitedConnectionsConfigurationTest.java @@ -19,6 +19,8 @@ package org.apache.flink.core.fs; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.core.fs.LimitedConnectionsFileSystem.ConnectionLimitingSettings; import org.apache.flink.testutils.TestFileSystem; import org.junit.Rule; @@ -29,6 +31,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; /** @@ -40,6 +44,10 @@ public class LimitedConnectionsConfigurationTest { @Rule public final TemporaryFolder tempDir = new TemporaryFolder(); + /** + * This test validates that the File System is correctly wrapped by the + * file system factories when the corresponding entries are in the configuration. + */ @Test public void testConfiguration() throws Exception { final String fsScheme = TestFileSystem.SCHEME; @@ -81,4 +89,52 @@ public void testConfiguration() throws Exception { FileSystem.initialize(new Configuration()); } } + + /** + * This test checks that the file system connection limiting configuration object + * is properly created. + */ + @Test + public void testConnectionLimitingSettings() { + final String scheme = "testscheme"; + + // empty config + assertNull(ConnectionLimitingSettings.fromConfig(new Configuration(), scheme)); + + // only total limit set + { + Configuration conf = new Configuration(); + conf.setInteger(CoreOptions.fileSystemConnectionLimit(scheme), 10); + + ConnectionLimitingSettings settings = ConnectionLimitingSettings.fromConfig(conf, scheme); + assertNotNull(settings); + assertEquals(10, settings.limitTotal); + assertEquals(0, settings.limitInput); + assertEquals(0, settings.limitOutput); + } + + // only input limit set + { + Configuration conf = new Configuration(); + conf.setInteger(CoreOptions.fileSystemConnectionLimitIn(scheme), 10); + + ConnectionLimitingSettings settings = ConnectionLimitingSettings.fromConfig(conf, scheme); + assertNotNull(settings); + assertEquals(0, settings.limitTotal); + assertEquals(10, settings.limitInput); + assertEquals(0, settings.limitOutput); + } + + // only output limit set + { + Configuration conf = new Configuration(); + conf.setInteger(CoreOptions.fileSystemConnectionLimitOut(scheme), 10); + + ConnectionLimitingSettings settings = ConnectionLimitingSettings.fromConfig(conf, scheme); + assertNotNull(settings); + assertEquals(0, settings.limitTotal); + assertEquals(0, settings.limitInput); + assertEquals(10, settings.limitOutput); + } + } } From 612f07bbd2dc9aa1abf01ebf817c8341c1ad7a10 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 4 Dec 2017 16:14:56 +0100 Subject: [PATCH 140/367] [FLINK-8186] Exclude flink-avro from flink-dist; fix AvroUtils loading Before, AvroUtils were loaded when the class was loaded which didn't take into account the user-code ClassLoader. Now, we try loading avro utils with the Thread context ClassLoader. --- .../flink/api/java/typeutils/AvroUtils.java | 18 ++++++------------ flink-dist/pom.xml | 10 ---------- 2 files changed, 6 insertions(+), 22 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java index 2983be0a7cf7e..0677338b7b2b9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java @@ -37,12 +37,14 @@ public abstract class AvroUtils { private static final String AVRO_KRYO_UTILS = "org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils"; - private static final AvroUtils INSTANCE = loadAvroKryoUtils(); - - private static AvroUtils loadAvroKryoUtils() { + /** + * Returns either the default {@link AvroUtils} which throw an exception in cases where Avro + * would be needed or loads the specific utils for Avro from flink-avro. + */ + public static AvroUtils getAvroUtils() { // try and load the special AvroUtils from the flink-avro package try { - Class clazz = Class.forName(AVRO_KRYO_UTILS, false, AvroUtils.class.getClassLoader()); + Class clazz = Class.forName(AVRO_KRYO_UTILS, false, Thread.currentThread().getContextClassLoader()); return clazz.asSubclass(AvroUtils.class).getConstructor().newInstance(); } catch (ClassNotFoundException e) { // cannot find the utils, return the default implementation @@ -52,14 +54,6 @@ private static AvroUtils loadAvroKryoUtils() { } } - /** - * Returns either the default {@link AvroUtils} which throw an exception in cases where Avro - * would be needed or loads the specific utils for Avro from flink-avro. - */ - public static AvroUtils getAvroUtils() { - return INSTANCE; - } - // ------------------------------------------------------------------------ /** diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 95b244ca20e93..124481fdf172a 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -79,12 +79,6 @@ under the License. ${project.version} - - org.apache.flink - flink-avro - ${project.version} - - org.apache.flink flink-streaming-java_${scala.binary.version} @@ -497,10 +491,6 @@ under the License. - - org.codehaus.jackson - org.apache.flink.formats.avro.shaded.org.codehaus.jackson - com.fasterxml.jackson From 9ba9e809ba326006e805dd3a5377be44e981261d Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 4 Dec 2017 13:38:17 +0100 Subject: [PATCH 141/367] [FLINK-8193][quickstart] Cleanup quickstart poms This closes #5118. --- .../resources/archetype-resources/pom.xml | 189 +++++----------- .../resources/archetype-resources/pom.xml | 206 ++++++------------ 2 files changed, 112 insertions(+), 283 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 34e2abbfe8956..e50e7b5b62ce4 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -59,38 +59,37 @@ under the License. a) Adding new dependencies: You can add dependencies to the list below. - Please check if the maven-shade-plugin below is filtering out your dependency - and remove the exclude from there. b) Build a jar for running on the cluster: - There are two options for creating a jar from this project - b.1) "mvn clean package" -> this will create a fat jar which contains all - dependencies necessary for running the jar created by this pom in a cluster. - The "maven-shade-plugin" excludes everything that is provided on a running Flink cluster. - - b.2) "mvn clean package -Pbuild-jar" -> This will also create a fat-jar, but with much - nicer dependency exclusion handling. This approach is preferred and leads to - much cleaner jar files. + "mvn clean package -Pbuild-jar" + This will create a fat-jar which contains all dependencies necessary for running the created jar in a cluster. --> org.apache.flink - flink-java + flink-core ${flink.version} org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-java ${flink.version} + org.apache.flink flink-clients_${scala.binary.version} ${flink.version} + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + @@ -118,13 +117,13 @@ under the License. org.apache.flink - flink-java + flink-core ${flink.version} provided org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-java ${flink.version} provided @@ -134,6 +133,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + provided + org.slf4j slf4j-log4j12 @@ -150,12 +155,15 @@ under the License. - + org.apache.maven.plugins maven-shade-plugin 2.4.1 + package @@ -163,8 +171,33 @@ under the License. - + + org.apache.flink:force-shading + com.google.code.findbgs:jsr305 + org.slf4j:slf4j-api + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + @@ -176,130 +209,6 @@ under the License. - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.1 - - - - package - - shade - - - - - - org.apache.flink:flink-annotations - org.apache.flink:flink-shaded-hadoop2 - org.apache.flink:flink-shaded-curator - org.apache.flink:flink-core - org.apache.flink:flink-java - org.apache.flink:flink-scala_${scala.binary.version} - org.apache.flink:flink-runtime_${scala.binary.version} - org.apache.flink:flink-optimizer_${scala.binary.version} - org.apache.flink:flink-clients_${scala.binary.version} - org.apache.flink:flink-avro - org.apache.flink:flink-examples-batch_${scala.binary.version} - org.apache.flink:flink-examples-streaming_${scala.binary.version} - org.apache.flink:flink-streaming-java_${scala.binary.version} - org.apache.flink:flink-streaming-scala_${scala.binary.version} - org.apache.flink:flink-scala-shell_${scala.binary.version} - org.apache.flink:flink-python - org.apache.flink:flink-metrics-core - org.apache.flink:flink-metrics-jmx - org.apache.flink:flink-statebackend-rocksdb_${scala.binary.version} - org.apache.flink:flink-shaded-jackson - - - - log4j:log4j - org.scala-lang:scala-library - org.scala-lang:scala-compiler - org.scala-lang:scala-reflect - com.typesafe.akka:akka-actor_* - com.typesafe.akka:akka-remote_* - com.typesafe.akka:akka-slf4j_* - io.netty:netty-all - io.netty:netty - commons-fileupload:commons-fileupload - org.apache.avro:avro - commons-collections:commons-collections - org.codehaus.jackson:jackson-core-asl - org.codehaus.jackson:jackson-mapper-asl - com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java - org.apache.commons:commons-compress - org.tukaani:xz - com.esotericsoftware.kryo:kryo - com.esotericsoftware.minlog:minlog - org.objenesis:objenesis - com.twitter:chill_* - com.twitter:chill-java - commons-lang:commons-lang - junit:junit - org.apache.commons:commons-lang3 - org.slf4j:slf4j-api - org.slf4j:slf4j-log4j12 - log4j:log4j - org.apache.commons:commons-math - org.apache.sling:org.apache.sling.commons.json - commons-logging:commons-logging - commons-codec:commons-codec - stax:stax-api - com.typesafe:config - org.uncommons.maths:uncommons-maths - com.github.scopt:scopt_* - commons-io:commons-io - commons-cli:commons-cli - - - - - org.apache.flink:* - - - org/apache/flink/shaded/com/** - web-docs/** - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - false - - - - - org.apache.maven.plugins maven-compiler-plugin diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index d705e8c137da9..da0a8e4455db7 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -49,6 +49,7 @@ under the License. @slf4j.version@ @log4j.version@ 2.11 + 2.11.11 org.apache.flink - flink-scala_${scala.binary.version} + flink-core ${flink.version} + org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-clients_${scala.binary.version} ${flink.version} org.apache.flink - flink-clients_${scala.binary.version} + flink-scala_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-streaming-scala_${scala.binary.version} ${flink.version} + + org.scala-lang + scala-library + ${scala.version} + + @@ -117,22 +123,33 @@ under the License. org.apache.flink - flink-scala_${scala.binary.version} + flink-core + ${flink.version} + + + org.apache.flink + flink-clients_${scala.binary.version} ${flink.version} provided org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-scala_${scala.binary.version} ${flink.version} provided org.apache.flink - flink-clients_${scala.binary.version} + flink-streaming-scala_${scala.binary.version} ${flink.version} provided + + org.scala-lang + scala-library + ${scala.version} + provided + org.slf4j slf4j-log4j12 @@ -149,12 +166,15 @@ under the License. - + org.apache.maven.plugins maven-shade-plugin 2.4.1 + package @@ -162,8 +182,33 @@ under the License. - + + org.apache.flink:force-shading + com.google.code.findbgs:jsr305 + org.slf4j:slf4j-api + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + @@ -173,133 +218,8 @@ under the License. - - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.1 - - - - package - - shade - - - - - - org.apache.flink:flink-annotations - org.apache.flink:flink-shaded-hadoop2 - org.apache.flink:flink-shaded-curator - org.apache.flink:flink-core - org.apache.flink:flink-java - org.apache.flink:flink-scala_${scala.binary.version} - org.apache.flink:flink-runtime_${scala.binary.version} - org.apache.flink:flink-optimizer_${scala.binary.version} - org.apache.flink:flink-clients_${scala.binary.version} - org.apache.flink:flink-avro - org.apache.flink:flink-examples-batch_${scala.binary.version} - org.apache.flink:flink-examples-streaming_${scala.binary.version} - org.apache.flink:flink-streaming-java_${scala.binary.version} - org.apache.flink:flink-streaming-scala_${scala.binary.version} - org.apache.flink:flink-scala-shell_${scala.binary.version} - org.apache.flink:flink-python - org.apache.flink:flink-metrics-core - org.apache.flink:flink-metrics-jmx - org.apache.flink:flink-statebackend-rocksdb_${scala.binary.version} - org.apache.flink:flink-shaded-jackson - - - - log4j:log4j - org.scala-lang:scala-library - org.scala-lang:scala-compiler - org.scala-lang:scala-reflect - com.typesafe.akka:akka-actor_* - com.typesafe.akka:akka-remote_* - com.typesafe.akka:akka-slf4j_* - io.netty:netty-all - io.netty:netty - commons-fileupload:commons-fileupload - org.apache.avro:avro - commons-collections:commons-collections - com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java - org.apache.commons:commons-compress - org.tukaani:xz - com.esotericsoftware.kryo:kryo - com.esotericsoftware.minlog:minlog - org.objenesis:objenesis - com.twitter:chill_* - com.twitter:chill-java - commons-lang:commons-lang - junit:junit - org.apache.commons:commons-lang3 - org.slf4j:slf4j-api - org.slf4j:slf4j-log4j12 - log4j:log4j - org.apache.commons:commons-math - org.apache.sling:org.apache.sling.commons.json - commons-logging:commons-logging - commons-codec:commons-codec - stax:stax-api - com.typesafe:config - org.uncommons.maths:uncommons-maths - com.github.scopt:scopt_* - commons-io:commons-io - commons-cli:commons-cli - - - - - org.apache.flink:* - - - org/apache/flink/shaded/com/** - web-docs/** - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - false - - - - - org.apache.maven.plugins maven-compiler-plugin From 4bc4f326580bbea290630999273d9d90711b44c3 Mon Sep 17 00:00:00 2001 From: Eron Wright Date: Sun, 3 Dec 2017 03:14:41 -0800 Subject: [PATCH 142/367] [FLINK-8174] Mesos RM unable to accept offers for unreserved resources [FLINK-8174] Mesos RM unable to accept offers for unreserved resources - added test `OfferTest` [FLINK-8174] Mesos RM unable to accept offers for unreserved resources - fix `LaunchCoordinatorTest` [FLINK-8174] Mesos RM unable to accept offers for unreserved resources - improved javadocs [FLINK-8174] Mesos RM unable to accept offers for unreserved resources - rename `print` to `toString` - precalculate offer resource values - extend TestLogger This closes #5114. --- .../java/org/apache/flink/mesos/Utils.java | 161 +++++++++- .../LaunchableMesosWorker.java | 39 +-- .../flink/mesos/scheduler/LaunchableTask.java | 7 +- .../apache/flink/mesos/scheduler/Offer.java | 187 ++++++++++++ .../flink/mesos/util/MesosConfiguration.java | 10 + .../mesos/util/MesosResourceAllocation.java | 206 +++++++++++++ .../mesos/scheduler/LaunchCoordinator.scala | 25 +- .../flink/mesos/scheduler/OfferTest.java | 174 +++++++++++ .../util/MesosResourceAllocationTest.java | 275 ++++++++++++++++++ .../scheduler/LaunchCoordinatorTest.scala | 6 +- 10 files changed, 1058 insertions(+), 32 deletions(-) create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/Offer.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosResourceAllocation.java create mode 100644 flink-mesos/src/test/java/org/apache/flink/mesos/scheduler/OfferTest.java create mode 100644 flink-mesos/src/test/java/org/apache/flink/mesos/util/MesosResourceAllocationTest.java diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java index eac73e0a176c7..0e63b9bda035a 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java @@ -25,17 +25,30 @@ import java.net.URL; import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.LongStream; import scala.Option; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Collection of utility methods. */ public class Utils { + + /** + * The special 'unreserved' role. + */ + public static final String UNRESERVED_ROLE = "*"; + /** * Construct a Mesos environment variable. */ public static Protos.Environment.Variable variable(String name, String value) { + checkNotNull(name); return Protos.Environment.Variable.newBuilder() .setName(name) .setValue(value) @@ -46,6 +59,7 @@ public static Protos.Environment.Variable variable(String name, String value) { * Construct a Mesos URI. */ public static Protos.CommandInfo.URI uri(URL url, boolean cacheable) { + checkNotNull(url); return Protos.CommandInfo.URI.newBuilder() .setValue(url.toExternalForm()) .setExtract(false) @@ -57,6 +71,8 @@ public static Protos.CommandInfo.URI uri(URL url, boolean cacheable) { * Construct a Mesos URI. */ public static Protos.CommandInfo.URI uri(MesosArtifactResolver resolver, ContainerSpecification.Artifact artifact) { + checkNotNull(resolver); + checkNotNull(artifact); Option url = resolver.resolve(artifact.dest); if (url.isEmpty()) { throw new IllegalArgumentException("Unresolvable artifact: " + artifact.dest); @@ -72,9 +88,90 @@ public static Protos.CommandInfo.URI uri(MesosArtifactResolver resolver, Contain } /** - * Construct a scalar resource value. + * Construct a list of resources. + */ + public static List resources(Protos.Resource... resources) { + checkNotNull(resources); + return Arrays.asList(resources); + } + + /** + * Construct a cpu resource. + */ + public static Protos.Resource cpus(double amount) { + return cpus(UNRESERVED_ROLE, amount); + } + + /** + * Construct a cpu resource. + */ + public static Protos.Resource cpus(String role, double amount) { + return scalar("cpus", role, amount); + } + + /** + * Construct a mem resource. + */ + public static Protos.Resource mem(double amount) { + return mem(UNRESERVED_ROLE, amount); + } + + /** + * Construct a mem resource. + */ + public static Protos.Resource mem(String role, double amount) { + return scalar("mem", role, amount); + } + + /** + * Construct a network resource. + */ + public static Protos.Resource network(double amount) { + return network(UNRESERVED_ROLE, amount); + } + + /** + * Construct a network resource. + */ + public static Protos.Resource network(String role, double amount) { + return scalar("network", role, amount); + } + + /** + * Construct a disk resource. + */ + public static Protos.Resource disk(double amount) { + return disk(UNRESERVED_ROLE, amount); + } + + /** + * Construct a disk resource. + */ + public static Protos.Resource disk(String role, double amount) { + return scalar("disk", role, amount); + } + + /** + * Construct a port resource. + */ + public static Protos.Resource ports(Protos.Value.Range... ranges) { + return ports(UNRESERVED_ROLE, ranges); + } + + /** + * Construct a port resource. + */ + public static Protos.Resource ports(String role, Protos.Value.Range... ranges) { + return ranges("ports", role, ranges); + } + + /** + * Construct a scalar resource. */ public static Protos.Resource scalar(String name, String role, double value) { + checkNotNull(name); + checkNotNull(role); + checkNotNull(value); return Protos.Resource.newBuilder() .setName(name) .setType(Protos.Value.Type.SCALAR) @@ -91,9 +188,12 @@ public static Protos.Value.Range range(long begin, long end) { } /** - * Construct a ranges resource value. + * Construct a range resource. */ public static Protos.Resource ranges(String name, String role, Protos.Value.Range... ranges) { + checkNotNull(name); + checkNotNull(role); + checkNotNull(ranges); return Protos.Resource.newBuilder() .setName(name) .setType(Protos.Value.Type.RANGES) @@ -101,4 +201,61 @@ public static Protos.Resource ranges(String name, String role, Protos.Value.Rang .setRole(role) .build(); } + + /** + * Gets a stream of values from a collection of range resources. + */ + public static LongStream rangeValues(Collection resources) { + checkNotNull(resources); + return resources.stream() + .filter(Protos.Resource::hasRanges) + .flatMap(r -> r.getRanges().getRangeList().stream()) + .flatMapToLong(Utils::rangeValues); + } + + /** + * Gets a stream of values from a range. + */ + public static LongStream rangeValues(Protos.Value.Range range) { + checkNotNull(range); + return LongStream.rangeClosed(range.getBegin(), range.getEnd()); + } + + /** + * Gets a string representation of a collection of resources. + */ + public static String toString(Collection resources) { + checkNotNull(resources); + return resources.stream().map(Utils::toString).collect(Collectors.joining("; ", "[", "]")); + } + + /** + * Gets a string representation of a resource. + */ + public static String toString(Protos.Resource resource) { + checkNotNull(resource); + if (resource.hasScalar()) { + return String.format("%s(%s):%.1f", resource.getName(), resource.getRole(), resource.getScalar().getValue()); + } + if (resource.hasRanges()) { + return String.format("%s(%s):%s", resource.getName(), resource.getRole(), toString(resource.getRanges())); + } + return resource.toString(); + } + + /** + * Gets a string representation of a collection of ranges. + */ + public static String toString(Protos.Value.Ranges ranges) { + checkNotNull(ranges); + return ranges.getRangeList().stream().map(Utils::toString).collect(Collectors.joining(",", "[", "]")); + } + + /** + * Gets a string representation of a range. + */ + public static String toString(Protos.Value.Range range) { + checkNotNull(range); + return String.format("%d-%d", range.getBegin(), range.getEnd()); + } } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 125258b42998f..e71c703b8ce58 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -25,12 +25,14 @@ import org.apache.flink.mesos.util.MesosArtifactResolver; import org.apache.flink.mesos.util.MesosArtifactServer; import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.mesos.util.MesosResourceAllocation; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.util.Preconditions; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; + import com.netflix.fenzo.ConstraintEvaluator; -import com.netflix.fenzo.TaskAssignmentResult; import com.netflix.fenzo.TaskRequest; import com.netflix.fenzo.VMTaskFitnessCalculator; import org.apache.mesos.Protos; @@ -39,16 +41,16 @@ import java.io.IOException; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import scala.Option; -import static org.apache.flink.mesos.Utils.range; -import static org.apache.flink.mesos.Utils.ranges; -import static org.apache.flink.mesos.Utils.scalar; +import static org.apache.flink.mesos.Utils.rangeValues; import static org.apache.flink.mesos.Utils.variable; /** @@ -180,11 +182,11 @@ public String toString() { /** * Construct the TaskInfo needed to launch the worker. * @param slaveId the assigned slave. - * @param assignment the assignment details. + * @param allocation the resource allocation (available resources). * @return a fully-baked TaskInfo. */ @Override - public Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult assignment) { + public Protos.TaskInfo launch(Protos.SlaveID slaveId, MesosResourceAllocation allocation) { ContaineredTaskManagerParameters tmParams = params.containeredParameters(); @@ -197,9 +199,12 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult assig final Protos.TaskInfo.Builder taskInfo = Protos.TaskInfo.newBuilder() .setSlaveId(slaveId) .setTaskId(taskID) - .setName(taskID.getValue()) - .addResources(scalar("cpus", mesosConfiguration.frameworkInfo().getRole(), assignment.getRequest().getCPUs())) - .addResources(scalar("mem", mesosConfiguration.frameworkInfo().getRole(), assignment.getRequest().getMemory())); + .setName(taskID.getValue()); + + // take needed resources from the overall allocation, under the assumption of adequate resources + Set roles = mesosConfiguration.roles(); + taskInfo.addAllResources(allocation.takeScalar("cpus", taskRequest.getCPUs(), roles)); + taskInfo.addAllResources(allocation.takeScalar("mem", taskRequest.getMemory(), roles)); final Protos.CommandInfo.Builder cmd = taskInfo.getCommandBuilder(); final Protos.Environment.Builder env = cmd.getEnvironmentBuilder(); @@ -217,15 +222,13 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult assig dynamicProperties.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, taskManagerHostname); } - // use the assigned ports for the TM - if (assignment.getAssignedPorts().size() < TM_PORT_KEYS.length) { - throw new IllegalArgumentException("unsufficient # of ports assigned"); - } - for (int i = 0; i < TM_PORT_KEYS.length; i++) { - int port = assignment.getAssignedPorts().get(i); - String key = TM_PORT_KEYS[i]; - taskInfo.addResources(ranges("ports", mesosConfiguration.frameworkInfo().getRole(), range(port, port))); - dynamicProperties.setInteger(key, port); + // take needed ports for the TM + List portResources = allocation.takeRanges("ports", TM_PORT_KEYS.length, roles); + taskInfo.addAllResources(portResources); + Iterator portsToAssign = Iterators.forArray(TM_PORT_KEYS); + rangeValues(portResources).forEach(port -> dynamicProperties.setLong(portsToAssign.next(), port)); + if (portsToAssign.hasNext()) { + throw new IllegalArgumentException("insufficient # of ports assigned"); } // ship additional files diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java index 203708bd7ecf0..fe8f2e378c9ef 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java @@ -18,7 +18,8 @@ package org.apache.flink.mesos.scheduler; -import com.netflix.fenzo.TaskAssignmentResult; +import org.apache.flink.mesos.util.MesosResourceAllocation; + import com.netflix.fenzo.TaskRequest; import org.apache.mesos.Protos; @@ -35,8 +36,8 @@ public interface LaunchableTask { /** * Prepare to launch the task by producing a Mesos TaskInfo record. * @param slaveId the slave assigned to the task. - * @param taskAssignmentResult the task assignment details. + * @param allocation the resource allocation to take from. * @return a TaskInfo. */ - Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult taskAssignmentResult); + Protos.TaskInfo launch(Protos.SlaveID slaveId, MesosResourceAllocation allocation); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/Offer.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/Offer.java new file mode 100644 index 0000000000000..64436fb33c143 --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/Offer.java @@ -0,0 +1,187 @@ +/* + * 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.flink.mesos.scheduler; + +import org.apache.flink.mesos.Utils; + +import com.netflix.fenzo.VirtualMachineLease; +import org.apache.mesos.Protos; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An adapter class to transform a Mesos resource offer to a Fenzo {@link VirtualMachineLease}. + * + *

    The default implementation provided by Fenzo isn't compatible with reserved resources. + * This implementation properly combines resources, e.g. a combination of reserved and unreserved cpus. + * + */ +public class Offer implements VirtualMachineLease { + + private static final Logger logger = LoggerFactory.getLogger(Offer.class); + + private final Protos.Offer offer; + private final String hostname; + private final String vmID; + private final long offeredTime; + + private final List resources; + private final Map attributeMap; + + private final double cpuCores; + private final double memoryMB; + private final double networkMbps; + private final double diskMB; + private final List portRanges; + + public Offer(Protos.Offer offer) { + this.offer = checkNotNull(offer); + this.hostname = offer.getHostname(); + this.vmID = offer.getSlaveId().getValue(); + this.offeredTime = System.currentTimeMillis(); + + List resources = new ArrayList<>(offer.getResourcesList().size()); + Map> resourceMap = new HashMap<>(); + for (Protos.Resource resource : offer.getResourcesList()) { + switch (resource.getType()) { + case SCALAR: + case RANGES: + resources.add(resource); + resourceMap.computeIfAbsent(resource.getName(), k -> new ArrayList<>(2)).add(resource); + break; + default: + logger.debug("Unknown resource type " + resource.getType() + " for resource " + resource.getName() + + " in offer, hostname=" + hostname + ", offerId=" + offer.getId()); + } + } + this.resources = Collections.unmodifiableList(resources); + + this.cpuCores = aggregateScalarResource(resourceMap, "cpus"); + this.memoryMB = aggregateScalarResource(resourceMap, "mem"); + this.networkMbps = aggregateScalarResource(resourceMap, "network"); + this.diskMB = aggregateScalarResource(resourceMap, "disk"); + this.portRanges = Collections.unmodifiableList(aggregateRangesResource(resourceMap, "ports")); + + if (offer.getAttributesCount() > 0) { + Map attributeMap = new HashMap<>(); + for (Protos.Attribute attribute: offer.getAttributesList()) { + attributeMap.put(attribute.getName(), attribute); + } + this.attributeMap = Collections.unmodifiableMap(attributeMap); + } else { + this.attributeMap = Collections.emptyMap(); + } + } + + public List getResources() { + return resources; + } + + @Override + public String hostname() { + return hostname; + } + + @Override + public String getVMID() { + return vmID; + } + + @Override + public double cpuCores() { + return cpuCores; + } + + @Override + public double memoryMB() { + return memoryMB; + } + + @Override + public double networkMbps() { + return networkMbps; + } + + @Override + public double diskMB() { + return diskMB; + } + + public Protos.Offer getOffer(){ + return offer; + } + + @Override + public String getId() { + return offer.getId().getValue(); + } + + @Override + public long getOfferedTime() { + return offeredTime; + } + + @Override + public List portRanges() { + return portRanges; + } + + @Override + public Map getAttributeMap() { + return attributeMap; + } + + @Override + public String toString() { + return "Offer{" + + "offer=" + offer + + ", resources='" + Utils.toString(resources) + '\'' + + ", hostname='" + hostname + '\'' + + ", vmID='" + vmID + '\'' + + ", attributeMap=" + attributeMap + + ", offeredTime=" + offeredTime + + '}'; + } + + private static double aggregateScalarResource(Map> resourceMap, String resourceName) { + if (resourceMap.get(resourceName) == null) { + return 0.0; + } + return resourceMap.get(resourceName).stream().mapToDouble(r -> r.getScalar().getValue()).sum(); + } + + private static List aggregateRangesResource(Map> resourceMap, String resourceName) { + if (resourceMap.get(resourceName) == null) { + return Collections.emptyList(); + } + return resourceMap.get(resourceName).stream() + .flatMap(r -> r.getRanges().getRangeList().stream()) + .map(r -> new Range((int) r.getBegin(), (int) r.getEnd())) + .collect(Collectors.toList()); + } +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java index 7660e9c549ce0..4db5e25a54136 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java @@ -24,7 +24,9 @@ import org.apache.mesos.SchedulerDriver; import org.slf4j.Logger; +import java.util.Collections; import java.util.Map; +import java.util.Set; import scala.Option; @@ -89,6 +91,14 @@ public MesosConfiguration withFrameworkInfo(Protos.FrameworkInfo.Builder framewo return new MesosConfiguration(masterUrl, frameworkInfo, credential); } + /** + * Gets the roles associated with the framework. + */ + public Set roles() { + return frameworkInfo.hasRole() && !"*".equals(frameworkInfo.getRole()) ? + Collections.singleton(frameworkInfo.getRole()) : Collections.emptySet(); + } + /** * Create the Mesos scheduler driver based on this configuration. * @param scheduler the scheduler to use. diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosResourceAllocation.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosResourceAllocation.java new file mode 100644 index 0000000000000..9ad5234c662d6 --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosResourceAllocation.java @@ -0,0 +1,206 @@ +/* + * 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.flink.mesos.util; + +import org.apache.flink.mesos.Utils; + +import org.apache.mesos.Protos; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.ListIterator; +import java.util.Set; + +import static org.apache.flink.mesos.Utils.UNRESERVED_ROLE; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An allocation of resources on a particular host from one or more Mesos offers, to be portioned out to tasks. + * + *

    A typical offer contains a mix of reserved and unreserved resources. + * The below example depicts 2 cpus reserved for 'myrole' plus 3 unreserved cpus for a total of 5 cpus: + *

    {@code
    + *   cpus(myrole):2.0; mem(myrole):4096.0; ports(myrole):[1025-2180];
    + *   disk(*):28829.0; cpus(*):3.0; mem(*):10766.0; ports(*):[2182-3887,8082-8180,8182-32000]
    + * }
    + * + *

    This class assumes that the resources were offered without the {@code RESERVATION_REFINEMENT} capability, + * as detailed in the "Resource Format" section of the Mesos protocol definition. + * + *

    This class is not thread-safe. + */ +public class MesosResourceAllocation { + + protected static final Logger LOG = LoggerFactory.getLogger(MesosResourceAllocation.class); + + static final double EPSILON = 1e-5; + + private final List resources; + + /** + * Creates an allocation of resources for tasks to take. + * + * @param resources the resources to add to the allocation. + */ + public MesosResourceAllocation(Collection resources) { + this.resources = new ArrayList<>(checkNotNull(resources)); + + // sort the resources to prefer reserved resources + this.resources.sort(Comparator.comparing(r -> UNRESERVED_ROLE.equals(r.getRole()))); + } + + /** + * Gets the remaining resources. + */ + public List getRemaining() { + return Collections.unmodifiableList(resources); + } + + /** + * Takes some amount of scalar resources (e.g. cpus, mem). + * + * @param amount the (approximate) amount to take from the available quantity. + * @param roles the roles to accept + */ + public List takeScalar(String resourceName, double amount, Set roles) { + if (LOG.isDebugEnabled()) { + LOG.debug("Allocating {} {}", amount, resourceName); + } + + List result = new ArrayList<>(1); + for (ListIterator i = resources.listIterator(); i.hasNext();) { + if (amount <= EPSILON) { + break; + } + + // take from next available scalar resource that is unreserved or reserved for an applicable role + Protos.Resource available = i.next(); + if (!resourceName.equals(available.getName()) || !available.hasScalar()) { + continue; + } + if (!UNRESERVED_ROLE.equals(available.getRole()) && !roles.contains(available.getRole())) { + continue; + } + + double amountToTake = Math.min(available.getScalar().getValue(), amount); + Protos.Resource taken = available.toBuilder().setScalar(Protos.Value.Scalar.newBuilder().setValue(amountToTake)).build(); + amount -= amountToTake; + result.add(taken); + if (LOG.isDebugEnabled()) { + LOG.debug("Taking {} from {}", amountToTake, Utils.toString(available)); + } + + // keep remaining amount (if any) + double remaining = available.getScalar().getValue() - taken.getScalar().getValue(); + if (remaining > EPSILON) { + i.set(available.toBuilder().setScalar(Protos.Value.Scalar.newBuilder().setValue(remaining)).build()); + } + else { + i.remove(); + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Allocated: {}, unsatisfied: {}", Utils.toString(result), amount); + } + return result; + } + + /** + * Takes some amount of range resources (e.g. ports). + * + * @param amount the number of values to take from the available range(s). + * @param roles the roles to accept + */ + public List takeRanges(String resourceName, int amount, Set roles) { + if (LOG.isDebugEnabled()) { + LOG.debug("Allocating {} {}", amount, resourceName); + } + + List result = new ArrayList<>(1); + for (ListIterator i = resources.listIterator(); i.hasNext();) { + if (amount <= 0) { + break; + } + + // take from next available range resource that is unreserved or reserved for an applicable role + Protos.Resource available = i.next(); + if (!resourceName.equals(available.getName()) || !available.hasRanges()) { + continue; + } + if (!UNRESERVED_ROLE.equals(available.getRole()) && !roles.contains(available.getRole())) { + continue; + } + + List takenRanges = new ArrayList<>(); + List remainingRanges = new ArrayList<>(available.getRanges().getRangeList()); + for (ListIterator j = remainingRanges.listIterator(); j.hasNext();) { + if (amount <= 0) { + break; + } + + // take from next available range (note: ranges are inclusive) + Protos.Value.Range availableRange = j.next(); + long amountToTake = Math.min(availableRange.getEnd() - availableRange.getBegin() + 1, amount); + Protos.Value.Range takenRange = availableRange.toBuilder().setEnd(availableRange.getBegin() + amountToTake - 1).build(); + amount -= amountToTake; + takenRanges.add(takenRange); + + // keep remaining range (if any) + long remaining = availableRange.getEnd() - takenRange.getEnd(); + if (remaining > 0) { + j.set(availableRange.toBuilder().setBegin(takenRange.getEnd() + 1).build()); + } + else { + j.remove(); + } + } + Protos.Resource taken = available.toBuilder().setRanges(Protos.Value.Ranges.newBuilder().addAllRange(takenRanges)).build(); + if (LOG.isDebugEnabled()) { + LOG.debug("Taking {} from {}", Utils.toString(taken.getRanges()), Utils.toString(available)); + } + result.add(taken); + + // keep remaining ranges (if any) + if (remainingRanges.size() > 0) { + i.set(available.toBuilder().setRanges(Protos.Value.Ranges.newBuilder().addAllRange(remainingRanges)).build()); + } + else { + i.remove(); + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Allocated: {}, unsatisfied: {}", Utils.toString(result), amount); + } + return result; + } + + @Override + public String toString() { + return "MesosResourceAllocation{" + + "resources=" + Utils.toString(resources) + + '}'; + } +} diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala index 1024b5cbbe8e0..124022adb1352 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala @@ -23,12 +23,13 @@ import java.util.Collections import akka.actor.{Actor, ActorRef, FSM, Props} import com.netflix.fenzo._ import com.netflix.fenzo.functions.Action1 -import com.netflix.fenzo.plugins.VMLeaseObject import grizzled.slf4j.Logger import org.apache.flink.api.java.tuple.{Tuple2=>FlinkTuple2} import org.apache.flink.configuration.Configuration +import org.apache.flink.mesos.Utils import org.apache.flink.mesos.scheduler.LaunchCoordinator._ import org.apache.flink.mesos.scheduler.messages._ +import org.apache.flink.mesos.util.MesosResourceAllocation import org.apache.mesos.{SchedulerDriver, Protos} import scala.collection.JavaConverters._ @@ -147,15 +148,17 @@ class LaunchCoordinator( goto(Suspended) using data.copy(newLeases = Nil) case Event(offers: ResourceOffers, data: GatherData) => - val leases = offers.offers().asScala.map( - new VMLeaseObject(_).asInstanceOf[VirtualMachineLease]) + val leases = offers.offers().asScala.map(new Offer(_)) if(LOG.isInfoEnabled) { val (cpus, mem) = leases.foldLeft((0.0,0.0)) { (z,o) => (z._1 + o.cpuCores(), z._2 + o.memoryMB()) } LOG.info(s"Received offer(s) of $mem MB, $cpus cpus:") for(l <- leases) { - LOG.info(s" ${l.getId} from ${l.hostname()} of ${l.memoryMB()} MB, ${l.cpuCores()} cpus") + val reservations = l.getResources.asScala.map(_.getRole).toSet + LOG.info( + s" ${l.getId} from ${l.hostname()} of ${l.memoryMB()} MB, ${l.cpuCores()} cpus" + + s" for ${reservations.mkString("[", ",", "]")}") } } stay using data.copy(newLeases = data.newLeases ++ leases) forMax (1 seconds) @@ -185,7 +188,7 @@ class LaunchCoordinator( // process the assignments into a set of operations (reserve and/or launch) val slaveId = assignments.getLeasesUsed.get(0).getOffer.getSlaveId val offerIds = assignments.getLeasesUsed.asScala.map(_.getOffer.getId) - val operations = processAssignments(slaveId, assignments, remaining.toMap) + val operations = processAssignments(LOG, slaveId, assignments, remaining.toMap) // update the state to reflect the launched tasks val launchedTasks = operations @@ -316,18 +319,26 @@ object LaunchCoordinator { * * The operations may include reservations and task launches. * + * @param log the logger to use. * @param slaveId the slave associated with the given assignments. * @param assignments the task assignments as provided by the optimizer. * @param allTasks all known tasks, keyed by taskId. * @return the operations to perform. */ private def processAssignments( + log: Logger, slaveId: Protos.SlaveID, assignments: VMAssignmentResult, allTasks: Map[String, LaunchableTask]): Seq[Protos.Offer.Operation] = { + val resources = + assignments.getLeasesUsed.asScala.flatMap(_.asInstanceOf[Offer].getResources.asScala) + val allocation = new MesosResourceAllocation(resources.asJava) + log.debug(s"Assigning resources: ${Utils.toString(allocation.getRemaining)}") + def taskInfo(assignment: TaskAssignmentResult): Protos.TaskInfo = { - allTasks(assignment.getTaskId).launch(slaveId, assignment) + log.debug(s"Processing task ${assignment.getTaskId}") + allTasks(assignment.getTaskId).launch(slaveId, allocation) } val launches = Protos.Offer.Operation.newBuilder() @@ -338,6 +349,8 @@ object LaunchCoordinator { )) .build() + log.debug(s"Remaining resources: ${Utils.toString(allocation.getRemaining)}") + Seq(launches) } diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/scheduler/OfferTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/scheduler/OfferTest.java new file mode 100644 index 0000000000000..cf84e8e9e77ad --- /dev/null +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/scheduler/OfferTest.java @@ -0,0 +1,174 @@ +/* + * 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.flink.mesos.scheduler; + +import org.apache.flink.util.TestLogger; + +import com.netflix.fenzo.VirtualMachineLease; +import org.apache.mesos.Protos; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.mesos.Utils.UNRESERVED_ROLE; +import static org.apache.flink.mesos.Utils.cpus; +import static org.apache.flink.mesos.Utils.disk; +import static org.apache.flink.mesos.Utils.mem; +import static org.apache.flink.mesos.Utils.network; +import static org.apache.flink.mesos.Utils.ports; +import static org.apache.flink.mesos.Utils.range; +import static org.apache.flink.mesos.Utils.resources; +import static org.apache.flink.mesos.Utils.scalar; + +/** + * Tests {@link Offer} which adapts a Mesos offer as a lease for use with Fenzo. + */ +public class OfferTest extends TestLogger { + + private static final double EPSILON = 1e-5; + + private static final Protos.FrameworkID FRAMEWORK_ID = Protos.FrameworkID.newBuilder().setValue("framework-1").build(); + private static final Protos.OfferID OFFER_ID = Protos.OfferID.newBuilder().setValue("offer-1").build(); + private static final String HOSTNAME = "host-1"; + private static final Protos.SlaveID AGENT_ID = Protos.SlaveID.newBuilder().setValue("agent-1").build(); + + private static final String ROLE_A = "A"; + + private static final String ATTR_1 = "A1"; + + // region Resources + + /** + * Tests basic properties (other than those of specific resources, covered elsewhere). + */ + @Test + public void testResourceProperties() { + Offer offer = new Offer(offer(resources(), attrs())); + Assert.assertNotNull(offer.getResources()); + Assert.assertEquals(HOSTNAME, offer.hostname()); + Assert.assertEquals(AGENT_ID.getValue(), offer.getVMID()); + Assert.assertNotNull(offer.getOffer()); + Assert.assertEquals(OFFER_ID.getValue(), offer.getId()); + Assert.assertNotEquals(0L, offer.getOfferedTime()); + Assert.assertNotNull(offer.getAttributeMap()); + Assert.assertNotNull(offer.toString()); + } + + /** + * Tests aggregation of resources in the presence of unreserved plus reserved resources. + */ + @Test + public void testResourceAggregation() { + Offer offer; + + offer = new Offer(offer(resources(), attrs())); + Assert.assertEquals(0.0, offer.cpuCores(), EPSILON); + Assert.assertEquals(Arrays.asList(), ranges(offer.portRanges())); + + offer = new Offer(offer(resources( + cpus(ROLE_A, 1.0), cpus(UNRESERVED_ROLE, 1.0), + ports(ROLE_A, range(80, 80), range(443, 444)), ports(UNRESERVED_ROLE, range(8080, 8081)), + otherScalar(42.0)), attrs())); + Assert.assertEquals(2.0, offer.cpuCores(), EPSILON); + Assert.assertEquals(Arrays.asList(range(80, 80), range(443, 444), range(8080, 8081)), ranges(offer.portRanges())); + } + + @Test + public void testCpuCores() { + Offer offer = new Offer(offer(resources(cpus(1.0)), attrs())); + Assert.assertEquals(1.0, offer.cpuCores(), EPSILON); + } + + @Test + public void testMemoryMB() { + Offer offer = new Offer(offer(resources(mem(1024.0)), attrs())); + Assert.assertEquals(1024.0, offer.memoryMB(), EPSILON); + } + + @Test + public void testNetworkMbps() { + Offer offer = new Offer(offer(resources(network(10.0)), attrs())); + Assert.assertEquals(10.0, offer.networkMbps(), EPSILON); + } + + @Test + public void testDiskMB() { + Offer offer = new Offer(offer(resources(disk(1024.0)), attrs())); + Assert.assertEquals(1024.0, offer.diskMB(), EPSILON); + } + + @Test + public void testPortRanges() { + Offer offer = new Offer(offer(resources(ports(range(8080, 8081))), attrs())); + Assert.assertEquals(Collections.singletonList(range(8080, 8081)), ranges(offer.portRanges())); + } + + // endregion + + // region Attributes + + @Test + public void testAttributeIndexing() { + Offer offer = new Offer(offer(resources(), attrs(attr(ATTR_1, 42.0)))); + Assert.assertEquals(attr(ATTR_1, 42.0), offer.getAttributeMap().get(ATTR_1)); + } + + // endregion + + // region Utilities + + private static Protos.Offer offer(List resources, List attributes) { + return Protos.Offer.newBuilder() + .setId(OFFER_ID) + .setFrameworkId(FRAMEWORK_ID) + .setHostname(HOSTNAME) + .setSlaveId(AGENT_ID) + .addAllAttributes(attributes) + .addAllResources(resources) + .build(); + } + + private static Protos.Attribute attr(String name, double scalar) { + return Protos.Attribute.newBuilder() + .setName(name) + .setType(Protos.Value.Type.SCALAR) + .setScalar(Protos.Value.Scalar.newBuilder().setValue(scalar)) + .build(); + } + + private static List attrs(Protos.Attribute... attributes) { + return Arrays.asList(attributes); + } + + private static List ranges(List ranges) { + return ranges.stream() + .map(r -> Protos.Value.Range.newBuilder().setBegin(r.getBeg()).setEnd(r.getEnd()).build()) + .collect(Collectors.toList()); + } + + private static Protos.Resource otherScalar(double value) { + return scalar("mem", UNRESERVED_ROLE, value); + } + + // endregion +} diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/util/MesosResourceAllocationTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/util/MesosResourceAllocationTest.java new file mode 100644 index 0000000000000..fe2e4444623c9 --- /dev/null +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/util/MesosResourceAllocationTest.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.flink.mesos.util; + +import org.apache.flink.util.TestLogger; + +import org.apache.mesos.Protos; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.mesos.Utils.UNRESERVED_ROLE; +import static org.apache.flink.mesos.Utils.cpus; +import static org.apache.flink.mesos.Utils.ports; +import static org.apache.flink.mesos.Utils.range; +import static org.apache.flink.mesos.Utils.resources; + +/** + * Tests {@link MesosResourceAllocation}. + */ +public class MesosResourceAllocationTest extends TestLogger { + + // possible roles + private static final String ROLE_A = "A"; + private static final String ROLE_B = "B"; + + // possible framework configurations + private static final Set AS_ROLE_A = Collections.singleton(ROLE_A); + private static final Set AS_NO_ROLE = Collections.emptySet(); + + // region Reservations + + /** + * Tests that reserved resources are prioritized. + */ + @Test + public void testReservationPrioritization() { + MesosResourceAllocation allocation = new MesosResourceAllocation( + resources(cpus(ROLE_A, 1.0), cpus(UNRESERVED_ROLE, 1.0), cpus(ROLE_B, 1.0))); + Assert.assertEquals( + resources(cpus(ROLE_A, 1.0), cpus(ROLE_B, 1.0), cpus(UNRESERVED_ROLE, 1.0)), + allocation.getRemaining()); + } + + /** + * Tests that resources are filtered according to the framework role (if any). + */ + @Test + public void testReservationFiltering() { + MesosResourceAllocation allocation; + + // unreserved resources + allocation = new MesosResourceAllocation( + resources(cpus(UNRESERVED_ROLE, 1.0), ports(UNRESERVED_ROLE, range(80, 80)))); + Assert.assertEquals(resources(cpus(UNRESERVED_ROLE, 1.0)), + allocation.takeScalar("cpus", 1.0, AS_NO_ROLE)); + Assert.assertEquals(resources(ports(UNRESERVED_ROLE, range(80, 80))), + allocation.takeRanges("ports", 1, AS_NO_ROLE)); + allocation = new MesosResourceAllocation( + resources(cpus(UNRESERVED_ROLE, 1.0), ports(UNRESERVED_ROLE, range(80, 80)))); + Assert.assertEquals(resources(cpus(UNRESERVED_ROLE, 1.0)), + allocation.takeScalar("cpus", 1.0, AS_ROLE_A)); + Assert.assertEquals(resources(ports(UNRESERVED_ROLE, range(80, 80))), + allocation.takeRanges("ports", 1, AS_ROLE_A)); + + // reserved for the framework role + allocation = new MesosResourceAllocation( + resources(cpus(ROLE_A, 1.0), ports(ROLE_A, range(80, 80)))); + Assert.assertEquals(resources(), + allocation.takeScalar("cpus", 1.0, AS_NO_ROLE)); + Assert.assertEquals(resources(), + allocation.takeRanges("ports", 1, AS_NO_ROLE)); + Assert.assertEquals(resources(cpus(ROLE_A, 1.0)), + allocation.takeScalar("cpus", 1.0, AS_ROLE_A)); + Assert.assertEquals(resources(ports(ROLE_A, range(80, 80))), + allocation.takeRanges("ports", 1, AS_ROLE_A)); + + // reserved for a different role + allocation = new MesosResourceAllocation( + resources(cpus(ROLE_B, 1.0), ports(ROLE_B, range(80, 80)))); + Assert.assertEquals(resources(), + allocation.takeScalar("cpus", 1.0, AS_NO_ROLE)); + Assert.assertEquals(resources(), + allocation.takeRanges("ports", 1, AS_NO_ROLE)); + Assert.assertEquals(resources(), + allocation.takeScalar("cpus", 1.0, AS_ROLE_A)); + Assert.assertEquals(resources(), + allocation.takeRanges("ports", 1, AS_ROLE_A)); + } + + // endregion + + // region General + + /** + * Tests resource naming and typing. + */ + @Test + public void testResourceSpecificity() { + MesosResourceAllocation allocation = new MesosResourceAllocation( + resources(cpus(1.0), ports(range(80, 80)))); + + // mismatched name + Assert.assertEquals( + resources(), + allocation.takeScalar("other", 1.0, AS_NO_ROLE)); + Assert.assertEquals( + resources(), + allocation.takeRanges("other", 1, AS_NO_ROLE)); + + // mismatched type + Assert.assertEquals( + resources(), + allocation.takeScalar("ports", 1.0, AS_NO_ROLE)); + Assert.assertEquals( + resources(), + allocation.takeRanges("cpus", 1, AS_NO_ROLE)); + + // nothing lost + Assert.assertEquals( + resources(cpus(1.0), ports(range(80, 80))), + allocation.getRemaining()); + } + + // endregion + + // region Scalar Resources + + /** + * Tests scalar resource accounting. + */ + @Test + public void testScalarResourceAccounting() { + MesosResourceAllocation allocation; + + // take part of a resource + allocation = new MesosResourceAllocation(resources(cpus(1.0))); + Assert.assertEquals( + resources(cpus(0.25)), + allocation.takeScalar("cpus", 0.25, AS_NO_ROLE)); + Assert.assertEquals( + resources(cpus(0.75)), + allocation.getRemaining()); + + // take a whole resource + allocation = new MesosResourceAllocation(resources(cpus(1.0))); + Assert.assertEquals( + resources(cpus(1.0)), + allocation.takeScalar("cpus", 1.0, AS_NO_ROLE)); + Assert.assertEquals(resources(), allocation.getRemaining()); + + // take multiple resources + allocation = new MesosResourceAllocation( + resources(cpus(ROLE_A, 1.0), cpus(UNRESERVED_ROLE, 1.0))); + Assert.assertEquals( + resources(cpus(ROLE_A, 1.0), cpus(UNRESERVED_ROLE, 0.25)), + allocation.takeScalar("cpus", 1.25, AS_ROLE_A)); + Assert.assertEquals( + resources(cpus(UNRESERVED_ROLE, 0.75)), + allocation.getRemaining()); + } + + /** + * Tests scalar resource exhaustion (i.e. insufficient resources). + */ + @Test + public void testScalarResourceExhaustion() { + MesosResourceAllocation allocation = new MesosResourceAllocation(resources(cpus(1.0))); + Assert.assertEquals( + resources(cpus(1.0)), + allocation.takeScalar("cpus", 2.0, AS_NO_ROLE)); + Assert.assertEquals(resources(), allocation.getRemaining()); + } + + // endregion + + // region Range Resources + + /** + * Tests range resource accounting. + */ + @Test + public void testRangeResourceAccounting() { + MesosResourceAllocation allocation; + List ports = resources( + ports(ROLE_A, range(80, 81), range(443, 444)), + ports(UNRESERVED_ROLE, range(1024, 1025), range(8080, 8081))); + + // take a partial range of one resource + allocation = new MesosResourceAllocation(ports); + Assert.assertEquals( + resources(ports(ROLE_A, range(80, 80))), + allocation.takeRanges("ports", 1, AS_ROLE_A)); + Assert.assertEquals( + resources( + ports(ROLE_A, range(81, 81), range(443, 444)), + ports(UNRESERVED_ROLE, range(1024, 1025), range(8080, 8081))), + allocation.getRemaining()); + + // take a whole range of one resource + allocation = new MesosResourceAllocation(ports); + Assert.assertEquals( + resources(ports(ROLE_A, range(80, 81))), + allocation.takeRanges("ports", 2, AS_ROLE_A)); + Assert.assertEquals( + resources( + ports(ROLE_A, range(443, 444)), + ports(UNRESERVED_ROLE, range(1024, 1025), range(8080, 8081))), + allocation.getRemaining()); + + // take numerous ranges of one resource + allocation = new MesosResourceAllocation(ports); + Assert.assertEquals( + resources(ports(ROLE_A, range(80, 81), range(443, 443))), + allocation.takeRanges("ports", 3, AS_ROLE_A)); + Assert.assertEquals( + resources( + ports(ROLE_A, range(444, 444)), + ports(UNRESERVED_ROLE, range(1024, 1025), range(8080, 8081))), + allocation.getRemaining()); + + // take a whole resource + allocation = new MesosResourceAllocation(ports); + Assert.assertEquals( + resources(ports(ROLE_A, range(80, 81), range(443, 444))), + allocation.takeRanges("ports", 4, AS_ROLE_A)); + Assert.assertEquals( + resources(ports(UNRESERVED_ROLE, range(1024, 1025), range(8080, 8081))), + allocation.getRemaining()); + + // take numerous resources + allocation = new MesosResourceAllocation(ports); + Assert.assertEquals( + resources( + ports(ROLE_A, range(80, 81), range(443, 444)), + ports(UNRESERVED_ROLE, range(1024, 1024))), + allocation.takeRanges("ports", 5, AS_ROLE_A)); + Assert.assertEquals( + resources(ports(UNRESERVED_ROLE, range(1025, 1025), range(8080, 8081))), + allocation.getRemaining()); + } + + /** + * Tests range resource exhaustion (i.e. insufficient resources). + */ + @Test + public void testRangeResourceExhaustion() { + MesosResourceAllocation allocation = new MesosResourceAllocation(resources(ports(range(80, 80)))); + Assert.assertEquals( + resources(ports(range(80, 80))), + allocation.takeRanges("ports", 2, AS_NO_ROLE)); + Assert.assertEquals(resources(), allocation.getRemaining()); + } + + // endregion +} diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala index f18c07dbc2dd7..eb8259aaafeb9 100644 --- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala +++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala @@ -26,7 +26,6 @@ import akka.testkit._ import com.netflix.fenzo.TaskRequest.{AssignedResources, NamedResourceSetRequest} import com.netflix.fenzo._ import com.netflix.fenzo.functions.{Action1, Action2} -import com.netflix.fenzo.plugins.VMLeaseObject import org.apache.flink.api.java.tuple.{Tuple2=>FlinkTuple2} import org.apache.flink.configuration.Configuration import org.apache.flink.mesos.scheduler.LaunchCoordinator._ @@ -47,6 +46,7 @@ import scala.collection.JavaConverters._ import org.apache.flink.mesos.Utils.range import org.apache.flink.mesos.Utils.ranges import org.apache.flink.mesos.Utils.scalar +import org.apache.flink.mesos.util.MesosResourceAllocation @RunWith(classOf[JUnitRunner]) class LaunchCoordinatorTest @@ -95,7 +95,7 @@ class LaunchCoordinatorTest override def taskRequest: TaskRequest = generateTaskRequest override def launch( slaveId: SlaveID, - taskAssignment: TaskAssignmentResult): Protos.TaskInfo = { + allocation: MesosResourceAllocation): Protos.TaskInfo = { Protos.TaskInfo.newBuilder .setTaskId(taskID).setName(taskID.getValue) .setCommand(Protos.CommandInfo.newBuilder.setValue("whoami")) @@ -129,7 +129,7 @@ class LaunchCoordinatorTest } def lease(offer: Protos.Offer) = { - new VMLeaseObject(offer) + new Offer(offer) } /** From e6d8b82ac09717b5555cbca62b4f0d426e7bf04a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 6 Dec 2017 12:07:36 +0100 Subject: [PATCH 143/367] [hotfix] Properly delete temp flink dir in create_source_release.sh --- tools/releasing/create_source_release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/releasing/create_source_release.sh b/tools/releasing/create_source_release.sh index 33b4e51d84701..0eff9f8b8ba94 100755 --- a/tools/releasing/create_source_release.sh +++ b/tools/releasing/create_source_release.sh @@ -67,4 +67,4 @@ $SHASUM flink-$RELEASE_VERSION-src.tgz > flink-$RELEASE_VERSION-src.tgz.sha mv flink-$RELEASE_VERSION-src.* ../ cd .. -rm -r flink-tmp-clone +rm -rf flink-tmp-clone From 68b5a61aab56cc88645383bd12c479d5ab4d92f2 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Thu, 9 Nov 2017 19:21:43 +0100 Subject: [PATCH 144/367] [FLINK-7975][QS] Wait for QS client to shutdown. --- .../client/QueryableStateClient.java | 30 ++- .../flink/queryablestate/network/Client.java | 171 ++++++++++++------ .../queryablestate/network/ClientTest.java | 88 +++++++-- .../query/AbstractQueryableStateOperator.java | 2 + 4 files changed, 215 insertions(+), 76 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java index 7abf6bc398725..f1c69edd773ad 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java @@ -108,9 +108,33 @@ public QueryableStateClient(final InetAddress remoteAddress, final int remotePor new DisabledKvStateRequestStats()); } - /** Shuts down the client. */ - public void shutdown() { - client.shutdown(); + /** + * Shuts down the client and returns a {@link CompletableFuture} that + * will be completed when the shutdown process is completed. + * + *

    If an exception is thrown for any reason, then the returned future + * will be completed exceptionally with that exception. + * + * @return A {@link CompletableFuture} for further handling of the + * shutdown result. + */ + public CompletableFuture shutdownAndHandle() { + return client.shutdown(); + } + + /** + * Shuts down the client and waits until shutdown is completed. + * + *

    If an exception is thrown, a warning is logged containing + * the exception message. + */ + public void shutdownAndWait() { + try { + client.shutdown().get(); + LOG.info("The Queryable State Client was shutdown successfully."); + } catch (Exception e) { + LOG.warn("The Queryable State Client shutdown failed: ", e); + } } /** diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java index 12286faa02505..364f835d20816 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java @@ -42,15 +42,19 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder; import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.net.InetSocketAddress; import java.nio.channels.ClosedChannelException; import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -64,6 +68,8 @@ @Internal public class Client { + private static final Logger LOG = LoggerFactory.getLogger(Client.class); + /** The name of the client. Used for logging and stack traces.*/ private final String clientName; @@ -82,8 +88,8 @@ public class Client { /** Pending connections. */ private final Map pendingConnections = new ConcurrentHashMap<>(); - /** Atomic shut down flag. */ - private final AtomicBoolean shutDown = new AtomicBoolean(); + /** Atomic shut down future. */ + private final AtomicReference> clientShutdownFuture = new AtomicReference<>(null); /** * Creates a client with the specified number of event loop threads. @@ -133,7 +139,7 @@ public String getClientName() { } public CompletableFuture sendRequest(final InetSocketAddress serverAddress, final REQ request) { - if (shutDown.get()) { + if (clientShutdownFuture.get() != null) { return FutureUtils.getFailedFuture(new IllegalStateException(clientName + " is already shut down.")); } @@ -166,28 +172,57 @@ public CompletableFuture sendRequest(final InetSocketAddress serverAddress * Shuts down the client and closes all connections. * *

    After a call to this method, all returned futures will be failed. + * + * @return A {@link CompletableFuture} that will be completed when the shutdown process is done. */ - public void shutdown() { - if (shutDown.compareAndSet(false, true)) { + public CompletableFuture shutdown() { + final CompletableFuture newShutdownFuture = new CompletableFuture<>(); + if (clientShutdownFuture.compareAndSet(null, newShutdownFuture)) { + + final List> connectionFutures = new ArrayList<>(); + for (Map.Entry conn : establishedConnections.entrySet()) { if (establishedConnections.remove(conn.getKey(), conn.getValue())) { - conn.getValue().close(); + connectionFutures.add(conn.getValue().close()); } } for (Map.Entry conn : pendingConnections.entrySet()) { if (pendingConnections.remove(conn.getKey()) != null) { - conn.getValue().close(); + connectionFutures.add(conn.getValue().close()); } } - if (bootstrap != null) { - EventLoopGroup group = bootstrap.group(); - if (group != null) { - group.shutdownGracefully(0L, 10L, TimeUnit.SECONDS); + CompletableFuture.allOf( + connectionFutures.toArray(new CompletableFuture[connectionFutures.size()]) + ).whenComplete((result, throwable) -> { + + if (throwable != null) { + LOG.warn("Problem while shutting down the connections at the {}: {}", clientName, throwable); } - } + + if (bootstrap != null) { + EventLoopGroup group = bootstrap.group(); + if (group != null && !group.isShutdown()) { + group.shutdownGracefully(0L, 0L, TimeUnit.MILLISECONDS) + .addListener(finished -> { + if (finished.isSuccess()) { + newShutdownFuture.complete(null); + } else { + newShutdownFuture.completeExceptionally(finished.cause()); + } + }); + } else { + newShutdownFuture.complete(null); + } + } else { + newShutdownFuture.complete(null); + } + }); + + return newShutdownFuture; } + return clientShutdownFuture.get(); } /** @@ -209,8 +244,8 @@ private class PendingConnection implements ChannelFutureListener { /** The established connection after the connect succeeds. */ private EstablishedConnection established; - /** Closed flag. */ - private boolean closed; + /** Atomic shut down future. */ + private final AtomicReference> connectionShutdownFuture = new AtomicReference<>(null); /** Failure cause if something goes wrong. */ private Throwable failureCause; @@ -250,7 +285,7 @@ public CompletableFuture sendRequest(REQ request) { synchronized (connectLock) { if (failureCause != null) { return FutureUtils.getFailedFuture(failureCause); - } else if (closed) { + } else if (connectionShutdownFuture.get() != null) { return FutureUtils.getFailedFuture(new ClosedChannelException()); } else { if (established != null) { @@ -272,7 +307,7 @@ public CompletableFuture sendRequest(REQ request) { */ private void handInChannel(Channel channel) { synchronized (connectLock) { - if (closed || failureCause != null) { + if (connectionShutdownFuture.get() != null || failureCause != null) { // Close the channel and we are done. Any queued requests // are removed on the close/failure call and after that no // new ones can be enqueued. @@ -300,7 +335,7 @@ private void handInChannel(Channel channel) { // Check shut down for possible race with shut down. We // don't want any lingering connections after shut down, // which can happen if we don't check this here. - if (shutDown.get()) { + if (clientShutdownFuture.get() != null) { if (establishedConnections.remove(serverAddress, established)) { established.close(); } @@ -312,32 +347,40 @@ private void handInChannel(Channel channel) { /** * Close the connecting channel with a ClosedChannelException. */ - private void close() { - close(new ClosedChannelException()); + private CompletableFuture close() { + return close(new ClosedChannelException()); } /** * Close the connecting channel with an Exception (can be {@code null}) * or forward to the established channel. */ - private void close(Throwable cause) { - synchronized (connectLock) { - if (!closed) { + private CompletableFuture close(Throwable cause) { + CompletableFuture future = new CompletableFuture<>(); + if (connectionShutdownFuture.compareAndSet(null, future)) { + synchronized (connectLock) { if (failureCause == null) { failureCause = cause; } if (established != null) { - established.close(); + established.close().whenComplete((result, throwable) -> { + if (throwable != null) { + future.completeExceptionally(throwable); + } else { + future.complete(null); + } + }); } else { PendingRequest pending; while ((pending = queuedRequests.poll()) != null) { pending.completeExceptionally(cause); } + future.complete(null); } - closed = true; } } + return connectionShutdownFuture.get(); } @Override @@ -347,7 +390,7 @@ public String toString() { "serverAddress=" + serverAddress + ", queuedRequests=" + queuedRequests.size() + ", established=" + (established != null) + - ", closed=" + closed + + ", closed=" + (connectionShutdownFuture.get() != null) + '}'; } } @@ -383,8 +426,8 @@ private class EstablishedConnection implements ClientHandlerCallback { /** Current request number used to assign unique request IDs. */ private final AtomicLong requestCount = new AtomicLong(); - /** Reference to a failure that was reported by the channel. */ - private final AtomicReference failureCause = new AtomicReference<>(); + /** Atomic shut down future. */ + private final AtomicReference> connectionShutdownFuture = new AtomicReference<>(null); /** * Creates an established connection with the given channel. @@ -412,8 +455,8 @@ private class EstablishedConnection implements ClientHandlerCallback { /** * Close the channel with a ClosedChannelException. */ - void close() { - close(new ClosedChannelException()); + CompletableFuture close() { + return close(new ClosedChannelException()); } /** @@ -422,20 +465,33 @@ void close() { * @param cause The cause to close the channel with. * @return Channel close future */ - private boolean close(Throwable cause) { - if (failureCause.compareAndSet(null, cause)) { - channel.close(); - stats.reportInactiveConnection(); + private CompletableFuture close(final Throwable cause) { + final CompletableFuture shutdownFuture = new CompletableFuture<>(); - for (long requestId : pendingRequests.keySet()) { - TimestampedCompletableFuture pending = pendingRequests.remove(requestId); - if (pending != null && pending.completeExceptionally(cause)) { - stats.reportFailedRequest(); + if (connectionShutdownFuture.compareAndSet(null, shutdownFuture)) { + channel.close().addListener(finished -> { + stats.reportInactiveConnection(); + for (long requestId : pendingRequests.keySet()) { + TimestampedCompletableFuture pending = pendingRequests.remove(requestId); + if (pending != null && pending.completeExceptionally(cause)) { + stats.reportFailedRequest(); + } } - } - return true; + + // when finishing, if netty successfully closes the channel, then the provided exception is used + // as the reason for the closing. If there was something wrong at the netty side, then that exception + // is prioritized over the provided one. + if (finished.isSuccess()) { + shutdownFuture.completeExceptionally(cause); + } else { + LOG.warn("Something went wrong when trying to close connection due to : ", cause); + shutdownFuture.completeExceptionally(finished.cause()); + } + }); } - return false; + + // in case we had a race condition, return the winner of the race. + return connectionShutdownFuture.get(); } /** @@ -464,16 +520,22 @@ CompletableFuture sendRequest(REQ request) { } }); - // Check failure for possible race. We don't want any lingering + // Check for possible race. We don't want any lingering // promises after a failure, which can happen if we don't check // this here. Note that close is treated as a failure as well. - Throwable failure = failureCause.get(); - if (failure != null) { - // Remove from pending requests to guard against concurrent - // removal and to make sure that we only count it once as failed. + CompletableFuture clShutdownFuture = clientShutdownFuture.get(); + if (clShutdownFuture != null) { TimestampedCompletableFuture pending = pendingRequests.remove(requestId); - if (pending != null && pending.completeExceptionally(failure)) { - stats.reportFailedRequest(); + if (pending != null) { + clShutdownFuture.whenComplete((ignored, throwable) -> { + if (throwable != null && pending.completeExceptionally(throwable)) { + stats.reportFailedRequest(); + } else { + // the shutdown future is always completed exceptionally so we should not arrive here. + // but in any case, we complete the pending connection request exceptionally. + pending.completeExceptionally(new ClosedChannelException()); + } + }); } } } catch (Throwable t) { @@ -486,27 +548,25 @@ CompletableFuture sendRequest(REQ request) { @Override public void onRequestResult(long requestId, RESP response) { TimestampedCompletableFuture pending = pendingRequests.remove(requestId); - if (pending != null && pending.complete(response)) { + if (pending != null && !pending.isDone()) { long durationMillis = (System.nanoTime() - pending.getTimestamp()) / 1_000_000L; stats.reportSuccessfulRequest(durationMillis); + pending.complete(response); } } @Override public void onRequestFailure(long requestId, Throwable cause) { TimestampedCompletableFuture pending = pendingRequests.remove(requestId); - if (pending != null && pending.completeExceptionally(cause)) { + if (pending != null && !pending.isDone()) { stats.reportFailedRequest(); + pending.completeExceptionally(cause); } } @Override public void onFailure(Throwable cause) { - if (close(cause)) { - // Remove from established channels, otherwise future - // requests will be handled by this failed channel. - establishedConnections.remove(serverAddress, this); - } + close(cause).handle((cancelled, ignored) -> establishedConnections.remove(serverAddress, this)); } @Override @@ -516,7 +576,6 @@ public String toString() { ", channel=" + channel + ", pendingRequests=" + pendingRequests.size() + ", requestCount=" + requestCount + - ", failureCause=" + failureCause + '}'; } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java index 1fa4deb953675..8638efa680fc0 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap; @@ -54,7 +55,9 @@ import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel; import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder; -import org.junit.AfterClass; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,15 +98,20 @@ public class ClientTest { private static final Logger LOG = LoggerFactory.getLogger(ClientTest.class); + private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(20L, TimeUnit.SECONDS); + // Thread pool for client bootstrap (shared between tests) - private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup(); + private NioEventLoopGroup nioGroup; - private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10L, TimeUnit.SECONDS); + @Before + public void setUp() throws Exception { + nioGroup = new NioEventLoopGroup(); + } - @AfterClass - public static void tearDown() throws Exception { - if (NIO_GROUP != null) { - NIO_GROUP.shutdownGracefully(); + @After + public void tearDown() throws Exception { + if (nioGroup != null) { + nioGroup.shutdownGracefully(); } } @@ -218,7 +226,24 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception assertEquals(expectedRequests, stats.getNumFailed()); } finally { if (client != null) { - client.shutdown(); + Exception exc = null; + try { + + // todo here we were seeing this problem: + // https://github.com/netty/netty/issues/4357 if we do a get(). + // this is why we now simply wait a bit so that everything is + // shut down and then we check + + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + exc = e; + LOG.error("An exception occurred while shutting down netty.", e); + } + + Assert.assertTrue( + ExceptionUtils.stringifyException(exc), + client.isEventGroupShutdown() + ); } if (serverChannel != null) { @@ -265,7 +290,12 @@ public void testRequestUnavailableHost() throws Exception { } } finally { if (client != null) { - client.shutdown(); + try { + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertTrue(client.isEventGroupShutdown()); } assertEquals("Channel leak", 0L, stats.getNumConnections()); @@ -366,7 +396,12 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } if (client != null) { - client.shutdown(); + try { + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertTrue(client.isEventGroupShutdown()); } assertEquals("Channel leak", 0L, stats.getNumConnections()); @@ -467,7 +502,12 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception assertEquals(2L, stats.getNumFailed()); } finally { if (client != null) { - client.shutdown(); + try { + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertTrue(client.isEventGroupShutdown()); } if (serverChannel != null) { @@ -548,7 +588,12 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception assertEquals(1L, stats.getNumFailed()); } finally { if (client != null) { - client.shutdown(); + try { + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertTrue(client.isEventGroupShutdown()); } if (serverChannel != null) { @@ -661,7 +706,7 @@ public void testClientServerIntegration() throws Throwable { Collections.shuffle(random); // Dispatch queries - List> futures = new ArrayList<>(batchSize); + List> futures = new ArrayList<>(batchSize); for (int j = 0; j < batchSize; j++) { int targetServer = random.get(j) % numServers; @@ -700,8 +745,12 @@ public void testClientServerIntegration() throws Throwable { LOG.info("Number of requests {}/100_000", numRequests); } - // Shut down - client.shutdown(); + try { + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertTrue(client.isEventGroupShutdown()); for (Future future : taskFutures) { try { @@ -739,7 +788,12 @@ public void testClientServerIntegration() throws Throwable { } } finally { if (client != null) { - client.shutdown(); + try { + client.shutdown().get(10L, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertTrue(client.isEventGroupShutdown()); } for (int i = 0; i < numServers; i++) { @@ -761,7 +815,7 @@ private Channel createServerChannel(final ChannelHandler... handlers) throws Unk // Bind address and port .localAddress(InetAddress.getLocalHost(), 0) // NIO server channels - .group(NIO_GROUP) + .group(nioGroup) .channel(NioServerSocketChannel.class) // See initializer for pipeline details .childHandler(new ChannelInitializer() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/query/AbstractQueryableStateOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/query/AbstractQueryableStateOperator.java index 7522a617bf792..5ca9c1e3cce0c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/query/AbstractQueryableStateOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/query/AbstractQueryableStateOperator.java @@ -36,6 +36,8 @@ abstract class AbstractQueryableStateOperator extends AbstractStreamOperator implements OneInputStreamOperator { + private static final long serialVersionUID = 7842489558298787382L; + /** State descriptor for the queryable state instance. */ protected final StateDescriptor stateDescriptor; From 415b74ce17cb1ebfa968d158accf32aa8f56230d Mon Sep 17 00:00:00 2001 From: kkloudas Date: Thu, 9 Nov 2017 19:30:29 +0100 Subject: [PATCH 145/367] [FLINK-7974][QS] Wait for QS abstract server to shutdown. --- .../program/rest/RestClusterClient.java | 3 +- .../org/apache/flink/util/ExecutorUtils.java | 77 ++++++++++ .../MesosApplicationMasterRunner.java | 3 +- .../network/AbstractServerBase.java | 95 +++++++++--- .../network/AbstractServerHandler.java | 11 +- .../proxy/KvStateClientProxyHandler.java | 34 +++-- .../client/proxy/KvStateClientProxyImpl.java | 8 +- .../server/KvStateServerHandler.java | 4 +- .../server/KvStateServerImpl.java | 8 +- .../HAAbstractQueryableStateTestBase.java | 5 +- .../NonHAAbstractQueryableStateTestBase.java | 4 +- .../network/AbstractServerTest.java | 135 ++++++++++-------- .../flink/runtime/concurrent/Executors.java | 49 +------ .../taskexecutor/TaskManagerRunner.java | 6 +- .../flink/runtime/jobmanager/JobManager.scala | 6 +- .../minicluster/FlinkMiniCluster.scala | 6 +- .../slotmanager/SlotProtocolTest.java | 3 +- .../legacy/ExecutionGraphCacheTest.java | 4 +- .../yarn/YarnApplicationMasterRunner.java | 3 +- 19 files changed, 306 insertions(+), 158 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 3916514627cff..d5c0e50990a89 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -47,6 +47,7 @@ import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerResponseBody; import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.ExecutorUtils; import javax.annotation.Nullable; @@ -89,7 +90,7 @@ public void shutdown() { log.error("An error occurred during the client shutdown.", e); } this.restClient.shutdown(Time.seconds(5)); - org.apache.flink.runtime.concurrent.Executors.gracefulShutdown(5, TimeUnit.SECONDS, this.executorService); + ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.executorService); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java new file mode 100644 index 0000000000000..d98bdd29cbd80 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java @@ -0,0 +1,77 @@ +/* + * 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.flink.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Utilities for {@link java.util.concurrent.Executor Executors}. + */ +public class ExecutorUtils { + + private static final Logger LOG = LoggerFactory.getLogger(ExecutorUtils.class); + + /** + * Gracefully shutdown the given {@link ExecutorService}. The call waits the given timeout that + * all ExecutorServices terminate. If the ExecutorServices do not terminate in this time, + * they will be shut down hard. + * + * @param timeout to wait for the termination of all ExecutorServices + * @param unit of the timeout + * @param executorServices to shut down + */ + public static void gracefulShutdown(long timeout, TimeUnit unit, ExecutorService... executorServices) { + for (ExecutorService executorService: executorServices) { + executorService.shutdown(); + } + + boolean wasInterrupted = false; + final long endTime = unit.toMillis(timeout) + System.currentTimeMillis(); + long timeLeft = unit.toMillis(timeout); + boolean hasTimeLeft = timeLeft > 0L; + + for (ExecutorService executorService: executorServices) { + if (wasInterrupted || !hasTimeLeft) { + executorService.shutdownNow(); + } else { + try { + if (!executorService.awaitTermination(timeLeft, TimeUnit.MILLISECONDS)) { + LOG.warn("ExecutorService did not terminate in time. Shutting it down now."); + executorService.shutdownNow(); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while shutting down executor services. Shutting all " + + "remaining ExecutorServices down now.", e); + executorService.shutdownNow(); + + wasInterrupted = true; + + Thread.currentThread().interrupt(); + } + + timeLeft = endTime - System.currentTimeMillis(); + hasTimeLeft = timeLeft > 0L; + } + } + } +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java index 93eb3c6622a52..544150b99db10 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java @@ -52,6 +52,7 @@ import org.apache.flink.runtime.webmonitor.WebMonitor; import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaJobManagerRetriever; import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; +import org.apache.flink.util.ExecutorUtils; import akka.actor.ActorRef; import akka.actor.ActorSystem; @@ -439,7 +440,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie } } - org.apache.flink.runtime.concurrent.Executors.gracefulShutdown( + ExecutorUtils.gracefulShutdown( AkkaUtils.getTimeout(config).toMillis(), TimeUnit.MILLISECONDS, futureExecutor, diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java index 82a05f2a09a30..d5afeb3b033d7 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.queryablestate.network.messages.MessageBody; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -45,10 +46,12 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * The base class for every server in the queryable state module. @@ -83,6 +86,9 @@ public abstract class AbstractServerBase> serverShutdownFuture = new AtomicReference<>(null); + /** Netty's ServerBootstrap. */ private ServerBootstrap bootstrap; @@ -179,8 +185,8 @@ public InetSocketAddress getServerAddress() { * @throws Exception If something goes wrong during the bind operation. */ public void start() throws Throwable { - Preconditions.checkState(serverAddress == null, - serverName + " is already running @ " + serverAddress + '.'); + Preconditions.checkState(serverAddress == null && serverShutdownFuture.get() == null, + serverName + " is already running @ " + serverAddress + ". "); Iterator portIterator = bindPortRange.iterator(); while (portIterator.hasNext() && !attemptToBind(portIterator.next())) {} @@ -251,7 +257,22 @@ private boolean attemptToBind(final int port) throws Throwable { throw future.cause(); } catch (BindException e) { log.debug("Failed to start {} on port {}: {}.", serverName, port, e.getMessage()); - shutdown(); + try { + // we shutdown the server but we reset the future every time because in + // case of failure to bind, we will call attemptToBind() here, and not resetting + // the flag will interfere with future shutdown attempts. + + shutdownServer() + .whenComplete((ignoredV, ignoredT) -> serverShutdownFuture.getAndSet(null)) + .get(); + } catch (Exception r) { + + // Here we were seeing this problem: + // https://github.com/netty/netty/issues/4357 if we do a get(). + // this is why we now simply wait a bit so that everything is shut down. + + log.warn("Problem while shutting down {}: {}", serverName, r.getMessage()); + } } // any other type of exception we let it bubble up. return false; @@ -259,26 +280,62 @@ private boolean attemptToBind(final int port) throws Throwable { /** * Shuts down the server and all related thread pools. + * @return A {@link CompletableFuture} that will be completed upon termination of the shutdown process. */ - public void shutdown() { - log.info("Shutting down {} @ {}", serverName, serverAddress); - - if (handler != null) { - handler.shutdown(); - handler = null; - } - - if (queryExecutor != null) { - queryExecutor.shutdown(); - } + public CompletableFuture shutdownServer() { + CompletableFuture shutdownFuture = new CompletableFuture<>(); + if (serverShutdownFuture.compareAndSet(null, shutdownFuture)) { + log.info("Shutting down {} @ {}", serverName, serverAddress); + + final CompletableFuture groupShutdownFuture = new CompletableFuture<>(); + if (bootstrap != null) { + EventLoopGroup group = bootstrap.group(); + if (group != null && !group.isShutdown()) { + group.shutdownGracefully(0L, 0L, TimeUnit.MILLISECONDS) + .addListener(finished -> { + if (finished.isSuccess()) { + groupShutdownFuture.complete(null); + } else { + groupShutdownFuture.completeExceptionally(finished.cause()); + } + }); + } else { + groupShutdownFuture.complete(null); + } + } else { + groupShutdownFuture.complete(null); + } - if (bootstrap != null) { - EventLoopGroup group = bootstrap.group(); - if (group != null) { - group.shutdownGracefully(0L, 10L, TimeUnit.SECONDS); + final CompletableFuture handlerShutdownFuture = new CompletableFuture<>(); + if (handler == null) { + handlerShutdownFuture.complete(null); + } else { + handler.shutdown().whenComplete((result, throwable) -> { + if (throwable != null) { + handlerShutdownFuture.completeExceptionally(throwable); + } else { + handlerShutdownFuture.complete(null); + } + }); } + + final CompletableFuture queryExecShutdownFuture = CompletableFuture.runAsync(() -> { + if (queryExecutor != null) { + ExecutorUtils.gracefulShutdown(10L, TimeUnit.MINUTES, queryExecutor); + } + }); + + CompletableFuture.allOf( + queryExecShutdownFuture, groupShutdownFuture, handlerShutdownFuture + ).whenComplete((result, throwable) -> { + if (throwable != null) { + shutdownFuture.completeExceptionally(throwable); + } else { + shutdownFuture.complete(null); + } + }); } - serverAddress = null; + return serverShutdownFuture.get(); } /** diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java index 7e71a11c3e860..a514723e668e6 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java @@ -183,9 +183,16 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E public abstract CompletableFuture handleRequest(final long requestId, final REQ request); /** - * Shuts down any handler specific resources, e.g. thread pools etc. + * Shuts down any handler-specific resources, e.g. thread pools etc and returns + * a {@link CompletableFuture}. + * + *

    If an exception is thrown during the shutdown process, then that exception + * will be included in the returned future. + * + * @return A {@link CompletableFuture} that will be completed when the shutdown + * process actually finishes. */ - public abstract void shutdown(); + public abstract CompletableFuture shutdown(); /** * Task to execute the actual query against the state instance. diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java index af3370198e1df..29ee0d7932910 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java @@ -33,7 +33,9 @@ import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats; import org.apache.flink.queryablestate.network.stats.KvStateRequestStats; import org.apache.flink.queryablestate.server.KvStateServerImpl; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.query.KvStateClientProxy; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.KvStateMessage; @@ -42,6 +44,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler; +import akka.dispatch.OnComplete; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -141,6 +144,7 @@ private void executeActionAsync( // KvStateLocation. Therefore we retry this query and // force look up the location. + LOG.debug("Retrying after failing to retrieve state due to: {}.", throwable.getCause().getMessage()); executeActionAsync(result, request, true); } else { result.completeExceptionally(throwable); @@ -203,20 +207,34 @@ private CompletableFuture getKvStateLookupInfo( LOG.debug("Retrieving location for state={} of job={} from the job manager.", jobId, queryableStateName); + final CompletableFuture location = new CompletableFuture<>(); + lookupCache.put(cacheKey, location); return proxy.getJobManagerFuture().thenComposeAsync( jobManagerGateway -> { final Object msg = new KvStateMessage.LookupKvStateLocation(jobId, queryableStateName); - final CompletableFuture locationFuture = FutureUtils.toJava( - jobManagerGateway.ask(msg, FiniteDuration.apply(1000L, TimeUnit.MILLISECONDS)) - .mapTo(ClassTag$.MODULE$.apply(KvStateLocation.class))); - - lookupCache.put(cacheKey, locationFuture); - return locationFuture; + jobManagerGateway.ask(msg, FiniteDuration.apply(1000L, TimeUnit.MILLISECONDS)) + .mapTo(ClassTag$.MODULE$.apply(KvStateLocation.class)) + .onComplete(new OnComplete() { + + @Override + public void onComplete(Throwable failure, KvStateLocation loc) throws Throwable { + if (failure != null) { + if (failure instanceof FlinkJobNotFoundException) { + // if the jobId was wrong, remove the entry from the cache. + lookupCache.remove(cacheKey); + } + location.completeExceptionally(failure); + } else { + location.complete(loc); + } + } + }, Executors.directExecutionContext()); + return location; }, queryExecutor); } @Override - public void shutdown() { - kvStateClient.shutdown(); + public CompletableFuture shutdown() { + return kvStateClient.shutdown(); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java index 6fcaf40107c77..aa5e7b652b016 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java @@ -35,6 +35,7 @@ import java.net.InetSocketAddress; import java.util.Iterator; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; /** * The default implementation of the {@link KvStateClientProxy}. @@ -96,7 +97,12 @@ public void start() throws Throwable { @Override public void shutdown() { - super.shutdown(); + try { + shutdownServer().get(10L, TimeUnit.SECONDS); + log.info("{} was shutdown successfully.", getServerName()); + } catch (Exception e) { + log.warn("{} shutdown failed: {}", getServerName(), e); + } } @Override diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java index 476f153107894..18a294429ae9b 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java @@ -101,7 +101,7 @@ public CompletableFuture handleRequest(final long requestId, fi } @Override - public void shutdown() { - // do nothing + public CompletableFuture shutdown() { + return CompletableFuture.completedFuture(null); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java index 3a37a3a620c3a..072026888a896 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java @@ -32,6 +32,7 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.Iterator; +import java.util.concurrent.TimeUnit; /** * The default implementation of the {@link KvStateServer}. @@ -101,6 +102,11 @@ public InetSocketAddress getServerAddress() { @Override public void shutdown() { - super.shutdown(); + try { + shutdownServer().get(10L, TimeUnit.SECONDS); + log.info("{} was shutdown successfully.", getServerName()); + } catch (Exception e) { + log.warn("{} shutdown failed: {}", getServerName(), e); + } } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java index 79809b3004781..b9ce7c2a2c9aa 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java @@ -88,11 +88,10 @@ public static void tearDown() { try { zkServer.stop(); zkServer.close(); - } catch (Exception e) { + client.shutdownAndWait(); + } catch (Throwable e) { e.printStackTrace(); fail(e.getMessage()); } - - client.shutdown(); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java index 6945cca6360aa..a5e24b2ec9595 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java @@ -69,10 +69,10 @@ public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { public static void tearDown() { try { cluster.stop(); - } catch (Exception e) { + client.shutdownAndWait(); + } catch (Throwable e) { e.printStackTrace(); fail(e.getMessage()); } - client.shutdown(); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java index 3d2ed4066ca69..103c638ee3f7e 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java @@ -22,7 +22,9 @@ import org.apache.flink.queryablestate.network.messages.MessageBody; import org.apache.flink.queryablestate.network.messages.MessageDeserializer; import org.apache.flink.queryablestate.network.messages.MessageSerializer; +import org.apache.flink.queryablestate.network.stats.AtomicKvStateRequestStats; import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats; +import org.apache.flink.queryablestate.network.stats.KvStateRequestStats; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -37,6 +39,7 @@ import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -60,22 +63,15 @@ public void testServerInitializationFailure() throws Throwable { expectedEx.expect(FlinkRuntimeException.class); expectedEx.expectMessage("Unable to start Test Server 2. All ports in provided range are occupied."); - TestServer server1 = null; - TestServer server2 = null; - try { + List portList = new ArrayList<>(); + portList.add(7777); - server1 = startServer("Test Server 1", 7777); - Assert.assertEquals(7777L, server1.getServerAddress().getPort()); + try (TestServer server1 = new TestServer("Test Server 1", new DisabledKvStateRequestStats(), portList.iterator())) { + server1.start(); - server2 = startServer("Test Server 2", 7777); - } finally { - - if (server1 != null) { - server1.shutdown(); - } - - if (server2 != null) { - server2.shutdown(); + try (TestServer server2 = new TestServer("Test Server 2", new DisabledKvStateRequestStats(), + Collections.singletonList(server1.getServerAddress().getPort()).iterator())) { + server2.start(); } } } @@ -86,69 +82,81 @@ public void testServerInitializationFailure() throws Throwable { */ @Test public void testPortRangeSuccess() throws Throwable { - TestServer server1 = null; - TestServer server2 = null; - Client client = null; - try { - server1 = startServer("Test Server 1", 7777, 7778, 7779); - Assert.assertEquals(7777L, server1.getServerAddress().getPort()); - - server2 = startServer("Test Server 2", 7777, 7778, 7779); - Assert.assertEquals(7778L, server2.getServerAddress().getPort()); - - client = new Client<>( - "Test Client", - 1, - new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()), - new DisabledKvStateRequestStats()); + // this is shared between the two servers. + AtomicKvStateRequestStats serverStats = new AtomicKvStateRequestStats(); + AtomicKvStateRequestStats clientStats = new AtomicKvStateRequestStats(); + + List portList = new ArrayList<>(); + portList.add(7777); + portList.add(7778); + portList.add(7779); + + try ( + TestServer server1 = new TestServer("Test Server 1", serverStats, portList.iterator()); + TestServer server2 = new TestServer("Test Server 2", serverStats, portList.iterator()); + TestClient client = new TestClient( + "Test Client", + 1, + new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()), + clientStats + ) + ) { + server1.start(); + Assert.assertTrue(server1.getServerAddress().getPort() >= 7777 && server1.getServerAddress().getPort() <= 7779); + + server2.start(); + Assert.assertTrue(server2.getServerAddress().getPort() >= 7777 && server2.getServerAddress().getPort() <= 7779); TestMessage response1 = client.sendRequest(server1.getServerAddress(), new TestMessage("ping")).join(); Assert.assertEquals(server1.getServerName() + "-ping", response1.getMessage()); TestMessage response2 = client.sendRequest(server2.getServerAddress(), new TestMessage("pong")).join(); Assert.assertEquals(server2.getServerName() + "-pong", response2.getMessage()); - } finally { - if (server1 != null) { - server1.shutdown(); - } - - if (server2 != null) { - server2.shutdown(); - } + // the client connects to both servers and the stats object is shared. + Assert.assertEquals(2L, serverStats.getNumConnections()); - if (client != null) { - client.shutdown(); - } + Assert.assertEquals(2L, clientStats.getNumConnections()); + Assert.assertEquals(0L, clientStats.getNumFailed()); + Assert.assertEquals(2L, clientStats.getNumSuccessful()); + Assert.assertEquals(2L, clientStats.getNumRequests()); } + + Assert.assertEquals(0L, clientStats.getNumConnections()); + Assert.assertEquals(0L, clientStats.getNumFailed()); + Assert.assertEquals(2L, clientStats.getNumSuccessful()); + Assert.assertEquals(2L, clientStats.getNumRequests()); } - /** - * Initializes a {@link TestServer} with the given port range. - * @param serverName the name of the server. - * @param ports a range of ports. - * @return A test server with the given name. - */ - private TestServer startServer(String serverName, int... ports) throws Throwable { - List portList = new ArrayList<>(ports.length); - for (int p : ports) { - portList.add(p); + private static class TestClient extends Client implements AutoCloseable { + + TestClient( + String clientName, + int numEventLoopThreads, + MessageSerializer serializer, + KvStateRequestStats stats) { + super(clientName, numEventLoopThreads, serializer, stats); } - final TestServer server = new TestServer(serverName, portList.iterator()); - server.start(); - return server; + @Override + public void close() throws Exception { + shutdown().join(); + Assert.assertTrue(isEventGroupShutdown()); + } } /** * A server that receives a {@link TestMessage test message} and returns another test * message containing the same string as the request with the name of the server prepended. */ - private class TestServer extends AbstractServerBase { + private static class TestServer extends AbstractServerBase implements AutoCloseable { - protected TestServer(String name, Iterator bindPort) throws UnknownHostException { + private final KvStateRequestStats requestStats; + + TestServer(String name, KvStateRequestStats stats, Iterator bindPort) throws UnknownHostException { super(name, InetAddress.getLocalHost(), bindPort, 1, 1); + this.requestStats = stats; } @Override @@ -156,7 +164,7 @@ public AbstractServerHandler initializeHandler() { return new AbstractServerHandler( this, new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()), - new DisabledKvStateRequestStats()) { + requestStats) { @Override public CompletableFuture handleRequest(long requestId, TestMessage request) { @@ -165,11 +173,22 @@ public CompletableFuture handleRequest(long requestId, TestMessage } @Override - public void shutdown() { - // do nothing + public CompletableFuture shutdown() { + return CompletableFuture.completedFuture(null); } }; } + + @Override + public void close() throws Exception { + shutdownServer().get(); + if (requestStats instanceof AtomicKvStateRequestStats) { + AtomicKvStateRequestStats stats = (AtomicKvStateRequestStats) requestStats; + Assert.assertEquals(0L, stats.getNumConnections()); + } + Assert.assertTrue(getQueryExecutor().isTerminated()); + Assert.assertTrue(isEventGroupShutdown()); + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java index 04cdce741a79b..703ac4eba7fec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Executors.java @@ -22,14 +22,13 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; + import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; import scala.concurrent.ExecutionContext; /** - * Collection of {@link Executor} implementations + * Collection of {@link Executor} implementations. */ public class Executors { @@ -94,48 +93,4 @@ public ExecutionContext prepare() { return this; } } - - /** - * Gracefully shutdown the given {@link ExecutorService}. The call waits the given timeout that - * all ExecutorServices terminate. If the ExecutorServices do not terminate in this time, - * they will be shut down hard. - * - * @param timeout to wait for the termination of all ExecutorServices - * @param unit of the timeout - * @param executorServices to shut down - */ - public static void gracefulShutdown(long timeout, TimeUnit unit, ExecutorService... executorServices) { - for (ExecutorService executorService: executorServices) { - executorService.shutdown(); - } - - boolean wasInterrupted = false; - final long endTime = unit.toMillis(timeout) + System.currentTimeMillis(); - long timeLeft = unit.toMillis(timeout); - boolean hasTimeLeft = timeLeft > 0L; - - for (ExecutorService executorService: executorServices) { - if (wasInterrupted || !hasTimeLeft) { - executorService.shutdownNow(); - } else { - try { - if (!executorService.awaitTermination(timeLeft, TimeUnit.MILLISECONDS)) { - LOG.warn("ExecutorService did not terminate in time. Shutting it down now."); - executorService.shutdownNow(); - } - } catch (InterruptedException e) { - LOG.warn("Interrupted while shutting down executor services. Shutting all " + - "remaining ExecutorServices down now.", e); - executorService.shutdownNow(); - - wasInterrupted = true; - - Thread.currentThread().interrupt(); - } - - timeLeft = endTime - System.currentTimeMillis(); - hasTimeLeft = timeLeft > 0L; - } - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index a24daf0f608fa..7258e52b3e574 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; @@ -47,6 +46,7 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.runtime.util.SignalHandler; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.Preconditions; import akka.actor.ActorSystem; @@ -88,7 +88,7 @@ public class TaskManagerRunner implements FatalErrorHandler { private final MetricRegistryImpl metricRegistry; - /** Executor used to run future callbacks */ + /** Executor used to run future callbacks. */ private final ExecutorService executor; private final TaskExecutor taskManager; @@ -165,7 +165,7 @@ protected void shutDownInternally() throws Exception { exception = ExceptionUtils.firstOrSuppressed(e, exception); } - Executors.gracefulShutdown(timeout.toMilliseconds(), TimeUnit.MILLISECONDS, executor); + ExecutorUtils.gracefulShutdown(timeout.toMilliseconds(), TimeUnit.MILLISECONDS, executor); if (exception != null) { throw exception; diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index f57637a5d71e7..af42870ab69b6 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -45,7 +45,7 @@ import org.apache.flink.runtime.clusterframework.{BootstrapTools, FlinkResourceM import org.apache.flink.runtime.clusterframework.messages._ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.concurrent.{FutureUtils, ScheduledExecutorServiceAdapter, Executors => FlinkExecutors} +import org.apache.flink.runtime.concurrent.{FutureUtils, ScheduledExecutorServiceAdapter} import org.apache.flink.runtime.execution.SuppressRestartsException import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders.ResolveOrder import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, LibraryCacheManager} @@ -84,7 +84,7 @@ import org.apache.flink.runtime.util._ import org.apache.flink.runtime.webmonitor.retriever.impl.{AkkaJobManagerRetriever, AkkaQueryServiceRetriever} import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils} import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages} -import org.apache.flink.util.{FlinkException, InstantiationUtil, NetUtils, SerializedThrowable} +import org.apache.flink.util._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -2066,7 +2066,7 @@ object JobManager { LOG.warn("Could not properly shut down the metric registry.", t) } - FlinkExecutors.gracefulShutdown( + ExecutorUtils.gracefulShutdown( timeout.toMillis, TimeUnit.MILLISECONDS, futureExecutor, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index 227b8549e108c..5554061997ff6 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -32,7 +32,7 @@ import org.apache.flink.configuration._ import org.apache.flink.core.fs.Path import org.apache.flink.runtime.akka.{AkkaJobManagerGateway, AkkaUtils} import org.apache.flink.runtime.client.{JobClient, JobExecutionException} -import org.apache.flink.runtime.concurrent.{FutureUtils, ScheduledExecutorServiceAdapter, Executors => FlinkExecutors} +import org.apache.flink.runtime.concurrent.{FutureUtils, ScheduledExecutorServiceAdapter} import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils} import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway} @@ -44,7 +44,7 @@ import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegi import org.apache.flink.runtime.util.{ExecutorThreadFactory, Hardware} import org.apache.flink.runtime.webmonitor.retriever.impl.{AkkaJobManagerRetriever, AkkaQueryServiceRetriever} import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils} -import org.apache.flink.util.NetUtils +import org.apache.flink.util.{ExecutorUtils, NetUtils} import org.slf4j.LoggerFactory import scala.concurrent.duration.{Duration, FiniteDuration} @@ -440,7 +440,7 @@ abstract class FlinkMiniCluster( isRunning = false - FlinkExecutors.gracefulShutdown( + ExecutorUtils.gracefulShutdown( timeout.toMillis, TimeUnit.MILLISECONDS, futureExecutor, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index 97942ea467890..79e38dfd9de0e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -64,7 +65,7 @@ public class SlotProtocolTest extends TestLogger { @AfterClass public static void afterClass() { - Executors.gracefulShutdown(timeout, TimeUnit.MILLISECONDS, scheduledExecutorService); + ExecutorUtils.gracefulShutdown(timeout, TimeUnit.MILLISECONDS, scheduledExecutorService); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java index 1a8ea84de2b6e..ecadaa5ae7efe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; -import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; @@ -33,6 +32,7 @@ import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -233,7 +233,7 @@ public void testConcurrentAccess() throws Exception { verify(jobManagerGateway, times(1)).requestJob(eq(jobId), any(Time.class)); } finally { - Executors.gracefulShutdown(5000L, TimeUnit.MILLISECONDS, executor); + ExecutorUtils.gracefulShutdown(5000L, TimeUnit.MILLISECONDS, executor); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 3bdc2ac0184d3..279981ad2c931 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -51,6 +51,7 @@ import org.apache.flink.runtime.webmonitor.WebMonitor; import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaJobManagerRetriever; import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -472,7 +473,7 @@ protected int runApplicationMaster(Configuration config) { } } - org.apache.flink.runtime.concurrent.Executors.gracefulShutdown( + ExecutorUtils.gracefulShutdown( AkkaUtils.getTimeout(config).toMillis(), TimeUnit.MILLISECONDS, futureExecutor, From 9b63cbaceca99b352f39b129b1f19b8ad9b9a645 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Wed, 6 Dec 2017 14:32:46 +0100 Subject: [PATCH 146/367] [FLINK-7880][QS] Wait for proper resource cleanup after each ITCase. --- .../AbstractQueryableStateTestBase.java | 1073 +++++++---------- .../HAAbstractQueryableStateTestBase.java | 22 +- .../HAQueryableStateRocksDBBackendITCase.java | 2 - .../NonHAAbstractQueryableStateTestBase.java | 11 +- ...nHAQueryableStateRocksDBBackendITCase.java | 2 - 5 files changed, 476 insertions(+), 634 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 65e9bb5183a7e..5a283677c571d 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -73,6 +73,7 @@ import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.util.ArrayList; @@ -92,7 +93,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongArray; -import scala.concurrent.Await; import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; import scala.reflect.ClassTag$; @@ -159,52 +159,40 @@ public void setUp() throws Exception { @Test @SuppressWarnings("unchecked") public void testQueryableState() throws Exception { - // Config + final Deadline deadline = TEST_TIMEOUT.fromNow(); final int numKeys = 256; - JobID jobId = null; + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - try { - // - // Test program - // - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestKeyRangeSource(numKeys)); - - // Reducing state - ReducingStateDescriptor> reducingState = new ReducingStateDescriptor<>( - "any-name", - new SumReduce(), - source.getType()); - - final String queryName = "hakuna-matata"; - - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 7143749578983540352L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState(queryName, reducingState); + DataStream> source = env.addSource(new TestKeyRangeSource(numKeys)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - cluster.submitJobDetached(jobGraph); + ReducingStateDescriptor> reducingState = new ReducingStateDescriptor<>( + "any-name", new SumReduce(), source.getType()); + + final String queryName = "hakuna-matata"; + + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 7143749578983540352L; + + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState(queryName, reducingState); - // - // Start querying - // - jobId = jobGraph.getJobID(); + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); + + cluster.submitJobDetached(jobGraph); final AtomicLongArray counts = new AtomicLongArray(numKeys); @@ -261,16 +249,6 @@ public Integer getKey(Tuple2 value) throws Exception { long count = counts.get(i); assertTrue("Count at position " + i + " is " + count, count > 0); } - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -282,91 +260,94 @@ public void testDuplicateRegistrationFailsJob() throws Exception { final Deadline deadline = TEST_TIMEOUT.fromNow(); final int numKeys = 256; - JobID jobId = null; + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - try { - // - // Test program - // - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestKeyRangeSource(numKeys)); - - // Reducing state - ReducingStateDescriptor> reducingState = new ReducingStateDescriptor<>( - "any-name", - new SumReduce(), - source.getType()); - - final String queryName = "duplicate-me"; - - final QueryableStateStream> queryableState = - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = -4126824763829132959L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState(queryName, reducingState); + DataStream> source = env.addSource(new TestKeyRangeSource(numKeys)); - final QueryableStateStream> duplicate = - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = -6265024000462809436L; + // Reducing state + ReducingStateDescriptor> reducingState = new ReducingStateDescriptor<>( + "any-name", + new SumReduce(), + source.getType()); - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState(queryName); + final String queryName = "duplicate-me"; - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + final QueryableStateStream> queryableState = + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = -4126824763829132959L; - CompletableFuture failedFuture = FutureUtils.toJava( - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.FAILED), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(TestingJobManagerMessages.JobStatusIs.class))); + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState(queryName, reducingState); - cluster.submitJobDetached(jobGraph); + final QueryableStateStream> duplicate = + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = -6265024000462809436L; - TestingJobManagerMessages.JobStatusIs jobStatus = + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState(queryName); + + // Submit the job graph + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + final JobID jobId = jobGraph.getJobID(); + + final CompletableFuture failedFuture = + notifyWhenJobStatusIs(jobId, JobStatus.FAILED, deadline); + + final CompletableFuture cancellationFuture = + notifyWhenJobStatusIs(jobId, JobStatus.CANCELED, deadline); + + cluster.submitJobDetached(jobGraph); + + try { + final TestingJobManagerMessages.JobStatusIs jobStatus = failedFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertEquals(JobStatus.FAILED, jobStatus.state()); + } catch (Exception e) { + + // if the assertion fails, it means that the job was (falsely) not cancelled. + // in this case, and given that the mini-cluster is shared with other tests, + // we cancel the job and wait for the cancellation so that the resources are freed. - // Get the job and check the cause - JobManagerMessages.JobFound jobFound = FutureUtils.toJava( - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.RequestJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(JobManagerMessages.JobFound.class))) - .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - - String failureCause = jobFound.executionGraph().getFailureCause().getExceptionAsString(); - - assertTrue("Not instance of SuppressRestartsException", failureCause.startsWith("org.apache.flink.runtime.execution.SuppressRestartsException")); - int causedByIndex = failureCause.indexOf("Caused by: "); - String subFailureCause = failureCause.substring(causedByIndex + "Caused by: ".length()); - assertTrue("Not caused by IllegalStateException", subFailureCause.startsWith("java.lang.IllegalStateException")); - assertTrue("Exception does not contain registration name", subFailureCause.contains(queryName)); - } finally { - // Free cluster resources if (jobId != null) { - scala.concurrent.Future cancellation = cluster - .getLeaderGateway(deadline.timeLeft()) + cluster.getLeaderGateway(deadline.timeLeft()) .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(JobManagerMessages.CancellationSuccess.class)); + .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class)); - Await.ready(cancellation, deadline.timeLeft()); + cancellationFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); } + + // and we re-throw the exception. + throw e; } + + // Get the job and check the cause + JobManagerMessages.JobFound jobFound = FutureUtils.toJava( + cluster.getLeaderGateway(deadline.timeLeft()) + .ask(new JobManagerMessages.RequestJob(jobId), deadline.timeLeft()) + .mapTo(ClassTag$.MODULE$.apply(JobManagerMessages.JobFound.class))) + .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + + String failureCause = jobFound.executionGraph().getFailureCause().getExceptionAsString(); + + assertEquals(JobStatus.FAILED, jobFound.executionGraph().getState()); + assertTrue("Not instance of SuppressRestartsException", failureCause.startsWith("org.apache.flink.runtime.execution.SuppressRestartsException")); + int causedByIndex = failureCause.indexOf("Caused by: "); + String subFailureCause = failureCause.substring(causedByIndex + "Caused by: ".length()); + assertTrue("Not caused by IllegalStateException", subFailureCause.startsWith("java.lang.IllegalStateException")); + assertTrue("Exception does not contain registration name", subFailureCause.contains(queryName)); } /** @@ -377,55 +358,40 @@ public Integer getKey(Tuple2 value) throws Exception { */ @Test public void testValueState() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Value state - ValueStateDescriptor> valueState = new ValueStateDescriptor<>( - "any", - source.getType()); - - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState("hakuna", valueState); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); - cluster.submitJobDetached(jobGraph); + // Value state + ValueStateDescriptor> valueState = new ValueStateDescriptor<>("any", source.getType()); - executeValueQuery(deadline, client, jobId, "hakuna", valueState, numElements); - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 7662520075515707428L; - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + @Override + public Integer getKey(Tuple2 value) { + return value.f0; } + }).asQueryableState("hakuna", valueState); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); + + cluster.submitJobDetached(jobGraph); + + executeValueQuery(deadline, client, jobId, "hakuna", valueState, numElements); } } @@ -434,48 +400,36 @@ public Integer getKey(Tuple2 value) throws Exception { * contains a wrong jobId or wrong queryable state name. */ @Test + @Ignore public void testWrongJobIdAndWrongQueryableStateName() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Value state - ValueStateDescriptor> valueState = - new ValueStateDescriptor<>("any", source.getType()); - - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState("hakuna", valueState); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); + ValueStateDescriptor> valueState = new ValueStateDescriptor<>("any", source.getType()); - CompletableFuture runningFuture = FutureUtils.toJava( - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.RUNNING), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(TestingJobManagerMessages.JobStatusIs.class))); + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 7662520075515707428L; - cluster.submitJobDetached(jobGraph); + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState("hakuna", valueState); + + try (AutoCancellableJob closableJobGraph = new AutoCancellableJob(cluster, env, deadline)) { + + // register to be notified when the job is running. + CompletableFuture runningFuture = + notifyWhenJobStatusIs(closableJobGraph.getJobId(), JobStatus.RUNNING, deadline); + + cluster.submitJobDetached(closableJobGraph.getJobGraph()); // expect for the job to be running TestingJobManagerMessages.JobStatusIs jobStatus = @@ -486,49 +440,38 @@ public Integer getKey(Tuple2 value) throws Exception { CompletableFuture>> unknownJobFuture = client.getKvState( wrongJobId, // this is the wrong job id - "hankuna", + "hakuna", 0, BasicTypeInfo.INT_TYPE_INFO, valueState); try { - unknownJobFuture.get(); - fail(); // by now the job must have failed. + unknownJobFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + fail(); // by now the request must have failed. } catch (ExecutionException e) { - Assert.assertTrue(e.getCause() instanceof RuntimeException); - Assert.assertTrue(e.getCause().getMessage().contains( + Assert.assertTrue("GOT: " + e.getCause().getMessage(), e.getCause() instanceof RuntimeException); + Assert.assertTrue("GOT: " + e.getCause().getMessage(), e.getCause().getMessage().contains( "FlinkJobNotFoundException: Could not find Flink job (" + wrongJobId + ")")); - } catch (Exception ignored) { - fail("Unexpected type of exception."); + } catch (Exception f) { + fail("Unexpected type of exception: " + f.getMessage()); } CompletableFuture>> unknownQSName = client.getKvState( - jobId, - "wrong-hankuna", // this is the wrong name. + closableJobGraph.getJobId(), + "wrong-hakuna", // this is the wrong name. 0, BasicTypeInfo.INT_TYPE_INFO, valueState); try { - unknownQSName.get(); - fail(); // by now the job must have failed. + unknownQSName.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + fail(); // by now the request must have failed. } catch (ExecutionException e) { - Assert.assertTrue(e.getCause() instanceof RuntimeException); - Assert.assertTrue(e.getCause().getMessage().contains( - "UnknownKvStateLocation: No KvStateLocation found for KvState instance with name 'wrong-hankuna'.")); - } catch (Exception ignored) { - fail("Unexpected type of exception."); - } - - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + Assert.assertTrue("GOT: " + e.getCause().getMessage(), e.getCause() instanceof RuntimeException); + Assert.assertTrue("GOT: " + e.getCause().getMessage(), e.getCause().getMessage().contains( + "UnknownKvStateLocation: No KvStateLocation found for KvState instance with name 'wrong-hakuna'.")); + } catch (Exception f) { + fail("Unexpected type of exception: " + f.getMessage()); } } } @@ -539,50 +482,44 @@ public Integer getKey(Tuple2 value) throws Exception { */ @Test public void testQueryNonStartedJobState() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Value state - ValueStateDescriptor> valueState = new ValueStateDescriptor<>( - "any", - source.getType(), - null); - - QueryableStateStream> queryableState = + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); + + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); + + ValueStateDescriptor> valueState = new ValueStateDescriptor<>( + "any", source.getType(), null); + + QueryableStateStream> queryableState = source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 7480503339992214681L; @Override - public Integer getKey(Tuple2 value) throws Exception { + public Integer getKey(Tuple2 value) { return value.f0; } }).asQueryableState("hakuna", valueState); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - // Now query long expected = numElements; // query once client.getKvState( - jobId, + autoCancellableJob.getJobId(), queryableState.getQueryableStateName(), 0, BasicTypeInfo.INT_TYPE_INFO, @@ -591,16 +528,6 @@ public Integer getKey(Tuple2 value) throws Exception { cluster.submitJobDetached(jobGraph); executeValueQuery(deadline, client, jobId, "hakuna", valueState, expected); - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -615,51 +542,37 @@ public Integer getKey(Tuple2 value) throws Exception { @Test(expected = UnknownKeyOrNamespaceException.class) public void testValueStateDefault() throws Throwable { - // Config final Deadline deadline = TEST_TIMEOUT.fromNow(); - final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies - .fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Value state - ValueStateDescriptor> valueState = - new ValueStateDescriptor<>( - "any", - source.getType(), - Tuple2.of(0, 1337L)); - - // only expose key "1" - QueryableStateStream> - queryableState = - source.keyBy( - new KeySelector, Integer>() { - private static final long serialVersionUID = 4509274556892655887L; - - @Override - public Integer getKey( - Tuple2 value) throws - Exception { - return 1; - } - }).asQueryableState("hakuna", valueState); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); + + ValueStateDescriptor> valueState = new ValueStateDescriptor<>( + "any", source.getType(), Tuple2.of(0, 1337L)); + + // only expose key "1" + QueryableStateStream> queryableState = source.keyBy( + new KeySelector, Integer>() { + private static final long serialVersionUID = 4509274556892655887L; + + @Override + public Integer getKey(Tuple2 value) { + return 1; + } + }).asQueryableState("hakuna", valueState); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); cluster.submitJobDetached(jobGraph); @@ -683,17 +596,6 @@ public Integer getKey( // exception in an ExecutionException. throw e.getCause(); } - } finally { - - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -707,55 +609,41 @@ public Integer getKey( */ @Test public void testValueStateShortcut() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Value state shortcut - QueryableStateStream> queryableState = - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 9168901838808830068L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState("matata"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); - cluster.submitJobDetached(jobGraph); + // Value state shortcut + final QueryableStateStream> queryableState = + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 9168901838808830068L; - final ValueStateDescriptor> stateDesc = - (ValueStateDescriptor>) queryableState.getStateDescriptor(); - executeValueQuery(deadline, client, jobId, "matata", stateDesc, numElements); - } finally { + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState("matata"); - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava( - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); + final ValueStateDescriptor> stateDesc = + (ValueStateDescriptor>) queryableState.getStateDescriptor(); - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); + + cluster.submitJobDetached(jobGraph); + executeValueQuery(deadline, client, jobId, "matata", stateDesc, numElements); } } @@ -768,50 +656,40 @@ public Integer getKey(Tuple2 value) throws Exception { */ @Test public void testFoldingState() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final int numElements = 1024; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Folding state - FoldingStateDescriptor, String> foldingState = - new FoldingStateDescriptor<>( - "any", - "0", - new SumFold(), - StringSerializer.INSTANCE); - - QueryableStateStream queryableState = - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = -842809958106747539L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState("pumba", foldingState); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); + + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); + + FoldingStateDescriptor, String> foldingState = new FoldingStateDescriptor<>( + "any", "0", new SumFold(), StringSerializer.INSTANCE); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = -842809958106747539L; + + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState("pumba", foldingState); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); cluster.submitJobDetached(jobGraph); - // Now query - String expected = Integer.toString(numElements * (numElements + 1) / 2); + final String expected = Integer.toString(numElements * (numElements + 1) / 2); for (int key = 0; key < maxParallelism; key++) { boolean success = false; @@ -840,16 +718,6 @@ public Integer getKey(Tuple2 value) throws Exception { assertTrue("Did not succeed query", success); } - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -861,48 +729,40 @@ public Integer getKey(Tuple2 value) throws Exception { */ @Test public void testReducingState() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - // Reducing state - ReducingStateDescriptor> reducingState = - new ReducingStateDescriptor<>( - "any", - new SumReduce(), - source.getType()); - - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 8470749712274833552L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).asQueryableState("jungle", reducingState); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); + + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + ReducingStateDescriptor> reducingState = new ReducingStateDescriptor<>( + "any", new SumReduce(), source.getType()); + + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 8470749712274833552L; + + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).asQueryableState("jungle", reducingState); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); cluster.submitJobDetached(jobGraph); - // Now query - long expected = numElements * (numElements + 1L) / 2L; + final long expected = numElements * (numElements + 1L) / 2L; for (int key = 0; key < maxParallelism; key++) { boolean success = false; @@ -931,16 +791,6 @@ public Integer getKey(Tuple2 value) throws Exception { assertTrue("Did not succeed query", success); } - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -952,66 +802,60 @@ public Integer getKey(Tuple2 value) throws Exception { */ @Test public void testMapState() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - final MapStateDescriptor> mapStateDescriptor = new MapStateDescriptor<>( - "timon", - BasicTypeInfo.INT_TYPE_INFO, - source.getType()); - mapStateDescriptor.setQueryable("timon-queryable"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 8470749712274833552L; + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).process(new ProcessFunction, Object>() { - private static final long serialVersionUID = -805125545438296619L; + final MapStateDescriptor> mapStateDescriptor = new MapStateDescriptor<>( + "timon", BasicTypeInfo.INT_TYPE_INFO, source.getType()); + mapStateDescriptor.setQueryable("timon-queryable"); - private transient MapState> mapState; + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 8470749712274833552L; - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - mapState = getRuntimeContext().getMapState(mapStateDescriptor); - } + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).process(new ProcessFunction, Object>() { + private static final long serialVersionUID = -805125545438296619L; - @Override - public void processElement(Tuple2 value, Context ctx, Collector out) throws Exception { - Tuple2 v = mapState.get(value.f0); - if (v == null) { - v = new Tuple2<>(value.f0, 0L); - } - mapState.put(value.f0, new Tuple2<>(v.f0, v.f1 + value.f1)); + private transient MapState> mapState; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + mapState = getRuntimeContext().getMapState(mapStateDescriptor); + } + + @Override + public void processElement(Tuple2 value, Context ctx, Collector out) throws Exception { + Tuple2 v = mapState.get(value.f0); + if (v == null) { + v = new Tuple2<>(value.f0, 0L); } - }); + mapState.put(value.f0, new Tuple2<>(v.f0, v.f1 + value.f1)); + } + }); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); cluster.submitJobDetached(jobGraph); - // Now query - long expected = numElements * (numElements + 1L) / 2L; + final long expected = numElements * (numElements + 1L) / 2L; for (int key = 0; key < maxParallelism; key++) { boolean success = false; @@ -1039,16 +883,6 @@ public void processElement(Tuple2 value, Context ctx, Collector cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -1061,62 +895,56 @@ public void processElement(Tuple2 value, Context ctx, Collector> source = env - .addSource(new TestAscendingValueSource(numElements)); - - final ListStateDescriptor listStateDescriptor = new ListStateDescriptor( - "list", - BasicTypeInfo.LONG_TYPE_INFO); - listStateDescriptor.setQueryable("list-queryable"); - - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 8470749712274833552L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).process(new ProcessFunction, Object>() { - private static final long serialVersionUID = -805125545438296619L; + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - private transient ListState listState; + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - listState = getRuntimeContext().getListState(listStateDescriptor); - } + final ListStateDescriptor listStateDescriptor = new ListStateDescriptor( + "list", BasicTypeInfo.LONG_TYPE_INFO); + listStateDescriptor.setQueryable("list-queryable"); - @Override - public void processElement(Tuple2 value, Context ctx, Collector out) throws Exception { - listState.add(value.f1); - } - }); + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 8470749712274833552L; - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).process(new ProcessFunction, Object>() { + private static final long serialVersionUID = -805125545438296619L; - cluster.submitJobDetached(jobGraph); + private transient ListState listState; - // Now query + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + listState = getRuntimeContext().getListState(listStateDescriptor); + } - Map> results = new HashMap<>(); + @Override + public void processElement(Tuple2 value, Context ctx, Collector out) throws Exception { + listState.add(value.f1); + } + }); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); + + cluster.submitJobDetached(jobGraph); + + final Map> results = new HashMap<>(); for (int key = 0; key < maxParallelism; key++) { boolean success = false; @@ -1159,66 +987,48 @@ public void processElement(Tuple2 value, Context ctx, Collector cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @Test public void testAggregatingState() throws Exception { - // Config - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = TEST_TIMEOUT.fromNow(); final long numElements = 1024L; - JobID jobId = null; - try { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setStateBackend(stateBackend); - env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we - // don't explicitly check that all slots are available before - // submitting. - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - - DataStream> source = env - .addSource(new TestAscendingValueSource(numElements)); - - final AggregatingStateDescriptor, String, String> aggrStateDescriptor = - new AggregatingStateDescriptor<>( - "aggregates", - new SumAggr(), - String.class); - aggrStateDescriptor.setQueryable("aggr-queryable"); - - source.keyBy(new KeySelector, Integer>() { - private static final long serialVersionUID = 8470749712274833552L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }).transform( - "TestAggregatingOperator", - BasicTypeInfo.STRING_TYPE_INFO, - new AggregatingTestOperator(aggrStateDescriptor) - ); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStateBackend(stateBackend); + env.setParallelism(maxParallelism); + // Very important, because cluster is shared between tests and we + // don't explicitly check that all slots are available before + // submitting. + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); - // Submit the job graph - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - jobId = jobGraph.getJobID(); + DataStream> source = env.addSource(new TestAscendingValueSource(numElements)); - cluster.submitJobDetached(jobGraph); + final AggregatingStateDescriptor, String, String> aggrStateDescriptor = + new AggregatingStateDescriptor<>("aggregates", new SumAggr(), String.class); + aggrStateDescriptor.setQueryable("aggr-queryable"); - // Now query + source.keyBy(new KeySelector, Integer>() { + private static final long serialVersionUID = 8470749712274833552L; + + @Override + public Integer getKey(Tuple2 value) { + return value.f0; + } + }).transform( + "TestAggregatingOperator", + BasicTypeInfo.STRING_TYPE_INFO, + new AggregatingTestOperator(aggrStateDescriptor) + ); + + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + + final JobID jobId = autoCancellableJob.getJobId(); + final JobGraph jobGraph = autoCancellableJob.getJobGraph(); + + cluster.submitJobDetached(jobGraph); for (int key = 0; key < maxParallelism; key++) { boolean success = false; @@ -1246,16 +1056,6 @@ public Integer getKey(Tuple2 value) throws Exception { assertTrue("Did not succeed query", success); } - } finally { - // Free cluster resources - if (jobId != null) { - CompletableFuture cancellation = FutureUtils.toJava(cluster - .getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class))); - - cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } } } @@ -1316,7 +1116,6 @@ public void cancel() { notifyAll(); } } - } /** @@ -1465,6 +1264,60 @@ public Tuple2 reduce(Tuple2 value1, Tuple2 cancellationFuture; + + AutoCancellableJob(final FlinkMiniCluster cluster, final StreamExecutionEnvironment env, final Deadline deadline) { + Preconditions.checkNotNull(env); + + this.cluster = Preconditions.checkNotNull(cluster); + this.jobGraph = env.getStreamGraph().getJobGraph(); + this.deadline = Preconditions.checkNotNull(deadline); + + this.jobId = jobGraph.getJobID(); + this.cancellationFuture = notifyWhenJobStatusIs(jobId, JobStatus.CANCELED, deadline); + } + + JobGraph getJobGraph() { + return jobGraph; + } + + JobID getJobId() { + return jobId; + } + + @Override + public void close() throws Exception { + // Free cluster resources + if (jobId != null) { + cluster.getLeaderGateway(deadline.timeLeft()) + .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) + .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class)); + + cancellationFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + } + } + } + + private static CompletableFuture notifyWhenJobStatusIs( + final JobID jobId, final JobStatus status, final Deadline deadline) { + + return FutureUtils.toJava( + cluster.getLeaderGateway(deadline.timeLeft()) + .ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, status), deadline.timeLeft()) + .mapTo(ClassTag$.MODULE$.apply(TestingJobManagerMessages.JobStatusIs.class))); + } + private static CompletableFuture getKvState( final Deadline deadline, final QueryableStateClient client, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java index b9ce7c2a2c9aa..8767b5214e98d 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java @@ -31,6 +31,8 @@ import org.junit.Assert; import org.junit.rules.TemporaryFolder; +import java.io.IOException; + import static org.junit.Assert.fail; /** @@ -79,19 +81,13 @@ public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { } @AfterClass - public static void tearDown() { - if (cluster != null) { - cluster.stop(); - cluster.awaitTermination(); - } + public static void tearDown() throws IOException { + client.shutdownAndWait(); - try { - zkServer.stop(); - zkServer.close(); - client.shutdownAndWait(); - } catch (Throwable e) { - e.printStackTrace(); - fail(e.getMessage()); - } + cluster.stop(); + cluster.awaitTermination(); + + zkServer.stop(); + zkServer.close(); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java index 18b167fb12b9c..cae02e2ba69c8 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java @@ -22,14 +22,12 @@ import org.apache.flink.runtime.state.AbstractStateBackend; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Rule; import org.junit.rules.TemporaryFolder; /** * Several integration tests for queryable state using the {@link RocksDBStateBackend}. */ -@Ignore public class HAQueryableStateRocksDBBackendITCase extends HAAbstractQueryableStateTestBase { @Rule diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java index a5e24b2ec9595..2686a2981f319 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java @@ -67,12 +67,9 @@ public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { @AfterClass public static void tearDown() { - try { - cluster.stop(); - client.shutdownAndWait(); - } catch (Throwable e) { - e.printStackTrace(); - fail(e.getMessage()); - } + client.shutdownAndWait(); + + cluster.stop(); + cluster.awaitTermination(); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java index 39fbe9ebe631b..7778a9446bd9d 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java @@ -22,14 +22,12 @@ import org.apache.flink.runtime.state.AbstractStateBackend; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Rule; import org.junit.rules.TemporaryFolder; /** * Several integration tests for queryable state using the {@link RocksDBStateBackend}. */ -@Ignore public class NonHAQueryableStateRocksDBBackendITCase extends NonHAAbstractQueryableStateTestBase { @Rule From 2435fe9091edfac212e1bcf7d1d0c534da32a824 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 7 Dec 2017 13:59:20 +0100 Subject: [PATCH 147/367] [hotfix] [docs] Upgrade Scala version to 2.11 --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 5a92bb99f8ac5..3b71a09d2305a 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -34,7 +34,7 @@ version_title: "1.4-SNAPSHOT" version_javadocs: "1.4" # This suffix is appended to the Scala-dependent Maven artifact names -scala_version_suffix: "_2.10" +scala_version_suffix: "_2.11" # Some commonly linked pages (this was more important to have as a variable # during incubator; by now it should also be fine to hardcode these.) From f00e7297a5b1e2dff8657f1f74f67dcd187f1a17 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 12 Dec 2017 11:18:38 +0100 Subject: [PATCH 148/367] Update 1.4.0 doc to "stable" --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 3b71a09d2305a..ecec0e6bd9bac 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -47,7 +47,7 @@ download_url: "http://flink.apache.org/downloads.html" baseurl: //ci.apache.org/projects/flink/flink-docs-release-1.4 # Flag whether this is a stable version or not. Used for the quickstart page. -is_stable: false +is_stable: true # Flag to indicate whether an outdated warning should be shown. show_outdated_warning: false From 0b024aad17e1d8a0ea69a9b5651eb750385ebdea Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Tue, 12 Dec 2017 18:10:15 +0800 Subject: [PATCH 149/367] [hotfix][javadoc] Fix typo in StreamElement javadoc This closes #5152. --- .../flink/streaming/runtime/streamstatus/StreamStatus.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StreamStatus.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StreamStatus.java index 8e58340947c60..3dee61c2879e6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StreamStatus.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StreamStatus.java @@ -63,7 +63,7 @@ * active. However, for watermarks, since there may be watermark generators that might produce watermarks * anywhere in the middle of topologies regardless of whether there are input data at the operator, the current * status of the task must be checked before forwarding watermarks emitted from - * an operator. It the status is actually idle, the watermark must be blocked. + * an operator. If the status is actually idle, the watermark must be blocked. * *
  5. For downstream tasks with multiple input streams, the watermarks of input streams that are temporarily idle, * or has resumed to be active but its watermark is behind the overall min watermark of the operator, should not From b8f1744b54c10968a5a8cabf9d461346ccf1b586 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 24 Nov 2017 11:31:48 +0100 Subject: [PATCH 150/367] [FLINK-8145][tests] fix various IOManagerAsync instances not being shut down This closes #5064. --- .../AsynchronousBufferFileWriterTest.java | 7 ++ ...BufferFileWriterFileSegmentReaderTest.java | 6 ++ .../iomanager/BufferFileWriterReaderTest.java | 6 ++ .../operators/hash/HashTableITCase.java | 11 +-- .../hash/HashTablePerformanceComparison.java | 6 +- .../runtime/operators/hash/HashTableTest.java | 92 ++++++++++--------- ...tractSortMergeOuterJoinIteratorITCase.java | 3 - .../sort/FixedLengthRecordSorterTest.java | 5 + .../test/manual/MassiveStringSorting.java | 24 ++++- .../manual/MassiveStringValueSorting.java | 24 ++++- 10 files changed, 117 insertions(+), 67 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java index 0397de5729449..40f3e32feabdb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java @@ -20,6 +20,8 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.util.TestNotificationListener; + +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -45,6 +47,11 @@ public class AsynchronousBufferFileWriterTest { private AsynchronousBufferFileWriter writer; + @AfterClass + public static void shutdown() { + ioManager.shutdown(); + } + @Before public void setUp() throws IOException { writer = new AsynchronousBufferFileWriter(ioManager.createChannel(), new RequestQueue()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java index 7fee0fd903844..0d554c7d4f221 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.testutils.DiscardingRecycler; import org.apache.flink.runtime.util.event.NotificationListener; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; @@ -55,6 +56,11 @@ public class BufferFileWriterFileSegmentReaderTest { private LinkedBlockingQueue returnedFileSegments = new LinkedBlockingQueue<>(); + @AfterClass + public static void shutdown() { + ioManager.shutdown(); + } + @Before public void setUpWriterAndReader() { final FileIOChannel.ID channel = ioManager.createChannel(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java index 2da0f7e83677d..31702f062e9c2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.testutils.DiscardingRecycler; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; @@ -52,6 +53,11 @@ public class BufferFileWriterReaderTest { private LinkedBlockingQueue returnedBuffers = new LinkedBlockingQueue<>(); + @AfterClass + public static void shutdown() { + ioManager.shutdown(); + } + @Before public void setUpWriterAndReader() { final FileIOChannel.ID channel = ioManager.createChannel(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java index f3eac19c3ef74..a94227c23d766 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java @@ -806,11 +806,8 @@ public void testInMemoryMutableHashTableIntPair() throws IOException return; } - // create the I/O access for spilling - final IOManager ioManager = new IOManagerAsync(); - // ---------------------------------------------------------------------------------------- - + final MutableHashTable join = new MutableHashTable( this.pairBuildSideAccesssor, this.pairProbeSideAccesssor, this.pairBuildSideComparator, this.pairProbeSideComparator, this.pairComparator, @@ -907,9 +904,6 @@ public void testSpillingHashJoinOneRecursionValidityIntPair() throws IOException return; } - // create the I/O access for spilling - IOManager ioManager = new IOManagerAsync(); - // create the map for validating the results HashMap map = new HashMap(NUM_KEYS); @@ -1022,9 +1016,6 @@ public void testSpillingHashJoinWithMassiveCollisionsIntPair() throws IOExceptio return; } - // create the I/O access for spilling - IOManager ioManager = new IOManagerAsync(); - // create the map for validating the results HashMap map = new HashMap(NUM_KEYS); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java index 10bac1f043c8c..f426a9428dfc4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer; import org.apache.flink.util.MutableObjectIterator; +import org.junit.AfterClass; import org.junit.Test; import static org.junit.Assert.*; @@ -54,8 +55,6 @@ public class HashTablePerformanceComparison { private final TypePairComparator pairComparator = new IntPairPairComparator(); - private IOManager ioManager = new IOManagerAsync(); - @Test public void testCompactingHashMapPerformance() { @@ -132,6 +131,7 @@ public void testCompactingHashMapPerformance() { @Test public void testMutableHashMapPerformance() { + final IOManager ioManager = new IOManagerAsync(); try { final int NUM_MEM_PAGES = SIZE * NUM_PAIRS / PAGE_SIZE; @@ -207,6 +207,8 @@ public void testMutableHashMapPerformance() { catch (Exception e) { e.printStackTrace(); fail("Error: " + e.getMessage()); + } finally { + ioManager.shutdown(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java index 7c385fc661a1c..bcf620c605407 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java @@ -205,54 +205,58 @@ public void testSpillingFreesOnlyOverflowSegments() { public void testSpillingWhenBuildingTableWithoutOverflow() throws Exception { final IOManager ioMan = new IOManagerAsync(); - final TypeSerializer serializer = BytePrimitiveArraySerializer.INSTANCE; - final TypeComparator buildComparator = new BytePrimitiveArrayComparator(true); - final TypeComparator probeComparator = new BytePrimitiveArrayComparator(true); + try { + final TypeSerializer serializer = BytePrimitiveArraySerializer.INSTANCE; + final TypeComparator buildComparator = new BytePrimitiveArrayComparator(true); + final TypeComparator probeComparator = new BytePrimitiveArrayComparator(true); - @SuppressWarnings("unchecked") - final TypePairComparator pairComparator = new GenericPairComparator<>( - new BytePrimitiveArrayComparator(true), new BytePrimitiveArrayComparator(true)); - - final int pageSize = 128; - final int numSegments = 33; - - List memory = getMemory(numSegments, pageSize); - - MutableHashTable table = new MutableHashTable( - serializer, - serializer, - buildComparator, - probeComparator, - pairComparator, - memory, - ioMan, - 1, - false); - - int numElements = 9; - - table.open( - new CombiningIterator( - new ByteArrayIterator(numElements, 128,(byte) 0), - new ByteArrayIterator(numElements, 128,(byte) 1)), - new CombiningIterator( - new ByteArrayIterator(1, 128,(byte) 0), - new ByteArrayIterator(1, 128,(byte) 1))); - - while(table.nextRecord()) { - MutableObjectIterator iterator = table.getBuildSideIterator(); - - int counter = 0; - - while(iterator.next() != null) { - counter++; + @SuppressWarnings("unchecked") final TypePairComparator pairComparator = + new GenericPairComparator<>( + new BytePrimitiveArrayComparator(true), new BytePrimitiveArrayComparator(true)); + + final int pageSize = 128; + final int numSegments = 33; + + List memory = getMemory(numSegments, pageSize); + + MutableHashTable table = new MutableHashTable( + serializer, + serializer, + buildComparator, + probeComparator, + pairComparator, + memory, + ioMan, + 1, + false); + + int numElements = 9; + + table.open( + new CombiningIterator( + new ByteArrayIterator(numElements, 128, (byte) 0), + new ByteArrayIterator(numElements, 128, (byte) 1)), + new CombiningIterator( + new ByteArrayIterator(1, 128, (byte) 0), + new ByteArrayIterator(1, 128, (byte) 1))); + + while (table.nextRecord()) { + MutableObjectIterator iterator = table.getBuildSideIterator(); + + int counter = 0; + + while (iterator.next() != null) { + counter++; + } + + // check that we retrieve all our elements + Assert.assertEquals(numElements, counter); } - // check that we retrieve all our elements - Assert.assertEquals(numElements, counter); + table.close(); + } finally { + ioMan.shutdown(); } - - table.close(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java index 28bded2559c10..94c0fd45d077e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java @@ -300,9 +300,6 @@ protected void testOuterJoinWithHighNumberOfCommonKeys(OuterJoinType outerJoinTy TypePairComparator, Tuple2> pairComparator = new GenericPairComparator<>(comparator1, comparator2); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); - this.ioManager = new IOManagerAsync(); - final int DUPLICATE_KEY = 13; try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java index 288e86dc5624c..bba713eb71cac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java @@ -77,6 +77,11 @@ public void afterTest() { if (!this.memoryManager.verifyEmpty()) { Assert.fail("Memory Leak: Some memory has not been returned to the memory manager."); } + + if (this.ioManager != null) { + ioManager.shutdown(); + ioManager = null; + } if (this.memoryManager != null) { this.memoryManager.shutdown(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java index c69e6fd606b98..46e44854b9282 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java @@ -82,10 +82,12 @@ public void testStringSorting() { UnilateralSortMerger sorter = null; BufferedReader reader = null; BufferedReader verifyReader = null; + MemoryManager mm = null; + IOManager ioMan = null; try { - MemoryManager mm = new MemoryManager(1024 * 1024, 1); - IOManager ioMan = new IOManagerAsync(); + mm = new MemoryManager(1024 * 1024, 1); + ioMan = new IOManagerAsync(); TypeSerializer serializer = StringSerializer.INSTANCE; TypeComparator comparator = new StringComparator(true); @@ -122,6 +124,12 @@ public void testStringSorting() { if (sorter != null) { sorter.close(); } + if (mm != null) { + mm.shutdown(); + } + if (ioMan != null) { + ioMan.shutdown(); + } } } catch (Exception e) { @@ -173,10 +181,12 @@ public void testStringTuplesSorting() { UnilateralSortMerger> sorter = null; BufferedReader reader = null; BufferedReader verifyReader = null; + MemoryManager mm = null; + IOManager ioMan = null; try { - MemoryManager mm = new MemoryManager(1024 * 1024, 1); - IOManager ioMan = new IOManagerAsync(); + mm = new MemoryManager(1024 * 1024, 1); + ioMan = new IOManagerAsync(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeInfoParser.>parse("Tuple2"); @@ -243,6 +253,12 @@ public void testStringTuplesSorting() { if (sorter != null) { sorter.close(); } + if (mm != null) { + mm.shutdown(); + } + if (ioMan != null) { + ioMan.shutdown(); + } } } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java index 453aa148890fc..1f72e4a8495a8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java @@ -83,10 +83,12 @@ public void testStringValueSorting() { UnilateralSortMerger sorter = null; BufferedReader reader = null; BufferedReader verifyReader = null; + MemoryManager mm = null; + IOManager ioMan = null; try { - MemoryManager mm = new MemoryManager(1024 * 1024, 1); - IOManager ioMan = new IOManagerAsync(); + mm = new MemoryManager(1024 * 1024, 1); + ioMan = new IOManagerAsync(); TypeSerializer serializer = new CopyableValueSerializer(StringValue.class); TypeComparator comparator = new CopyableValueComparator(true, StringValue.class); @@ -124,6 +126,12 @@ public void testStringValueSorting() { if (sorter != null) { sorter.close(); } + if (mm != null) { + mm.shutdown(); + } + if (ioMan != null) { + ioMan.shutdown(); + } } } catch (Exception e) { @@ -177,10 +185,12 @@ public void testStringValueTuplesSorting() { UnilateralSortMerger> sorter = null; BufferedReader reader = null; BufferedReader verifyReader = null; + MemoryManager mm = null; + IOManager ioMan = null; try { - MemoryManager mm = new MemoryManager(1024 * 1024, 1); - IOManager ioMan = new IOManagerAsync(); + mm = new MemoryManager(1024 * 1024, 1); + ioMan = new IOManagerAsync(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeInfoParser.>parse("Tuple2"); @@ -247,6 +257,12 @@ public void testStringValueTuplesSorting() { if (sorter != null) { sorter.close(); } + if (mm != null) { + mm.shutdown(); + } + if (ioMan != null) { + ioMan.shutdown(); + } } } catch (Exception e) { From 897e6a7a8949272f6e8a6159fd2beeee9c7847db Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 11 Dec 2017 11:44:57 +0100 Subject: [PATCH 151/367] [FLINK-8235][build] Spotbugs exclusion file path now absolute This closes #5146. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 63f2015619239..abbd5dbd4b4e7 100644 --- a/pom.xml +++ b/pom.xml @@ -615,7 +615,7 @@ under the License. Low default ${project.build.directory}/spotbugs - tools/maven/spotbugs-exclude.xml + ${rootDir}/tools/maven/spotbugs-exclude.xml true From d94bd7cbe5bb67676bcac96ceea195517582cf36 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 13 Dec 2017 09:59:37 +0100 Subject: [PATCH 152/367] [hotfix] [docs] Correct version in _config.yml --- docs/_config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index ecec0e6bd9bac..866f346271451 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -27,10 +27,10 @@ # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) -version: "1.4-SNAPSHOT" +version: "1.4.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version -version_title: "1.4-SNAPSHOT" +version_title: "1.4" version_javadocs: "1.4" # This suffix is appended to the Scala-dependent Maven artifact names From 6981412f736e58118490e24a2c1df09f415d921e Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Wed, 6 Dec 2017 11:37:54 -0800 Subject: [PATCH 153/367] [hotfix][javadocs] Clarify replacement for deprecated FoldingState This closes #5129. --- docs/dev/stream/state/state.md | 2 +- .../org/apache/flink/api/common/functions/RuntimeContext.java | 2 +- .../java/org/apache/flink/api/common/state/FoldingState.java | 2 +- .../java/org/apache/flink/api/common/state/KeyedStateStore.java | 2 +- .../java/org/apache/flink/api/common/state/StateBinder.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index c9a68ab976ac1..e9770bd757b0b 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -119,7 +119,7 @@ views for mappings, keys and values can be retrieved using `entries()`, `keys()` All types of state also have a method `clear()` that clears the state for the currently active key, i.e. the key of the input element. -Attention `FoldingState` and `FoldingStateDescriptor` have been deprecated in Flink 1.4 and will be completely removed in the future. A more general alternative will be provided. +Attention `FoldingState` and `FoldingStateDescriptor` have been deprecated in Flink 1.4 and will be completely removed in the future. Please use `AggregatingState` and `AggregatingStateDescriptor` instead. It is important to keep in mind that these state objects are only used for interfacing with state. The state is not necessarily stored inside but might reside on disk or somewhere else. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 7a89e10c32ac4..3429143bfc751 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -441,7 +441,7 @@ public interface RuntimeContext { * @throws UnsupportedOperationException Thrown, if no partitioned state is available for the * function (function is not part of a KeyedStream). * - * @deprecated will be removed in a future version + * @deprecated will be removed in a future version in favor of {@link AggregatingState} */ @PublicEvolving @Deprecated diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java index b47a1a1b60f72..df9a0c6aa4bc9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java @@ -36,7 +36,7 @@ * @param Type of the values folded into the state * @param Type of the value in the state * - * @deprecated will be removed in a future version + * @deprecated will be removed in a future version in favor of {@link AggregatingState} */ @PublicEvolving @Deprecated diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java b/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java index 7f45214deca81..a1038a84fc98a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java @@ -237,7 +237,7 @@ public interface KeyedStateStore { * @throws UnsupportedOperationException Thrown, if no partitioned state is available for the * function (function is not part of a KeyedStream). * - * @deprecated will be removed in a future version + * @deprecated will be removed in a future version in favor of {@link AggregatingState} */ @PublicEvolving @Deprecated diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java index a37392394a4f2..a1f7d8d2939f5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java @@ -69,7 +69,7 @@ AggregatingState createAggregatingState( * @param Type of the values folded into the state * @param Type of the value in the state * - * @deprecated will be removed in a future version + * @deprecated will be removed in a future version in favor of {@link AggregatingState} */ @Deprecated FoldingState createFoldingState(FoldingStateDescriptor stateDesc) throws Exception; From a4b401c8eb5463fd7e4ef56d92b46685cc4c43ed Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 14 Dec 2017 12:49:19 +0100 Subject: [PATCH 154/367] [FLINK-8261] [quickstarts] Fix typos in exclusion patterns. --- .../src/main/resources/archetype-resources/pom.xml | 2 +- .../src/main/resources/archetype-resources/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index e50e7b5b62ce4..50bc561322097 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -173,7 +173,7 @@ under the License. org.apache.flink:force-shading - com.google.code.findbgs:jsr305 + com.google.code.findbugs:jsr305 org.slf4j:slf4j-api diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index da0a8e4455db7..0946baf5b7b0e 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -184,7 +184,7 @@ under the License. org.apache.flink:force-shading - com.google.code.findbgs:jsr305 + com.google.code.findbugs:jsr305 org.slf4j:slf4j-api From dcd3e3ab4d5053cbff6491396f52f5b635d72c7a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 14 Dec 2017 12:51:59 +0100 Subject: [PATCH 155/367] [hotfix] [quickstarts] Exclude not only slf4j api, but also bridges --- .../src/main/resources/archetype-resources/pom.xml | 2 +- .../src/main/resources/archetype-resources/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 50bc561322097..73bd3bb8ab21f 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -174,7 +174,7 @@ under the License. org.apache.flink:force-shading com.google.code.findbugs:jsr305 - org.slf4j:slf4j-api + org.slf4j:* diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 0946baf5b7b0e..81367eb16286a 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -185,7 +185,7 @@ under the License. org.apache.flink:force-shading com.google.code.findbugs:jsr305 - org.slf4j:slf4j-api + org.slf4j:* From d84e215136c170eada3a0af6d09fcce23203319e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 14 Dec 2017 13:29:43 +0100 Subject: [PATCH 156/367] [hotfix] [quickstarts] Fix indentation in Java Quickstart pom.xml --- .../main/resources/archetype-resources/pom.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 73bd3bb8ab21f..71bc0a1006640 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -156,8 +156,8 @@ under the License. + except flink and its transitive dependencies. The resulting fat-jar can be executed + on a cluster. Change the value of Program-Class if your program entry point changes. --> org.apache.maven.plugins maven-shade-plugin @@ -190,14 +190,14 @@ under the License. + This will add a Main-Class entry to the manifest file --> + + + ${package}.StreamingJob + + + --> From 18efbea67169fc2d70de4040bf8c1c4650ce2ab1 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 14 Dec 2017 13:30:17 +0100 Subject: [PATCH 157/367] [FLINK-8263] [quickstarts] Correctly set 'flink-core' to provided in Scala Quickstart pom.xml --- .../src/main/resources/archetype-resources/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 81367eb16286a..b76f9b910c146 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -125,6 +125,7 @@ under the License. org.apache.flink flink-core ${flink.version} + provided org.apache.flink From 57de25d8950272fc6c6ac9ed400c79b0680f40e4 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 14 Dec 2017 13:50:39 +0100 Subject: [PATCH 158/367] [FLINK-8264] [core] Add 'scala.' to the 'parent-first' classloading patterns. This closes #5167 --- .../main/java/org/apache/flink/configuration/CoreOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index 928e8102a2ac4..cf10012e3d8f1 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -36,7 +36,7 @@ public class CoreOptions { public static final ConfigOption ALWAYS_PARENT_FIRST_LOADER = ConfigOptions .key("classloader.parent-first-patterns") - .defaultValue("java.;org.apache.flink.;javax.annotation;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); + .defaultValue("java.;scala.;org.apache.flink.;javax.annotation;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); // ------------------------------------------------------------------------ From 0e27e20886df635fcfb4845d1979dcbeccc5f279 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Wed, 13 Dec 2017 23:59:07 +0800 Subject: [PATCH 159/367] [hotfix] [javadoc] Fix typo in StreamExecutionEnvironment javadoc This closes #5164. --- .../streaming/api/environment/StreamExecutionEnvironment.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index cc45ddc580ffb..355d2776a0baf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -439,7 +439,7 @@ public CheckpointingMode getCheckpointingMode() { * from operations on {@link org.apache.flink.streaming.api.datastream.KeyedStream}) is maintained * (heap, managed memory, externally), and where state snapshots/checkpoints are stored, both for * the key/value state, and for checkpointed functions (implementing the interface - * {@link org.apache.flink.streaming.api.checkpoint.Checkpointed}). + * {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction}). * *

    The {@link org.apache.flink.runtime.state.memory.MemoryStateBackend} for example * maintains the state in heap memory, as objects. It is lightweight without extra dependencies, From dbbaa9a4a761a22e402f08745775ce357dddac06 Mon Sep 17 00:00:00 2001 From: eskabetxe Date: Wed, 13 Dec 2017 12:43:27 +0100 Subject: [PATCH 160/367] [FLINK-8249] [kinesis] Fix setting region on KinesisProducerConfiguration This closes #5160. --- .../connectors/kinesis/util/KinesisConfigUtil.java | 1 + .../connectors/kinesis/util/KinesisConfigUtilTest.java | 10 ++++++++++ 2 files changed, 11 insertions(+) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java index cadde8da3ae29..6c91206044883 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java @@ -191,6 +191,7 @@ public static KinesisProducerConfiguration getValidatedProducerConfiguration(Pro } KinesisProducerConfiguration kpc = KinesisProducerConfiguration.fromProperties(config); + kpc.setRegion(config.getProperty(AWSConfigConstants.AWS_REGION)); kpc.setCredentialsProvider(AWSUtil.getCredentialsProvider(config)); diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java index b52dce2cb7bf6..053b0c3bfa4e8 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java @@ -131,6 +131,16 @@ public void testReplaceDeprecatedKeys() { assertEquals("2", replacedConfig.getProperty(KinesisConfigUtil.COLLECTION_MAX_COUNT)); } + @Test + public void testCorrectlySetRegionInProducerConfiguration() { + String region = "us-east-1"; + Properties testConfig = new Properties(); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, region); + KinesisProducerConfiguration kpc = KinesisConfigUtil.getValidatedProducerConfiguration(testConfig); + + assertEquals("incorrect region", region, kpc.getRegion()); + } + // ---------------------------------------------------------------------- // validateAwsConfiguration() tests // ---------------------------------------------------------------------- From f638deddb78bc236606cacf19b2f9f973d447b25 Mon Sep 17 00:00:00 2001 From: Cristian Date: Thu, 7 Dec 2017 11:00:00 -0800 Subject: [PATCH 161/367] [hotfix] [doc] Fix typo in TaskManager and EnvironmentInformation doc This closes #5135. --- .../apache/flink/runtime/util/EnvironmentInformation.java | 2 +- .../org/apache/flink/runtime/taskmanager/TaskManager.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java index a2a64cfd5b906..e7002567db544 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java @@ -259,7 +259,7 @@ public static long getOpenFileHandlesLimit() { } /** - * Logs a information about the environment, like code revision, current user, java version, + * Logs information about the environment, like code revision, current user, Java version, * and JVM parameters. * * @param log The logger to log the information to. diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index f948df4c08ea6..9979618cec7af 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -585,8 +585,8 @@ class TaskManager( config.getMaxRegistrationPause().toMilliseconds, TimeUnit.MILLISECONDS)) - // schedule (with our timeout s delay) a check triggers a new registration - // attempt, if we are not registered by then + // schedule a check to trigger a new registration attempt if not registered + // by the timeout scheduledTaskManagerRegistration = Option(context.system.scheduler.scheduleOnce( timeout, self, @@ -1898,7 +1898,7 @@ object TaskManager { } /** - * Starts and runs the TaskManager. with all its components trying to bind to + * Starts and runs the TaskManager with all its components trying to bind to * a port in the specified range. * * @param taskManagerHostname The hostname/address of the interface where the actor system From 7e497f744a67c8011a8e1f353eddc11114f1d514 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 19 Dec 2017 18:14:19 +0100 Subject: [PATCH 162/367] [FLINK-8295] [cassandra] [build] Properly shade netty for the datastax driver com.datastax.driver.core.NettyUtil expects netty to be present either at its original package or relocated to com.datastax.shaded.netty. By relocating it to this package we make sure the driver follows its designated path. This closes #5183. --- flink-connectors/flink-connector-cassandra/pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml index 3c1d3e158a38c..3d2b3c06bce19 100644 --- a/flink-connectors/flink-connector-cassandra/pom.xml +++ b/flink-connectors/flink-connector-cassandra/pom.xml @@ -85,13 +85,13 @@ under the License. io.netty - org.apache.flink.cassandra.shaded.io.netty + com.datastax.shaded.netty From 47c067a5924e56abeb5a3d0769b6356f19a66b1e Mon Sep 17 00:00:00 2001 From: Xpray Date: Mon, 25 Dec 2017 10:51:02 +0800 Subject: [PATCH 163/367] [FLINK-8301][table] Support Unicode in codegen for TableAPI && SQL This closes #5203 --- .../flink/table/codegen/CodeGenerator.scala | 4 +- .../table/codegen/ExpressionReducer.scala | 7 +++ .../flink/table/expressions/literals.scala | 8 +++- .../utils/userDefinedScalarFunctions.scala | 13 ++++++ .../table/runtime/batch/sql/CalcITCase.scala | 37 ++++++++++++++++ .../runtime/batch/table/CalcITCase.scala | 31 +++++++++++-- .../table/runtime/stream/sql/SqlITCase.scala | 43 +++++++++++++++++-- .../runtime/stream/table/CalcITCase.scala | 30 ++++++++++++- 8 files changed, 163 insertions(+), 10 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index 61ae0ec636c4f..609a4c5b56301 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -691,7 +691,9 @@ abstract class CodeGenerator( generateNonNullLiteral(resultType, decimalField) case VARCHAR | CHAR => - val escapedValue = StringEscapeUtils.ESCAPE_JAVA.translate(value.toString) + val escapedValue = StringEscapeUtils.escapeJava( + StringEscapeUtils.unescapeJava(value.toString) + ) generateNonNullLiteral(resultType, "\"" + escapedValue + "\"") case SYMBOL => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala index 9696ced32c4b4..a9dbf19e42592 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala @@ -23,6 +23,7 @@ import java.util import org.apache.calcite.plan.RelOptPlanner import org.apache.calcite.rex.{RexBuilder, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.commons.lang3.StringEscapeUtils import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.java.typeutils.RowTypeInfo @@ -119,6 +120,12 @@ class ExpressionReducer(config: TableConfig) SqlTypeName.MAP | SqlTypeName.MULTISET => reducedValues.add(unreduced) + // after expression reduce, the literal string has to be escaped + case SqlTypeName.VARCHAR | SqlTypeName.CHAR => + val escapeVarchar = StringEscapeUtils + .escapeJava(reduced.getField(reducedIdx).asInstanceOf[String]) + reducedValues.add(rexBuilder.makeLiteral(escapeVarchar, unreduced.getType, true)) + reducedIdx += 1 case _ => val reducedValue = reduced.getField(reducedIdx) // RexBuilder handle double literal incorrectly, convert it into BigDecimal manually diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala index e6905ef998956..863dfc1aef230 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala @@ -27,10 +27,11 @@ import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo} - import java.sql.{Date, Time, Timestamp} import java.util.{Calendar, TimeZone} +import org.apache.commons.lang3.StringEscapeUtils + object Literal { private[flink] val UTC = TimeZone.getTimeZone("UTC") @@ -103,6 +104,11 @@ case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpre SqlParserPos.ZERO) relBuilder.getRexBuilder.makeIntervalLiteral(interval, intervalQualifier) + case BasicTypeInfo.STRING_TYPE_INFO => + relBuilder.getRexBuilder.makeLiteral( + StringEscapeUtils.escapeJava(value.asInstanceOf[String]) + ) + case _ => relBuilder.literal(value) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala index 3f6ebbd5f3749..32e5d71662e66 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.expressions.utils import java.sql.{Date, Time, Timestamp} +import org.apache.commons.lang3.StringUtils import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.Types import org.apache.flink.table.functions.{FunctionContext, ScalarFunction} @@ -288,3 +289,15 @@ object Func19 extends ScalarFunction { Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)) } + +class SplitUDF(deterministic: Boolean) extends ScalarFunction { + def eval(x: String, sep: String, index: Int): String = { + val splits = StringUtils.splitByWholeSeparator(x, sep) + if (splits.length > index) { + splits(index) + } else { + null + } + } + override def isDeterministic: Boolean = deterministic +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala index 71df4e6ace3a6..416998ccafa6e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala @@ -25,6 +25,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{TableEnvironment, ValidationException} +import org.apache.flink.table.expressions.utils.SplitUDF import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.runtime.batch.table.OldHashCode import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode @@ -325,6 +326,42 @@ class CalcITCase( val results = result.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } + + @Test + def testUdfWithUnicodeParameter(): Unit = { + val data = List( + ("a\u0001b", "c\"d", "e\\\"\u0004f"), + ("x\u0001y", "y\"z", "z\\\"\u0004z") + ) + + val env = ExecutionEnvironment.getExecutionEnvironment + + val tEnv = TableEnvironment.getTableEnvironment(env) + + val splitUDF0 = new SplitUDF(deterministic = true) + val splitUDF1 = new SplitUDF(deterministic = false) + + tEnv.registerFunction("splitUDF0", splitUDF0) + tEnv.registerFunction("splitUDF1", splitUDF1) + + // user have to specify '\' with '\\' in SQL + val sqlQuery = "SELECT " + + "splitUDF0(a, '\u0001', 0) as a0, " + + "splitUDF1(a, '\u0001', 0) as a1, " + + "splitUDF0(b, '\"', 1) as b0, " + + "splitUDF1(b, '\"', 1) as b1, " + + "splitUDF0(c, '\\\\\"\u0004', 0) as c0, " + + "splitUDF1(c, '\\\\\"\u0004', 0) as c1 from T1" + + val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val results = tEnv.sql(sqlQuery).toDataSet[Row].collect() + + val expected = List("a,a,d,d,e,e", "x,x,z,z,z,z").mkString("\n") + TestBaseUtils.compareResultAsText(results.asJava, expected) + } } object MyHashCode extends ScalarFunction { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala index 22373d2ac001d..1a274d3b3cbb5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala @@ -27,11 +27,10 @@ import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.api.Types._ import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.Literal -import org.apache.flink.table.expressions.utils.{Func13, RichFunc1, RichFunc2, RichFunc3} +import org.apache.flink.table.expressions.utils._ import org.apache.flink.table.functions.ScalarFunction -import org.apache.flink.table.runtime.utils.{TableProgramsCollectionTestBase, TableProgramsTestBase} +import org.apache.flink.table.runtime.utils.{TableProgramsCollectionTestBase, TableProgramsTestBase, UserDefinedFunctionTestUtils} import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.table.runtime.utils.UserDefinedFunctionTestUtils import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.test.util.TestBaseUtils.compareResultAsText import org.apache.flink.types.Row @@ -517,6 +516,32 @@ class CalcITCase( "default-nosharp,Sunny-nosharp,kevin2-nosharp" TestBaseUtils.compareResultAsText(results.asJava, expected) } + + @Test + def testUDFWithUnicodeParameter(): Unit = { + val data = List( + ("a\u0001b", "c\"d", "e\\\"\u0004f"), + ("x\u0001y", "y\"z", "z\\\"\u0004z") + ) + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + val splitUDF0 = new SplitUDF(deterministic = true) + val splitUDF1 = new SplitUDF(deterministic = false) + val ds = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c) + .select(splitUDF0('a, "\u0001", 0) as 'a0, + splitUDF1('a, "\u0001", 0) as 'a1, + splitUDF0('b, "\"", 1) as 'b0, + splitUDF1('b, "\"", 1) as 'b1, + splitUDF0('c, "\\\"\u0004", 0) as 'c0, + splitUDF1('c, "\\\"\u0004", 0) as 'c1 + ) + val results = ds.collect() + val expected = List( + "a,a,d,d,e,e", "x,x,z,z,z,z" + ).mkString("\n") + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + } object CalcITCase { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 3acdd58ee5f39..18b45a36ea26b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -18,8 +18,6 @@ package org.apache.flink.table.runtime.stream.sql -import java.util - import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala._ @@ -27,12 +25,11 @@ import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.{TableEnvironment, Types} import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.utils.SplitUDF import org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeSourceFunction import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase} import org.apache.flink.types.Row import org.apache.flink.table.utils.MemoryTableSinkUtil - -import scala.collection.JavaConverters._ import org.junit.Assert._ import org.junit._ @@ -481,4 +478,42 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(expected.sorted, MemoryTableSinkUtil.results.sorted) } + @Test + def testUdfWithUnicodeParameter(): Unit = { + val data = List( + ("a\u0001b", "c\"d", "e\\\"\u0004f"), + ("x\u0001y", "y\"z", "z\\\"\u0004z") + ) + + val env = StreamExecutionEnvironment.getExecutionEnvironment + + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val splitUDF0 = new SplitUDF(deterministic = true) + val splitUDF1 = new SplitUDF(deterministic = false) + + tEnv.registerFunction("splitUDF0", splitUDF0) + tEnv.registerFunction("splitUDF1", splitUDF1) + + // user have to specify '\' with '\\' in SQL + val sqlQuery = "SELECT " + + "splitUDF0(a, '\u0001', 0) as a0, " + + "splitUDF1(a, '\u0001', 0) as a1, " + + "splitUDF0(b, '\"', 1) as b0, " + + "splitUDF1(b, '\"', 1) as b1, " + + "splitUDF0(c, '\\\\\"\u0004', 0) as c0, " + + "splitUDF1(c, '\\\\\"\u0004', 0) as c1 from T1" + + val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = List("a,a,d,d,e,e", "x,x,z,z,z,z") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala index ca6da809d1fa4..94cc32619f23d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala @@ -24,7 +24,7 @@ import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.expressions.Literal -import org.apache.flink.table.expressions.utils.{Func13, RichFunc1, RichFunc2} +import org.apache.flink.table.expressions.utils.{Func13, RichFunc1, RichFunc2, SplitUDF} import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData} import org.apache.flink.table.runtime.utils.UserDefinedFunctionTestUtils import org.apache.flink.types.Row @@ -313,4 +313,32 @@ class CalcITCase extends StreamingMultipleProgramsTestBase { ) assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + @Test + def testUDFWithUnicodeParameter(): Unit = { + val data = List( + ("a\u0001b", "c\"d", "e\\\"\u0004f"), + ("x\u0001y", "y\"z", "z\\\"\u0004z") + ) + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.testResults = mutable.MutableList() + val splitUDF0 = new SplitUDF(deterministic = true) + val splitUDF1 = new SplitUDF(deterministic = false) + val ds = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c) + .select(splitUDF0('a, "\u0001", 0) as 'a0, + splitUDF1('a, "\u0001", 0) as 'a1, + splitUDF0('b, "\"", 1) as 'b0, + splitUDF1('b, "\"", 1) as 'b1, + splitUDF0('c, "\\\"\u0004", 0) as 'c0, + splitUDF1('c, "\\\"\u0004", 0) as 'c1 + ) + val results = ds.toAppendStream[Row] + results.addSink(new StreamITCase.StringSink[Row]) + env.execute() + val expected = mutable.MutableList( + "a,a,d,d,e,e", "x,x,z,z,z,z" + ) + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } } From e42182d40bbf0b0ce6721f91ed28d348ed36847d Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Mon, 18 Dec 2017 21:55:30 +0800 Subject: [PATCH 164/367] [FLINK-8278] [docs] Fix Scala examples of metrics docs (var initialization). --- docs/dev/stream/state/state.md | 2 +- docs/monitoring/metrics.md | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index e9770bd757b0b..9956998ed6aae 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -397,7 +397,7 @@ class BufferingSink(threshold: Int = 0) with CheckpointedRestoring[List[(String, Int)]] { @transient - private var checkpointedState: ListState[(String, Int)] = null + private var checkpointedState: ListState[(String, Int)] = _ private val bufferedElements = ListBuffer[(String, Int)]() diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 7963e1a015468..e4a7a735416da 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -69,7 +69,7 @@ public class MyMapper extends RichMapFunction { {% highlight scala %} class MyMapper extends RichMapFunction[String,String] { - @transient private var counter: Counter + @transient private var counter: Counter = _ override def open(parameters: Configuration): Unit = { counter = getRuntimeContext() @@ -119,7 +119,7 @@ public class MyMapper extends RichMapFunction { {% highlight scala %} class MyMapper extends RichMapFunction[String,String] { - @transient private var counter: Counter + @transient private var counter: Counter = _ override def open(parameters: Configuration): Unit = { counter = getRuntimeContext() @@ -229,7 +229,7 @@ public class MyMapper extends RichMapFunction { {% highlight scala %} class MyMapper extends RichMapFunction[Long,Long] { - @transient private var histogram: Histogram + @transient private var histogram: Histogram = _ override def open(parameters: Configuration): Unit = { histogram = getRuntimeContext() @@ -289,7 +289,7 @@ public class MyMapper extends RichMapFunction { {% highlight scala %} class MyMapper extends RichMapFunction[Long, Long] { - @transient private var histogram: Histogram + @transient private var histogram: Histogram = _ override def open(config: Configuration): Unit = { com.codahale.metrics.Histogram dropwizardHistogram = @@ -342,7 +342,7 @@ public class MyMapper extends RichMapFunction { {% highlight scala %} class MyMapper extends RichMapFunction[Long,Long] { - @transient private var meter: Meter + @transient private var meter: Meter = _ override def open(config: Configuration): Unit = { meter = getRuntimeContext() @@ -401,7 +401,7 @@ public class MyMapper extends RichMapFunction { {% highlight scala %} class MyMapper extends RichMapFunction[Long,Long] { - @transient private var meter: Meter + @transient private var meter: Meter = _ override def open(config: Configuration): Unit = { com.codahale.metrics.Meter dropwizardMeter = new com.codahale.metrics.Meter() From 96051e516bdc114b04c5a3cbb75874c420e27e7b Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Tue, 5 Dec 2017 12:38:50 -0500 Subject: [PATCH 165/367] [FLINK-5506] [gelly] Fix CommunityDetection NullPointerException Double.MIN_VALUE != min(double) This closes #5126 --- .../graph/library/CommunityDetection.java | 2 +- .../graph/library/CommunityDetectionTest.java | 116 ++++++++++++++++++ 2 files changed, 117 insertions(+), 1 deletion(-) create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/CommunityDetectionTest.java diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java index ccf2bb18c7078..26291055b9a79 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java @@ -144,7 +144,7 @@ public void updateVertex(Vertex> vertex, if (receivedLabelsWithScores.size() > 0) { // find the label with the highest score from the ones received - double maxScore = Double.MIN_VALUE; + double maxScore = -Double.MAX_VALUE; long maxScoreLabel = vertex.getValue().f0; for (long curLabel : receivedLabelsWithScores.keySet()) { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/CommunityDetectionTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/CommunityDetectionTest.java new file mode 100644 index 0000000000000..cbabcfebe9e2e --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/CommunityDetectionTest.java @@ -0,0 +1,116 @@ +/* + * 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.flink.graph.library; + +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.graph.asm.dataset.ChecksumHashCode; +import org.apache.flink.graph.asm.dataset.ChecksumHashCode.Checksum; +import org.apache.flink.graph.generator.SingletonEdgeGraph; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link CommunityDetection}. + */ +public class CommunityDetectionTest extends AsmTestBase { + + @Test + public void testWithSimpleGraph() throws Exception { + Graph result = undirectedSimpleGraph + .mapVertices(v -> (long) v.getId().getValue(), + new TypeHint>(){}.getTypeInfo()) + .mapEdges(e -> (double) e.getTarget().getValue() + e.getSource().getValue(), + new TypeHint>(){}.getTypeInfo()) + .run(new CommunityDetection<>(10, 0.5)); + + String expectedResult = + "(0,3)\n" + + "(1,5)\n" + + "(2,5)\n" + + "(3,3)\n" + + "(4,5)\n" + + "(5,5)\n"; + + TestBaseUtils.compareResultAsText(result.getVertices().collect(), expectedResult); + } + + @Test + public void testWithSingletonEdgeGraph() throws Exception { + Graph result = new SingletonEdgeGraph(env, 1) + .generate() + .mapVertices(v -> v.getId().getValue(), + new TypeHint>(){}.getTypeInfo()) + .mapEdges(e -> 1.0, + new TypeHint>(){}.getTypeInfo()) + .run(new CommunityDetection<>(10, 0.5)); + + String expectedResult = + "(0,0)\n" + + "(1,1)\n"; + + TestBaseUtils.compareResultAsText(result.getVertices().collect(), expectedResult); + } + + @Test + public void testWithEmptyGraphWithVertices() throws Exception { + emptyGraphWithVertices + .mapVertices(v -> 0L, + new TypeHint>(){}.getTypeInfo()) + .mapEdges(e -> 0.0, + new TypeHint>(){}.getTypeInfo()) + .run(new CommunityDetection<>(10, 0.5)); + } + + @Test + public void testWithEmptyGraphWithoutVertices() throws Exception { + emptyGraphWithoutVertices + .mapVertices(v -> 0L, + new TypeHint>(){}.getTypeInfo()) + .mapEdges(e -> 0.0, + new TypeHint>(){}.getTypeInfo()) + .run(new CommunityDetection<>(10, 0.5)); + } + + @Test + public void testWithRMatGraph() throws Exception { + Graph result = undirectedRMatGraph(8, 4) + .mapVertices(v -> v.getId().getValue(), + new TypeHint>(){}.getTypeInfo()) + .mapEdges(e -> (double) e.getTarget().getValue() - e.getSource().getValue(), + new TypeHint>(){}.getTypeInfo()) + .run(new CommunityDetection<>(10, 0.5)); + + Checksum checksum = new ChecksumHashCode>() + .run(result.getVertices()) + .execute(); + + assertEquals(184, checksum.getCount()); + assertEquals(0x00000000000cdc96L, checksum.getChecksum()); + } +} From 87d8dc1b0a1d915912e78d68aa99cf1965299a07 Mon Sep 17 00:00:00 2001 From: Eron Wright Date: Wed, 27 Dec 2017 14:26:47 -0800 Subject: [PATCH 166/367] [FLINK-8265] Missing jackson dependency for flink-mesos --- flink-mesos/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml index d4a0dccf984a7..cbd8a7354ea65 100644 --- a/flink-mesos/pom.xml +++ b/flink-mesos/pom.xml @@ -267,6 +267,9 @@ under the License. com.google.protobuf:protobuf-java org.apache.mesos:mesos com.netflix.fenzo:fenzo-core + com.fasterxml.jackson.core:jackson-core + com.fasterxml.jackson.core:jackson-annotations + com.fasterxml.jackson.core:jackson-databind From 5cf0f57b998e800fdd3648da976faac38badbc07 Mon Sep 17 00:00:00 2001 From: okumin Date: Mon, 1 Jan 2018 15:58:37 +0900 Subject: [PATCH 167/367] [hotfix] [docs] Remove duplicated 'program' in docs/dev/api_concepts.md This closes #5222. --- docs/dev/api_concepts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/api_concepts.md b/docs/dev/api_concepts.md index eaf0e17612b3d..cb43a42bc8e9a 100644 --- a/docs/dev/api_concepts.md +++ b/docs/dev/api_concepts.md @@ -64,7 +64,7 @@ derived from these by transforming them using API methods such as `map`, `filter Anatomy of a Flink Program -------------------------- -Flink program programs look like regular programs that transform collections of data. +Flink programs look like regular programs that transform collections of data. Each program consists of the same basic parts: 1. Obtain an `execution environment`, From ff8b2098b206f5e95fadde3dec5f1b09df95f9bb Mon Sep 17 00:00:00 2001 From: okumin Date: Mon, 1 Jan 2018 15:25:54 +0900 Subject: [PATCH 168/367] [hotfix] [docs] Fix Scala code snippets in docs. * remove unneeded semi-colons * add `()` to `print` method * typically, methods with some side-effects are invoked with `()` * fix a few misc issues This closes #5221. --- docs/dev/batch/index.md | 20 +++--- docs/dev/connectors/elasticsearch.md | 6 +- docs/dev/connectors/kafka.md | 24 +++---- docs/dev/connectors/kinesis.md | 52 +++++++------- docs/dev/connectors/twitter.md | 12 ++-- docs/dev/datastream_api.md | 2 +- docs/dev/event_timestamps_watermarks.md | 12 ++-- docs/dev/libs/cep.md | 30 ++++---- docs/dev/libs/gelly/graph_api.md | 2 +- docs/dev/libs/gelly/library_methods.md | 2 +- docs/dev/stream/operators/asyncio.md | 2 +- docs/dev/stream/state/custom_serialization.md | 2 +- docs/dev/stream/testing.md | 4 +- docs/dev/table/tableApi.md | 68 +++++++++---------- docs/dev/table/udfs.md | 14 ++-- 15 files changed, 126 insertions(+), 126 deletions(-) diff --git a/docs/dev/batch/index.md b/docs/dev/batch/index.md index 7fb84e8d2728d..cb3b42c6bbad5 100644 --- a/docs/dev/batch/index.md +++ b/docs/dev/batch/index.md @@ -571,7 +571,7 @@ data.reduceGroup { elements => elements.sum } data set.

    {% highlight scala %} val input: DataSet[(Int, String, Double)] = // [...] -val output: DataSet[(Int, String, Doublr)] = input.aggregate(SUM, 0).aggregate(MIN, 2); +val output: DataSet[(Int, String, Double)] = input.aggregate(SUM, 0).aggregate(MIN, 2) {% endhighlight %}

    You can also use short-hand syntax for minimum, maximum, and sum aggregations.

    {% highlight scala %} @@ -1037,7 +1037,7 @@ val csvInput = env.readCsvFile[Person]( val values = env.fromElements("Foo", "bar", "foobar", "fubar") // generate a number sequence -val numbers = env.generateSequence(1, 10000000); +val numbers = env.generateSequence(1, 10000000) // read a file from the specified path of type TextInputFormat val tuples = env.readHadoopFile(new TextInputFormat, classOf[LongWritable], @@ -1288,7 +1288,7 @@ val values: DataSet[(String, Int, Double)] = // [...] values.writeAsCsv("file:///path/to/the/result/file", "\n", "|") // this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines -values.writeAsText("file:///path/to/the/result/file"); +values.writeAsText("file:///path/to/the/result/file") // this writes values as strings using a user-defined formatting values map { tuple => tuple._1 + " - " + tuple._2 } @@ -1309,19 +1309,19 @@ val pData: DataSet[(BookPojo, Double)] = // [...] val sData: DataSet[String] = // [...] // sort output on String field in ascending order -tData.sortPartition(1, Order.ASCENDING).print; +tData.sortPartition(1, Order.ASCENDING).print() // sort output on Double field in descending and Int field in ascending order -tData.sortPartition(2, Order.DESCENDING).sortPartition(0, Order.ASCENDING).print; +tData.sortPartition(2, Order.DESCENDING).sortPartition(0, Order.ASCENDING).print() // sort output on the "author" field of nested BookPojo in descending order -pData.sortPartition("_1.author", Order.DESCENDING).writeAsText(...); +pData.sortPartition("_1.author", Order.DESCENDING).writeAsText(...) // sort output on the full tuple in ascending order -tData.sortPartition("_", Order.ASCENDING).writeAsCsv(...); +tData.sortPartition("_", Order.ASCENDING).writeAsCsv(...) // sort atomic type (String) output in descending order -sData.sortPartition("_", Order.DESCENDING).writeAsText(...); +sData.sortPartition("_", Order.DESCENDING).writeAsText(...) {% endhighlight %} @@ -1486,7 +1486,7 @@ val result = count map { c => c / 10000.0 * 4 } result.print() -env.execute("Iterative Pi Example"); +env.execute("Iterative Pi Example") {% endhighlight %} You can also check out the @@ -1693,7 +1693,7 @@ val env = ExecutionEnvironment.createLocalEnvironment() val lines = env.readTextFile(pathToTextFile) // build your program -env.execute(); +env.execute() {% endhighlight %} diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index b6ee63c0449f8..8774fcbcee868 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -159,7 +159,7 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc return Requests.indexRequest() .index("my-index") .type("my-type") - .source(json); + .source(json) } })) {% endhighlight %} @@ -185,7 +185,7 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc return Requests.indexRequest() .index("my-index") .type("my-type") - .source(json); + .source(json) } })) {% endhighlight %} @@ -298,7 +298,7 @@ input.addSink(new ElasticsearchSink(config, new ElasticsearchSinkFunction[String return Requests.indexRequest() .index("my-index") .type("my-type") - .source(json); + .source(json) } })) {% endhighlight %} diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 6c80370db4089..daf1903502ffa 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -130,14 +130,14 @@ DataStream stream = env
    {% highlight scala %} -val properties = new Properties(); -properties.setProperty("bootstrap.servers", "localhost:9092"); +val properties = new Properties() +properties.setProperty("bootstrap.servers", "localhost:9092") // only required for Kafka 0.8 -properties.setProperty("zookeeper.connect", "localhost:2181"); -properties.setProperty("group.id", "test"); +properties.setProperty("zookeeper.connect", "localhost:2181") +properties.setProperty("group.id", "test") stream = env .addSource(new FlinkKafkaConsumer08[String]("topic", new SimpleStringSchema(), properties)) - .print + .print() {% endhighlight %}
    @@ -422,17 +422,17 @@ DataStream stream = env
    {% highlight scala %} -val properties = new Properties(); -properties.setProperty("bootstrap.servers", "localhost:9092"); +val properties = new Properties() +properties.setProperty("bootstrap.servers", "localhost:9092") // only required for Kafka 0.8 -properties.setProperty("zookeeper.connect", "localhost:2181"); -properties.setProperty("group.id", "test"); +properties.setProperty("zookeeper.connect", "localhost:2181") +properties.setProperty("group.id", "test") -val myConsumer = new FlinkKafkaConsumer08[String]("topic", new SimpleStringSchema(), properties); -myConsumer.assignTimestampsAndWatermarks(new CustomWatermarkEmitter()); +val myConsumer = new FlinkKafkaConsumer08[String]("topic", new SimpleStringSchema(), properties) +myConsumer.assignTimestampsAndWatermarks(new CustomWatermarkEmitter()) stream = env .addSource(myConsumer) - .print + .print() {% endhighlight %}
    diff --git a/docs/dev/connectors/kinesis.md b/docs/dev/connectors/kinesis.md index 2c8b88a6797a7..ff22ee038a770 100644 --- a/docs/dev/connectors/kinesis.md +++ b/docs/dev/connectors/kinesis.md @@ -86,11 +86,11 @@ DataStream kinesis = env.addSource(new FlinkKinesisConsumer<>(
    {% highlight scala %} -val consumerConfig = new Properties(); -consumerConfig.put(ConsumerConfigConstants.AWS_REGION, "us-east-1"); -consumerConfig.put(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -consumerConfig.put(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); -consumerConfig.put(ConsumerConfigConstants.STREAM_INITIAL_POSITION, "LATEST"); +val consumerConfig = new Properties() +consumerConfig.put(ConsumerConfigConstants.AWS_REGION, "us-east-1") +consumerConfig.put(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +consumerConfig.put(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") +consumerConfig.put(ConsumerConfigConstants.STREAM_INITIAL_POSITION, "LATEST") val env = StreamExecutionEnvironment.getEnvironment @@ -295,28 +295,28 @@ simpleStringStream.addSink(kinesis);
    {% highlight scala %} -val producerConfig = new Properties(); +val producerConfig = new Properties() // Required configs -producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1"); -producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1") +producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") // Optional KPL configs -producerConfig.put("AggregationMaxCount", "4294967295"); -producerConfig.put("CollectionMaxCount", "1000"); -producerConfig.put("RecordTtl", "30000"); -producerConfig.put("RequestTimeout", "6000"); -producerConfig.put("ThreadPoolSize", "15"); +producerConfig.put("AggregationMaxCount", "4294967295") +producerConfig.put("CollectionMaxCount", "1000") +producerConfig.put("RecordTtl", "30000") +producerConfig.put("RequestTimeout", "6000") +producerConfig.put("ThreadPoolSize", "15") // Switch KinesisProducer's threading model -// producerConfig.put("ThreadingModel", "PER_REQUEST"); +// producerConfig.put("ThreadingModel", "PER_REQUEST") -val kinesis = new FlinkKinesisProducer[String](new SimpleStringSchema, producerConfig); -kinesis.setFailOnError(true); -kinesis.setDefaultStream("kinesis_stream_name"); -kinesis.setDefaultPartition("0"); +val kinesis = new FlinkKinesisProducer[String](new SimpleStringSchema, producerConfig) +kinesis.setFailOnError(true) +kinesis.setDefaultStream("kinesis_stream_name") +kinesis.setDefaultPartition("0") -val simpleStringStream = ...; -simpleStringStream.addSink(kinesis); +val simpleStringStream = ... +simpleStringStream.addSink(kinesis) {% endhighlight %}
    @@ -359,11 +359,11 @@ producerConfig.put(AWSConfigConstants.AWS_ENDPOINT, "http://localhost:4567");
    {% highlight scala %} -val producerConfig = new Properties(); -producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1"); -producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); -producerConfig.put(AWSConfigConstants.AWS_ENDPOINT, "http://localhost:4567"); +val producerConfig = new Properties() +producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1") +producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") +producerConfig.put(AWSConfigConstants.AWS_ENDPOINT, "http://localhost:4567") {% endhighlight %}
    diff --git a/docs/dev/connectors/twitter.md b/docs/dev/connectors/twitter.md index a563be6e726f3..e6fe32abd9596 100644 --- a/docs/dev/connectors/twitter.md +++ b/docs/dev/connectors/twitter.md @@ -67,12 +67,12 @@ DataStream streamSource = env.addSource(new TwitterSource(props));
    {% highlight scala %} -val props = new Properties(); -props.setProperty(TwitterSource.CONSUMER_KEY, ""); -props.setProperty(TwitterSource.CONSUMER_SECRET, ""); -props.setProperty(TwitterSource.TOKEN, ""); -props.setProperty(TwitterSource.TOKEN_SECRET, ""); -DataStream streamSource = env.addSource(new TwitterSource(props)); +val props = new Properties() +props.setProperty(TwitterSource.CONSUMER_KEY, "") +props.setProperty(TwitterSource.CONSUMER_SECRET, "") +props.setProperty(TwitterSource.TOKEN, "") +props.setProperty(TwitterSource.TOKEN_SECRET, "") +val streamSource = env.addSource(new TwitterSource(props)) {% endhighlight %}
    diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md index 307679d4ba656..6bb755edf9148 100644 --- a/docs/dev/datastream_api.md +++ b/docs/dev/datastream_api.md @@ -113,7 +113,7 @@ object WindowWordCount { .timeWindow(Time.seconds(5)) .sum(1) - counts.print + counts.print() env.execute("Window Stream WordCount") } diff --git a/docs/dev/event_timestamps_watermarks.md b/docs/dev/event_timestamps_watermarks.md index 802a079a5c9fc..acde9e48254d0 100644 --- a/docs/dev/event_timestamps_watermarks.md +++ b/docs/dev/event_timestamps_watermarks.md @@ -154,7 +154,7 @@ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val stream: DataStream[MyEvent] = env.readFile( myFormat, myFilePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 100, - FilePathFilter.createDefaultFilter()); + FilePathFilter.createDefaultFilter()) val withTimestampsAndWatermarks: DataStream[MyEvent] = stream .filter( _.severity == WARNING ) @@ -240,19 +240,19 @@ public class TimeLagWatermarkGenerator extends AssignerWithPeriodicWatermarks @@ -729,7 +729,7 @@ pattern.times(2)

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

    {% highlight scala %} -pattern.times(2, 4); +pattern.times(2, 4) {% endhighlight %}

Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

- {% highlight scala %} +{% highlight java %} Table orders = tableEnv.scan("Orders"); Table result = orders // define window @@ -364,8 +364,8 @@ Table result = orders .orderBy("rowtime") .preceding("UNBOUNDED_RANGE") .following("CURRENT_RANGE") - .as("w") - .select("a, b.avg over w, b.max over w, b.min over w") // sliding aggregate + .as("w")) + .select("a, b.avg over w, b.max over w, b.min over w"); // sliding aggregate {% endhighlight %}

Note: All aggregates must be defined over the same window, i.e., same partitioning, sorting, and range. Currently, only windows with PRECEDING (UNBOUNDED and bounded) to CURRENT ROW range are supported. Ranges with FOLLOWING are not supported yet. ORDER BY must be specified on a single time attribute.

Similar to a SQL JOIN clause. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined through join operator or using a where or filter operator.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'd, 'e, 'f); -val result = left.join(right).where('a === 'd).select('a, 'b, 'e); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'd, 'e, 'f) +val result = left.join(right).where('a === 'd).select('a, 'b, 'e) {% endhighlight %}

Similar to a SQL UNION clause. Unions two tables with duplicate records removed, both tables must have identical field types.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'a, 'b, 'c); -val result = left.union(right); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'a, 'b, 'c) +val result = left.union(right) {% endhighlight %}

Similar to a SQL UNION ALL clause. Unions two tables, both tables must have identical field types.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'a, 'b, 'c); -val result = left.unionAll(right); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'a, 'b, 'c) +val result = left.unionAll(right) {% endhighlight %}

Similar to a SQL INTERSECT clause. Intersect returns records that exist in both tables. If a record is present in one or both tables more than once, it is returned just once, i.e., the resulting table has no duplicate records. Both tables must have identical field types.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'e, 'f, 'g); -val result = left.intersect(right); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'e, 'f, 'g) +val result = left.intersect(right) {% endhighlight %}

Similar to a SQL INTERSECT ALL clause. IntersectAll returns records that exist in both tables. If a record is present in both tables more than once, it is returned as many times as it is present in both tables, i.e., the resulting table might have duplicate records. Both tables must have identical field types.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'e, 'f, 'g); -val result = left.intersectAll(right); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'e, 'f, 'g) +val result = left.intersectAll(right) {% endhighlight %}

Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not exist in the right table. Duplicate records in the left table are returned exactly once, i.e., duplicates are removed. Both tables must have identical field types.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'a, 'b, 'c); -val result = left.minus(right); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'a, 'b, 'c) +val result = left.minus(right) {% endhighlight %}

Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in the right table. A record that is present n times in the left table and m times in the right table is returned (n - m) times, i.e., as many duplicates as are present in the right table are removed. Both tables must have identical field types.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'a, 'b, 'c); -val result = left.minusAll(right); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'a, 'b, 'c) +val result = left.minusAll(right) {% endhighlight %}

Similar to a SQL IN clause. In returns true if an expression exists in a given table sub-query. The sub-query table must consist of one column. This column must have the same data type as the expression.

{% highlight scala %} -val left = ds1.toTable(tableEnv, 'a, 'b, 'c); -val right = ds2.toTable(tableEnv, 'a); -val result = left.select('a, 'b, 'c).where('a.in(right)); +val left = ds1.toTable(tableEnv, 'a, 'b, 'c) +val right = ds2.toTable(tableEnv, 'a) +val result = left.select('a, 'b, 'c).where('a.in(right)) {% endhighlight %}

Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.

{% highlight scala %} -val in = ds.toTable(tableEnv, 'a, 'b, 'c); -val result = in.orderBy('a.asc); +val in = ds.toTable(tableEnv, 'a, 'b, 'c) +val result = in.orderBy('a.asc) {% endhighlight %}
Job-/TaskManagerStatus.JVM.ClassLoaderThreads.CountStatus.JVM.ThreadsCount The total number of live threads. Gauge
Aggregations on windows
WindowedStream → DataStream

Aggregates the contents of a window. The difference between min - and minBy is that min returns the minimun value, whereas minBy returns + and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy).

{% highlight java %} windowedStream.sum(0); @@ -540,7 +540,7 @@ val result: DataStream[String] =
Aggregations
KeyedStream → DataStream

Rolling aggregations on a keyed data stream. The difference between min - and minBy is that min returns the minimun value, whereas minBy returns + and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy).

{% highlight scala %} keyedStream.sum(0) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index e161854bdcd9d..0327d0629bf50 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -811,7 +811,7 @@ input #### Incremental Window Aggregation with AggregateFunction The following example shows how an incremental `AggregateFunction` can be combined with -a `ProcesWindowFunction` to compute the average and also emit the key and window along with +a `ProcessWindowFunction` to compute the average and also emit the key and window along with the average.
diff --git a/docs/dev/stream/state/checkpointing.md b/docs/dev/stream/state/checkpointing.md index 67d95d5c9988c..cd9816d7e0c92 100644 --- a/docs/dev/stream/state/checkpointing.md +++ b/docs/dev/stream/state/checkpointing.md @@ -50,7 +50,7 @@ By default, checkpointing is disabled. To enable checkpointing, call `enableChec Other parameters for checkpointing include: - *exactly-once vs. at-least-once*: You can optionally pass a mode to the `enableCheckpointing(n)` method to choose between the two guarantee levels. - Exactly-once is preferrable for most applications. At-least-once may be relevant for certain super-low-latency (consistently few milliseconds) applications. + Exactly-once is preferable for most applications. At-least-once may be relevant for certain super-low-latency (consistently few milliseconds) applications. - *checkpoint timeout*: The time after which a checkpoint-in-progress is aborted, if it did not complete by then. diff --git a/docs/dev/stream/state/index.md b/docs/dev/stream/state/index.md index 1cfadcac1e18a..fe725afc210f8 100644 --- a/docs/dev/stream/state/index.md +++ b/docs/dev/stream/state/index.md @@ -33,7 +33,7 @@ For example: - When an application searches for certain event patterns, the state will store the sequence of events encountered so far. - When aggregating events per minute/hour/day, the state holds the pending aggregates. - When training a machine learning model over a stream of data points, the state holds the current version of the model parameters. - - When historic data needs to be managed, the state allows efficient access to events occured in the past. + - When historic data needs to be managed, the state allows efficient access to events that occurred in the past. Flink needs to be aware of the state in order to make state fault tolerant using [checkpoints](checkpointing.html) and to allow [savepoints]({{ site.baseurl }}/ops/state/savepoints.html) of streaming applications. @@ -53,4 +53,4 @@ Where to go next? * [Queryable State](queryable_state.html): Explains how to access state from outside of Flink during runtime. * [Custom Serialization for Managed State](custom_serialization.html): Discusses custom serialization logic for state and its upgrades. -{% top %} \ No newline at end of file +{% top %} diff --git a/docs/dev/stream/testing.md b/docs/dev/stream/testing.md index ce31629ee74a1..0a43d313e6dbd 100644 --- a/docs/dev/stream/testing.md +++ b/docs/dev/stream/testing.md @@ -68,7 +68,7 @@ public class SumReduceTest { @Test public void testSum() throws Exception { - // intiantiate your function + // instantiate your function SumReduce sumReduce = new SumReduce(); // call the methods that you have implemented @@ -83,7 +83,7 @@ public class SumReduceTest { class SumReduceTest extends FlatSpec with Matchers { "SumReduce" should "add values" in { - // intiantiate your function + // instantiate your function val sumReduce: SumReduce = new SumReduce() // call the methods that you have implemented diff --git a/docs/dev/table/common.md b/docs/dev/table/common.md index a63c4dc5fd36f..f562f6a726768 100644 --- a/docs/dev/table/common.md +++ b/docs/dev/table/common.md @@ -160,7 +160,7 @@ An input table can be registered from various sources: * a `TableSource`, which accesses external data, such as a file, database, or messaging system. * a `DataStream` or `DataSet` from a DataStream or DataSet program. Registering a `DataStream` or `DataSet` is discussed in the [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api) section. -An output table can be registerd using a `TableSink`. +An output table can be registered using a `TableSink`. ### Register a Table @@ -271,7 +271,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) val csvSink: TableSink = new CsvTableSink("/path/to/file", ...) // define the field names and types -val fieldNames: Arary[String] = Array("a", "b", "c") +val fieldNames: Array[String] = Array("a", "b", "c") val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.STRING, Types.LONG) // register the TableSink as table "CsvSinkTable" diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 2b1027835d43c..413d7a95cb796 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -463,7 +463,7 @@ The following `TimestampExtractor` implementations are currently available: * `ExistingField(fieldName)`: Extracts the value of a rowtime attribute from an existing `LONG` or `SQL_TIMESTAMP` field. * `StreamRecordTimestamp()`: Extracts the value of a rowtime attribute from the timestamp of the `DataStream` `StreamRecord`. Note, this `TimestampExtractor` is not available for batch table sources. -A custom `TimestampExtrator` can be defined by implementing the corresponding interface. +A custom `TimestampExtractor` can be defined by implementing the corresponding interface. #### Provided WatermarkStrategies @@ -479,7 +479,7 @@ A custom `WatermarkStrategy` can be defined by implementing the corresponding in ### CsvTableSource -The `CsvTableSource` is already included in `flink-table` without additional dependecies. +The `CsvTableSource` is already included in `flink-table` without additional dependencies. The easiest way to create a `CsvTableSource` is by using the enclosed builder `CsvTableSource.builder()`, the builder has the following methods to configure properties: diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 3097d9edea769..dd6adc1ba66ed 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -94,7 +94,7 @@ val result2 = tableEnv.sqlQuery( // SQL update with a registered table // create and register a TableSink TableSink csvSink = new CsvTableSink("/path/to/file", ...) -val fieldNames: Arary[String] = Array("product", "amount") +val fieldNames: Array[String] = Array("product", "amount") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT) tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink) // run a SQL update query on the Table and emit the result to the TableSink diff --git a/docs/dev/table/udfs.md b/docs/dev/table/udfs.md index 0e0930211b171..02c047e67db27 100644 --- a/docs/dev/table/udfs.md +++ b/docs/dev/table/udfs.md @@ -665,7 +665,7 @@ We recommended that user-defined functions should be written by Java instead of Integrating UDFs with the Runtime --------------------------------- -Sometimes it might be necessary for a user-defined function to get global runtime information or do some setup/clean-up work before the actual work. User-defined functions provide `open()` and `close()` methods that can be overriden and provide similar functionality as the methods in `RichFunction` of DataSet or DataStream API. +Sometimes it might be necessary for a user-defined function to get global runtime information or do some setup/clean-up work before the actual work. User-defined functions provide `open()` and `close()` methods that can be overridden and provide similar functionality as the methods in `RichFunction` of DataSet or DataStream API. The `open()` method is called once before the evaluation method. The `close()` method after the last call to the evaluation method. diff --git a/docs/dev/types_serialization.md b/docs/dev/types_serialization.md index 1f0c4661c3a08..b0a4b28950e72 100644 --- a/docs/dev/types_serialization.md +++ b/docs/dev/types_serialization.md @@ -64,7 +64,7 @@ The most frequent issues where users need to interact with Flink's data type han Call `.getConfig().addDefaultKryoSerializer(clazz, serializer)` on the `StreamExecutionEnvironment` or `ExecutionEnvironment`. Additional Kryo serializers are available in many libraries. See [Custom Serializers]({{ site.baseurl }}/dev/custom_serializers.html) for more details on working with custom serializers. -* **Adding Type Hints:** Sometimes, when Flink cannot infer the generic types despits all tricks, a user must pass a *type hint*. That is generally +* **Adding Type Hints:** Sometimes, when Flink cannot infer the generic types despite all tricks, a user must pass a *type hint*. That is generally only necessary in the Java API. The [Type Hints Section](#type-hints-in-the-java-api) describes that in more detail. * **Manually creating a `TypeInformation`:** This may be necessary for some API calls where it is not possible for Flink to infer diff --git a/docs/internals/ide_setup.md b/docs/internals/ide_setup.md index ea80f56ed16e9..340544d734670 100644 --- a/docs/internals/ide_setup.md +++ b/docs/internals/ide_setup.md @@ -47,7 +47,7 @@ git clone https://github.com/apache/flink.git ## IntelliJ IDEA A brief guide on how to set up IntelliJ IDEA IDE for development of the Flink core. -As Eclipse is known to have issues with mixed Scala and Java projects, more and more contributers are migrating to IntelliJ IDEA. +As Eclipse is known to have issues with mixed Scala and Java projects, more and more contributors are migrating to IntelliJ IDEA. The following documentation describes the steps to setup IntelliJ IDEA 2016.2.5 ([https://www.jetbrains.com/idea/download/](https://www.jetbrains.com/idea/download/)) diff --git a/docs/internals/job_scheduling.md b/docs/internals/job_scheduling.md index 668dfa3f42d8e..1d44e86511c04 100644 --- a/docs/internals/job_scheduling.md +++ b/docs/internals/job_scheduling.md @@ -39,7 +39,7 @@ but also for batch programs, it happens frequently. The figure below illustrates that. Consider a program with a data source, a *MapFunction*, and a *ReduceFunction*. The source and MapFunction are executed with a parallelism of 4, while the ReduceFunction is executed with a -parallism of 3. A pipeline consists of the sequence Source - Map - Reduce. On a cluster with 2 TaskManagers with +parallelism of 3. A pipeline consists of the sequence Source - Map - Reduce. On a cluster with 2 TaskManagers with 3 slots each, the program will be executed as described below.
diff --git a/docs/monitoring/checkpoint_monitoring.md b/docs/monitoring/checkpoint_monitoring.md index 6c2c2899a3953..25ed3dde74d68 100644 --- a/docs/monitoring/checkpoint_monitoring.md +++ b/docs/monitoring/checkpoint_monitoring.md @@ -75,7 +75,7 @@ jobmanager.web.checkpoints.history: 15 ### Summary Tab -The summary computes a simple min/average/maximum statitics over all completed checkpoints for the End to End Duration, State Size, and Bytes Buffered During Alignment (see [History](#history) for details about what these mean). +The summary computes a simple min/average/maximum statistics over all completed checkpoints for the End to End Duration, State Size, and Bytes Buffered During Alignment (see [History](#history) for details about what these mean).
Checkpoint Monitoring: Summary @@ -96,7 +96,7 @@ The configuration list your streaming configuration: ### Checkpoint Details -When you click on a *More details* link for a checkpoint, you get a Minumum/Average/Maximum summary over all its operators and also the detailed numbers per single subtask. +When you click on a *More details* link for a checkpoint, you get a Minimum/Average/Maximum summary over all its operators and also the detailed numbers per single subtask.
Checkpoint Monitoring: Details diff --git a/docs/ops/config.md b/docs/ops/config.md index bcf7671bd9ac3..375e6bdf5369b 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -62,7 +62,7 @@ for JobManager or TaskManager-specific options, respectively. - `taskmanager.numberOfTaskSlots`: The number of parallel operator or user function instances that a single TaskManager can run (DEFAULT: 1). If this value is larger than 1, a single TaskManager takes multiple instances of a function or operator. That way, the TaskManager can utilize multiple CPU cores, but at the same time, the available memory is divided between the different operator or function instances. This value is typically proportional to the number of physical CPU cores that the TaskManager's machine has (e.g., equal to the number of cores, or half the number of cores). [More about task slots](config.html#configuring-taskmanager-processing-slots). -- `parallelism.default`: The default parallelism to use for programs that have no parallelism specified. (DEFAULT: 1). For setups that have no concurrent jobs running, setting this value to NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all available execution resources for the program's execution. **Note**: The default parallelism can be overwriten for an entire job by calling `setParallelism(int parallelism)` on the `ExecutionEnvironment` or by passing `-p ` to the Flink Command-line frontend. It can be overwritten for single transformations by calling `setParallelism(int +- `parallelism.default`: The default parallelism to use for programs that have no parallelism specified. (DEFAULT: 1). For setups that have no concurrent jobs running, setting this value to NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all available execution resources for the program's execution. **Note**: The default parallelism can be overwritten for an entire job by calling `setParallelism(int parallelism)` on the `ExecutionEnvironment` or by passing `-p ` to the Flink Command-line frontend. It can be overwritten for single transformations by calling `setParallelism(int parallelism)` on an operator. See [Parallel Execution]({{site.baseurl}}/dev/parallel.html) for more information about parallelism. - `fs.default-scheme`: The default filesystem scheme to be used, with the necessary authority to contact, e.g. the host:port of the NameNode in the case of HDFS (if needed). diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md index 56d0cded7d169..5bbdc7d6c548e 100644 --- a/docs/ops/deployment/mesos.md +++ b/docs/ops/deployment/mesos.md @@ -55,7 +55,7 @@ the state before the failure. The artifact server is responsible for providing resources to the worker nodes. The resources can be anything from the Flink binaries to shared secrets -or configuration files. For instance, in non-containered environments, the +or configuration files. For instance, in non-containerized environments, the artifact server will provide the Flink binaries. What files will be served depends on the configuration overlay used. diff --git a/docs/ops/filesystems.md b/docs/ops/filesystems.md index 5b2a1e78d138f..ea0caa972aaf5 100644 --- a/docs/ops/filesystems.md +++ b/docs/ops/filesystems.md @@ -46,7 +46,7 @@ including any NFS or SAN that is mounted into that local file system. - **S3**: Flink directly provides file systems to talk to Amazon S3, registered under the scheme *"s3://"*. There are two alternative implementations, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`, based on code from the [Presto project](https://prestodb.io/) and the [Hadoop Project](https://hadoop.apache.org/). Both implementations are self-contained with no dependency footprint. -To use those when using Flink as a library, add the resective maven dependency (`org.apache.flink:flink-s3-fs-presto:{{ site.version }}` or `org.apache.flink:flink-s3-fs-hadoop:{{ site.version }}`). +To use those when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-s3-fs-presto:{{ site.version }}` or `org.apache.flink:flink-s3-fs-hadoop:{{ site.version }}`). When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder. See [AWS setup](deployment/aws.html) for details. @@ -57,7 +57,7 @@ See [AWS setup](deployment/aws.html) for details. For a scheme where Flink does not implemented a file system itself, Flink will try to use Hadoop to instantiate a file system for the respective scheme. All Hadoop file systems are automatically available once `flink-runtime` and the relevant Hadoop libraries are in classpath. -That way, Flink seamslessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS), for example: +That way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS), for example: - **hdfs** - **ftp** @@ -104,7 +104,7 @@ the total number of concurrent streams (`fs..limit.total`). If the file If the opening of the stream takes longer than `fs..limit.timeout`, the stream opening will fail. To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams: -`fs..limit.stream-timeout`. If a stream does not read/write any bytes for at least that amout of time, it is forcibly closed. +`fs..limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed. These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections. In addition, the The limit are also enforced only per FileSystem instance. Because File Systems are created per scheme and authority, different diff --git a/docs/ops/jobmanager_high_availability.md b/docs/ops/jobmanager_high_availability.md index 2b590cc070b6a..a99455849c418 100644 --- a/docs/ops/jobmanager_high_availability.md +++ b/docs/ops/jobmanager_high_availability.md @@ -82,7 +82,7 @@ In order to start an HA-cluster add the following configuration keys to `conf/fl
high-availability.cluster-id: /default_ns # important: customize per cluster
- **Important**: You should not set this value manually when runnig a YARN + **Important**: You should not set this value manually when running a YARN cluster, a per-job YARN session, or on another cluster manager. In those cases a cluster-id is automatically being generated based on the application id. Manually setting a cluster-id overrides this behaviour in YARN. diff --git a/docs/ops/security-ssl.md b/docs/ops/security-ssl.md index 8c7bf2bdbd324..961af892f5826 100644 --- a/docs/ops/security-ssl.md +++ b/docs/ops/security-ssl.md @@ -140,6 +140,6 @@ security.ssl.truststore-password: password flink run -m yarn-cluster -yt deploy-keys/ TestJob.jar ~~~ -When deployed using YARN, flink's web dashboard is accessible through YARN proxy's Tracking URL. To ensure that the YARN proxy is able to access flink's https url you need to configure YARN proxy to accept flink's SSL certificates. Add the custom CA certificate into Java's default trustore on the YARN Proxy node. +When deployed using YARN, flink's web dashboard is accessible through YARN proxy's Tracking URL. To ensure that the YARN proxy is able to access flink's https url you need to configure YARN proxy to accept flink's SSL certificates. Add the custom CA certificate into Java's default truststore on the YARN Proxy node. {% top %} diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md index 85ffd99c5b519..28ef99ba3eb54 100644 --- a/docs/ops/state/large_state_tuning.md +++ b/docs/ops/state/large_state_tuning.md @@ -58,7 +58,7 @@ The two numbers that are of particular interest when scaling up checkpoints are: - The amount of data buffered during alignments. For exactly-once semantics, Flink *aligns* the streams at operators that receive multiple input streams, buffering some data for that alignment. - The buffered data volume is ideally low - higher amounts means that checkpoint barriers are reveived at + The buffered data volume is ideally low - higher amounts means that checkpoint barriers are received at very different times from the different input streams. Note that when the here indicated numbers can be occasionally high in the presence of transient backpressure, data skew, diff --git a/docs/ops/upgrading.md b/docs/ops/upgrading.md index 2a34c172ae148..f5c8ee02e0974 100644 --- a/docs/ops/upgrading.md +++ b/docs/ops/upgrading.md @@ -107,7 +107,7 @@ When upgrading an application by changing its topology, a few things need to be * **Adding a stateful operator:** The state of the operator will be initialized with the default state unless it takes over the state of another operator. * **Removing a stateful operator:** The state of the removed operator is lost unless another operator takes it over. When starting the upgraded application, you have to explicitly agree to discard the state. * **Changing of input and output types of operators:** When adding a new operator before or behind an operator with internal state, you have to ensure that the input or output type of the stateful operator is not modified to preserve the data type of the internal operator state (see above for details). -* **Changing operator chaining:** Operators can be chained together for improved performance. When restoring from a savepoint taken since 1.3.x it is possible to modify chains while preversing state consistency. It is possible a break the chain such that a stateful operator is moved out of the chain. It is also possible to append or inject a new or existing stateful operator into a chain, or to modify the operator order within a chain. However, when upgrading a savepoint to 1.3.x it is paramount that the topology did not change in regards to chaining. All operators that are part of a chain should be assigned an ID as described in the [Matching Operator State](#Matching Operator State) section above. +* **Changing operator chaining:** Operators can be chained together for improved performance. When restoring from a savepoint taken since 1.3.x it is possible to modify chains while preserving state consistency. It is possible a break the chain such that a stateful operator is moved out of the chain. It is also possible to append or inject a new or existing stateful operator into a chain, or to modify the operator order within a chain. However, when upgrading a savepoint to 1.3.x it is paramount that the topology did not change in regards to chaining. All operators that are part of a chain should be assigned an ID as described in the [Matching Operator State](#Matching Operator State) section above. ## Upgrading the Flink Framework Version diff --git a/docs/start/flink_on_windows.md b/docs/start/flink_on_windows.md index 2cbc16377fa43..1cc45c677e645 100644 --- a/docs/start/flink_on_windows.md +++ b/docs/start/flink_on_windows.md @@ -56,7 +56,7 @@ Starting jobmanager. ## Installing Flink from Git -If you are installing Flink from the git repository and you are using the Windows git shell, Cygwin can produce a failure similiar to this one: +If you are installing Flink from the git repository and you are using the Windows git shell, Cygwin can produce a failure similar to this one: ~~~bash c:/flink/bin/start-local.sh: line 30: $'\r': command not found diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java index 249a5cb8fc3cc..c74385fa32810 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java @@ -88,7 +88,7 @@ public interface UdfOperator> { *

The runtime context itself is available in all UDFs via * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getRuntimeContext()}. * - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. * @param name The name under which the broadcast data set retrieved. * @return The operator itself, to allow chaining function calls. */ diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java index 096e309516a0d..3ca32fd2bf78b 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java @@ -35,7 +35,7 @@ import static org.junit.Assert.fail; /** - * Tests for {@link CollectionExecutor} with broadcasted variables. + * Tests for {@link CollectionExecutor} with broadcast variables. */ @SuppressWarnings("serial") public class CollectionExecutionWithBroadcastVariableTest { diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java index f09a89086274f..36734ddde1530 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java @@ -55,7 +55,7 @@ public GSAConfiguration() {} * Adds a data set as a broadcast set to the gather function. * * @param name The name under which the broadcast data is available in the gather function. - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. */ public void addBroadcastSetForGatherFunction(String name, DataSet data) { this.bcVarsGather.add(new Tuple2<>(name, data)); @@ -65,7 +65,7 @@ public void addBroadcastSetForGatherFunction(String name, DataSet data) { * Adds a data set as a broadcast set to the sum function. * * @param name The name under which the broadcast data is available in the sum function. - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. */ public void addBroadcastSetForSumFunction(String name, DataSet data) { this.bcVarsSum.add(new Tuple2<>(name, data)); @@ -75,7 +75,7 @@ public void addBroadcastSetForSumFunction(String name, DataSet data) { * Adds a data set as a broadcast set to the apply function. * * @param name The name under which the broadcast data is available in the apply function. - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. */ public void addBroadcastSetForApplyFunction(String name, DataSet data) { this.bcVarsApply.add(new Tuple2<>(name, data)); diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java index 39b9bcfb93ab6..3a05c448c9b6d 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java @@ -45,7 +45,7 @@ public VertexCentricConfiguration() {} * Adds a data set as a broadcast set to the compute function. * * @param name The name under which the broadcast data set is available in the compute function. - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. */ public void addBroadcastSet(String name, DataSet data) { this.bcVars.add(new Tuple2<>(name, data)); diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java index 0422f138330c4..d44ff7ed1ed88 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java @@ -56,7 +56,7 @@ public ScatterGatherConfiguration() {} * Adds a data set as a broadcast set to the scatter function. * * @param name The name under which the broadcast data is available in the scatter function. - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. */ public void addBroadcastSetForScatterFunction(String name, DataSet data) { this.bcVarsScatter.add(new Tuple2<>(name, data)); @@ -66,7 +66,7 @@ public void addBroadcastSetForScatterFunction(String name, DataSet data) { * Adds a data set as a broadcast set to the gather function. * * @param name The name under which the broadcast data is available in the gather function. - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. */ public void addBroadcastSetForGatherFunction(String name, DataSet data) { this.bcVarsGather.add(new Tuple2<>(name, data)); diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java index be6804ba425ec..a7576dc73df05 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java @@ -424,12 +424,12 @@ public void testConsecutiveUnionsWithBroadcast() throws Exception { DualInputPlanNode join = resolver.getNode("join"); - // check input of join is broadcasted + // check input of join is broadcast assertEquals("First join input should be fully replicated.", PartitioningProperty.FULL_REPLICATION, join.getInput1().getGlobalProperties().getPartitioning()); NAryUnionPlanNode union = (NAryUnionPlanNode)join.getInput1().getSource(); - // check that all union inputs are broadcasted + // check that all union inputs are broadcast for (Channel c : union.getInputs()) { assertEquals("Union input should be fully replicated", PartitioningProperty.FULL_REPLICATION, c.getGlobalProperties().getPartitioning()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java index 9c2c1091fb03c..4e90f8c6d17ce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java @@ -40,7 +40,7 @@ * This class represents a single materialization of a broadcast variable and maintains a reference count for it. If the * reference count reaches zero the variable is no longer accessible and will eventually be garbage-collected. * - * @param The type of the elements in the broadcasted data set. + * @param The type of the elements in the broadcast data set. */ public class BroadcastVariableMaterialization { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java index 7c493383ec2b0..a357dc4c03699 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java @@ -37,7 +37,7 @@ public class OperatorStateHandle implements StreamStateHandle { */ public enum Mode { SPLIT_DISTRIBUTE, // The operator state partitions in the state handle are split and distributed to one task each. - BROADCAST // The operator state partitions are broadcasted to all task. + BROADCAST // The operator state partitions are broadcast to all task. } private static final long serialVersionUID = 35876522969227335L; diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala index bfe7567b5efa6..78bed4533c63e 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala @@ -268,7 +268,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { * The runtime context itself is available in all UDFs via * `org.apache.flink.api.common.functions.AbstractRichFunction#getRuntimeContext()` * - * @param data The data set to be broadcasted. + * @param data The data set to be broadcast. * @param name The name under which the broadcast data set retrieved. * @return The operator itself, to allow chaining function calls. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 2274968a343a5..83c11266ce00a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -363,7 +363,7 @@ private DataStream partitionCustom(Partitioner partitioner, Keys ke /** * Sets the partitioning of the {@link DataStream} so that the output elements - * are broadcasted to every parallel instance of the next operation. + * are broadcast to every parallel instance of the next operation. * * @return The DataStream with broadcast partitioning set. */ From 0677f08b8f49a5db0be6fe1916bc4eb8dd156867 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 18 Jan 2018 17:57:10 +0100 Subject: [PATCH 197/367] [FLINK-8455] [core] Make 'org.apache.hadoop.' a 'parent-first' classloading pattern. This change avoid duplication of Hadoop classes between the Flink runtime and the user code. Hadoop (and transitively its dependencies) should be part of the application class loader. The user code classloader is allowed to duplicate transitive dependencies, but not Hadoop's classes directly. This also adds tests to validate parent-first classloading patterns. --- .../flink/configuration/CoreOptions.java | 2 +- .../ParentFirstPatternsTest.java | 78 +++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index cf10012e3d8f1..c48e5ef12bf86 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -36,7 +36,7 @@ public class CoreOptions { public static final ConfigOption ALWAYS_PARENT_FIRST_LOADER = ConfigOptions .key("classloader.parent-first-patterns") - .defaultValue("java.;scala.;org.apache.flink.;javax.annotation;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); + .defaultValue("java.;scala.;org.apache.flink.;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); // ------------------------------------------------------------------------ diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java new file mode 100644 index 0000000000000..784d0998768b4 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java @@ -0,0 +1,78 @@ +/* + * 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.flink.configuration; + +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashSet; + +import static org.junit.Assert.assertTrue; + +/** + * Test that checks that all packages that need to be loaded 'parent-first' are also + * in the parent-first patterns. + */ +public class ParentFirstPatternsTest extends TestLogger { + + private static final HashSet PARENT_FIRST_PACKAGES = new HashSet<>( + Arrays.asList(CoreOptions.ALWAYS_PARENT_FIRST_LOADER.defaultValue().split(";"))); + + /** + * All java and Flink classes must be loaded parent first. + */ + @Test + public void testAllCorePatterns() { + assertTrue(PARENT_FIRST_PACKAGES.contains("java.")); + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.flink.")); + assertTrue(PARENT_FIRST_PACKAGES.contains("javax.annotation.")); + } + + /** + * To avoid multiple binding problems and warnings for logger frameworks, we load them + * parent-first. + */ + @Test + public void testLoggersParentFirst() { + assertTrue(PARENT_FIRST_PACKAGES.contains("org.slf4j")); + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.log4j")); + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.logging.log4j")); + assertTrue(PARENT_FIRST_PACKAGES.contains("ch.qos.logback")); + } + + /** + * As long as Scala is not a pure user library, but is also used in the Flink runtime, we need + * to load all Scala classes parent-first. + */ + @Test + public void testScalaParentFirst() { + assertTrue(PARENT_FIRST_PACKAGES.contains("scala.")); + } + + /** + * As long as we have Hadoop classes leaking through some of Flink's APIs (example bucketing sink), + * we need to make them parent first. + */ + @Test + public void testHadoopParentFirst() { + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.hadoop.")); + } +} From 537552267a4e095fdf4120d17425814093bbd92a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 19 Jan 2018 13:53:08 +0100 Subject: [PATCH 198/367] [FLINK-8461] [build] Adjust logger configurations for shaded Netty classnames --- flink-dist/src/main/flink-bin/conf/log4j-cli.properties | 2 +- flink-dist/src/main/flink-bin/conf/log4j-console.properties | 2 +- .../src/main/flink-bin/conf/log4j-yarn-session.properties | 2 +- flink-dist/src/main/flink-bin/conf/log4j.properties | 2 +- flink-dist/src/main/flink-bin/conf/logback-console.xml | 2 +- flink-dist/src/main/flink-bin/conf/logback.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties index 2aba6af8a3a18..a6c8d365437b3 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties @@ -40,4 +40,4 @@ log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p log4j.logger.org.apache.hadoop.util.NativeCodeLoader=OFF # suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file +log4j.logger.org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file diff --git a/flink-dist/src/main/flink-bin/conf/log4j-console.properties b/flink-dist/src/main/flink-bin/conf/log4j-console.properties index 08c09961734ab..b458d5824225b 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-console.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-console.properties @@ -36,4 +36,4 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # Suppress the irrelevant (wrong) warnings from the Netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console +log4j.logger.org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console diff --git a/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties b/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties index fd8d978cd4f4a..5d415357f5000 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties @@ -24,7 +24,7 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, stdout +log4j.logger.org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, stdout log4j.logger.org.apache.zookeeper=WARN, stdout log4j.logger.org.apache.flink.shaded.org.apache.curator.framework=WARN, stdout log4j.logger.org.apache.flink.runtime.util.ZooKeeperUtils=WARN, stdout diff --git a/flink-dist/src/main/flink-bin/conf/log4j.properties b/flink-dist/src/main/flink-bin/conf/log4j.properties index 8e00ce369d514..4df066bf9160e 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j.properties @@ -38,4 +38,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # Suppress the irrelevant (wrong) warnings from the Netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file +log4j.logger.org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file diff --git a/flink-dist/src/main/flink-bin/conf/logback-console.xml b/flink-dist/src/main/flink-bin/conf/logback-console.xml index 4813ce592691f..15765e86f3e23 100644 --- a/flink-dist/src/main/flink-bin/conf/logback-console.xml +++ b/flink-dist/src/main/flink-bin/conf/logback-console.xml @@ -50,7 +50,7 @@ - + diff --git a/flink-dist/src/main/flink-bin/conf/logback.xml b/flink-dist/src/main/flink-bin/conf/logback.xml index f3c433105a7f9..ae0bfe7a33adb 100644 --- a/flink-dist/src/main/flink-bin/conf/logback.xml +++ b/flink-dist/src/main/flink-bin/conf/logback.xml @@ -52,7 +52,7 @@ - + From bfe3ff07c3b4cfa710b78db0db588086182884ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=87=91=E7=AB=B9?= Date: Wed, 3 Jan 2018 23:13:49 +0800 Subject: [PATCH 199/367] [FLINK-8355] [table] Remove DataSetAggregateWithNullValuesRule. This closes #5320. --- .../plan/nodes/dataset/DataSetAggregate.scala | 13 +- .../table/plan/rules/FlinkRuleSets.scala | 1 - .../rules/dataSet/DataSetAggregateRule.scala | 6 - .../DataSetAggregateWithNullValuesRule.scala | 89 ---------- .../runtime/aggregate/AggregateUtil.scala | 6 +- .../aggregate/DataSetAggFunction.scala | 24 ++- .../table/api/batch/sql/AggregateTest.scala | 44 +---- .../api/batch/sql/DistinctAggregateTest.scala | 167 +++++------------- .../api/batch/sql/GroupingSetsTest.scala | 28 +-- .../api/batch/sql/SetOperatorsTest.scala | 19 +- .../api/batch/sql/SingleRowJoinTest.scala | 130 +++----------- .../table/api/batch/table/AggregateTest.scala | 47 +---- .../table/api/batch/table/CalcTest.scala | 17 +- .../table/plan/QueryDecorrelationTest.scala | 75 ++++---- .../aggfunctions/AggFunctionTestBase.scala | 23 ++- .../runtime/batch/sql/AggregateITCase.scala | 11 ++ .../runtime/batch/table/AggregateITCase.scala | 28 ++- .../table/utils/UserDefinedAggFunctions.scala | 25 ++- 18 files changed, 235 insertions(+), 518 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala index bdc3d7a7e0677..0d7a4f44102a8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala @@ -23,7 +23,6 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} -import org.apache.flink.api.common.functions.GroupReduceFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet import org.apache.flink.api.java.typeutils.RowTypeInfo @@ -31,8 +30,8 @@ import org.apache.flink.table.api.BatchTableEnvironment import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.AggregationCodeGenerator import org.apache.flink.table.plan.nodes.CommonAggregate -import org.apache.flink.table.runtime.aggregate.{AggregateUtil, DataSetPreAggFunction} import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair +import org.apache.flink.table.runtime.aggregate.{AggregateUtil, DataSetAggFunction, DataSetFinalAggFunction, DataSetPreAggFunction} import org.apache.flink.types.Row /** @@ -102,7 +101,7 @@ class DataSetAggregate( val ( preAgg: Option[DataSetPreAggFunction], preAggType: Option[TypeInformation[Row]], - finalAgg: GroupReduceFunction[Row, Row] + finalAgg: Either[DataSetAggFunction, DataSetFinalAggFunction] ) = AggregateUtil.createDataSetAggregateFunctions( generator, namedAggregates, @@ -127,13 +126,13 @@ class DataSetAggregate( .name(aggOpName) // final aggregation .groupBy(grouping.indices: _*) - .reduceGroup(finalAgg) + .reduceGroup(finalAgg.right.get) .returns(rowTypeInfo) .name(aggOpName) } else { inputDS .groupBy(grouping: _*) - .reduceGroup(finalAgg) + .reduceGroup(finalAgg.left.get) .returns(rowTypeInfo) .name(aggOpName) } @@ -149,12 +148,12 @@ class DataSetAggregate( .returns(preAggType.get) .name(aggOpName) // final aggregation - .reduceGroup(finalAgg) + .reduceGroup(finalAgg.right.get) .returns(rowTypeInfo) .name(aggOpName) } else { inputDS - .reduceGroup(finalAgg) + .mapPartition(finalAgg.left.get).setParallelism(1) .returns(rowTypeInfo) .name(aggOpName) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 10d68814bf24e..bc95babee4825 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -164,7 +164,6 @@ object FlinkRuleSets { // translate to Flink DataSet nodes DataSetWindowAggregateRule.INSTANCE, DataSetAggregateRule.INSTANCE, - DataSetAggregateWithNullValuesRule.INSTANCE, DataSetDistinctRule.INSTANCE, DataSetCalcRule.INSTANCE, DataSetJoinRule.INSTANCE, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala index 9a31617d7ba97..e73c76e7fe87d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala @@ -37,12 +37,6 @@ class DataSetAggregateRule override def matches(call: RelOptRuleCall): Boolean = { val agg: FlinkLogicalAggregate = call.rel(0).asInstanceOf[FlinkLogicalAggregate] - // for non-grouped agg sets we attach null row to source data - // we need to apply DataSetAggregateWithNullValuesRule - if (agg.getGroupSet.isEmpty) { - return false - } - // distinct is translated into dedicated operator if (agg.getAggCallList.isEmpty && agg.getGroupCount == agg.getRowType.getFieldCount && diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala deleted file mode 100644 index 4a1e6d620e824..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.flink.table.plan.rules.dataSet - - -import com.google.common.collect.ImmutableList -import org.apache.calcite.plan._ -import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.convert.ConverterRule -import org.apache.calcite.rex.RexLiteral -import org.apache.flink.table.plan.nodes.FlinkConventions -import org.apache.flink.table.plan.nodes.dataset.DataSetAggregate -import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalAggregate, FlinkLogicalUnion, FlinkLogicalValues} - -import scala.collection.JavaConversions._ - -/** - * Rule for insert [[org.apache.flink.types.Row]] with null records into a [[DataSetAggregate]]. - * Rule apply for non grouped aggregate query. - */ -class DataSetAggregateWithNullValuesRule - extends ConverterRule( - classOf[FlinkLogicalAggregate], - FlinkConventions.LOGICAL, - FlinkConventions.DATASET, - "DataSetAggregateWithNullValuesRule") { - - override def matches(call: RelOptRuleCall): Boolean = { - val agg: FlinkLogicalAggregate = call.rel(0).asInstanceOf[FlinkLogicalAggregate] - - // group sets shouldn't attach a null row - // we need to apply other rules. i.e. DataSetAggregateRule - if (!agg.getGroupSet.isEmpty) { - return false - } - - // check if we have distinct aggregates - val distinctAggs = agg.getAggCallList.exists(_.isDistinct) - - !distinctAggs - } - - override def convert(rel: RelNode): RelNode = { - val agg: FlinkLogicalAggregate = rel.asInstanceOf[FlinkLogicalAggregate] - val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASET) - val cluster: RelOptCluster = rel.getCluster - - val fieldTypes = agg.getInput.getRowType.getFieldList.map(_.getType) - val nullLiterals: ImmutableList[ImmutableList[RexLiteral]] = - ImmutableList.of(ImmutableList.copyOf[RexLiteral]( - for (fieldType <- fieldTypes) - yield { - cluster.getRexBuilder. - makeLiteral(null, fieldType, false).asInstanceOf[RexLiteral] - })) - - val logicalValues = FlinkLogicalValues.create(cluster, agg.getInput.getRowType, nullLiterals) - val logicalUnion = FlinkLogicalUnion.create(List(logicalValues, agg.getInput), all = true) - - new DataSetAggregate( - cluster, - traitSet, - RelOptRule.convert(logicalUnion, FlinkConventions.DATASET), - agg.getNamedAggCalls, - rel.getRowType, - agg.getInput.getRowType, - agg.getGroupSet.toArray - ) - } -} - -object DataSetAggregateWithNullValuesRule { - val INSTANCE: RelOptRule = new DataSetAggregateWithNullValuesRule -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index a867b1cd8b013..a652981f5f499 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -829,7 +829,7 @@ object AggregateUtil { outputType: RelDataType, groupings: Array[Int]): (Option[DataSetPreAggFunction], Option[TypeInformation[Row]], - RichGroupReduceFunction[Row, Row]) = { + Either[DataSetAggFunction, DataSetFinalAggFunction]) = { val needRetract = false val (aggInFields, aggregates, accTypes, _) = transformToAggregateFunctions( @@ -899,7 +899,7 @@ object AggregateUtil { ( Some(new DataSetPreAggFunction(genPreAggFunction)), Some(preAggRowType), - new DataSetFinalAggFunction(genFinalAggFunction) + Right(new DataSetFinalAggFunction(genFinalAggFunction)) ) } else { @@ -922,7 +922,7 @@ object AggregateUtil { ( None, None, - new DataSetAggFunction(genFunction) + Left(new DataSetAggFunction(genFunction)) ) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala index ced14504e8e37..313dae08bbae7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.aggregate import java.lang.Iterable -import org.apache.flink.api.common.functions.RichGroupReduceFunction +import org.apache.flink.api.common.functions.{MapPartitionFunction, RichGroupReduceFunction} import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.util.Logging @@ -27,14 +27,15 @@ import org.apache.flink.types.Row import org.apache.flink.util.Collector /** - * [[RichGroupReduceFunction]] to compute aggregates that do not support pre-aggregation for batch - * (DataSet) queries. + * [[RichGroupReduceFunction]] and [[MapPartitionFunction]] to compute aggregates that do + * not support pre-aggregation for batch(DataSet) queries. * * @param genAggregations Code-generated [[GeneratedAggregations]] */ class DataSetAggFunction( private val genAggregations: GeneratedAggregationsFunction) extends RichGroupReduceFunction[Row, Row] + with MapPartitionFunction[Row, Row] with Compiler[GeneratedAggregations] with Logging { private var output: Row = _ @@ -56,6 +57,12 @@ class DataSetAggFunction( accumulators = function.createAccumulators() } + /** + * Computes a non-pre-aggregated aggregation. + * + * @param records An iterator over all records of the group. + * @param out The collector to hand results to. + */ override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = { // reset accumulators @@ -79,4 +86,15 @@ class DataSetAggFunction( out.collect(output) } + + /** + * Computes a non-pre-aggregated aggregation and returns a row even if the input is empty. + * + * @param records An iterator over all records of the partition. + * @param out The collector to hand results to. + */ + override def mapPartition(records: Iterable[Row], out: Collector[Row]): Unit = { + reduce(records, out) + } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/AggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/AggregateTest.scala index f2e250b3d1452..921c139a4d541 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/AggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/AggregateTest.scala @@ -36,21 +36,9 @@ class AggregateTest extends TableTestBase { val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable" - val setValues = unaryNode( - "DataSetValues", - batchTableNode(0), - tuples(List(null,null,null)), - term("values","a","b","c") - ) - val union = unaryNode( - "DataSetUnion", - setValues, - term("union","a","b","c") - ) - val aggregate = unaryNode( "DataSetAggregate", - union, + batchTableNode(0), term("select", "AVG(a) AS EXPR$0", "SUM(b) AS EXPR$1", @@ -73,22 +61,9 @@ class AggregateTest extends TableTestBase { term("where", "=(a, 1)") ) - val setValues = unaryNode( - "DataSetValues", - calcNode, - tuples(List(null,null,null)), - term("values","a","b","c") - ) - - val union = unaryNode( - "DataSetUnion", - setValues, - term("union","a","b","c") - ) - val aggregate = unaryNode( "DataSetAggregate", - union, + calcNode, term("select", "AVG(a) AS EXPR$0", "SUM(b) AS EXPR$1", @@ -111,22 +86,9 @@ class AggregateTest extends TableTestBase { term("where", "=(a, 1)") ) - val setValues = unaryNode( - "DataSetValues", - calcNode, - tuples(List(null,null,null,null)), - term("values","a","b","c","$f3") - ) - - val union = unaryNode( - "DataSetUnion", - setValues, - term("union","a","b","c","$f3") - ) - val aggregate = unaryNode( "DataSetAggregate", - union, + calcNode, term("select", "AVG(a) AS EXPR$0", "SUM(b) AS EXPR$1", diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala index ce008e4062ca7..ced07e4e98bfe 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala @@ -36,22 +36,13 @@ class DistinctAggregateTest extends TableTestBase { val expected = unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetDistinct", unaryNode( - "DataSetValues", - unaryNode( - "DataSetDistinct", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a") - ), - term("distinct", "a") - ), - tuples(List(null)), - term("values", "a") + "DataSetCalc", + batchTableNode(0), + term("select", "a") ), - term("union", "a") + term("distinct", "a") ), term("select", "COUNT(a) AS EXPR$0") ) @@ -69,22 +60,13 @@ class DistinctAggregateTest extends TableTestBase { val expected = unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetDistinct", unaryNode( - "DataSetValues", - unaryNode( - "DataSetDistinct", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a") - ), - term("distinct", "a") - ), - tuples(List(null)), - term("values", "a") + "DataSetCalc", + batchTableNode(0), + term("select", "a") ), - term("union", "a") + term("distinct", "a") ), term("select", "COUNT(a) AS EXPR$0", "SUM(a) AS EXPR$1", "MAX(a) AS EXPR$2") ) @@ -103,23 +85,14 @@ class DistinctAggregateTest extends TableTestBase { val expected0 = unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetAggregate", unaryNode( - "DataSetValues", - unaryNode( - "DataSetAggregate", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a", "b") - ), - term("groupBy", "a"), - term("select", "a", "SUM(b) AS EXPR$1") - ), - tuples(List(null, null)), - term("values", "a", "EXPR$1") + "DataSetCalc", + batchTableNode(0), + term("select", "a", "b") ), - term("union", "a", "EXPR$1") + term("groupBy", "a"), + term("select", "a", "SUM(b) AS EXPR$1") ), term("select", "COUNT(a) AS EXPR$0", "SUM(EXPR$1) AS EXPR$1") ) @@ -132,23 +105,14 @@ class DistinctAggregateTest extends TableTestBase { val expected1 = unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetAggregate", unaryNode( - "DataSetValues", - unaryNode( - "DataSetAggregate", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a", "b") - ), - term("groupBy", "b"), - term("select", "b", "COUNT(a) AS EXPR$0") - ), - tuples(List(null, null)), - term("values", "b", "EXPR$0") + "DataSetCalc", + batchTableNode(0), + term("select", "a", "b") ), - term("union", "b", "EXPR$0") + term("groupBy", "b"), + term("select", "b", "COUNT(a) AS EXPR$0") ), term("select", "$SUM0(EXPR$0) AS EXPR$0", "SUM(b) AS EXPR$1") ) @@ -168,44 +132,26 @@ class DistinctAggregateTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetDistinct", unaryNode( - "DataSetValues", - unaryNode( - "DataSetDistinct", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a") - ), - term("distinct", "a") - ), - tuples(List(null)), - term("values", "a") + "DataSetCalc", + batchTableNode(0), + term("select", "a") ), - term("union", "a") + term("distinct", "a") ), term("select", "COUNT(a) AS EXPR$0") ), unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetDistinct", unaryNode( - "DataSetValues", - unaryNode( - "DataSetDistinct", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "b") - ), - term("distinct", "b") - ), - tuples(List(null)), - term("values", "b") + "DataSetCalc", + batchTableNode(0), + term("select", "b") ), - term("union", "b") + term("distinct", "b") ), term("select", "SUM(b) AS EXPR$1") ), @@ -232,37 +178,19 @@ class DistinctAggregateTest extends TableTestBase { "DataSetSingleRowJoin", unaryNode( "DataSetAggregate", - unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - batchTableNode(0), - tuples(List(null, null, null)), - term("values", "a, b, c") - ), - term("union", "a, b, c") - ), + batchTableNode(0), term("select", "COUNT(c) AS EXPR$2") ), unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetDistinct", unaryNode( - "DataSetValues", - unaryNode( - "DataSetDistinct", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a") - ), - term("distinct", "a") - ), - tuples(List(null)), - term("values", "a") + "DataSetCalc", + batchTableNode(0), + term("select", "a") ), - term("union", "a") + term("distinct", "a") ), term("select", "COUNT(a) AS EXPR$0") ), @@ -273,22 +201,13 @@ class DistinctAggregateTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", + "DataSetDistinct", unaryNode( - "DataSetValues", - unaryNode( - "DataSetDistinct", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "b") - ), - term("distinct", "b") - ), - tuples(List(null)), - term("values", "b") + "DataSetCalc", + batchTableNode(0), + term("select", "b") ), - term("union", "b") + term("distinct", "b") ), term("select", "SUM(b) AS EXPR$1") ), diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala index 9f3d2b6c77007..57a4c5a03f0cd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala @@ -114,19 +114,11 @@ class GroupingSetsTest extends TableTestBase { "DataSetCalc", unaryNode( "DataSetAggregate", - unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - batchTableNode(0), - tuples(List(null, null, null)), - term("values", "a", "b", "c") - ), - term("union", "a", "b", "c") - ), + batchTableNode(0), term("select", "AVG(a) AS a") ), - term("select", "null AS b", "null AS c", "a", "0 AS g", "0 AS gb", "0 AS gc", + term( + "select", "null AS b", "null AS c", "a", "0 AS g", "0 AS gb", "0 AS gc", "0 AS gib", "0 AS gic", "0 AS gid") ) @@ -189,19 +181,11 @@ class GroupingSetsTest extends TableTestBase { "DataSetCalc", unaryNode( "DataSetAggregate", - unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - batchTableNode(0), - tuples(List(null, null, null)), - term("values", "a", "b", "c") - ), - term("union", "a", "b", "c") - ), + batchTableNode(0), term("select", "AVG(a) AS a") ), - term("select", "null AS b", "null AS c", "a", "0 AS g", "0 AS gb", "0 AS gc", + term( + "select", "null AS b", "null AS c", "a", "0 AS g", "0 AS gb", "0 AS gc", "0 AS gib", "0 AS gic", "0 AS gid") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala index bff0b78505c88..d51fc42aba5e5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala @@ -101,20 +101,11 @@ class SetOperatorsTest extends TableTestBase { batchTableNode(0), unaryNode( "DataSetAggregate", - binaryNode( - "DataSetUnion", - values( - "DataSetValues", - term("tuples", "[{ null }]"), - term("values", "b") - ), - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "b"), - term("where", "OR(=(b, 6), =(b, 1))") - ), - term("union", "b") + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "b"), + term("where", "OR(=(b, 6), =(b, 1))") ), term("select", "COUNT(*) AS $f0", "COUNT(b) AS $f1") ), diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SingleRowJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SingleRowJoinTest.scala index 8bfb61b08a2bb..59156d632d08b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SingleRowJoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SingleRowJoinTest.scala @@ -47,16 +47,7 @@ class SingleRowJoinTest extends TableTestBase { "DataSetCalc", unaryNode( "DataSetAggregate", - unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - batchTableNode(0), - tuples(List(null, null)), - term("values", "a1", "a2") - ), - term("union","a1","a2") - ), + batchTableNode(0), term("select", "SUM(a1) AS $f0", "SUM(a2) AS $f1") ), term("select", "+($f0, $f1) AS asum") @@ -88,18 +79,9 @@ class SingleRowJoinTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a1") - ), - tuples(List(null)), - term("values", "a1") - ), - term("union","a1") + "DataSetCalc", + batchTableNode(0), + term("select", "a1") ), term("select", "COUNT(a1) AS cnt") ), @@ -132,18 +114,9 @@ class SingleRowJoinTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a1") - ), - tuples(List(null)), - term("values", "a1") - ), - term("union", "a1") + "DataSetCalc", + batchTableNode(0), + term("select", "a1") ), term("select", "COUNT(a1) AS cnt") ), @@ -173,16 +146,7 @@ class SingleRowJoinTest extends TableTestBase { batchTableNode(0), unaryNode( "DataSetAggregate", - unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - batchTableNode(1), - tuples(List(null, null)), - term("values", "b1", "b2") - ), - term("union","b1","b2") - ), + batchTableNode(1), term("select", "MIN(b1) AS b1", "MAX(b2) AS b2") ), term("where", "AND(<(a1, b1)", "=(a2, b2))"), @@ -221,17 +185,9 @@ class SingleRowJoinTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(1), - term("select", "0 AS $f0")), - tuples(List(null)), term("values", "$f0") - ), - term("union", "$f0") - ), + "DataSetCalc", + batchTableNode(1), + term("select", "0 AS $f0")), term("select", "COUNT(*) AS cnt") ) @@ -266,17 +222,9 @@ class SingleRowJoinTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(1), - term("select", "0 AS $f0")), - tuples(List(null)), term("values", "$f0") - ), - term("union", "$f0") - ), + "DataSetCalc", + batchTableNode(1), + term("select", "0 AS $f0")), term("select", "COUNT(*) AS cnt") ) @@ -308,21 +256,13 @@ class SingleRowJoinTest extends TableTestBase { ), term("select", "a1") ) + unaryNode( - "DataSetAggregate", - unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(1), - term("select", "0 AS $f0")), - tuples(List(null)), term("values", "$f0") - ), - term("union", "$f0") - ), - term("select", "COUNT(*) AS cnt") - ) + "\n" + + "DataSetAggregate", + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "0 AS $f0")), + term("select", "COUNT(*) AS cnt") + ) + "\n" + batchTableNode(0) util.verifySql(queryRightJoin, expected) @@ -356,17 +296,9 @@ class SingleRowJoinTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(1), - term("select", "0 AS $f0")), - tuples(List(null)), term("values", "$f0") - ), - term("union", "$f0") - ), + "DataSetCalc", + batchTableNode(1), + term("select", "0 AS $f0")), term("select", "COUNT(*) AS cnt") ) + "\n" + batchTableNode(0) @@ -406,17 +338,9 @@ class SingleRowJoinTest extends TableTestBase { unaryNode( "DataSetAggregate", unaryNode( - "DataSetUnion", - unaryNode( - "DataSetValues", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a1") - ), - tuples(List(null)), term("values", "a1") - ), - term("union", "a1") + "DataSetCalc", + batchTableNode(0), + term("select", "a1") ), term("select", "SUM(a1) AS $f0") ), diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/AggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/AggregateTest.scala index 0a135d1764924..df65481e9e87d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/AggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/AggregateTest.scala @@ -66,21 +66,9 @@ class AggregateTest extends TableTestBase { val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c) val resultTable = sourceTable.select('a.avg,'b.sum,'c.count) - val setValues = unaryNode( - "DataSetValues", - batchTableNode(0), - tuples(List(null,null,null)), - term("values","a","b","c") - ) - val union = unaryNode( - "DataSetUnion", - setValues, - term("union","a","b","c") - ) - val expected = unaryNode( "DataSetAggregate", - union, + batchTableNode(0), term("select", "AVG(a) AS TMP_0", "SUM(b) AS TMP_1", @@ -106,22 +94,9 @@ class AggregateTest extends TableTestBase { term("where", "=(a, 1)") ) - val setValues = unaryNode( - "DataSetValues", - calcNode, - tuples(List(null,null,null)), - term("values","a","b","c") - ) - - val union = unaryNode( - "DataSetUnion", - setValues, - term("union","a","b","c") - ) - val expected = unaryNode( "DataSetAggregate", - union, + calcNode, term("select", "AVG(a) AS TMP_0", "SUM(b) AS TMP_1", @@ -148,23 +123,11 @@ class AggregateTest extends TableTestBase { term("where", "=(a, 1)") ) - val setValues = unaryNode( - "DataSetValues", - calcNode, - tuples(List(null,null,null,null)), - term("values","a","b","c","$f3") - ) - - val union = unaryNode( - "DataSetUnion", - setValues, - term("union","a","b","c","$f3") - ) - val expected = unaryNode( "DataSetAggregate", - union, - term("select", + calcNode, + term( + "select", "AVG(a) AS TMP_0", "SUM(b) AS TMP_1", "COUNT(c) AS TMP_2", diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala index ff6dcf13c0030..bba1a5b58aaaf 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala @@ -139,19 +139,10 @@ class CalcTest extends TableTestBase { val expected = unaryNode( "DataSetAggregate", - binaryNode( - "DataSetUnion", - values( - "DataSetValues", - tuples(List(null, null)), - term("values", "a", "b") - ), - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "a", "b") - ), - term("union", "a", "b") + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "a", "b") ), term("select", "SUM(a) AS TMP_0", "MAX(b) AS TMP_1") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/QueryDecorrelationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/QueryDecorrelationTest.scala index 0c3796f238cfb..c952578b3b7b0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/QueryDecorrelationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/QueryDecorrelationTest.scala @@ -98,55 +98,46 @@ class QueryDecorrelationTest extends TableTestBase { val expectedQuery = unaryNode( "DataSetAggregate", - binaryNode( - "DataSetUnion", - values( - "DataSetValues", - tuples(List(null)), - term("values", "empno") - ), - unaryNode( - "DataSetCalc", - binaryNode( - "DataSetJoin", - unaryNode( - "DataSetCalc", - binaryNode( - "DataSetJoin", - unaryNode( - "DataSetCalc", - batchTableNode(0), - term("select", "empno", "salary", "deptno") - ), - unaryNode( - "DataSetCalc", - batchTableNode(1), - term("select", "deptno") - ), - term("where", "=(deptno, deptno0)"), - term("join", "empno", "salary", "deptno", "deptno0"), - term("joinType", "InnerJoin") - ), - term("select", "empno", "salary", "deptno0") - ), - unaryNode( - "DataSetAggregate", + unaryNode( + "DataSetCalc", + binaryNode( + "DataSetJoin", + unaryNode( + "DataSetCalc", + binaryNode( + "DataSetJoin", unaryNode( "DataSetCalc", batchTableNode(0), - term("select", "salary", "deptno"), - term("where", "IS NOT NULL(deptno)") + term("select", "empno", "salary", "deptno") + ), + unaryNode( + "DataSetCalc", + batchTableNode(1), + term("select", "deptno") ), - term("groupBy", "deptno"), - term("select", "deptno", "AVG(salary) AS EXPR$0") + term("where", "=(deptno, deptno0)"), + term("join", "empno", "salary", "deptno", "deptno0"), + term("joinType", "InnerJoin") ), - term("where", "AND(=(deptno0, deptno), >(salary, EXPR$0))"), - term("join", "empno", "salary", "deptno0", "deptno", "EXPR$0"), - term("joinType", "InnerJoin") + term("select", "empno", "salary", "deptno0") + ), + unaryNode( + "DataSetAggregate", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "salary", "deptno"), + term("where", "IS NOT NULL(deptno)") + ), + term("groupBy", "deptno"), + term("select", "deptno", "AVG(salary) AS EXPR$0") ), - term("select", "empno") + term("where", "AND(=(deptno0, deptno), >(salary, EXPR$0))"), + term("join", "empno", "salary", "deptno0", "deptno", "EXPR$0"), + term("joinType", "InnerJoin") ), - term("union", "empno") + term("select", "empno") ), term("select", "SUM(empno) AS EXPR$0") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AggFunctionTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AggFunctionTestBase.scala index 458f80d7f49a5..bdd1df04894e7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AggFunctionTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AggFunctionTestBase.scala @@ -146,14 +146,31 @@ abstract class AggFunctionTestBase[T, ACC] { val accumulator = aggregator.createAccumulator() vals.foreach( v => - accumulateFunc.invoke(aggregator, accumulator.asInstanceOf[Object], v.asInstanceOf[Object]) + if (accumulateFunc.getParameterCount == 1) { + this.accumulateFunc.invoke(aggregator, accumulator.asInstanceOf[Object]) + } else { + this.accumulateFunc.invoke( + aggregator, + accumulator.asInstanceOf[Object], + v.asInstanceOf[Object]) + } ) accumulator } - private def retractVals(accumulator:ACC, vals: Seq[_]) = { + private def retractVals(accumulator: ACC, vals: Seq[_]) = { vals.foreach( - v => retractFunc.invoke(aggregator, accumulator.asInstanceOf[Object], v.asInstanceOf[Object]) + v => + if (retractFunc.getParameterCount == 1) { + this.retractFunc.invoke( + aggregator, + accumulator.asInstanceOf[Object]) + } else { + this.retractFunc.invoke( + aggregator, + accumulator.asInstanceOf[Object], + v.asInstanceOf[Object]) + } ) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala index b105ec0252c4e..ac0b7057c9f5d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.functions.aggfunctions.CountAggFunction import org.apache.flink.table.runtime.utils.TableProgramsCollectionTestBase import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode +import org.apache.flink.table.utils.NonMergableCount import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row import org.junit._ @@ -34,6 +35,7 @@ import org.junit.runner.RunWith import org.junit.runners.Parameterized import scala.collection.JavaConverters._ +import scala.collection.mutable @RunWith(classOf[Parameterized]) class AggregateITCase( @@ -262,6 +264,8 @@ class AggregateITCase( val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) + val myAgg = new NonMergableCount + tEnv.registerFunction("myAgg", myAgg) val sqlQuery = "SELECT avg(a), sum(a), count(b) " + "FROM MyTable where a = 4 group by a" @@ -272,6 +276,9 @@ class AggregateITCase( val sqlQuery3 = "SELECT avg(a), sum(a), count(b) " + "FROM MyTable" + val sqlQuery4 = "SELECT avg(a), sum(a), count(b), myAgg(b)" + + "FROM MyTable where a = 4" + val ds = env.fromElements( (1: Byte, 1: Short), (2: Byte, 2: Short)) @@ -282,6 +289,7 @@ class AggregateITCase( val result = tEnv.sqlQuery(sqlQuery) val result2 = tEnv.sqlQuery(sqlQuery2) val result3 = tEnv.sqlQuery(sqlQuery3) + val result4 = tEnv.sqlQuery(sqlQuery4) val results = result.toDataSet[Row].collect() val expected = Seq.empty @@ -289,11 +297,14 @@ class AggregateITCase( val expected2 = "null,null,0" val results3 = result3.toDataSet[Row].collect() val expected3 = "1,3,2" + val results4 = result4.toDataSet[Row].collect() + val expected4 = "null,null,0,0" assert(results.equals(expected), "Empty result is expected for grouped set, but actual: " + results) TestBaseUtils.compareResultAsText(results2.asJava, expected2) TestBaseUtils.compareResultAsText(results3.asJava, expected3) + TestBaseUtils.compareResultAsText(results4.asJava, expected4) } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala index e1348f600b52f..892e4f3084e2a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala @@ -28,7 +28,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.functions.aggfunctions.CountAggFunction import org.apache.flink.table.runtime.utils.TableProgramsCollectionTestBase import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.table.utils.Top10 +import org.apache.flink.table.utils.{NonMergableCount, Top10} import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row import org.junit._ @@ -36,6 +36,7 @@ import org.junit.runner.RunWith import org.junit.runners.Parameterized import scala.collection.JavaConverters._ +import scala.collection.mutable @RunWith(classOf[Parameterized]) class AggregationsITCase( @@ -266,12 +267,35 @@ class AggregationsITCase( .select('a.sum as 'd, 'b) .groupBy('b, 'd) .select('b) - val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n" val results = t.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test + def testAggregateEmptyDataSets(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val myAgg = new NonMergableCount + + val t1 = env.fromCollection(new mutable.MutableList[(Int, String)]).toTable(tEnv, 'a, 'b) + .select('a.sum, 'a.count) + val t2 = env.fromCollection(new mutable.MutableList[(Int, String)]).toTable(tEnv, 'a, 'b) + .select('a.sum, myAgg('b), 'a.count) + + val expected1 = "null,0" + val expected2 = "null,0,0" + + val results1 = t1.toDataSet[Row].collect() + val results2 = t2.toDataSet[Row].collect() + + TestBaseUtils.compareResultAsText(results1.asJava, expected1) + TestBaseUtils.compareResultAsText(results2.asJava, expected2) + + } + @Test def testGroupedAggregateWithLongKeys(): Unit = { // This uses very long keys to force serialized comparison. diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala index 7d4393c72be86..14c8461541d14 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedAggFunctions.scala @@ -45,8 +45,8 @@ class Top10 extends AggregateFunction[Array[JTuple2[JInt, JFloat]], Array[JTuple /** * Adds a new entry and count to the top 10 entries if necessary. * - * @param acc The current top 10 - * @param id The ID + * @param acc The current top 10 + * @param id The ID * @param value The value for the ID */ def accumulate(acc: Array[JTuple2[JInt, JFloat]], id: Int, value: Float) { @@ -91,7 +91,7 @@ class Top10 extends AggregateFunction[Array[JTuple2[JInt, JFloat]], Array[JTuple its: java.lang.Iterable[Array[JTuple2[JInt, JFloat]]]): Unit = { val it = its.iterator() - while(it.hasNext) { + while (it.hasNext) { val acc2 = it.next() var i = 0 @@ -124,3 +124,22 @@ class Top10 extends AggregateFunction[Array[JTuple2[JInt, JFloat]], Array[JTuple ObjectArrayTypeInfo.getInfoFor(new TupleTypeInfo[JTuple2[JInt, JFloat]](Types.INT, Types.FLOAT)) } } + +case class NonMergableCountAcc(var count: Long) + +class NonMergableCount extends AggregateFunction[Long, NonMergableCountAcc] { + + def accumulate(acc: NonMergableCountAcc, value: Any): Unit = { + if (null != value) { + acc.count = acc.count + 1 + } + } + + def resetAccumulator(acc: NonMergableCountAcc): Unit = { + acc.count = 0 + } + + override def createAccumulator(): NonMergableCountAcc = NonMergableCountAcc(0) + + override def getValue(acc: NonMergableCountAcc): Long = acc.count +} From aa5e4918049a59b965e7f7df385130ebb462c879 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=87=91=E7=AB=B9?= Date: Wed, 3 Jan 2018 23:13:49 +0800 Subject: [PATCH 200/367] [FLINK-8325] [table] Fix COUNT(*) and COUNT(1). This closes #5241. --- .../codegen/AggregationCodeGenerator.scala | 12 +- .../aggfunctions/CountAggFunction.scala | 15 +- .../datastream/DataStreamOverAggregate.scala | 14 + .../runtime/aggregate/AggregateUtil.scala | 373 +++++++++--------- .../aggfunctions/CountAggFunctionTest.scala | 2 +- .../CountAggFunctionWithNonParamTest.scala | 44 +++ .../runtime/stream/sql/OverWindowITCase.scala | 11 +- .../table/runtime/stream/sql/SqlITCase.scala | 90 +++++ 8 files changed, 366 insertions(+), 195 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionWithNonParamTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala index 32cbde2390c66..a9ec112e3b4fe 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala @@ -121,7 +121,7 @@ class AggregationCodeGenerator( // get parameter lists for aggregation functions val parametersCode = aggFields.map { inFields => - val fields = inFields.map { f => + val fields = inFields.filter(_ > -1).map { f => // index to constant if (f >= physicalInputTypes.length) { constantFields(f - physicalInputTypes.length) @@ -139,7 +139,7 @@ class AggregationCodeGenerator( val classes = UserDefinedFunctionUtils.typeInfoToClass(physicalInputTypes) val constantClasses = UserDefinedFunctionUtils.typeInfoToClass(constantTypes) val methodSignaturesList = aggFields.map { inFields => - inFields.map { f => + inFields.filter(_ > -1).map { f => // index to constant if (f >= physicalInputTypes.length) { constantClasses(f - physicalInputTypes.length) @@ -363,8 +363,8 @@ class AggregationCodeGenerator( | ${accTypes(i)} acc$i = (${accTypes(i)}) accs.getField($i); | ${genDataViewFieldSetter(s"acc$i", i)} | ${aggs(i)}.accumulate( - | acc$i, - | ${parametersCode(i)});""".stripMargin + | acc$i ${if (!parametersCode(i).isEmpty) "," else "" } ${parametersCode(i)}); + """.stripMargin } }.mkString("\n") @@ -387,8 +387,8 @@ class AggregationCodeGenerator( | ${accTypes(i)} acc$i = (${accTypes(i)}) accs.getField($i); | ${genDataViewFieldSetter(s"acc$i", i)} | ${aggs(i)}.retract( - | acc$i, - | ${parametersCode(i)});""".stripMargin + | acc$i ${if (!parametersCode(i).isEmpty) "," else "" } ${parametersCode(i)}); + """.stripMargin } }.mkString("\n") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/CountAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/CountAggFunction.scala index c94e05396c5fb..7147e17108473 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/CountAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/CountAggFunction.scala @@ -33,7 +33,20 @@ class CountAccumulator extends JTuple1[Long] { /** * built-in count aggregate function */ -class CountAggFunction extends AggregateFunction[JLong, CountAccumulator] { +class CountAggFunction + extends AggregateFunction[JLong, CountAccumulator] { + + // process argument is optimized by Calcite. + // For instance count(42) or count(*) will be optimized to count(). + def accumulate(acc: CountAccumulator): Unit = { + acc.f0 += 1L + } + + // process argument is optimized by Calcite. + // For instance count(42) or count(*) will be optimized to count(). + def retract(acc: CountAccumulator): Unit = { + acc.f0 -= 1L + } def accumulate(acc: CountAccumulator, value: Any): Unit = { if (value != null) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala index c41c1a99dd5db..635c7bc2d0322 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala @@ -145,6 +145,14 @@ class DataStreamOverAggregate( inputSchema.typeInfo, Some(constants)) + val constantTypes = constants.map(_.getType) + val fieldTypes = input.getRowType.getFieldList.asScala.map(_.getType) + val aggInTypes = fieldTypes ++ constantTypes + val aggInNames = aggInTypes.indices.map("f" + _) + + val aggregateInputType = + getCluster.getTypeFactory.createStructType(aggInTypes.asJava, aggInNames.asJava) + val timeType = schema.relDataType .getFieldList .get(orderKey.getFieldIndex) @@ -167,6 +175,7 @@ class DataStreamOverAggregate( generator, inputDS, rowTimeIdx, + aggregateInputType, isRowsClause = overWindow.isRows) } else if ( overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded && @@ -178,6 +187,7 @@ class DataStreamOverAggregate( generator, inputDS, rowTimeIdx, + aggregateInputType, isRowsClause = overWindow.isRows) } else { throw new TableException("OVER RANGE FOLLOWING windows are not supported yet.") @@ -189,6 +199,7 @@ class DataStreamOverAggregate( generator: AggregationCodeGenerator, inputDS: DataStream[CRow], rowTimeIdx: Option[Int], + aggregateInputType: RelDataType, isRowsClause: Boolean): DataStream[CRow] = { val overWindow: Group = logicWindow.groups.get(0) @@ -203,6 +214,7 @@ class DataStreamOverAggregate( val processFunction = AggregateUtil.createUnboundedOverProcessFunction( generator, namedAggregates, + aggregateInputType, inputSchema.relDataType, inputSchema.typeInfo, inputSchema.fieldTypeInfos, @@ -236,6 +248,7 @@ class DataStreamOverAggregate( generator: AggregationCodeGenerator, inputDS: DataStream[CRow], rowTimeIdx: Option[Int], + aggregateInputType: RelDataType, isRowsClause: Boolean): DataStream[CRow] = { val overWindow: Group = logicWindow.groups.get(0) @@ -253,6 +266,7 @@ class DataStreamOverAggregate( val processFunction = AggregateUtil.createBoundedOverProcessFunction( generator, namedAggregates, + aggregateInputType, inputSchema.relDataType, inputSchema.typeInfo, inputSchema.fieldTypeInfos, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index a652981f5f499..74b79e6ba1f2a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -62,6 +62,7 @@ object AggregateUtil { * * @param generator code generator instance * @param namedAggregates Physical calls to aggregate functions and their output field names + * @param aggregateInputType Physical type of the aggregate functions's input row. * @param inputType Physical type of the row. * @param inputTypeInfo Physical type information of the row. * @param inputFieldTypeInfo Physical type information of the row's fields. @@ -72,6 +73,7 @@ object AggregateUtil { private[flink] def createUnboundedOverProcessFunction( generator: AggregationCodeGenerator, namedAggregates: Seq[CalcitePair[AggregateCall, String]], + aggregateInputType: RelDataType, inputType: RelDataType, inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], @@ -84,7 +86,7 @@ object AggregateUtil { val (aggFields, aggregates, accTypes, accSpecs) = transformToAggregateFunctions( namedAggregates.map(_.getKey), - inputType, + aggregateInputType, needRetraction = false, isStateBackedDataViews = true) @@ -203,6 +205,7 @@ object AggregateUtil { * * @param generator code generator instance * @param namedAggregates Physical calls to aggregate functions and their output field names + * @param aggregateInputType Physical type of the aggregate functions's input row. * @param inputType Physical type of the row. * @param inputTypeInfo Physical type information of the row. * @param inputFieldTypeInfo Physical type information of the row's fields. @@ -214,6 +217,7 @@ object AggregateUtil { private[flink] def createBoundedOverProcessFunction( generator: AggregationCodeGenerator, namedAggregates: Seq[CalcitePair[AggregateCall, String]], + aggregateInputType: RelDataType, inputType: RelDataType, inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], @@ -227,7 +231,7 @@ object AggregateUtil { val (aggFields, aggregates, accTypes, accSpecs) = transformToAggregateFunctions( namedAggregates.map(_.getKey), - inputType, + aggregateInputType, needRetract, isStateBackedDataViews = true) @@ -827,7 +831,8 @@ object AggregateUtil { inputType: RelDataType, inputFieldTypeInfo: Seq[TypeInformation[_]], outputType: RelDataType, - groupings: Array[Int]): (Option[DataSetPreAggFunction], + groupings: Array[Int]): ( + Option[DataSetPreAggFunction], Option[TypeInformation[Row]], Either[DataSetAggFunction, DataSetFinalAggFunction]) = { @@ -1114,7 +1119,7 @@ object AggregateUtil { private def transformToAggregateFunctions( aggregateCalls: Seq[AggregateCall], - inputType: RelDataType, + aggregateInputType: RelDataType, needRetraction: Boolean, isStateBackedDataViews: Boolean = false) : (Array[Array[Int]], @@ -1130,235 +1135,239 @@ object AggregateUtil { // create aggregate function instances by function type and aggregate field data type. aggregateCalls.zipWithIndex.foreach { case (aggregateCall, index) => val argList: util.List[Integer] = aggregateCall.getArgList - if (argList.isEmpty) { - if (aggregateCall.getAggregation.isInstanceOf[SqlCountAggFunction]) { - aggFieldIndexes(index) = Array[Int](0) + + if (aggregateCall.getAggregation.isInstanceOf[SqlCountAggFunction]) { + aggregates(index) = new CountAggFunction + if (argList.isEmpty) { + aggFieldIndexes(index) = Array[Int](-1) } else { - throw new TableException("Aggregate fields should not be empty.") + aggFieldIndexes(index) = argList.asScala.map(i => i.intValue).toArray } } else { - aggFieldIndexes(index) = argList.asScala.map(i => i.intValue).toArray - } - val relDataType = inputType.getFieldList.get(aggFieldIndexes(index)(0)).getType - val sqlTypeName = relDataType.getSqlTypeName - aggregateCall.getAggregation match { - - case _: SqlSumAggFunction => - if (needRetraction) { - aggregates(index) = sqlTypeName match { - case TINYINT => - new ByteSumWithRetractAggFunction - case SMALLINT => - new ShortSumWithRetractAggFunction - case INTEGER => - new IntSumWithRetractAggFunction - case BIGINT => - new LongSumWithRetractAggFunction - case FLOAT => - new FloatSumWithRetractAggFunction - case DOUBLE => - new DoubleSumWithRetractAggFunction - case DECIMAL => - new DecimalSumWithRetractAggFunction - case sqlType: SqlTypeName => - throw new TableException(s"Sum aggregate does no support type: '$sqlType'") - } - } else { - aggregates(index) = sqlTypeName match { - case TINYINT => - new ByteSumAggFunction - case SMALLINT => - new ShortSumAggFunction - case INTEGER => - new IntSumAggFunction - case BIGINT => - new LongSumAggFunction - case FLOAT => - new FloatSumAggFunction - case DOUBLE => - new DoubleSumAggFunction - case DECIMAL => - new DecimalSumAggFunction - case sqlType: SqlTypeName => - throw new TableException(s"Sum aggregate does no support type: '$sqlType'") - } - } - - case _: SqlSumEmptyIsZeroAggFunction => - if (needRetraction) { - aggregates(index) = sqlTypeName match { - case TINYINT => - new ByteSum0WithRetractAggFunction - case SMALLINT => - new ShortSum0WithRetractAggFunction - case INTEGER => - new IntSum0WithRetractAggFunction - case BIGINT => - new LongSum0WithRetractAggFunction - case FLOAT => - new FloatSum0WithRetractAggFunction - case DOUBLE => - new DoubleSum0WithRetractAggFunction - case DECIMAL => - new DecimalSum0WithRetractAggFunction - case sqlType: SqlTypeName => - throw new TableException(s"Sum0 aggregate does no support type: '$sqlType'") - } - } else { - aggregates(index) = sqlTypeName match { - case TINYINT => - new ByteSum0AggFunction - case SMALLINT => - new ShortSum0AggFunction - case INTEGER => - new IntSum0AggFunction - case BIGINT => - new LongSum0AggFunction - case FLOAT => - new FloatSum0AggFunction - case DOUBLE => - new DoubleSum0AggFunction - case DECIMAL => - new DecimalSum0AggFunction - case sqlType: SqlTypeName => - throw new TableException(s"Sum0 aggregate does no support type: '$sqlType'") - } - } + if (argList.isEmpty) { + throw new TableException("Aggregate fields should not be empty.") + } else { + aggFieldIndexes(index) = argList.asScala.map(i => i.intValue).toArray + } - case _: SqlAvgAggFunction => - aggregates(index) = sqlTypeName match { - case TINYINT => - new ByteAvgAggFunction - case SMALLINT => - new ShortAvgAggFunction - case INTEGER => - new IntAvgAggFunction - case BIGINT => - new LongAvgAggFunction - case FLOAT => - new FloatAvgAggFunction - case DOUBLE => - new DoubleAvgAggFunction - case DECIMAL => - new DecimalAvgAggFunction - case sqlType: SqlTypeName => - throw new TableException(s"Avg aggregate does no support type: '$sqlType'") - } + val relDataType = aggregateInputType.getFieldList.get(aggFieldIndexes(index)(0)).getType + val sqlTypeName = relDataType.getSqlTypeName + aggregateCall.getAggregation match { - case sqlMinMaxFunction: SqlMinMaxAggFunction => - aggregates(index) = if (sqlMinMaxFunction.getKind == SqlKind.MIN) { + case _: SqlSumAggFunction => if (needRetraction) { - sqlTypeName match { + aggregates(index) = sqlTypeName match { case TINYINT => - new ByteMinWithRetractAggFunction + new ByteSumWithRetractAggFunction case SMALLINT => - new ShortMinWithRetractAggFunction + new ShortSumWithRetractAggFunction case INTEGER => - new IntMinWithRetractAggFunction + new IntSumWithRetractAggFunction case BIGINT => - new LongMinWithRetractAggFunction + new LongSumWithRetractAggFunction case FLOAT => - new FloatMinWithRetractAggFunction + new FloatSumWithRetractAggFunction case DOUBLE => - new DoubleMinWithRetractAggFunction + new DoubleSumWithRetractAggFunction case DECIMAL => - new DecimalMinWithRetractAggFunction - case BOOLEAN => - new BooleanMinWithRetractAggFunction - case VARCHAR | CHAR => - new StringMinWithRetractAggFunction + new DecimalSumWithRetractAggFunction case sqlType: SqlTypeName => - throw new TableException( - s"Min with retract aggregate does no support type: '$sqlType'") + throw new TableException(s"Sum aggregate does no support type: '$sqlType'") } } else { - sqlTypeName match { + aggregates(index) = sqlTypeName match { case TINYINT => - new ByteMinAggFunction + new ByteSumAggFunction case SMALLINT => - new ShortMinAggFunction + new ShortSumAggFunction case INTEGER => - new IntMinAggFunction + new IntSumAggFunction case BIGINT => - new LongMinAggFunction + new LongSumAggFunction case FLOAT => - new FloatMinAggFunction + new FloatSumAggFunction case DOUBLE => - new DoubleMinAggFunction + new DoubleSumAggFunction case DECIMAL => - new DecimalMinAggFunction - case BOOLEAN => - new BooleanMinAggFunction - case VARCHAR | CHAR => - new StringMinAggFunction + new DecimalSumAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Min aggregate does no support type: '$sqlType'") + throw new TableException(s"Sum aggregate does no support type: '$sqlType'") } } - } else { + + case _: SqlSumEmptyIsZeroAggFunction => if (needRetraction) { - sqlTypeName match { + aggregates(index) = sqlTypeName match { case TINYINT => - new ByteMaxWithRetractAggFunction + new ByteSum0WithRetractAggFunction case SMALLINT => - new ShortMaxWithRetractAggFunction + new ShortSum0WithRetractAggFunction case INTEGER => - new IntMaxWithRetractAggFunction + new IntSum0WithRetractAggFunction case BIGINT => - new LongMaxWithRetractAggFunction + new LongSum0WithRetractAggFunction case FLOAT => - new FloatMaxWithRetractAggFunction + new FloatSum0WithRetractAggFunction case DOUBLE => - new DoubleMaxWithRetractAggFunction + new DoubleSum0WithRetractAggFunction case DECIMAL => - new DecimalMaxWithRetractAggFunction - case BOOLEAN => - new BooleanMaxWithRetractAggFunction - case VARCHAR | CHAR => - new StringMaxWithRetractAggFunction + new DecimalSum0WithRetractAggFunction case sqlType: SqlTypeName => - throw new TableException( - s"Max with retract aggregate does no support type: '$sqlType'") + throw new TableException(s"Sum0 aggregate does no support type: '$sqlType'") } } else { - sqlTypeName match { + aggregates(index) = sqlTypeName match { case TINYINT => - new ByteMaxAggFunction + new ByteSum0AggFunction case SMALLINT => - new ShortMaxAggFunction + new ShortSum0AggFunction case INTEGER => - new IntMaxAggFunction + new IntSum0AggFunction case BIGINT => - new LongMaxAggFunction + new LongSum0AggFunction case FLOAT => - new FloatMaxAggFunction + new FloatSum0AggFunction case DOUBLE => - new DoubleMaxAggFunction + new DoubleSum0AggFunction case DECIMAL => - new DecimalMaxAggFunction - case BOOLEAN => - new BooleanMaxAggFunction - case VARCHAR | CHAR => - new StringMaxAggFunction + new DecimalSum0AggFunction case sqlType: SqlTypeName => - throw new TableException(s"Max aggregate does no support type: '$sqlType'") + throw new TableException(s"Sum0 aggregate does no support type: '$sqlType'") } } - } - case _: SqlCountAggFunction => - aggregates(index) = new CountAggFunction + case _: SqlAvgAggFunction => + aggregates(index) = sqlTypeName match { + case TINYINT => + new ByteAvgAggFunction + case SMALLINT => + new ShortAvgAggFunction + case INTEGER => + new IntAvgAggFunction + case BIGINT => + new LongAvgAggFunction + case FLOAT => + new FloatAvgAggFunction + case DOUBLE => + new DoubleAvgAggFunction + case DECIMAL => + new DecimalAvgAggFunction + case sqlType: SqlTypeName => + throw new TableException(s"Avg aggregate does no support type: '$sqlType'") + } + + case sqlMinMaxFunction: SqlMinMaxAggFunction => + aggregates(index) = if (sqlMinMaxFunction.getKind == SqlKind.MIN) { + if (needRetraction) { + sqlTypeName match { + case TINYINT => + new ByteMinWithRetractAggFunction + case SMALLINT => + new ShortMinWithRetractAggFunction + case INTEGER => + new IntMinWithRetractAggFunction + case BIGINT => + new LongMinWithRetractAggFunction + case FLOAT => + new FloatMinWithRetractAggFunction + case DOUBLE => + new DoubleMinWithRetractAggFunction + case DECIMAL => + new DecimalMinWithRetractAggFunction + case BOOLEAN => + new BooleanMinWithRetractAggFunction + case VARCHAR | CHAR => + new StringMinWithRetractAggFunction + case sqlType: SqlTypeName => + throw new TableException( + s"Min with retract aggregate does no support type: '$sqlType'") + } + } else { + sqlTypeName match { + case TINYINT => + new ByteMinAggFunction + case SMALLINT => + new ShortMinAggFunction + case INTEGER => + new IntMinAggFunction + case BIGINT => + new LongMinAggFunction + case FLOAT => + new FloatMinAggFunction + case DOUBLE => + new DoubleMinAggFunction + case DECIMAL => + new DecimalMinAggFunction + case BOOLEAN => + new BooleanMinAggFunction + case VARCHAR | CHAR => + new StringMinAggFunction + case sqlType: SqlTypeName => + throw new TableException(s"Min aggregate does no support type: '$sqlType'") + } + } + } else { + if (needRetraction) { + sqlTypeName match { + case TINYINT => + new ByteMaxWithRetractAggFunction + case SMALLINT => + new ShortMaxWithRetractAggFunction + case INTEGER => + new IntMaxWithRetractAggFunction + case BIGINT => + new LongMaxWithRetractAggFunction + case FLOAT => + new FloatMaxWithRetractAggFunction + case DOUBLE => + new DoubleMaxWithRetractAggFunction + case DECIMAL => + new DecimalMaxWithRetractAggFunction + case BOOLEAN => + new BooleanMaxWithRetractAggFunction + case VARCHAR | CHAR => + new StringMaxWithRetractAggFunction + case sqlType: SqlTypeName => + throw new TableException( + s"Max with retract aggregate does no support type: '$sqlType'") + } + } else { + sqlTypeName match { + case TINYINT => + new ByteMaxAggFunction + case SMALLINT => + new ShortMaxAggFunction + case INTEGER => + new IntMaxAggFunction + case BIGINT => + new LongMaxAggFunction + case FLOAT => + new FloatMaxAggFunction + case DOUBLE => + new DoubleMaxAggFunction + case DECIMAL => + new DecimalMaxAggFunction + case BOOLEAN => + new BooleanMaxAggFunction + case VARCHAR | CHAR => + new StringMaxAggFunction + case sqlType: SqlTypeName => + throw new TableException(s"Max aggregate does no support type: '$sqlType'") + } + } + } - case collect: SqlAggFunction if collect.getKind == SqlKind.COLLECT => - aggregates(index) = new CollectAggFunction(FlinkTypeFactory.toTypeInfo(relDataType)) - accTypes(index) = aggregates(index).getAccumulatorType + case collect: SqlAggFunction if collect.getKind == SqlKind.COLLECT => + aggregates(index) = new CollectAggFunction(FlinkTypeFactory.toTypeInfo(relDataType)) + accTypes(index) = aggregates(index).getAccumulatorType - case udagg: AggSqlFunction => - aggregates(index) = udagg.getFunction - accTypes(index) = udagg.accType + case udagg: AggSqlFunction => + aggregates(index) = udagg.getFunction + accTypes(index) = udagg.accType - case unSupported: SqlAggFunction => - throw new TableException(s"unsupported Function: '${unSupported.getName}'") + case unSupported: SqlAggFunction => + throw new TableException(s"unsupported Function: '${unSupported.getName}'") + } } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionTest.scala index 87aaff9f1438f..867435ba9a3a3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionTest.scala @@ -35,7 +35,7 @@ class CountAggFunctionTest extends AggFunctionTestBase[JLong, CountAccumulator] override def expectedResults: Seq[JLong] = Seq(6L, 0L) - override def aggregator: AggregateFunction[JLong, CountAccumulator] = new CountAggFunction() + override def aggregator: AggregateFunction[JLong, CountAccumulator] = new CountAggFunction override def retractFunc = aggregator.getClass.getMethod("retract", accType, classOf[Any]) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionWithNonParamTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionWithNonParamTest.scala new file mode 100644 index 0000000000000..da98c57abf9ad --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/CountAggFunctionWithNonParamTest.scala @@ -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. + */ + +package org.apache.flink.table.runtime.aggfunctions + +import java.lang.reflect.Method +import java.lang.{Long => JLong} + +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.table.functions.aggfunctions.{CountAccumulator, CountAggFunction} + +/** + * Test case for built-in count aggregate function with non-nullable parameter. + */ +class CountAggFunctionWithNonParamTest extends AggFunctionTestBase[JLong, CountAccumulator] { + + override def inputValueSets: Seq[Seq[_]] = Seq( + Seq("a", "b", null, "c", null, "d", "e", null, "f"), + Seq(null, null, null, null, null, null) + ) + + override def expectedResults: Seq[JLong] = Seq(9L, 6L) + + override def aggregator: AggregateFunction[JLong, CountAccumulator] = new CountAggFunction + + override def retractFunc = aggregator.getClass.getMethod("retract", accType) + + override def accumulateFunc: Method = aggregator.getClass.getMethod("accumulate", accType) +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala index 9bfdc4cac2f88..411cbb1961a04 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala @@ -152,7 +152,8 @@ class OverWindowITCase extends StreamingWithStateTestBase { val sqlQuery = "SELECT " + "c, " + "count(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding), " + - "sum(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding) " + + "sum(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding), " + + "sum(2) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding) " + "from T1" val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] @@ -160,8 +161,8 @@ class OverWindowITCase extends StreamingWithStateTestBase { env.execute() val expected = List( - "Hello World,1,7", "Hello World,2,15", "Hello World,3,35", - "Hello,1,1", "Hello,2,3", "Hello,3,6", "Hello,4,10", "Hello,5,15", "Hello,6,21") + "Hello World,1,7,2", "Hello World,2,15,4", "Hello World,3,35,6", + "Hello,1,1,2", "Hello,2,3,4", "Hello,3,6,6", "Hello,4,10,8", "Hello,5,15,10", "Hello,6,21,12") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -179,7 +180,7 @@ class OverWindowITCase extends StreamingWithStateTestBase { val sqlQuery = "SELECT c, cnt1 from " + "(SELECT " + "c, " + - "count(a) " + + "count(1) " + " OVER (PARTITION BY c ORDER BY proctime ROWS BETWEEN UNBOUNDED preceding AND CURRENT ROW) " + "as cnt1 from T1)" @@ -366,7 +367,7 @@ class OverWindowITCase extends StreamingWithStateTestBase { " c, a, " + " LTCNT(a, CAST('4' AS BIGINT)) " + " OVER (PARTITION BY c ORDER BY rowtime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), " + - " COUNT(a) " + + " COUNT(1) " + " OVER (PARTITION BY c ORDER BY rowtime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), " + " SUM(a) " + " OVER (PARTITION BY c ORDER BY rowtime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) " + diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 18b45a36ea26b..6556450c6f768 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -22,10 +22,13 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.TimeCharacteristic +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.table.api.{TableEnvironment, Types} import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.utils.SplitUDF +import org.apache.flink.table.runtime.stream.sql.SqlITCase.TimestampAndWatermarkWithOffset import org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeSourceFunction import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase} import org.apache.flink.types.Row @@ -33,8 +36,75 @@ import org.apache.flink.table.utils.MemoryTableSinkUtil import org.junit.Assert._ import org.junit._ +import scala.collection.mutable + class SqlITCase extends StreamingWithStateTestBase { + val data = List( + (1000L, "1", "Hello"), + (2000L, "2", "Hello"), + (3000L, null.asInstanceOf[String], "Hello"), + (4000L, "4", "Hello"), + (5000L, null.asInstanceOf[String], "Hello"), + (6000L, "6", "Hello"), + (7000L, "7", "Hello World"), + (8000L, "8", "Hello World"), + (20000L, "20", "Hello World")) + + @Test + def testRowTimeTumbleWindow(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.testResults = mutable.MutableList() + StreamITCase.clear + env.setParallelism(1) + + val stream = env + .fromCollection(data) + .assignTimestampsAndWatermarks( + new TimestampAndWatermarkWithOffset[(Long, String, String)](0L)) + val table = stream.toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime) + + tEnv.registerTable("T1", table) + + val sqlQuery = "SELECT c, COUNT(*), COUNT(1), COUNT(b) FROM T1 " + + "GROUP BY TUMBLE(rowtime, interval '5' SECOND), c" + + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = List("Hello World,2,2,2", "Hello World,1,1,1", "Hello,4,4,3", "Hello,2,2,1") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + @Test + def testNonWindowedCount(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.retractedResults = mutable.ArrayBuffer() + StreamITCase.clear + + env.setParallelism(1) + + val stream = env.fromCollection(data) + val table = stream.toTable(tEnv, 'a, 'b, 'c) + + tEnv.registerTable("T1", table) + + val sqlQuery = "SELECT c, COUNT(*), COUNT(1), COUNT(b) FROM T1 GROUP BY c" + + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] + result.addSink(new StreamITCase.RetractingSink) + env.execute() + + val expected = List("Hello World,3,3,3", "Hello,6,6,4") + assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) + } + /** test row stream registered table **/ @Test def testRowRegister(): Unit = { @@ -517,3 +587,23 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } } + +object SqlITCase { + + class TimestampAndWatermarkWithOffset[T <: Product]( + offset: Long) extends AssignerWithPunctuatedWatermarks[T] { + + override def checkAndGetNextWatermark( + lastElement: T, + extractedTimestamp: Long): Watermark = { + new Watermark(extractedTimestamp - offset) + } + + override def extractTimestamp( + element: T, + previousElementTimestamp: Long): Long = { + element.productElement(0).asInstanceOf[Long] + } + } + +} From df0526172334ec619ee9a5a70006f4ad2f3e2167 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Fri, 12 Jan 2018 19:56:42 +0100 Subject: [PATCH 201/367] [FLINK-8411] [State Backends] HeapListState#add(null) will wipe out entire list state --- .../flink/contrib/streaming/state/RocksDBListState.java | 4 ++++ .../org/apache/flink/api/common/state/AppendingState.java | 2 ++ .../org/apache/flink/runtime/state/heap/HeapListState.java | 5 ++--- .../org/apache/flink/runtime/state/StateBackendTestBase.java | 3 +++ 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java index f8ed2447c4a7d..ccd7e647c1785 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java @@ -107,6 +107,10 @@ public Iterable get() { @Override public void add(V value) throws IOException { + if (value == null) { + return; + } + try { writeCurrentKeyWithGroupAndNamespace(); byte[] key = keySerializationStream.toByteArray(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java index dd070a93f7df6..c17a849786651 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java @@ -63,6 +63,8 @@ public interface AppendingState extends State { * Updates the operator state accessible by {@link #get()} by adding the given value * to the list of values. The next time {@link #get()} is called (for the same state * partition) the returned state will represent the updated list. + * + * If `null` is passed in, the state value will remain unchanged * * @param value * The new value for the state. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java index d3f67f0a82634..737897c01afb4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java @@ -66,13 +66,12 @@ public Iterable get() { @Override public void add(V value) { - final N namespace = currentNamespace; - if (value == null) { - clear(); return; } + final N namespace = currentNamespace; + final StateTable> map = stateTable; ArrayList list = map.get(namespace); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index 0ac607f625d3f..3b638465cbd6b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -1288,6 +1288,8 @@ public void testListStateAddAndGet() throws Exception { keyedBackend.setCurrentKey("abc"); assertNull(state.get()); + state.add(null); + assertNull(state.get()); keyedBackend.setCurrentKey("def"); assertNull(state.get()); @@ -1312,6 +1314,7 @@ public void testListStateAddAndGet() throws Exception { keyedBackend.setCurrentKey("g"); state.add(3L); state.add(2L); + state.add(null); state.add(1L); keyedBackend.setCurrentKey("def"); From 20be204b96edd5c92683013a4c5af9ea4096acca Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 22 Jan 2018 13:29:34 +0100 Subject: [PATCH 202/367] [FLINK-8473][webUI] Improve error behavior of JarListHandler This closes #5331. --- .../webmonitor/HttpRequestHandler.java | 4 +++ .../runtime/webmonitor/WebRuntimeMonitor.java | 32 +++++++++++++++---- .../webmonitor/handlers/JarActionHandler.java | 11 +++++++ .../webmonitor/handlers/JarDeleteHandler.java | 13 ++++++++ .../webmonitor/handlers/JarListHandler.java | 11 +++++++ 5 files changed, 64 insertions(+), 7 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java index f8b51d7e815d3..a0fda9d1b75e7 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java @@ -133,6 +133,10 @@ else if (currentDecoder != null && msg instanceof HttpContent) { String name = file.getFilename(); File target = new File(tmpDir, UUID.randomUUID() + "_" + name); + if (!tmpDir.exists()) { + WebRuntimeMonitor.logExternalUploadDirDeletion(tmpDir); + WebRuntimeMonitor.checkAndCreateUploadDir(tmpDir); + } file.renameTo(target); QueryStringEncoder encoder = new QueryStringEncoder(currentRequestPath); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java index fe5f10693d425..e7bb157a206d2 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java @@ -188,13 +188,7 @@ public WebRuntimeMonitor( if (webSubmitAllow) { // create storage for uploads this.uploadDir = getUploadDir(config); - // the upload directory should either 1. exist and writable or 2. can be created and writable - if (!(uploadDir.exists() && uploadDir.canWrite()) && !(uploadDir.mkdirs() && uploadDir.canWrite())) { - throw new IOException( - String.format("Jar upload directory %s cannot be created or is not writable.", - uploadDir.getAbsolutePath())); - } - LOG.info("Using directory {} for web frontend JAR file uploads", uploadDir); + checkAndCreateUploadDir(uploadDir); } else { this.uploadDir = null; @@ -578,4 +572,28 @@ private File getUploadDir(Configuration configuration) { boolean uploadDirSpecified = configuration.contains(WebOptions.UPLOAD_DIR); return uploadDirSpecified ? baseDir : new File(baseDir, "flink-web-" + UUID.randomUUID()); } + + public static void logExternalUploadDirDeletion(File uploadDir) { + LOG.warn("Jar storage directory {} has been deleted externally. Previously uploaded jars are no longer available.", uploadDir.getAbsolutePath()); + } + + /** + * Checks whether the given directory exists and is writable. If it doesn't exist this method will attempt to create + * it. + * + * @param uploadDir directory to check + * @throws IOException if the directory does not exist and cannot be created, or if the directory isn't writable + */ + public static synchronized void checkAndCreateUploadDir(File uploadDir) throws IOException { + if (uploadDir.exists() && uploadDir.canWrite()) { + LOG.info("Using directory {} for web frontend JAR file uploads.", uploadDir); + } else if (uploadDir.mkdirs() && uploadDir.canWrite()) { + LOG.info("Created directory {} for web frontend JAR file uploads.", uploadDir); + } else { + LOG.warn("Jar upload directory {} cannot be created or is not writable.", uploadDir.getAbsolutePath()); + throw new IOException( + String.format("Jar upload directory %s cannot be created or is not writable.", + uploadDir.getAbsolutePath())); + } + } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java index c601a8d1dd253..eea3a873e0f5f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java @@ -37,11 +37,13 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler; import org.apache.flink.runtime.rest.handler.legacy.JsonFactory; +import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import java.io.File; +import java.io.IOException; import java.io.StringWriter; import java.net.URISyntaxException; import java.net.URL; @@ -66,6 +68,15 @@ protected Tuple2 getJobGraphAndClassLoader(JarActionHandl // generate the graph JobGraph graph = null; + if (!jarDir.exists()) { + WebRuntimeMonitor.logExternalUploadDirDeletion(jarDir); + try { + WebRuntimeMonitor.checkAndCreateUploadDir(jarDir); + } catch (IOException ioe) { + // the following code will throw an exception since the jar can't be found + } + } + PackagedProgram program = new PackagedProgram( new File(jarDir, config.getJarFile()), config.getEntryClass(), diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java index fb7fb4085a692..c1a5f602daf8e 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java @@ -21,12 +21,14 @@ import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler; import org.apache.flink.runtime.rest.handler.legacy.JsonFactory; +import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor; import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import java.io.File; import java.io.FilenameFilter; +import java.io.IOException; import java.io.StringWriter; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -64,6 +66,17 @@ public boolean accept(File dir, String name) { return name.equals(file); } }); + + if (list == null) { + WebRuntimeMonitor.logExternalUploadDirDeletion(jarDir); + try { + WebRuntimeMonitor.checkAndCreateUploadDir(jarDir); + } catch (IOException ioe) { + // entire directory doesn't exist anymore, continue as if deletion succeeded + } + list = new File[0]; + } + boolean success = false; for (File f: list) { // although next to impossible for multiple files, we still delete them. diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java index 59a436f3b7cc6..2b56ecd81a897 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler; import org.apache.flink.runtime.rest.handler.legacy.JsonFactory; import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler; +import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor; import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; @@ -77,6 +78,16 @@ public boolean accept(File dir, String name) { } }); + if (list == null) { + WebRuntimeMonitor.logExternalUploadDirDeletion(jarDir); + try { + WebRuntimeMonitor.checkAndCreateUploadDir(jarDir); + } catch (IOException ioe) { + // re-throwing an exception here breaks the UI + } + list = new File[0]; + } + // last modified ascending order Arrays.sort(list, (f1, f2) -> Long.compare(f2.lastModified(), f1.lastModified())); From e4d7ea264125322a68ea05ca141eef18a3016846 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 12 Dec 2017 14:04:34 +0100 Subject: [PATCH 203/367] [FLINK-8079][tests] Stop end-to-end test execution after first failure This closes #5156. --- tools/travis_mvn_watchdog.sh | 71 +++++++++++++++++++++--------------- 1 file changed, 41 insertions(+), 30 deletions(-) diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 42f7ae3355dd3..5edeb89ba7315 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -519,8 +519,9 @@ case $TEST in (connectors) if [ $EXIT_CODE == 0 ]; then check_shaded_artifacts_s3_fs hadoop + EXIT_CODE=$(($EXIT_CODE+$?)) check_shaded_artifacts_s3_fs presto - EXIT_CODE=$? + EXIT_CODE=$(($EXIT_CODE+$?)) else echo "==============================================================================" echo "Compilation/test failure detected, skipping shaded dependency check." @@ -543,35 +544,45 @@ case $TEST in printf "Running end-to-end tests\n" printf "==============================================================================\n" - printf "\n==============================================================================\n" - printf "Running Wordcount end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_batch_wordcount.sh - EXIT_CODE=$(($EXIT_CODE+$?)) - - printf "\n==============================================================================\n" - printf "Running Kafka end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_streaming_kafka010.sh - EXIT_CODE=$(($EXIT_CODE+$?)) - - printf "\n==============================================================================\n" - printf "Running class loading end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_streaming_classloader.sh - EXIT_CODE=$(($EXIT_CODE+$?)) - - printf "\n==============================================================================\n" - printf "Running Shaded Hadoop S3A end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh - EXIT_CODE=$(($EXIT_CODE+$?)) - - printf "\n==============================================================================\n" - printf "Running Shaded Presto S3 end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_shaded_presto_s3.sh - EXIT_CODE=$(($EXIT_CODE+$?)) + if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Wordcount end-to-end test\n" + printf "==============================================================================\n" + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_batch_wordcount.sh + EXIT_CODE=$? + fi + + if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Kafka end-to-end test\n" + printf "==============================================================================\n" + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_streaming_kafka010.sh + EXIT_CODE=$? + fi + + if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running class loading end-to-end test\n" + printf "==============================================================================\n" + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_streaming_classloader.sh + EXIT_CODE=$? + fi + + if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Shaded Hadoop S3A end-to-end test\n" + printf "==============================================================================\n" + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh + EXIT_CODE=$? + fi + + if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Shaded Presto S3 end-to-end test\n" + printf "==============================================================================\n" + FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_shaded_presto_s3.sh + EXIT_CODE=$? + fi else printf "\n==============================================================================\n" printf "Previous build failure detected, skipping end-to-end tests.\n" From 77df20bc50322166a517cb293abfbf35320586a1 Mon Sep 17 00:00:00 2001 From: zhangminglei Date: Mon, 15 Jan 2018 17:37:41 +0800 Subject: [PATCH 204/367] [FLINK-8433] [doc] Remove ununsed CheckpointedRestoring interface --- docs/dev/stream/state/state.md | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index 9956998ed6aae..ce1ce825f4d52 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -331,8 +331,7 @@ the basic even-split redistribution list state: {% highlight java %} public class BufferingSink implements SinkFunction>, - CheckpointedFunction, - CheckpointedRestoring>> { + CheckpointedFunction { private final int threshold; @@ -379,12 +378,6 @@ public class BufferingSink } } } - - @Override - public void restoreState(ArrayList> state) throws Exception { - // this is from the CheckpointedRestoring interface. - this.bufferedElements.addAll(state); - } } {% endhighlight %}

From c701a335bef4caa9745498d1505e6ef46942d764 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Wed, 10 Jan 2018 14:07:57 -0500 Subject: [PATCH 205/367] [hotfix] [build] Print cache info Print the size of the Maven cache copied for each TravisCI job. --- tools/travis_mvn_watchdog.sh | 41 ++++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 5edeb89ba7315..7c27392f9b5f6 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -186,6 +186,28 @@ ARTIFACTS_FILE=${TRAVIS_JOB_NUMBER}.tar.gz # FUNCTIONS # ============================================================================= +print_system_info() { + FOLD_ESCAPE="\x0d\x1b" + COLOR_ON="\x5b\x30\x4b\x1b\x5b\x33\x33\x3b\x31\x6d" + COLOR_OFF="\x1b\x5b\x30\x6d" + + echo -e "travis_fold:start:cpu_info${FOLD_ESCAPE}${COLOR_ON}CPU information${COLOR_OFF}" + lscpu + echo -en "travis_fold:end:cpu_info${FOLD_ESCAPE}" + + echo -e "travis_fold:start:mem_info${FOLD_ESCAPE}${COLOR_ON}Memory information${COLOR_OFF}" + cat /proc/meminfo + echo -en "travis_fold:end:mem_info${FOLD_ESCAPE}" + + echo -e "travis_fold:start:disk_info${FOLD_ESCAPE}${COLOR_ON}Disk information${COLOR_OFF}" + df -hH + echo -en "travis_fold:end:disk_info${FOLD_ESCAPE}" + + echo -e "travis_fold:start:cache_info${FOLD_ESCAPE}${COLOR_ON}Cache information${COLOR_OFF}" + du -s --si $HOME/.m2 + echo -en "travis_fold:end:cache_info${FOLD_ESCAPE}" +} + upload_artifacts_s3() { echo "PRODUCED build artifacts." @@ -436,24 +458,7 @@ WD_PID=$! echo "STARTED watchdog (${WD_PID})." - -# Print and fold CPU, memory, and filesystem info -FOLD_ESCAPE="\x0d\x1b" -COLOR_ON="\x5b\x30\x4b\x1b\x5b\x33\x33\x3b\x31\x6d" -COLOR_OFF="\x1b\x5b\x30\x6d" - -echo -e "travis_fold:start:cpu_info${FOLD_ESCAPE}${COLOR_ON}CPU information${COLOR_OFF}" -lscpu -echo -en "travis_fold:end:cpu_info${FOLD_ESCAPE}" - -echo -e "travis_fold:start:mem_info${FOLD_ESCAPE}${COLOR_ON}Memory information${COLOR_OFF}" -cat /proc/meminfo -echo -en "travis_fold:end:mem_info${FOLD_ESCAPE}" - -echo -e "travis_fold:start:disk_info${FOLD_ESCAPE}${COLOR_ON}Disk information${COLOR_OFF}" -df -hH -echo -en "travis_fold:end:disk_info${FOLD_ESCAPE}" - +print_system_info # Make sure to be in project root cd $HERE/../ From da8446ee5d1f305ef633e95908d2ff6e14a31206 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 23 Jan 2018 21:01:36 +0100 Subject: [PATCH 206/367] [hotfix] [build] Fix diverging snappy versions. This removes the snappy dependency from flink-core, which is no longer needed since we do not have an Avro dependency in flink-core any more. --- flink-core/pom.xml | 6 ------ flink-runtime/pom.xml | 1 - pom.xml | 2 +- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/flink-core/pom.xml b/flink-core/pom.xml index f82e1ed65fb21..674c8bd5d1582 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -80,12 +80,6 @@ under the License. - - - org.xerial.snappy - snappy-java - - diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 7be858fa468e9..abbe47f8975ec 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -168,7 +168,6 @@ under the License. org.xerial.snappy snappy-java - 1.1.4 From 15cb057bffd32ba8a853b46b207a5b7ea6bba430 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 23 Jan 2018 19:58:10 +0100 Subject: [PATCH 207/367] [FLINK-8499] [core] Force Kryo to be parent-first loaded. --- .../flink/configuration/CoreOptions.java | 2 +- .../avro/AvroKryoClassloadingTest.java | 89 +++++++++++++++++++ .../core/testutils/FilteredClassLoader.java | 60 +++++++++++++ 3 files changed, 150 insertions(+), 1 deletion(-) create mode 100644 flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoClassloadingTest.java create mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/FilteredClassLoader.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index c48e5ef12bf86..27f39a4e816f0 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -36,7 +36,7 @@ public class CoreOptions { public static final ConfigOption ALWAYS_PARENT_FIRST_LOADER = ConfigOptions .key("classloader.parent-first-patterns") - .defaultValue("java.;scala.;org.apache.flink.;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); + .defaultValue("java.;scala.;org.apache.flink.;com.esotericsoftware.kryo;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); // ------------------------------------------------------------------------ diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoClassloadingTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoClassloadingTest.java new file mode 100644 index 0000000000000..6eaca15240efd --- /dev/null +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoClassloadingTest.java @@ -0,0 +1,89 @@ +/* + * 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.flink.formats.avro; + +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.core.testutils.FilteredClassLoader; +import org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; + +import com.esotericsoftware.kryo.Kryo; +import org.junit.Test; + +import java.lang.reflect.Method; +import java.net.URL; +import java.util.LinkedHashMap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * This test makes sure that reversed classloading works for the Avro/Kryo integration when + * Kryo is in the application jar file. + * + *

If Kryo is not loaded consistently through the same classloader (parent-first), the following + * error happens: + * + *

+ * java.lang.VerifyError: Bad type on operand stack
+ * Exception Details:
+ *   Location:
+ *  org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.addAvroGenericDataArrayRegistration(Ljava/util/LinkedHashMap;)V @23: invokespecial
+ *   Reason:
+ *     Type 'org/apache/flink/api/java/typeutils/runtime/kryo/Serializers$SpecificInstanceCollectionSerializerForArrayList' (current frame, stack[7]) is not assignable to 'com/esotericsoftware/kryo/Serializer'
+ *   Current Frame:
+ *     bci: @23
+ *     flags: { }
+ *     locals: { 'org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils', 'java/util/LinkedHashMap' }
+ *     stack: { 'java/util/LinkedHashMap', 'java/lang/String', uninitialized 6, uninitialized 6, 'java/lang/Class', uninitialized 12, uninitialized 12, 'org/apache/flink/api/java/typeutils/runtime/kryo/Serializers$SpecificInstanceCollectionSerializerForArrayList' }
+ *   Bytecode:
+ *     0x0000000: 2b12 05b6 000b bb00 0c59 1205 bb00 0d59
+ *     0x0000010: bb00 0659 b700 0eb7 000f b700 10b6 0011
+ *     0x0000020: 57b1
+ * 
+ */ +public class AvroKryoClassloadingTest { + + @Test + public void testKryoInChildClasspath() throws Exception { + final Class avroClass = AvroKryoSerializerUtils.class; + + final URL avroLocation = avroClass.getProtectionDomain().getCodeSource().getLocation(); + final URL kryoLocation = Kryo.class.getProtectionDomain().getCodeSource().getLocation(); + + final ClassLoader parentClassLoader = new FilteredClassLoader( + avroClass.getClassLoader(), AvroKryoSerializerUtils.class.getName()); + + final ClassLoader userAppClassLoader = FlinkUserCodeClassLoaders.childFirst( + new URL[] { avroLocation, kryoLocation }, + parentClassLoader, + CoreOptions.ALWAYS_PARENT_FIRST_LOADER.defaultValue().split(";")); + + final Class userLoadedAvroClass = Class.forName(avroClass.getName(), false, userAppClassLoader); + assertNotEquals(avroClass, userLoadedAvroClass); + + // call the 'addAvroGenericDataArrayRegistration(...)' method + final Method m = userLoadedAvroClass.getMethod("addAvroGenericDataArrayRegistration", LinkedHashMap.class); + + final LinkedHashMap map = new LinkedHashMap<>(); + m.invoke(userLoadedAvroClass.newInstance(), map); + + assertEquals(1, map.size()); + } +} diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/FilteredClassLoader.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/FilteredClassLoader.java new file mode 100644 index 0000000000000..f04393bb788cd --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/FilteredClassLoader.java @@ -0,0 +1,60 @@ +/* + * 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.flink.core.testutils; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Objects; + +/** + * A ClassLoader that filters out certain classes (by name) and throws a ClassNotFoundException + * when they should be loaded. + * + *

This utility is useful when trying to eliminate certain classes from a class loader + * force loading them through another class loader. + */ +public class FilteredClassLoader extends ClassLoader { + + /** The set of class names for the filtered classes. */ + private final HashSet filteredClassNames; + + /** + * Creates a new filtered classloader. + * + * @param delegate The class loader that is filtered by this classloader. + * @param filteredClassNames The class names to filter out. + */ + public FilteredClassLoader(ClassLoader delegate, String... filteredClassNames) { + super(Objects.requireNonNull(delegate)); + + this.filteredClassNames = new HashSet<>(Arrays.asList(filteredClassNames)); + } + + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + synchronized (this) { + if (filteredClassNames.contains(name)) { + throw new ClassNotFoundException(name); + } + else { + return super.loadClass(name, resolve); + } + } + } +} From a8ea1698f44614ec9f35d9307b3294e4c165fcf0 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 24 Jan 2018 10:02:00 +0100 Subject: [PATCH 208/367] [hotfix] [build] Converge Kryo dependency Previously, the Kryo dependency was diverging between the flink-core dependency and the chill dependency. [INFO] +- org.apache.flink:flink-java:jar:1.4.0:compile [INFO] | +- org.apache.flink:flink-core:jar:1.4.0:compile [INFO] | | +- com.esotericsoftware.kryo:kryo:jar:2.24.0:compile .... [INFO] +- org.apache.flink:flink-streaming-java_2.11:jar:1.4.0:compile [INFO] | +- (org.apache.flink:flink-core:jar:1.4.0:compile - omitted for duplicate) [INFO] | +- org.apache.flink:flink-runtime_2.11:jar:1.4.0:compile [INFO] | | +- com.twitter:chill_2.11:jar:0.7.4:compile [INFO] | | | +- com.twitter:chill-java:jar:0.7.4:compile [INFO] | | | | \- (com.esotericsoftware.kryo:kryo:jar:2.21:compile - omitted for conflict with 2.24.0) [INFO] | | | \- (com.esotericsoftware.kryo:kryo:jar:2.21:compile - omitted for conflict with 2.24.0) --- flink-runtime/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index abbe47f8975ec..a8c583bbcd686 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -180,6 +180,13 @@ under the License. com.twitter chill_${scala.binary.version} ${chill.version} + + + + com.esotericsoftware.kryo + kryo + + From 82f3957811d2e5bcefabaa42326d3d4476e45df0 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 2 Feb 2018 15:53:13 +0100 Subject: [PATCH 253/367] [FLINK-8275] [security, yarn] Remove test-specific code path in YarnTaskManagerRunner MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, the YarnTaskManagerRunner contained a code path that exists for the sole purpose of injecting mock runners. Having code paths just to utilize tests in production code is in general a bad idea. This commit fixes this be making YarnTaskManagerRunner a factory-like class, which creates a Runner that contains all the runner’s properties, such as configuration. Unit tests can than test against the contained configuration in the created Runner to validate that everything is configured properly. This closes #5172. --- .../yarn/TestingYarnTaskManagerRunner.java | 12 ++- .../flink/yarn/TestingYarnTaskManager.scala | 14 +++- .../yarn/YarnApplicationMasterRunner.java | 8 +- ...java => YarnTaskManagerRunnerFactory.java} | 83 ++++++++++++------- .../apache/flink/yarn/YarnTaskManager.scala | 30 +++++-- ... => YarnTaskManagerRunnerFactoryTest.java} | 50 +++++------ 6 files changed, 128 insertions(+), 69 deletions(-) rename flink-yarn/src/main/java/org/apache/flink/yarn/{YarnTaskManagerRunner.java => YarnTaskManagerRunnerFactory.java} (77%) rename flink-yarn/src/test/java/org/apache/flink/yarn/{YarnTaskManagerRunnerTest.java => YarnTaskManagerRunnerFactoryTest.java} (58%) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnTaskManagerRunner.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnTaskManagerRunner.java index 90cdf9fb2e00f..ff030bec2840c 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnTaskManagerRunner.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnTaskManagerRunner.java @@ -18,6 +18,8 @@ package org.apache.flink.yarn; +import org.apache.flink.runtime.security.SecurityUtils; + import java.io.IOException; /** @@ -25,7 +27,13 @@ */ public class TestingYarnTaskManagerRunner { public static void main(String[] args) throws IOException { - YarnTaskManagerRunner.runYarnTaskManager( - args, TestingYarnTaskManager.class, System.getenv(), null); + YarnTaskManagerRunnerFactory.Runner tmRunner = YarnTaskManagerRunnerFactory.create( + args, TestingYarnTaskManager.class, System.getenv()); + + try { + SecurityUtils.getInstalledContext().runSecured(tmRunner); + } catch (Exception e) { + throw new RuntimeException(e); + } } } diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala index a194e7973a6c6..ed2b2f94022be 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala @@ -24,6 +24,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup +import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike @@ -68,12 +69,19 @@ class TestingYarnTaskManager( object YarnTaskManager { /** Entry point (main method) to run the TaskManager on YARN. - * + * * @param args The command line arguments. */ def main(args: Array[String]): Unit = { - YarnTaskManagerRunner.runYarnTaskManager( - args, classOf[TestingYarnTaskManager], System.getenv(), null) + val tmRunner = YarnTaskManagerRunnerFactory.create( + args, classOf[TestingYarnTaskManager], System.getenv()) + + try { + SecurityUtils.getInstalledContext.runSecured(tmRunner) + } catch { + case e: Exception => + throw new RuntimeException(e) + } } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 1c18e90708ea5..462682f7e5fd6 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -162,6 +162,9 @@ protected int run(String[] args) { File f = new File(currDir, Utils.KEYTAB_FILE_NAME); if (remoteKeytabPrincipal != null && f.exists()) { + String keytabPath = f.getAbsolutePath(); + LOG.debug("keytabPath: {}", keytabPath); + // set keytab principal and replace path with the local path of the shipped keytab file in NodeManager flinkConfig.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, f.getAbsolutePath()); flinkConfig.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, remoteKeytabPrincipal); @@ -247,7 +250,10 @@ protected int runApplicationMaster(Configuration config) { File f = new File(currDir, Utils.KEYTAB_FILE_NAME); if (remoteKeytabPrincipal != null && f.exists()) { - config.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, f.getAbsolutePath()); + String keytabPath = f.getAbsolutePath(); + LOG.debug("keytabPath: {}", keytabPath); + + config.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, keytabPath); config.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, remoteKeytabPrincipal); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunnerFactory.java similarity index 77% rename from flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunnerFactory.java index d1eb1edbdfe94..eef176b77f51c 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunnerFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -47,33 +48,56 @@ /** * The entry point for running a TaskManager in a YARN container. */ -public class YarnTaskManagerRunner { - - private static final Logger LOG = LoggerFactory.getLogger(YarnTaskManagerRunner.class); - - private static Callable createMainRunner( - Configuration configuration, - ResourceID resourceId, - final Class taskManager) { - return new Callable() { - @Override - public Integer call() { - try { - TaskManager.selectNetworkInterfaceAndRunTaskManager( - configuration, resourceId, taskManager); - } catch (Throwable t) { - LOG.error("Error while starting the TaskManager", t); - System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE()); - } - return null; +public class YarnTaskManagerRunnerFactory { + + private static final Logger LOG = LoggerFactory.getLogger(YarnTaskManagerRunnerFactory.class); + + /** + * Runner for a {@link YarnTaskManager}. + */ + public static class Runner implements Callable { + + private final Configuration configuration; + private final ResourceID resourceId; + private final Class taskManager; + + Runner(Configuration configuration, ResourceID resourceId, Class taskManager) { + this.configuration = Preconditions.checkNotNull(configuration); + this.resourceId = Preconditions.checkNotNull(resourceId); + this.taskManager = Preconditions.checkNotNull(taskManager); + } + + @Override + public Object call() throws Exception { + try { + TaskManager.selectNetworkInterfaceAndRunTaskManager( + configuration, resourceId, taskManager); + } catch (Throwable t) { + LOG.error("Error while starting the TaskManager", t); + System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE()); } - }; + return null; + } + + @VisibleForTesting + Configuration getConfiguration() { + return configuration; + } + + @VisibleForTesting + ResourceID getResourceId() { + return resourceId; + } } - public static void runYarnTaskManager(String[] args, - final Class taskManager, - Map envs, - Callable mainRunner) throws IOException { + /** + * Creates a {@link YarnTaskManagerRunnerFactory.Runner}. + */ + public static Runner create( + String[] args, + final Class taskManager, + Map envs) throws IOException { + EnvironmentInformation.logEnvironmentInfo(LOG, "YARN TaskManager", args); SignalHandler.register(LOG); JvmShutdownSafeguard.installAsShutdownHook(LOG); @@ -86,7 +110,7 @@ public static void runYarnTaskManager(String[] args, catch (Throwable t) { LOG.error(t.getMessage(), t); System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE()); - return; + return null; } // read the environment variables for YARN @@ -130,8 +154,8 @@ public static void runYarnTaskManager(String[] args, configuration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, f.getAbsolutePath()); configuration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, remoteKeytabPrincipal); } - try { + try { SecurityConfiguration sc; //To support Yarn Secure Integration Test Scenario @@ -153,12 +177,9 @@ public static void runYarnTaskManager(String[] args, SecurityUtils.install(sc); - if (mainRunner == null) { - mainRunner = createMainRunner(configuration, resourceId, taskManager); - } - SecurityUtils.getInstalledContext().runSecured(mainRunner); + return new Runner(configuration, resourceId, taskManager); } catch (Exception e) { - LOG.error("Exception occurred while launching Task Manager", e); + LOG.error("Exception occurred while building Task Manager runner", e); throw new RuntimeException(e); } diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 5b8f4ca8b454f..08d24b440ba0e 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -24,9 +24,12 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup +import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} +import grizzled.slf4j.Logger + /** An extension of the TaskManager that listens for additional YARN related * messages. */ @@ -56,14 +59,25 @@ class YarnTaskManager( } } - object YarnTaskManager { - /** Entry point (main method) to run the TaskManager on YARN. - * - * @param args The command line arguments. - */ - def main(args: Array[String]): Unit = { - YarnTaskManagerRunner.runYarnTaskManager( - args, classOf[YarnTaskManager], System.getenv(), null) +object YarnTaskManager { + + val LOG = Logger(classOf[TaskManager]) + + /** Entry point (main method) to run the TaskManager on YARN. + * + * @param args The command line arguments. + */ + def main(args: Array[String]): Unit = { + val tmRunner = YarnTaskManagerRunnerFactory.create( + args, classOf[YarnTaskManager], System.getenv()) + + try { + SecurityUtils.getInstalledContext.runSecured(tmRunner) + } catch { + case e: Exception => + LOG.error("Exception occurred while launching Task Manager runner", e) + throw new RuntimeException(e) } + } } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskManagerRunnerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskManagerRunnerFactoryTest.java similarity index 58% rename from flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskManagerRunnerTest.java rename to flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskManagerRunnerFactoryTest.java index cdacfc001ce0b..7f518f1409288 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskManagerRunnerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskManagerRunnerFactoryTest.java @@ -18,6 +18,8 @@ package org.apache.flink.yarn; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.security.modules.HadoopModule; import org.apache.flink.runtime.security.modules.SecurityModule; @@ -34,12 +36,11 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.Callable; /** - * Tests for {@link YarnTaskManagerRunner}. + * Tests for {@link YarnTaskManagerRunnerFactory}. */ -public class YarnTaskManagerRunnerTest { +public class YarnTaskManagerRunnerFactoryTest { @Test public void testKerberosKeytabConfiguration() throws IOException { @@ -49,26 +50,27 @@ public void testKerberosKeytabConfiguration() throws IOException { envs.put(YarnFlinkResourceManager.ENV_FLINK_CONTAINER_ID, "test_container_00001"); envs.put(YarnConfigKeys.KEYTAB_PRINCIPAL, "testuser1@domain"); envs.put(ApplicationConstants.Environment.PWD.key(), resourceDirPath); - YarnTaskManagerRunner.runYarnTaskManager( - new String[]{"--configDir", resourceDirPath}, - YarnTaskManager.class, - envs, - new Callable() { - @Override - public Integer call() { - final List modules = SecurityUtils.getInstalledModules(); - Optional moduleOpt = - modules.stream().filter(s -> s instanceof HadoopModule).findFirst(); - if (moduleOpt.isPresent()) { - HadoopModule hadoopModule = (HadoopModule) moduleOpt.get(); - assertEquals("testuser1@domain", hadoopModule.getSecurityConfig().getPrincipal()); - assertEquals(resourceDirPath + "/krb5.keytab", - hadoopModule.getSecurityConfig().getKeytab()); - } else { - fail("Can not find HadoopModule!"); - } - return null; - } - }); + + final YarnTaskManagerRunnerFactory.Runner tmRunner = YarnTaskManagerRunnerFactory.create( + new String[]{"--configDir", resourceDirPath}, + YarnTaskManager.class, + envs); + + final List modules = SecurityUtils.getInstalledModules(); + Optional moduleOpt = + modules.stream().filter(s -> s instanceof HadoopModule).findFirst(); + if (moduleOpt.isPresent()) { + HadoopModule hadoopModule = (HadoopModule) moduleOpt.get(); + assertEquals("testuser1@domain", hadoopModule.getSecurityConfig().getPrincipal()); + assertEquals(resourceDirPath + "/" + Utils.KEYTAB_FILE_NAME, + hadoopModule.getSecurityConfig().getKeytab()); + } else { + fail("Can not find HadoopModule!"); + } + + final Configuration configuration = tmRunner.getConfiguration(); + assertEquals(resourceDirPath + "/" + Utils.KEYTAB_FILE_NAME, configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB)); + assertEquals("testuser1@domain", configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL)); + assertEquals("test_container_00001", tmRunner.getResourceId().toString()); } } From 5f9e367be6429383be5d0f1ff80e3b77d5a0dda8 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 7 Feb 2018 17:33:03 +0100 Subject: [PATCH 254/367] [FLINK-7760] Fix deserialization of NFA state in CEP library Before, the condition was being read via in.read() and not in.readFully() --- .../flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java index 7092d7377185c..963efa87f17d7 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java @@ -1163,7 +1163,7 @@ private IterativeCondition deserializeCondition(DataInputView in) throws IOEx int length = in.readInt(); byte[] serCondition = new byte[length]; - in.read(serCondition); + in.readFully(serCondition); ByteArrayInputStream bais = new ByteArrayInputStream(serCondition); ObjectInputStream ois = new ObjectInputStream(bais); From 0c53e798cb25bacdde241d4b4696108dbae29117 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 4 Jan 2018 21:09:09 +0100 Subject: [PATCH 255/367] [FLINK-8362][elasticsearch] shade all dependencies --- docs/dev/connectors/elasticsearch.md | 33 +---- .../flink-connector-elasticsearch/pom.xml | 100 ++++++++++++++ .../flink-connector-elasticsearch2/pom.xml | 125 +++++++++++++++++ .../flink-connector-elasticsearch5/pom.xml | 126 ++++++++++++++++++ tools/travis_mvn_watchdog.sh | 29 ++++ 5 files changed, 382 insertions(+), 31 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 8774fcbcee868..52d1b58bf514d 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -440,36 +440,7 @@ For the execution of your Flink program, it is recommended to build a so-called uber-jar (executable jar) containing all your dependencies (see [here]({{site.baseurl}}/dev/linking.html) for further information). -However, when an uber-jar containing an Elasticsearch sink is executed, -an `IllegalArgumentException` may occur, which is caused by conflicting -files of Elasticsearch and it's dependencies in `META-INF/services`: - -``` -IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist. You need to add the corresponding JAR file supporting this SPI to your classpath. The current classpath supports the following names: [es090, completion090, XBloomFilter]] -``` - -If the uber-jar is built using Maven, this issue can be avoided by -adding the following to the Maven POM file in the plugins section: - -~~~xml - - org.apache.maven.plugins - maven-shade-plugin - 2.4.3 - - - package - - shade - - - - - - - - - -~~~ +Alternatively, you can put the connector's jar file into Flink's `lib/` folder to make it available +system-wide, i.e. for all job being run. {% top %} diff --git a/flink-connectors/flink-connector-elasticsearch/pom.xml b/flink-connectors/flink-connector-elasticsearch/pom.xml index 7785a771dfb9e..b6b876ae68e06 100644 --- a/flink-connectors/flink-connector-elasticsearch/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch/pom.xml @@ -93,6 +93,106 @@ under the License. 3 + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + *:* + + + + + + com.fasterxml.jackson + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.elasticsearch.common.jackson + + + com.spatial4j + org.apache.flink.streaming.connectors.elasticsearch.shaded.com.spatial4j + + + + + org.apache.flink.streaming.connectors.elasticsearch. + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.apache.flink.streaming.connectors.elasticsearch. + + + org.apache.flink.streaming.connectors.elasticsearch.Elasticsearch1ApiCallBridge + org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink + org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder + org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilderWrapperFunction + + + + org.apache + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.apache + + + org.apache.flink.** + org.apache.log4j.** + + + + + org.antlr + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.antlr + + + org.elasticsearch + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.elasticsearch + + + org.joda + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.joda + + + org.tartarus + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.tartarus + + + org.yaml + org.apache.flink.streaming.connectors.elasticsearch.shaded.org.yaml + + + + + + * + + log4j.properties + config/favicon.ico + mozilla/** + META-INF/maven/com*/** + META-INF/maven/io*/** + META-INF/maven/joda*/** + META-INF/maven/net*/** + META-INF/maven/org.an*/** + META-INF/maven/org.apache.h*/** + META-INF/maven/org.apache.commons/** + META-INF/maven/org.apache.flink/flink-connector-elasticsearch-base*/** + META-INF/maven/org.apache.flink/force-shading/** + META-INF/maven/org.apache.logging*/** + META-INF/maven/org.e*/** + META-INF/maven/org.h*/** + META-INF/maven/org.j*/** + META-INF/maven/org.y*/** + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml index 1f342bc2b2044..1d6ce04fd0c2f 100644 --- a/flink-connectors/flink-connector-elasticsearch2/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml @@ -91,4 +91,129 @@ under the License. + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + *:* + + + + + com.carrotsearch + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.carrotsearch + + + com.fasterxml + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.fasterxml + + + com.google + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.google + + + com.ning + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.ning + + + com.spatial4j + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.spatial4j + + + com.tdunning + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.tdunning + + + com.twitter + org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.twitter + + + + + org.apache.flink.streaming.connectors.elasticsearch + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.apache.flink.streaming.connectors.elasticsearch + + + org.apache.flink.streaming.connectors.elasticsearch2.** + + + + org.apache + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.apache + + + org.apache.flink.** + org.apache.log4j.** + + + + + org.elasticsearch + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.elasticsearch + + + org.jboss + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.jboss + + + org.joda + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.joda + + + org.HdrHistogram + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.HdrHistogram + + + org.tartarus + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.tartarus + + + org.yaml + org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.yaml + + + + + + * + + log4j.properties + config/favicon.ico + mozilla/** + META-INF/maven/com*/** + META-INF/maven/io*/** + META-INF/maven/joda*/** + META-INF/maven/net*/** + META-INF/maven/org.an*/** + META-INF/maven/org.apache.h*/** + META-INF/maven/org.apache.commons/** + META-INF/maven/org.apache.flink/flink-connector-elasticsearch-base*/** + META-INF/maven/org.apache.flink/force-shading/** + META-INF/maven/org.apache.logging*/** + META-INF/maven/org.e*/** + META-INF/maven/org.h*/** + META-INF/maven/org.j*/** + META-INF/maven/org.y*/** + + + + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml index 72a0d18ef9fcf..82d578bbd8ea9 100644 --- a/flink-connectors/flink-connector-elasticsearch5/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml @@ -156,6 +156,132 @@ under the License. + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + *:* + + + + + com.carrotsearch + org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.carrotsearch + + + com.fasterxml + org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.fasterxml + + + com.github + org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.github + + + com.sun + org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.sun + + + com.github + org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.github + + + com.tdunning + org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.tdunning + + + + io.netty + org.apache.flink.streaming.connectors.elasticsearch5.shaded.io.netty + + + + joptsimple + org.apache.flink.streaming.connectors.elasticsearch5.shaded.joptsimple + + + + + org.apache.flink.streaming.connectors.elasticsearch + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.apache.flink.streaming.connectors.elasticsearch + + + org.apache.flink.streaming.connectors.elasticsearch5.** + + + + org.apache + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.apache + + + org.apache.flink.** + org.apache.log4j.** + + + + + org.elasticsearch + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.elasticsearch + + + org.HdrHistogram + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.HdrHistogram + + + org.jboss + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.jboss + + + org.joda + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.joda + + + org.tartarus + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.tartarus + + + org.yaml + org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.yaml + + + + + + * + + log4j.properties + config/favicon.ico + mozilla/** + META-INF/maven/com*/** + META-INF/maven/io*/** + META-INF/maven/joda*/** + META-INF/maven/net*/** + META-INF/maven/org.an*/** + META-INF/maven/org.apache.h*/** + META-INF/maven/org.apache.commons/** + META-INF/maven/org.apache.flink/flink-connector-elasticsearch-base*/** + META-INF/maven/org.apache.flink/force-shading/** + META-INF/maven/org.apache.logging*/** + META-INF/maven/org.e*/** + META-INF/maven/org.h*/** + META-INF/maven/org.j*/** + META-INF/maven/org.y*/** + + + + + + + diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 7c27392f9b5f6..4fb7be9e4f086 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -447,6 +447,32 @@ check_shaded_artifacts_s3_fs() { return 0 } +# Check the elasticsearch connectors' fat jars for illegal or missing artifacts +check_shaded_artifacts_connector_elasticsearch() { + VARIANT=$1 + find flink-connectors/flink-connector-elasticsearch${VARIANT}/target/flink-connector-elasticsearch${VARIANT}*.jar ! -name "*-tests.jar" -exec jar tf {} \; > allClasses + + UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" | grep '\.class$'` + if [ "$?" == "0" ]; then + echo "==============================================================================" + echo "Detected unshaded dependencies in flink-connector-elasticsearch${VARIANT}'s fat jar:" + echo "${UNSHADED_CLASSES}" + echo "==============================================================================" + return 1 + fi + + UNSHADED_SERVICES=`cat allClasses | grep '^META-INF/services/' | grep -v -e '^META-INF/services/org\.apache\.flink\.core\.fs\.FileSystemFactory$' -e "^META-INF/services/org\.apache\.flink\.fs\.s3${VARIANT}\.shaded" -e '^META-INF/services/'` + if [ "$?" == "0" ]; then + echo "==============================================================================" + echo "Detected unshaded service files in flink-connector-elasticsearch${VARIANT}'s fat jar:" + echo "${UNSHADED_SERVICES}" + echo "==============================================================================" + return 1 + fi + + return 0 +} + # ============================================================================= # WATCHDOG # ============================================================================= @@ -526,6 +552,9 @@ case $TEST in check_shaded_artifacts_s3_fs hadoop EXIT_CODE=$(($EXIT_CODE+$?)) check_shaded_artifacts_s3_fs presto + check_shaded_artifacts_connector_elasticsearch "" + check_shaded_artifacts_connector_elasticsearch 2 + check_shaded_artifacts_connector_elasticsearch 5 EXIT_CODE=$(($EXIT_CODE+$?)) else echo "==============================================================================" From 33ebc85c281fcd4869fa743f88c2c71d339f9857 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 5 Feb 2018 14:16:56 +0100 Subject: [PATCH 256/367] [FLINK-8362] [elasticsearch] Further improvements for Elasticsearch connector shading - Do not shade Elasticsearch dependencies - Do not shade Flink Elasticseach Connector classes - Also shade log4j-api dependency in Elasticsearch 5 connector. This is required for the log4j-to-slf4j bridge adapter to work properly. - Add NOTICE files for license statements for all ES connectors This closes #5426. This closes #5243. --- .../flink-connector-elasticsearch/pom.xml | 17 -- .../src/main/resources/META-INF/NOTICE | 109 ++++++++++ .../flink-connector-elasticsearch2/pom.xml | 14 -- .../src/main/resources/META-INF/NOTICE | 133 ++++++++++++ .../flink-connector-elasticsearch5/pom.xml | 43 ++-- .../src/main/resources/META-INF/NOTICE | 201 ++++++++++++++++++ .../META-INF/log4j-provider.properties | 23 ++ tools/travis_mvn_watchdog.sh | 2 +- 8 files changed, 482 insertions(+), 60 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/log4j-provider.properties diff --git a/flink-connectors/flink-connector-elasticsearch/pom.xml b/flink-connectors/flink-connector-elasticsearch/pom.xml index b6b876ae68e06..987dccaf5028f 100644 --- a/flink-connectors/flink-connector-elasticsearch/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch/pom.xml @@ -121,18 +121,6 @@ under the License. org.apache.flink.streaming.connectors.elasticsearch.shaded.com.spatial4j - - - org.apache.flink.streaming.connectors.elasticsearch. - org.apache.flink.streaming.connectors.elasticsearch.shaded.org.apache.flink.streaming.connectors.elasticsearch. - - - org.apache.flink.streaming.connectors.elasticsearch.Elasticsearch1ApiCallBridge - org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink - org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder - org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilderWrapperFunction - - org.apache org.apache.flink.streaming.connectors.elasticsearch.shaded.org.apache @@ -147,10 +135,6 @@ under the License. org.antlr org.apache.flink.streaming.connectors.elasticsearch.shaded.org.antlr - - org.elasticsearch - org.apache.flink.streaming.connectors.elasticsearch.shaded.org.elasticsearch - org.joda org.apache.flink.streaming.connectors.elasticsearch.shaded.org.joda @@ -179,7 +163,6 @@ under the License. META-INF/maven/org.an*/** META-INF/maven/org.apache.h*/** META-INF/maven/org.apache.commons/** - META-INF/maven/org.apache.flink/flink-connector-elasticsearch-base*/** META-INF/maven/org.apache.flink/force-shading/** META-INF/maven/org.apache.logging*/** META-INF/maven/org.e*/** diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..f588e0e873c14 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE @@ -0,0 +1,109 @@ +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +------------------------------------------------------------- + +This project bundles the following dependencies under +the Apache Software License 2.0 + + - org.apache.lucene : lucene-core version 4.10.4 + - org.apache.lucene : lucene-analyzers-common version 4.10.4 + - org.apache.lucene : lucene-grouping version 4.10.4 + - org.apache.lucene : lucene-highlighter version 4.10.4 + - org.apache.lucene : lucene-join version 4.10.4 + - org.apache.lucene : lucene-memory version 4.10.4 + - org.apache.lucene : lucene-misc version 4.10.4 + - org.apache.lucene : lucene-queries version 4.10.4 + - org.apache.lucene : lucene-queryparser version 4.10.4 + - org.apache.lucene : lucene-sandbox version 4.10.4 + - org.apache.lucene : lucene-spatial version 4.10.4 + - org.apache.lucene : lucene-suggest version 4.10.4 + - com.spatial4j : spatial4j version 0.4.1 + - com.fasterxml.jackson.core : jackson-core version 2.5.3 + - com.fasterxml.jackson.dataformat : jackson-dataformat-smile version 2.5.3 + - com.fasterxml.jackson.dataformat : jackson-dataformat-yaml version 2.5.3 + - com.fasterxml.jackson.dataformat : jackson-dataformat-cbor version 2.5.3 + - org.joda : joda-convert (copied classes) + +=================================== + Notice for Yaml +=================================== + +This project bundles yaml (v. 1.12) under the Creative Commons License (CC-BY 2.0). + +Original project website: http://www.yaml.de + +Copyright (c) 2005-2013, Dirk Jesse + +YAML under Creative Commons License (CC-BY 2.0) +=============================================== + +The YAML framework is published under the Creative Commons Attribution 2.0 License (CC-BY 2.0), which permits +both private and commercial use (http://creativecommons.org/licenses/by/2.0/). + +Condition: For the free use of the YAML framework, a backlink to the YAML homepage (http://www.yaml.de) in a +suitable place (e.g.: footer of the website or in the imprint) is required. + +=================================== + Notice for Tartarus +=================================== + +This project bundles tartarus under the MIT License. + +Original source repository: https://github.com/sergiooramas/tartarus + +Copyright (c) 2017 Sergio Oramas and Oriol Nieto + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +=================================== + Notice for Antlr +=================================== + +This project bundles antlr-runtime (v. 3.5) under the BSD 2-Clause License + +Copyright (c) 2010 Terence Parr + +----------------------------------------------------------------------------- +** Beginning of "BSD 2-Clause License" text. ** + + Copyright (c) 2010 Terence Parr + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml index 1d6ce04fd0c2f..56e4d9e4b1548 100644 --- a/flink-connectors/flink-connector-elasticsearch2/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml @@ -140,15 +140,6 @@ under the License. org.apache.flink.streaming.connectors.elasticsearch2.shaded.com.twitter - - - org.apache.flink.streaming.connectors.elasticsearch - org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.apache.flink.streaming.connectors.elasticsearch - - - org.apache.flink.streaming.connectors.elasticsearch2.** - - org.apache org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.apache @@ -159,10 +150,6 @@ under the License. - - org.elasticsearch - org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.elasticsearch - org.jboss org.apache.flink.streaming.connectors.elasticsearch2.shaded.org.jboss @@ -199,7 +186,6 @@ under the License. META-INF/maven/org.an*/** META-INF/maven/org.apache.h*/** META-INF/maven/org.apache.commons/** - META-INF/maven/org.apache.flink/flink-connector-elasticsearch-base*/** META-INF/maven/org.apache.flink/force-shading/** META-INF/maven/org.apache.logging*/** META-INF/maven/org.e*/** diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..c3d95ca99c016 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE @@ -0,0 +1,133 @@ +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +------------------------------------------------------------- + +This project bundles the following dependencies under +the Apache Software License 2.0 + + - org.apache.lucene : lucene-core version 5.5.0 + - org.apache.lucene : lucene-analyzers-common version 5.5.0 + - org.apache.lucene : lucene-backward-codecs version 5.5.0 + - org.apache.lucene : lucene-grouping version 5.5.0 + - org.apache.lucene : lucene-highlighter version 5.5.0 + - org.apache.lucene : lucene-join version 5.5.0 + - org.apache.lucene : lucene-memory version 5.5.0 + - org.apache.lucene : lucene-misc version 5.5.0 + - org.apache.lucene : lucene-queries version 5.5.0 + - org.apache.lucene : lucene-queryparser version 5.5.0 + - org.apache.lucene : lucene-sandbox version 5.5.0 + - org.apache.lucene : lucene-spatial version 5.5.0 + - org.apache.lucene : lucene-spatial-extras version 5.5.0 + - org.apache.lucene : lucene-spatial3d version 5.5.0 + - org.apache.lucene : lucene-suggest version 5.5.0 + - com.carrotsearch : hppc version 0.7.1 + - com.google.guava : guava version 18.0 + - com.google.code.findbugs : jsr305 version 1.3.9 + - com.ning : compress-lzf version 1.0.2 + - com.spatial4j : spatial4j version 0.5 + - com.twitter : chill-java version 0.7.4 + - com.fasterxml.jackson.core : jackson-core version 2.6.6 + - com.fasterxml.jackson.dataformat : jackson-dataformat-smile version 2.6.6 + - com.fasterxml.jackson.dataformat : jackson-dataformat-yaml version 2.6.6 + - com.fasterxml.jackson.dataformat : jackson-dataformat-cbor version 2.6.6 + - com.tdunning : t-digest version 3.0 + - io.netty : netty version 3.10.5.Final + - joda-time : joda-time version 2.5 + +=================================== + Notice for Yaml +=================================== + +This project bundles yaml (v. 1.15) under the Creative Commons License (CC-BY 2.0). + +Original project website: http://www.yaml.de + +Copyright (c) 2005-2013, Dirk Jesse + +YAML under Creative Commons License (CC-BY 2.0) +=============================================== + +The YAML framework is published under the Creative Commons Attribution 2.0 License (CC-BY 2.0), which permits +both private and commercial use (http://creativecommons.org/licenses/by/2.0/). + +Condition: For the free use of the YAML framework, a backlink to the YAML homepage (http://www.yaml.de) in a +suitable place (e.g.: footer of the website or in the imprint) is required. + +=================================== + Notice for Tartarus +=================================== + +This project bundles tartarus under the MIT License. + +Original source repository: https://github.com/sergiooramas/tartarus + +Copyright (c) 2017 Sergio Oramas and Oriol Nieto + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +=================================== + Notice for HdrHistogram +=================================== + +This project bundles HdrHistogram (v. 2.1.9) under the BSD 2-Clause License + +Original source repository: https://github.com/HdrHistogram/HdrHistogram + +The code in this repository code was Written by Gil Tene, Michael Barker, +and Matt Warren, and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ + +For users of this code who wish to consume it under the "BSD" license +rather than under the public domain or CC0 contribution text mentioned +above, the code found under this directory is *also* provided under the +following license (commonly referred to as the BSD 2-Clause License). This +license does not detract from the above stated release of the code into +the public domain, and simply represents an additional license granted by +the Author. + +----------------------------------------------------------------------------- +** Beginning of "BSD 2-Clause License" text. ** + + Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene + Copyright (c) 2014 Michael Barker + Copyright (c) 2014 Matt Warren + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml index 82d578bbd8ea9..01eb4d3578fd5 100644 --- a/flink-connectors/flink-connector-elasticsearch5/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml @@ -124,7 +124,6 @@ under the License. org.apache.logging.log4j log4j-api 2.7 - test @@ -190,34 +189,14 @@ under the License. com.sun org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.sun - - com.github - org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.github - com.tdunning org.apache.flink.streaming.connectors.elasticsearch5.shaded.com.tdunning - io.netty org.apache.flink.streaming.connectors.elasticsearch5.shaded.io.netty - - - joptsimple - org.apache.flink.streaming.connectors.elasticsearch5.shaded.joptsimple - - - - - org.apache.flink.streaming.connectors.elasticsearch - org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.apache.flink.streaming.connectors.elasticsearch - - - org.apache.flink.streaming.connectors.elasticsearch5.** - - org.apache org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.apache @@ -227,11 +206,6 @@ under the License. org.apache.log4j.** - - - org.elasticsearch - org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.elasticsearch - org.HdrHistogram org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.HdrHistogram @@ -252,7 +226,10 @@ under the License. org.yaml org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.yaml - + + joptsimple + org.apache.flink.streaming.connectors.elasticsearch5.shaded.joptsimple + @@ -268,7 +245,6 @@ under the License. META-INF/maven/org.an*/** META-INF/maven/org.apache.h*/** META-INF/maven/org.apache.commons/** - META-INF/maven/org.apache.flink/flink-connector-elasticsearch-base*/** META-INF/maven/org.apache.flink/force-shading/** META-INF/maven/org.apache.logging*/** META-INF/maven/org.e*/** @@ -277,6 +253,17 @@ under the License. META-INF/maven/org.y*/** + + + org.apache.logging.log4j:log4j-to-slf4j + + META-INF/log4j-provider.properties + + diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..19e96faa46003 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE @@ -0,0 +1,201 @@ +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +------------------------------------------------------------- + +This project bundles the following dependencies under +the Apache Software License 2.0 + + - org.apache.lucene : lucene-core version 6.3.0 + - org.apache.lucene : lucene-analyzers-common version 6.3.0 + - org.apache.lucene : lucene-backward-codecs version 6.3.0 + - org.apache.lucene : lucene-grouping version 6.3.0 + - org.apache.lucene : lucene-highlighter version 6.3.0 + - org.apache.lucene : lucene-join version 6.3.0 + - org.apache.lucene : lucene-memory version 6.3.0 + - org.apache.lucene : lucene-misc version 6.3.0 + - org.apache.lucene : lucene-queries version 6.3.0 + - org.apache.lucene : lucene-queryparser version 6.3.0 + - org.apache.lucene : lucene-sandbox version 6.3.0 + - org.apache.lucene : lucene-spatial version 6.3.0 + - org.apache.lucene : lucene-spatial-extras version 6.3.0 + - org.apache.lucene : lucene-spatial3d version 6.3.0 + - org.apache.lucene : lucene-suggest version 6.3.0 + - org.apache.httpcomponents : httpclient version 4.5.3 + - org.apache.httpcomponents : httpcore version 4.4.6 + - org.apache.httpcomponents : httpasynclcient version 4.1.2 + - org.apache.httpcomponents : httpcore-nio version 4.4.5 + - com.carrotsearch : hppc version 0.7.1 + - com.fasterxml.jackson.core : jackson-core version 2.8.1 + - com.fasterxml.jackson.dataformat : jackson-dataformat-smile version 2.8.1 + - com.fasterxml.jackson.dataformat : jackson-dataformat-yaml version 2.8.1 + - com.fasterxml.jackson.dataformat : jackson-dataformat-cbor version 2.8.1 + - com.tdunning : t-digest version 3.0 + - io.netty : netty version 3.10.6.Final + - io.netty : netty-buffer version 4.1.6.Final + - io.netty : netty-codec version 4.1.6.Final + - io.netty : netty-codec-http version 4.1.6.Final + - io.netty : netty-common version 4.1.6.Final + - io.netty : netty-handler version 4.1.6.Final + - io.netty : netty-resolver version 4.1.6.Final + - io.netty : netty-transport version 4.1.6.Final + - org.jboss.netty : netty version 3.2.0.Final + - joda-time : joda-time version 2.5 + - com.github.spullara.mustache.java : compiler version 0.9.3 + +=================================== + Notice for Yaml +=================================== + +This project bundles yaml (v. 1.15) under the Creative Commons License (CC-BY 2.0). + +Original project website: http://www.yaml.de + +Copyright (c) 2005-2013, Dirk Jesse + +YAML under Creative Commons License (CC-BY 2.0) +=============================================== + +The YAML framework is published under the Creative Commons Attribution 2.0 License (CC-BY 2.0), which permits +both private and commercial use (http://creativecommons.org/licenses/by/2.0/). + +Condition: For the free use of the YAML framework, a backlink to the YAML homepage (http://www.yaml.de) in a +suitable place (e.g.: footer of the website or in the imprint) is required. + +=================================== + Notice for Tartarus +=================================== + +This project bundles tartarus under the MIT License. + +Original source repository: https://github.com/sergiooramas/tartarus + +Copyright (c) 2017 Sergio Oramas and Oriol Nieto + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +=================================== + Notice for joptsimple +=================================== + +This project bundles joptsimple under the MIT License. + +Original source repository: https://github.com/jopt-simple/jopt-simple + +Copyright (c) 2004-2016 Paul R. Holser, Jr. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +=================================== + Notice for scopt +=================================== + +This project bundles scopt (v. 3.5.0) underr the MIT License. + +Original source repository: https://github.com/scopt/scopt + +scopt - Copyright (c) scopt contributors + +See source files for details. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +``Software''), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED ``AS IS'', WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY +CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, +TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE +SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +=================================== + Notice for HdrHistogram +=================================== + +This project bundles HdrHistogram (v. 2.1.9) under the BSD 2-Clause License + +Original source repository: https://github.com/HdrHistogram/HdrHistogram + +The code in this repository code was Written by Gil Tene, Michael Barker, +and Matt Warren, and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ + +For users of this code who wish to consume it under the "BSD" license +rather than under the public domain or CC0 contribution text mentioned +above, the code found under this directory is *also* provided under the +following license (commonly referred to as the BSD 2-Clause License). This +license does not detract from the above stated release of the code into +the public domain, and simply represents an additional license granted by +the Author. + +----------------------------------------------------------------------------- +** Beginning of "BSD 2-Clause License" text. ** + + Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene + Copyright (c) 2014 Michael Barker + Copyright (c) 2014 Matt Warren + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/log4j-provider.properties b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/log4j-provider.properties new file mode 100644 index 0000000000000..c67f5e4e6e912 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/log4j-provider.properties @@ -0,0 +1,23 @@ +# 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. + +# Since we relocate the log4j2-to-slf4j dependency, +# we also re-package this provider file with the correct relocated paths +# (the original provider file with the incorrect paths is excluded) + +LoggerContextFactory = org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.apache.logging.slf4j.SLF4JLoggerContextFactory +Log4jAPIVersion = 2.0.0 +FactoryPriority= 15 +ThreadContextMap = org.apache.flink.streaming.connectors.elasticsearch5.shaded.org.apache.logging.slf4j.MDCContextMap diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 4fb7be9e4f086..3b99ec3832463 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -452,7 +452,7 @@ check_shaded_artifacts_connector_elasticsearch() { VARIANT=$1 find flink-connectors/flink-connector-elasticsearch${VARIANT}/target/flink-connector-elasticsearch${VARIANT}*.jar ! -name "*-tests.jar" -exec jar tf {} \; > allClasses - UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" | grep '\.class$'` + UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/streaming/connectors/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" -e "^org/elasticsearch/" | grep '\.class$'` if [ "$?" == "0" ]; then echo "==============================================================================" echo "Detected unshaded dependencies in flink-connector-elasticsearch${VARIANT}'s fat jar:" From beff62d2e6e41b9406ac02ec8b206c80b9df50ed Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 9 Feb 2018 11:30:37 +0100 Subject: [PATCH 257/367] [FLINK-8571] [DataStream] Introduce utility function that reinterprets a data stream as keyed stream (backport from 1.5 branch) This closes #5439. --- .../contrib/streaming/DataStreamUtils.java | 61 +++++ .../streaming/scala/utils/package.scala | 25 +- .../ReinterpretAsKeyedStreamITCase.java | 233 ++++++++++++++++++ ...terpretDataStreamAsKeyedStreamITCase.scala | 42 ++++ .../streaming/api/datastream/KeyedStream.java | 33 ++- 5 files changed, 389 insertions(+), 5 deletions(-) create mode 100644 flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/ReinterpretAsKeyedStreamITCase.java create mode 100644 flink-contrib/flink-streaming-contrib/src/test/scala/org/apache/flink/streaming/api/scala/ReinterpretDataStreamAsKeyedStreamITCase.scala diff --git a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java index 430c98cb893db..6d0055c2965bd 100644 --- a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java +++ b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/DataStreamUtils.java @@ -17,13 +17,19 @@ package org.apache.flink.contrib.streaming; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.runtime.net.ConnectionUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.KeyedStream; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import java.io.IOException; import java.net.InetAddress; @@ -80,6 +86,61 @@ public static Iterator collect(DataStream stream) throws IOExcep return iter; } + /** + * Reinterprets the given {@link DataStream} as a {@link KeyedStream}, which extracts keys with the given + * {@link KeySelector}. + * + *

IMPORTANT: For every partition of the base stream, the keys of events in the base stream must be + * partitioned exactly in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}. + * + * @param stream The data stream to reinterpret. For every partition, this stream must be partitioned exactly + * in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}. + * @param keySelector Function that defines how keys are extracted from the data stream. + * @param Type of events in the data stream. + * @param Type of the extracted keys. + * @return The reinterpretation of the {@link DataStream} as a {@link KeyedStream}. + */ + public static KeyedStream reinterpretAsKeyedStream( + DataStream stream, + KeySelector keySelector) { + + return reinterpretAsKeyedStream( + stream, + keySelector, + TypeExtractor.getKeySelectorTypes(keySelector, stream.getType())); + } + + /** + * Reinterprets the given {@link DataStream} as a {@link KeyedStream}, which extracts keys with the given + * {@link KeySelector}. + * + *

IMPORTANT: For every partition of the base stream, the keys of events in the base stream must be + * partitioned exactly in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}. + * + * @param stream The data stream to reinterpret. For every partition, this stream must be partitioned exactly + * in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}. + * @param keySelector Function that defines how keys are extracted from the data stream. + * @param typeInfo Explicit type information about the key type. + * @param Type of events in the data stream. + * @param Type of the extracted keys. + * @return The reinterpretation of the {@link DataStream} as a {@link KeyedStream}. + */ + public static KeyedStream reinterpretAsKeyedStream( + DataStream stream, + KeySelector keySelector, + TypeInformation typeInfo) { + + PartitionTransformation partitionTransformation = new PartitionTransformation<>( + stream.getTransformation(), + new ForwardPartitioner<>()); + + return new KeyedStream<>( + stream, + partitionTransformation, + keySelector, + typeInfo); + } + private static class CallExecute extends Thread { private final StreamExecutionEnvironment toTrigger; diff --git a/flink-contrib/flink-streaming-contrib/src/main/scala/org/apache/flink/contrib/streaming/scala/utils/package.scala b/flink-contrib/flink-streaming-contrib/src/main/scala/org/apache/flink/contrib/streaming/scala/utils/package.scala index 86a2bdcc23ea4..a2dcb188bcfd8 100644 --- a/flink-contrib/flink-streaming-contrib/src/main/scala/org/apache/flink/contrib/streaming/scala/utils/package.scala +++ b/flink-contrib/flink-streaming-contrib/src/main/scala/org/apache/flink/contrib/streaming/scala/utils/package.scala @@ -43,6 +43,29 @@ package object utils { JavaStreamUtils.collect(self.javaStream).asScala } - } + /** + * Reinterprets the given [[DataStream]] as a [[KeyedStream]], which extracts keys with the + * given [[KeySelectorWithType]]. + * + * IMPORTANT: For every partition of the base stream, the keys of events in the base stream + * must be partitioned exactly in the same way as if it was created through a + * [[DataStream#keyBy(KeySelectorWithType)]]. + * + * @param keySelector Function that defines how keys are extracted from the data stream. + * @return The reinterpretation of the [[DataStream]] as a [[KeyedStream]]. + */ + def reinterpretAsKeyedStream[K: TypeInformation]( + keySelector: T => K): KeyedStream[T, K] = { + + val keySelectorWithType = + new KeySelectorWithType[T, K](clean(keySelector), implicitly[TypeInformation[K]]) + asScalaStream( + JavaStreamUtils.reinterpretAsKeyedStream(self.javaStream, keySelectorWithType)) + } + + private[flink] def clean[F <: AnyRef](f: F): F = { + new StreamExecutionEnvironment(self.javaStream.getExecutionEnvironment).scalaClean(f) + } + } } diff --git a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/ReinterpretAsKeyedStreamITCase.java b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/ReinterpretAsKeyedStreamITCase.java new file mode 100644 index 0000000000000..e068ff1e94405 --- /dev/null +++ b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/ReinterpretAsKeyedStreamITCase.java @@ -0,0 +1,233 @@ +/* + * 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.flink.contrib.streaming; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.util.Preconditions; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +/** + * ITCase for {@link DataStreamUtils#reinterpretAsKeyedStream(DataStream, KeySelector)}. + */ +public class ReinterpretAsKeyedStreamITCase { + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + /** + * This test checks that reinterpreting a data stream to a keyed stream works as expected. This test consists of + * two jobs. The first job materializes a keyBy into files, one files per partition. The second job opens the + * files created by the first jobs as sources (doing the correct assignment of files to partitions) and + * reinterprets the sources as keyed, because we know they have been partitioned in a keyBy from the first job. + */ + @Test + public void testReinterpretAsKeyedStream() throws Exception { + + final int numEventsPerInstance = 100; + final int maxParallelism = 8; + final int parallelism = 3; + final int numUniqueKeys = 12; + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.setMaxParallelism(maxParallelism); + env.setParallelism(parallelism); + + final List partitionFiles = new ArrayList<>(parallelism); + for (int i = 0; i < parallelism; ++i) { + File partitionFile = temporaryFolder.newFile(); + partitionFiles.add(i, partitionFile); + } + + env.addSource(new RandomTupleSource(numEventsPerInstance, numUniqueKeys)) + .keyBy(0) + .addSink(new ToPartitionFileSink(partitionFiles)); + + env.execute(); + + DataStreamUtils.reinterpretAsKeyedStream( + env.addSource(new FromPartitionFileSource(partitionFiles)), + (KeySelector, Integer>) value -> value.f0, + TypeInformation.of(Integer.class)) + .timeWindow(Time.seconds(1)) // test that also timers and aggregated state work as expected + .reduce((ReduceFunction>) (value1, value2) -> + new Tuple2<>(value1.f0, value1.f1 + value2.f1)) + .addSink(new ValidatingSink(numEventsPerInstance * parallelism)).setParallelism(1); + + env.execute(); + } + + private static class RandomTupleSource implements ParallelSourceFunction> { + private static final long serialVersionUID = 1L; + + private int numKeys; + private int remainingEvents; + + public RandomTupleSource(int numEvents, int numKeys) { + this.numKeys = numKeys; + this.remainingEvents = numEvents; + } + + @Override + public void run(SourceContext> out) throws Exception { + Random random = new Random(42); + while (--remainingEvents >= 0) { + out.collect(new Tuple2<>(random.nextInt(numKeys), 1)); + } + } + + @Override + public void cancel() { + this.remainingEvents = 0; + } + } + + private static class ToPartitionFileSink extends RichSinkFunction> { + + private static final long serialVersionUID = 1L; + + private final List allPartitions; + private DataOutputStream dos; + + public ToPartitionFileSink(List allPartitions) { + this.allPartitions = allPartitions; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + dos = new DataOutputStream( + new BufferedOutputStream( + new FileOutputStream(allPartitions.get(subtaskIdx)))); + } + + @Override + public void close() throws Exception { + super.close(); + dos.close(); + } + + @Override + public void invoke(Tuple2 value, Context context) throws Exception { + dos.writeInt(value.f0); + dos.writeInt(value.f1); + } + } + + private static class FromPartitionFileSource extends RichParallelSourceFunction> { + private static final long serialVersionUID = 1L; + + private List allPartitions; + private DataInputStream din; + private volatile boolean running; + + public FromPartitionFileSource(List allPartitons) { + this.allPartitions = allPartitons; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + din = new DataInputStream( + new BufferedInputStream( + new FileInputStream(allPartitions.get(subtaskIdx)))); + } + + @Override + public void close() throws Exception { + super.close(); + din.close(); + } + + @Override + public void run(SourceContext> out) throws Exception { + this.running = true; + try { + while (running) { + Integer key = din.readInt(); + Integer val = din.readInt(); + out.collect(new Tuple2<>(key, val)); + } + } catch (EOFException ignore) { + } + } + + @Override + public void cancel() { + this.running = false; + } + } + + private static class ValidatingSink extends RichSinkFunction> { + + private static final long serialVersionUID = 1L; + private final int expectedSum; + private int runningSum = 0; + + private ValidatingSink(int expectedSum) { + this.expectedSum = expectedSum; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + Preconditions.checkState(getRuntimeContext().getNumberOfParallelSubtasks() == 1); + } + + @Override + public void invoke(Tuple2 value, Context context) throws Exception { + runningSum += value.f1; + } + + @Override + public void close() throws Exception { + Assert.assertEquals(expectedSum, runningSum); + super.close(); + } + } +} diff --git a/flink-contrib/flink-streaming-contrib/src/test/scala/org/apache/flink/streaming/api/scala/ReinterpretDataStreamAsKeyedStreamITCase.scala b/flink-contrib/flink-streaming-contrib/src/test/scala/org/apache/flink/streaming/api/scala/ReinterpretDataStreamAsKeyedStreamITCase.scala new file mode 100644 index 0000000000000..2a36f1a2e4a46 --- /dev/null +++ b/flink-contrib/flink-streaming-contrib/src/test/scala/org/apache/flink/streaming/api/scala/ReinterpretDataStreamAsKeyedStreamITCase.scala @@ -0,0 +1,42 @@ +/* + * 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.flink.streaming.api.scala + +import org.apache.flink.contrib.streaming.scala.utils.DataStreamUtils +import org.apache.flink.streaming.api.functions.sink.DiscardingSink +import org.apache.flink.streaming.api.windowing.time.Time +import org.junit.Test + + +/** + * Integration test for [[DataStreamUtils.reinterpretAsKeyedStream()]]. + */ +class ReinterpretDataStreamAsKeyedStreamITCase { + + @Test + def testReinterpretAsKeyedStream(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setParallelism(1) + val source = env.fromElements(1, 2, 3) + new DataStreamUtils(source).reinterpretAsKeyedStream((in) => in) + .timeWindow(Time.seconds(1)) + .reduce((a, b) => a + b) + .addSink(new DiscardingSink[Int]) + env.execute() + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java index ebcd7d57f0652..94c69969c7597 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java @@ -120,12 +120,37 @@ public KeyedStream(DataStream dataStream, KeySelector keySelector) { * Function for determining state partitions */ public KeyedStream(DataStream dataStream, KeySelector keySelector, TypeInformation keyType) { - super( - dataStream.getExecutionEnvironment(), + this( + dataStream, new PartitionTransformation<>( dataStream.getTransformation(), - new KeyGroupStreamPartitioner<>(keySelector, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM))); - this.keySelector = keySelector; + new KeyGroupStreamPartitioner<>(keySelector, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM)), + keySelector, + keyType); + } + + /** + * Creates a new {@link KeyedStream} using the given {@link KeySelector} and {@link TypeInformation} + * to partition operator state by key, where the partitioning is defined by a {@link PartitionTransformation}. + * + * @param stream + * Base stream of data + * @param partitionTransformation + * Function that determines how the keys are distributed to downstream operator(s) + * @param keySelector + * Function to extract keys from the base stream + * @param keyType + * Defines the type of the extracted keys + */ + @Internal + public KeyedStream( + DataStream stream, + PartitionTransformation partitionTransformation, + KeySelector keySelector, + TypeInformation keyType) { + + super(stream.getExecutionEnvironment(), partitionTransformation); + this.keySelector = clean(keySelector); this.keyType = validateKeyType(keyType); } From bafb91eeb50a2821771a852919b2358fb43622f2 Mon Sep 17 00:00:00 2001 From: zhangminglei Date: Tue, 13 Feb 2018 10:33:04 +0800 Subject: [PATCH 258/367] [FLINK-8423] OperatorChain#pushToOperator catch block may fail with NPE This closes #5447. --- .../runtime/tasks/OperatorChain.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index a44cffb437e7d..1c117ac9909e4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -548,17 +548,20 @@ protected void pushToOperator(StreamRecord record) { operator.setKeyContextElement1(copy); operator.processElement(copy); } catch (ClassCastException e) { - // Enrich error message - ClassCastException replace = new ClassCastException( - String.format( - "%s. Failed to push OutputTag with id '%s' to operator. " + - "This can occur when multiple OutputTags with different types " + - "but identical names are being used.", - e.getMessage(), - outputTag.getId())); - - throw new ExceptionInChainedOperatorException(replace); - + if (outputTag != null) { + // Enrich error message + ClassCastException replace = new ClassCastException( + String.format( + "%s. Failed to push OutputTag with id '%s' to operator. " + + "This can occur when multiple OutputTags with different types " + + "but identical names are being used.", + e.getMessage(), + outputTag.getId())); + + throw new ExceptionInChainedOperatorException(replace); + } else { + throw new ExceptionInChainedOperatorException(e); + } } catch (Exception e) { throw new ExceptionInChainedOperatorException(e); } From a044d9d6caa5e6af2d9edb9321d40617f0ca6f78 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 14 Feb 2018 17:46:51 +0100 Subject: [PATCH 259/367] [hotfix] Remove costly logging statements from CEP SharedBuffer --- .../src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java index cb1a68d9c9693..36a365619487e 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java @@ -944,15 +944,12 @@ public void serialize(SharedBuffer record, DataOutputView target) throws I // the ids of the source and target SharedBufferEntry Integer id = entryIDs.get(sharedBuffer); - Preconditions.checkState(id != null, "Could not find id for entry: " + sharedBuffer); for (SharedBufferEdge edge: sharedBuffer.edges) { // in order to serialize the previous relation we simply serialize the ids // of the source and target SharedBufferEntry if (edge.target != null) { Integer targetId = entryIDs.get(edge.getTarget()); - Preconditions.checkState(targetId != null, - "Could not find id for entry: " + edge.getTarget()); target.writeInt(id); target.writeInt(targetId); From 59f9ded8fd36d17b339f46a7a95286ff703719ef Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 14 Feb 2018 15:43:16 +0100 Subject: [PATCH 260/367] [FLINK-8652] [QS] Reduce log level in getKvState to DEBUG. This closes #5489. --- .../flink/queryablestate/client/QueryableStateClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java index f1c69edd773ad..2a6baf07bbfa6 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java @@ -268,7 +268,7 @@ private CompletableFuture getKvState( final String queryableStateName, final int keyHashCode, final byte[] serializedKeyAndNamespace) { - LOG.info("Sending State Request to {}.", remoteAddress); + LOG.debug("Sending State Request to {}.", remoteAddress); try { KvStateRequest request = new KvStateRequest(jobId, queryableStateName, keyHashCode, serializedKeyAndNamespace); return client.sendRequest(remoteAddress, request); From f2b5635bc83682e90cb3488316dee363b9622504 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 15 Feb 2018 10:20:58 +0100 Subject: [PATCH 261/367] [hotfix] Remove more checkState() calls from SharedBuffer serialization --- .../src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java index 36a365619487e..608f20405d4be 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java @@ -31,7 +31,6 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.flink.util.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -995,12 +994,8 @@ public SharedBuffer deserialize(DataInputView source) throws IOException { for (int j = 0; j < totalEdges; j++) { int sourceIndex = source.readInt(); - Preconditions.checkState(sourceIndex < entryList.size() && sourceIndex >= 0, - "Could not find source entry with index " + sourceIndex + ". This indicates a corrupted state."); int targetIndex = source.readInt(); - Preconditions.checkState(targetIndex < entryList.size(), - "Could not find target entry with index " + sourceIndex + ". This indicates a corrupted state."); DeweyNumber version = versionSerializer.deserialize(source); From 1b70f50d93805d13c7f37b6cd14e95ea8303412d Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Feb 2018 10:31:41 +0100 Subject: [PATCH 262/367] [FLINK-8576][QS] Reduce verbosity when classes can't be found This closes #5420. --- .../runtime/query/QueryableStateUtils.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java index adbe15d90675b..521b72047beea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java @@ -36,6 +36,10 @@ public final class QueryableStateUtils { private static final Logger LOG = LoggerFactory.getLogger(QueryableStateUtils.class); + private static final String ERROR_MESSAGE_ON_LOAD_FAILURE = + "Probable reason: flink-queryable-state-runtime is not in the classpath. " + + "To enable Queryable State, please move the flink-queryable-state-runtime jar from the opt to the lib folder."; + /** * Initializes the {@link KvStateClientProxy client proxy} responsible for * receiving requests from the external (to the cluster) client and forwarding them internally. @@ -73,10 +77,12 @@ public static KvStateClientProxy createKvStateClientProxy( KvStateRequestStats.class); return constructor.newInstance(address, ports, eventLoopThreads, queryThreads, stats); } catch (ClassNotFoundException e) { - LOG.warn("Could not load Queryable State Client Proxy. " + - "Probable reason: flink-queryable-state-runtime is not in the classpath. " + - "Please put the corresponding jar from the opt to the lib folder."); - LOG.debug("Caught exception", e); + final String msg = "Could not load Queryable State Client Proxy. " + ERROR_MESSAGE_ON_LOAD_FAILURE; + if (LOG.isDebugEnabled()) { + LOG.debug(msg + " Cause: " + e.getMessage()); + } else { + LOG.warn(msg); + } return null; } catch (InvocationTargetException e) { LOG.error("Queryable State Client Proxy could not be created: ", e.getTargetException()); @@ -128,10 +134,12 @@ public static KvStateServer createKvStateServer( KvStateRequestStats.class); return constructor.newInstance(address, ports, eventLoopThreads, queryThreads, kvStateRegistry, stats); } catch (ClassNotFoundException e) { - LOG.warn("Could not load Queryable State Server. " + - "Probable reason: flink-queryable-state-runtime is not in the classpath. " + - "Please put the corresponding jar from the opt to the lib folder."); - LOG.debug("Caught exception", e); + final String msg = "Could not load Queryable State Server. " + ERROR_MESSAGE_ON_LOAD_FAILURE; + if (LOG.isDebugEnabled()) { + LOG.debug(msg + " Cause: " + e.getMessage()); + } else { + LOG.warn(msg); + } return null; } catch (InvocationTargetException e) { LOG.error("Queryable State Server could not be created: ", e.getTargetException()); From 054af9974638bb4f1b994622bb0ade6cf35468ac Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 13 Feb 2018 12:10:22 +0100 Subject: [PATCH 263/367] [FLINK-8520][cassandra] Fix race condition This closes #5474. --- .../streaming/connectors/cassandra/CassandraSinkBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java index 7a6efd948c505..6d1b095b0f722 100644 --- a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java @@ -134,8 +134,8 @@ public void snapshotState(FunctionSnapshotContext ctx) throws Exception { } private void waitForPendingUpdates() throws InterruptedException { - while (updatesPending.get() > 0) { - synchronized (updatesPending) { + synchronized (updatesPending) { + while (updatesPending.get() > 0) { updatesPending.wait(); } } From 50b648473f1ec7e4e11f511c6027d66694db7e7b Mon Sep 17 00:00:00 2001 From: Steven Langbroek Date: Wed, 31 Jan 2018 17:26:28 +0100 Subject: [PATCH 264/367] [FLINK-8308] Remove explicit yajl-ruby dependency, update Jekyll to 3+ --- docs/Gemfile | 8 +-- docs/Gemfile.lock | 99 ++++++++++++++------------------ docs/_config.yml | 7 +-- docs/_layouts/plain.html | 2 +- docs/_plugins/highlightCode.rb | 100 --------------------------------- docs/build_docs.sh | 1 - docs/ruby2/Gemfile | 32 ----------- docs/ruby2/Gemfile.lock | 84 --------------------------- 8 files changed, 47 insertions(+), 286 deletions(-) delete mode 100644 docs/_plugins/highlightCode.rb delete mode 100644 docs/ruby2/Gemfile delete mode 100644 docs/ruby2/Gemfile.lock diff --git a/docs/Gemfile b/docs/Gemfile index b642e7603cea5..683bd557ca5fc 100644 --- a/docs/Gemfile +++ b/docs/Gemfile @@ -18,14 +18,10 @@ source 'https://rubygems.org' -ruby '>= 1.9.0' +ruby '>= 2.1.0' -gem 'jekyll', '2.5.3' -gem 'kramdown', '1.10.0' +gem 'jekyll', '3.7.2' gem 'addressable', '2.4.0' gem 'octokit', '~> 4.3.0' -gem 'pygments.rb', '0.6.3' gem 'therubyracer', '0.12.2' gem 'json', '2.0.4' -gem 'yajl-ruby', '1.2.2' -gem 'jekyll-coffeescript', '1.0.2' diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index 889a5fea3cd40..ab3a1d3ecae9e 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -2,94 +2,81 @@ GEM remote: https://rubygems.org/ specs: addressable (2.4.0) - blankslate (2.1.2.4) - classifier-reborn (2.0.4) - fast-stemmer (~> 1.0) - coffee-script (2.4.1) - coffee-script-source - execjs - coffee-script-source (1.11.1) - colorator (0.1) - execjs (2.7.0) + colorator (1.1.0) + concurrent-ruby (1.0.5) + em-websocket (0.5.1) + eventmachine (>= 0.12.9) + http_parser.rb (~> 0.6.0) + eventmachine (1.2.5) faraday (0.9.2) multipart-post (>= 1.2, < 3) - fast-stemmer (1.0.2) - ffi (1.9.14) - jekyll (2.5.3) - classifier-reborn (~> 2.0) - colorator (~> 0.1) - jekyll-coffeescript (~> 1.0) - jekyll-gist (~> 1.0) - jekyll-paginate (~> 1.0) + ffi (1.9.18) + forwardable-extended (2.6.0) + http_parser.rb (0.6.0) + i18n (0.9.3) + concurrent-ruby (~> 1.0) + jekyll (3.7.2) + addressable (~> 2.4) + colorator (~> 1.0) + em-websocket (~> 0.5) + i18n (~> 0.7) jekyll-sass-converter (~> 1.0) - jekyll-watch (~> 1.1) - kramdown (~> 1.3) - liquid (~> 2.6.1) + jekyll-watch (~> 2.0) + kramdown (~> 1.14) + liquid (~> 4.0) mercenary (~> 0.3.3) - pygments.rb (~> 0.6.0) - redcarpet (~> 3.1) + pathutil (~> 0.9) + rouge (>= 1.7, < 4) safe_yaml (~> 1.0) - toml (~> 0.1.0) - jekyll-coffeescript (1.0.2) - coffee-script (~> 2.2) - coffee-script-source (~> 1.11.1) - jekyll-gist (1.4.0) - octokit (~> 4.2) - jekyll-paginate (1.1.0) - jekyll-sass-converter (1.5.0) + jekyll-sass-converter (1.5.1) sass (~> 3.4) - jekyll-watch (1.5.0) - listen (~> 3.0, < 3.1) + jekyll-watch (2.0.0) + listen (~> 3.0) json (2.0.4) - kramdown (1.10.0) - libv8 (3.16.14.17) - liquid (2.6.3) - listen (3.0.8) + kramdown (1.16.2) + libv8 (3.16.14.19) + liquid (4.0.0) + listen (3.1.5) rb-fsevent (~> 0.9, >= 0.9.4) rb-inotify (~> 0.9, >= 0.9.7) + ruby_dep (~> 1.2) mercenary (0.3.6) multipart-post (2.0.0) octokit (4.3.0) sawyer (~> 0.7.0, >= 0.5.3) - parslet (1.5.0) - blankslate (~> 2.0) - posix-spawn (0.3.12) - pygments.rb (0.6.3) - posix-spawn (~> 0.3.6) - yajl-ruby (~> 1.2.0) - rb-fsevent (0.9.8) - rb-inotify (0.9.7) - ffi (>= 0.5.0) - redcarpet (3.3.4) + pathutil (0.16.1) + forwardable-extended (~> 2.6) + rb-fsevent (0.10.2) + rb-inotify (0.9.10) + ffi (>= 0.5.0, < 2) ref (2.0.0) + rouge (3.1.1) + ruby_dep (1.5.0) safe_yaml (1.0.4) - sass (3.4.22) + sass (3.5.5) + sass-listen (~> 4.0.0) + sass-listen (4.0.0) + rb-fsevent (~> 0.9, >= 0.9.4) + rb-inotify (~> 0.9, >= 0.9.7) sawyer (0.7.0) addressable (>= 2.3.5, < 2.5) faraday (~> 0.8, < 0.10) therubyracer (0.12.2) libv8 (~> 3.16.14.0) ref - toml (0.1.2) - parslet (~> 1.5.0) - yajl-ruby (1.2.2) PLATFORMS ruby DEPENDENCIES addressable (= 2.4.0) - jekyll (= 2.5.3) - jekyll-coffeescript (= 1.0.2) + jekyll (= 3.7.2) json (= 2.0.4) - kramdown (= 1.10.0) octokit (~> 4.3.0) - pygments.rb (= 0.6.3) therubyracer (= 0.12.2) - yajl-ruby (= 1.2.2) RUBY VERSION ruby 2.3.1p112 BUNDLED WITH - 1.15.0 + 1.16.1 diff --git a/docs/_config.yml b/docs/_config.yml index 866f346271451..4c52b73627c1c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -76,12 +76,7 @@ defaults: layout: plain nav-pos: 99999 # Move to end if no pos specified -markdown: KramdownPygments -highlighter: pygments +host: 0.0.0.0 kramdown: - input: GFM # GitHub syntax - hard_wrap: false # Don't translate new lines to
s toc_levels: 1..3 # Include h1-h3 for ToC - -host: 0.0.0.0 diff --git a/docs/_layouts/plain.html b/docs/_layouts/plain.html index e991f78f82be3..7bd0030341b11 100644 --- a/docs/_layouts/plain.html +++ b/docs/_layouts/plain.html @@ -26,7 +26,7 @@ {% for i in (1..10) %} {% assign active_pages = active_pages | push: active %} {% if active.nav-parent_id %} - {% assign next = (site.pages | where: "nav-id" , active.nav-parent_id ) %} + {% assign next = site.pages | where: "nav-id" , active.nav-parent_id %} {% if next.size > 0 %} {% assign active = next[0] %} {% else %} diff --git a/docs/_plugins/highlightCode.rb b/docs/_plugins/highlightCode.rb deleted file mode 100644 index 74f6d6f0ff144..0000000000000 --- a/docs/_plugins/highlightCode.rb +++ /dev/null @@ -1,100 +0,0 @@ -# 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. - -# This file was derived from https://github.com/navarroj/krampygs -# written by Juan Navarro and released under the Unlicense (http://unlicense.org). - -require 'kramdown' -require 'pygments' - -module Kramdown - module Converter - - class Pygs < Html - - def convert_codeblock(el, indent) - attr = el.attr.dup - lang = extract_code_language!(attr) || @options[:kramdown_default_lang] - code = pygmentize(el.value, lang) - code_attr = {} - code_attr['class'] = "language-#{lang}" if lang - "#{' '*indent}

#{code}
\n" - end - - def convert_codespan(el, indent) - attr = el.attr.dup - lang = extract_code_language!(attr) || @options[:kramdown_default_lang] - code = pygmentize(el.value, lang) - if lang - attr['class'] = "highlight" - if attr.has_key?('class') - attr['class'] += " language-#{lang}" - else - attr['class'] = "language-#{lang}" - end - end - "#{code}" - end - - def pygmentize(code, lang) - if lang - Pygments.highlight(code, - :lexer => lang, - :options => { :encoding => 'utf-8', :nowrap => true }) - else - escape_html(code) - end - end - - end - end -end - -class Jekyll::Converters::Markdown::KramdownPygments - - def initialize(config) - require 'kramdown' - @config = config - rescue LoadError - STDERR.puts 'You are missing a library required for Markdown. Please run:' - STDERR.puts ' $ [sudo] gem install kramdown' - raise FatalException.new("Missing dependency: kramdown") - end - - def matches(ext) - ext =~ /^\.md$/i - end - - def output_ext(ext) - ".html" - end - - def convert(content) - html = Kramdown::Document.new(content, { - :auto_ids => @config['kramdown']['auto_ids'], - :footnote_nr => @config['kramdown']['footnote_nr'], - :entity_output => @config['kramdown']['entity_output'], - :toc_levels => @config['kramdown']['toc_levels'], - :smart_quotes => @config['kramdown']['smart_quotes'], - :coderay_default_lang => @config['kramdown']['default_lang'], - :input => @config['kramdown']['input'], - :hard_wrap => @config['kramdown']['hard_wrap'] - }).to_pygs - return html - end - -end \ No newline at end of file diff --git a/docs/build_docs.sh b/docs/build_docs.sh index cbdd1d42cd591..0ef7170bd7ebb 100755 --- a/docs/build_docs.sh +++ b/docs/build_docs.sh @@ -54,7 +54,6 @@ while getopts "pi" opt; do ;; i) [[ `ruby -v` =~ 'ruby 1' ]] && echo "Error: building the docs with the incremental option requires at least ruby 2.0" && exit 1 - cd ruby2 bundle install --path .rubydeps JEKYLL_CMD="liveserve --baseurl= --watch --incremental" ;; diff --git a/docs/ruby2/Gemfile b/docs/ruby2/Gemfile deleted file mode 100644 index f71ceb013ad03..0000000000000 --- a/docs/ruby2/Gemfile +++ /dev/null @@ -1,32 +0,0 @@ -################################################################################ -# 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. -################################################################################ - -source 'https://rubygems.org' - -ruby '~> 2' - -gem 'jekyll', '~> 3.3.0' -gem 'kramdown', '~> 1.13.0' -gem 'json', '2.0.4' -group :jekyll_plugins do - gem 'hawkins' -end -gem 'pygments.rb', '0.6.3' -gem 'therubyracer', '0.12.2' -gem 'yajl-ruby', '1.2.2' -gem 'jekyll-coffeescript', '1.0.2' diff --git a/docs/ruby2/Gemfile.lock b/docs/ruby2/Gemfile.lock deleted file mode 100644 index ade8cf7b2d470..0000000000000 --- a/docs/ruby2/Gemfile.lock +++ /dev/null @@ -1,84 +0,0 @@ -GEM - remote: https://rubygems.org/ - specs: - addressable (2.5.0) - public_suffix (~> 2.0, >= 2.0.2) - coffee-script (2.4.1) - coffee-script-source - execjs - coffee-script-source (1.11.1) - colorator (1.1.0) - em-websocket (0.5.1) - eventmachine (>= 0.12.9) - http_parser.rb (~> 0.6.0) - eventmachine (1.2.1) - execjs (2.7.0) - ffi (1.9.14) - forwardable-extended (2.6.0) - hawkins (2.0.4) - em-websocket (~> 0.5) - jekyll (~> 3.1) - http_parser.rb (0.6.0) - jekyll (3.3.1) - addressable (~> 2.4) - colorator (~> 1.0) - jekyll-sass-converter (~> 1.0) - jekyll-watch (~> 1.1) - kramdown (~> 1.3) - liquid (~> 3.0) - mercenary (~> 0.3.3) - pathutil (~> 0.9) - rouge (~> 1.7) - safe_yaml (~> 1.0) - jekyll-coffeescript (1.0.2) - coffee-script (~> 2.2) - coffee-script-source (~> 1.11.1) - jekyll-sass-converter (1.5.0) - sass (~> 3.4) - jekyll-watch (1.5.0) - listen (~> 3.0, < 3.1) - json (2.0.4) - kramdown (1.13.1) - libv8 (3.16.14.17) - liquid (3.0.6) - listen (3.0.8) - rb-fsevent (~> 0.9, >= 0.9.4) - rb-inotify (~> 0.9, >= 0.9.7) - mercenary (0.3.6) - pathutil (0.14.0) - forwardable-extended (~> 2.6) - posix-spawn (0.3.12) - public_suffix (2.0.4) - pygments.rb (0.6.3) - posix-spawn (~> 0.3.6) - yajl-ruby (~> 1.2.0) - rb-fsevent (0.9.8) - rb-inotify (0.9.7) - ffi (>= 0.5.0) - ref (2.0.0) - rouge (1.11.1) - safe_yaml (1.0.4) - sass (3.4.22) - therubyracer (0.12.2) - libv8 (~> 3.16.14.0) - ref - yajl-ruby (1.2.2) - -PLATFORMS - ruby - -DEPENDENCIES - hawkins - jekyll (~> 3.3.0) - jekyll-coffeescript (= 1.0.2) - json (= 2.0.4) - kramdown (~> 1.13.0) - pygments.rb (= 0.6.3) - therubyracer (= 0.12.2) - yajl-ruby (= 1.2.2) - -RUBY VERSION - ruby 2.3.0p0 - -BUNDLED WITH - 1.13.6 From d4435e159681bed5baa351d2cac06a5e9626fc11 Mon Sep 17 00:00:00 2001 From: Steven Langbroek Date: Thu, 1 Feb 2018 17:52:24 +0100 Subject: [PATCH 265/367] [FLINK-8303] Add hawkins back to Gemfile --- docs/Gemfile | 4 ++++ docs/Gemfile.lock | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/docs/Gemfile b/docs/Gemfile index 683bd557ca5fc..6b3d0b4fbb1e6 100644 --- a/docs/Gemfile +++ b/docs/Gemfile @@ -25,3 +25,7 @@ gem 'addressable', '2.4.0' gem 'octokit', '~> 4.3.0' gem 'therubyracer', '0.12.2' gem 'json', '2.0.4' + +group :jekyll_plugins do + gem 'hawkins' +end diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index ab3a1d3ecae9e..1faadc0bc1d57 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -12,6 +12,9 @@ GEM multipart-post (>= 1.2, < 3) ffi (1.9.18) forwardable-extended (2.6.0) + hawkins (2.0.5) + em-websocket (~> 0.5) + jekyll (~> 3.1) http_parser.rb (0.6.0) i18n (0.9.3) concurrent-ruby (~> 1.0) @@ -70,6 +73,7 @@ PLATFORMS DEPENDENCIES addressable (= 2.4.0) + hawkins jekyll (= 3.7.2) json (= 2.0.4) octokit (~> 4.3.0) From cc76c323f081fa9c3fb3a28abd253b7a67a26559 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Thu, 15 Feb 2018 11:58:31 +0100 Subject: [PATCH 266/367] [FLINK-8303] [docs] Allow to overwrite ruby/gem binary This closes #5395. --- docs/build_docs.sh | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/build_docs.sh b/docs/build_docs.sh index 0ef7170bd7ebb..f9ffc5a02c9b6 100755 --- a/docs/build_docs.sh +++ b/docs/build_docs.sh @@ -17,6 +17,9 @@ # limitations under the License. ################################################################################ +RUBY=${RUBY:-ruby} +GEM=${GEM:-gem} + set -e cd "$(dirname ${BASH_SOURCE[0]})" @@ -28,12 +31,12 @@ if [ "`command -v bundle`" == "" ]; then echo "Attempting to install locally. If this doesn't work, please install with 'gem install bundler'." # Adjust the PATH to discover the locally installed Ruby gem - if which ruby >/dev/null && which gem >/dev/null; then - export PATH="$(ruby -rubygems -e 'puts Gem.user_dir')/bin:$PATH" + if which ${RUBY} >/dev/null && which gem >/dev/null; then + export PATH="$(${RUBY} -rubygems -e 'puts Gem.user_dir')/bin:$PATH" fi # install bundler locally - gem install --user-install bundler + ${GEM} install --user-install bundler fi # Install Ruby dependencies locally @@ -53,7 +56,7 @@ while getopts "pi" opt; do JEKYLL_CMD="serve --baseurl= --watch" ;; i) - [[ `ruby -v` =~ 'ruby 1' ]] && echo "Error: building the docs with the incremental option requires at least ruby 2.0" && exit 1 + [[ `${RUBY} -v` =~ 'ruby 1' ]] && echo "Error: building the docs with the incremental option requires at least ruby 2.0" && exit 1 bundle install --path .rubydeps JEKYLL_CMD="liveserve --baseurl= --watch --incremental" ;; From 528317c8f23a7f20493510e57c21c72a1fc42b3d Mon Sep 17 00:00:00 2001 From: zentol Date: Sun, 18 Feb 2018 21:54:56 +0100 Subject: [PATCH 267/367] [FLINK-8692][docs] Remove extra parenthesis in scala code samples --- docs/dev/api_concepts.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/dev/api_concepts.md b/docs/dev/api_concepts.md index c6752254cd666..c72478d359188 100644 --- a/docs/dev/api_concepts.md +++ b/docs/dev/api_concepts.md @@ -495,7 +495,7 @@ The most basic way is to implement one of the provided interfaces: {% highlight java %} class MyMapFunction implements MapFunction { public Integer map(String value) { return Integer.parseInt(value); } -}); +}; data.map(new MyMapFunction()); {% endhighlight %} @@ -528,7 +528,7 @@ instead take as argument a *rich* function. For example, instead of {% highlight java %} class MyMapFunction implements MapFunction { public Integer map(String value) { return Integer.parseInt(value); } -}); +}; {% endhighlight %} you can write @@ -536,7 +536,7 @@ you can write {% highlight java %} class MyMapFunction extends RichMapFunction { public Integer map(String value) { return Integer.parseInt(value); } -}); +}; {% endhighlight %} and pass the function as usual to a `map` transformation: @@ -586,7 +586,7 @@ you can write {% highlight scala %} class MyMapFunction extends RichMapFunction[String, Int] { def map(in: String):Int = { in.toInt } -}) +}; {% endhighlight %} and pass the function to a `map` transformation: From 45efe4702ac1d932afa1b322e93264c28aefbc4a Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 13 Feb 2018 10:31:31 +0100 Subject: [PATCH 268/367] [FLINK-8621][prometheus][tests] Remove endpointIsUnavailableAfterReporterIsClosed() The test is inherently unstable as it will always fail if any other server is started on the port between the closing of the reporter and the polling of metrics. This closes #5473. --- .../flink/metrics/prometheus/PrometheusReporterTest.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java index 6704189da2713..1a70865e355da 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java @@ -156,15 +156,6 @@ public void histogramIsReportedAsPrometheusSummary() throws UnirestException { } } - @Test - public void endpointIsUnavailableAfterReporterIsClosed() throws UnirestException { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); - PrometheusReporter reporter = (PrometheusReporter) registry.getReporters().get(0); - reporter.close(); - thrown.expect(UnirestException.class); - pollMetrics(reporter.getPort()); - } - @Test public void invalidCharactersAreReplacedWithUnderscore() { assertThat(PrometheusReporter.replaceInvalidChars(""), equalTo("")); From d88f43bb064ff34ec389915f0205fb1728d1bf56 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 13 Feb 2018 10:33:03 +0100 Subject: [PATCH 269/367] [hotfix][prometheus] Document internal usage of CollectorRegistry.defaultRegistry It appeared as if the HTTPServer wasn't actually doing anything, but it internally accessed the singleton registry that we also access to register metrics. --- .../org/apache/flink/metrics/prometheus/PrometheusReporter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java index fad3ced58c1a2..48fd8a42b53cc 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java @@ -98,6 +98,7 @@ public void open(MetricConfig config) { while (ports.hasNext()) { int port = ports.next(); try { + // internally accesses CollectorRegistry.defaultRegistry httpServer = new HTTPServer(port); this.port = port; LOG.info("Started PrometheusReporter HTTP server on port {}.", port); From 527faf63e21eaeedd3ab471ec7d2556985422734 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 13 Feb 2018 10:33:31 +0100 Subject: [PATCH 270/367] [hotfix][prometheus][tests] Add utility for generating port ranges --- .../prometheus/PrometheusReporterTest.java | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java index 1a70865e355da..8b4d286cb1ada 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java @@ -45,6 +45,8 @@ import org.junit.rules.ExpectedException; import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; import static org.apache.flink.metrics.prometheus.PrometheusReporter.ARG_PORT; import static org.hamcrest.Matchers.containsString; @@ -66,6 +68,8 @@ public class PrometheusReporterTest extends TestLogger { private static final String DEFAULT_LABELS = "{" + DIMENSIONS + ",}"; private static final String SCOPE_PREFIX = "flink_taskmanager_"; + private static final PortRangeProvider portRangeProvider = new PortRangeProvider(); + @Rule public ExpectedException thrown = ExpectedException.none(); @@ -75,7 +79,7 @@ public class PrometheusReporterTest extends TestLogger { @Before public void setupReporter() { - registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", portRangeProvider.next()))); metricGroup = new FrontMetricGroup<>(0, new TaskManagerMetricGroup(registry, HOST_NAME, TASK_MANAGER)); reporter = (PrometheusReporter) registry.getReporters().get(0); } @@ -235,7 +239,7 @@ class SomeMetricType implements Metric{} @Test public void cannotStartTwoReportersOnSamePort() { - final MetricRegistryImpl fixedPort1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); + final MetricRegistryImpl fixedPort1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", portRangeProvider.next()))); assertThat(fixedPort1.getReporters(), hasSize(1)); PrometheusReporter firstReporter = (PrometheusReporter) fixedPort1.getReporters().get(0); @@ -249,8 +253,9 @@ public void cannotStartTwoReportersOnSamePort() { @Test public void canStartTwoReportersWhenUsingPortRange() { - final MetricRegistryImpl portRange1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9200-9300"))); - final MetricRegistryImpl portRange2 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", "9200-9300"))); + String portRange = portRangeProvider.next(); + final MetricRegistryImpl portRange1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", portRange))); + final MetricRegistryImpl portRange2 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", portRange))); assertThat(portRange1.getReporters(), hasSize(1)); assertThat(portRange2.getReporters(), hasSize(1)); @@ -280,4 +285,32 @@ static Configuration createConfigWithOneReporter(String reporterName, String por public void closeReporterAndShutdownRegistry() { registry.shutdown(); } + + /** + * Utility class providing distinct port ranges. + */ + private static class PortRangeProvider implements Iterator { + + private int base = 9000; + + @Override + public boolean hasNext() { + return base < 14000; // arbitrary limit that should be sufficient for test purposes + } + + /** + * Returns the next port range containing exactly 100 ports. + * + * @return next port range + */ + public String next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + int lowEnd = base; + int highEnd = base + 99; + base += 100; + return String.valueOf(lowEnd) + "-" + String.valueOf(highEnd); + } + } } From a7df42485a6d96bb7167c6597b9b0d1c59390f4d Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Feb 2018 09:52:23 +0100 Subject: [PATCH 271/367] [FLINK-8574][travis] Add timestamp to logging messages This closes #5419. --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 5e2ef74aa8d28..7d2035b4651bf 100644 --- a/.travis.yml +++ b/.travis.yml @@ -110,6 +110,7 @@ before_install: - "rm apache-maven-3.2.5-bin.zip" - "export M2_HOME=$PWD/apache-maven-3.2.5" - "export PATH=$M2_HOME/bin:$PATH" + - "export MAVEN_OPTS=\"-Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS\"" # just in case: clean up the .m2 home and remove invalid jar files - 'test ! -d $HOME/.m2/repository/ || find $HOME/.m2/repository/ -name "*.jar" -exec sh -c ''if ! zip -T {} >/dev/null ; then echo "deleting invalid file: {}"; rm {} ; fi'' \;' From 82e6f8d5b8623f97f854ae9936e6754dc09ef5af Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 21 Feb 2018 18:08:13 +0100 Subject: [PATCH 272/367] [FLINK-8735] Rename StatefulJobSavepointMigrationITCase This is preparation for modifying a new ITCase to use modern state features. --- ...se.java => LegacyStatefulJobSavepointMigrationITCase.java} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/{StatefulJobSavepointMigrationITCase.java => LegacyStatefulJobSavepointMigrationITCase.java} (99%) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java index 641e450b23352..21c6ee2306ad1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java @@ -63,7 +63,7 @@ * migrating for multiple previous Flink versions, as well as for different state backends. */ @RunWith(Parameterized.class) -public class StatefulJobSavepointMigrationITCase extends SavepointMigrationTestBase { +public class LegacyStatefulJobSavepointMigrationITCase extends SavepointMigrationTestBase { private static final int NUM_SOURCE_ELEMENTS = 4; @@ -89,7 +89,7 @@ public static Collection> parameters () { private final MigrationVersion testMigrateVersion; private final String testStateBackend; - public StatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) { + public LegacyStatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) { this.testMigrateVersion = testMigrateVersionAndBackend.f0; this.testStateBackend = testMigrateVersionAndBackend.f1; } From f06ec38f223e94c926964b8760115d89988478c1 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 21 Feb 2018 18:10:55 +0100 Subject: [PATCH 273/367] [FLINK-8735] Add new StatefulJobSavepointMigrationITCase This new test does not pretend to use legacy state but now instead uses the more modern operator state varieties. --- .../StatefulJobSavepointMigrationITCase.java | 634 ++++++++++++++++++ .../_metadata | Bin 0 -> 44848 bytes .../_metadata | Bin 0 -> 44776 bytes 3 files changed, 634 insertions(+) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java create mode 100644 flink-tests/src/test/resources/new-stateful-udf-migration-itcase-flink1.4-rocksdb-savepoint/_metadata create mode 100644 flink-tests/src/test/resources/new-stateful-udf-migration-itcase-flink1.4-savepoint/_metadata diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java new file mode 100644 index 0000000000000..a8895e60ed7f1 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java @@ -0,0 +1,634 @@ +/* + * 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.flink.test.checkpointing.utils; + +import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.InternalTimer; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.migration.MigrationVersion; +import org.apache.flink.util.Collector; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +/** + * Migration ITCases for a stateful job. The tests are parameterized to cover + * migrating for multiple previous Flink versions, as well as for different state backends. + */ +@RunWith(Parameterized.class) +public class StatefulJobSavepointMigrationITCase extends SavepointMigrationTestBase { + + private static final int NUM_SOURCE_ELEMENTS = 4; + + /** + * This test runs in either of two modes: 1) we want to generate the binary savepoint, i.e. + * we have to run the checkpointing functions 2) we want to verify restoring, so we have to run + * the checking functions. + */ + public enum ExecutionMode { + PERFORM_SAVEPOINT, + VERIFY_SAVEPOINT + } + + // TODO change this to PERFORM_SAVEPOINT to regenerate binary savepoints + private final ExecutionMode executionMode = ExecutionMode.VERIFY_SAVEPOINT; + + @Parameterized.Parameters(name = "Migrate Savepoint / Backend: {0}") + public static Collection> parameters () { + return Arrays.asList( + Tuple2.of(MigrationVersion.v1_4, AbstractStateBackend.MEMORY_STATE_BACKEND_NAME), + Tuple2.of(MigrationVersion.v1_4, AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME)); + } + + private final MigrationVersion testMigrateVersion; + private final String testStateBackend; + + public StatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) { + this.testMigrateVersion = testMigrateVersionAndBackend.f0; + this.testStateBackend = testMigrateVersionAndBackend.f1; + } + + @Test + public void testSavepoint() throws Exception { + + final int parallelism = 4; + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + switch (testStateBackend) { + case AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME: + env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend())); + break; + case AbstractStateBackend.MEMORY_STATE_BACKEND_NAME: + env.setStateBackend(new MemoryStateBackend()); + break; + default: + throw new UnsupportedOperationException(); + } + + env.enableCheckpointing(500); + env.setParallelism(parallelism); + env.setMaxParallelism(parallelism); + + SourceFunction> nonParallelSource; + SourceFunction> parallelSource; + RichFlatMapFunction, Tuple2> flatMap; + OneInputStreamOperator, Tuple2> timelyOperator; + + if (executionMode == ExecutionMode.PERFORM_SAVEPOINT) { + nonParallelSource = new CheckpointingNonParallelSourceWithListState(NUM_SOURCE_ELEMENTS); + parallelSource = new CheckpointingParallelSourceWithUnionListState(NUM_SOURCE_ELEMENTS); + flatMap = new CheckpointingKeyedStateFlatMap(); + timelyOperator = new CheckpointingTimelyStatefulOperator(); + } else if (executionMode == ExecutionMode.VERIFY_SAVEPOINT) { + nonParallelSource = new CheckingNonParallelSourceWithListState(NUM_SOURCE_ELEMENTS); + parallelSource = new CheckingParallelSourceWithUnionListState(NUM_SOURCE_ELEMENTS); + flatMap = new CheckingKeyedStateFlatMap(); + timelyOperator = new CheckingTimelyStatefulOperator(); + } else { + throw new IllegalStateException("Unknown ExecutionMode " + executionMode); + } + + env + .addSource(nonParallelSource).uid("CheckpointingSource1") + .keyBy(0) + .flatMap(flatMap).startNewChain().uid("CheckpointingKeyedStateFlatMap1") + .keyBy(0) + .transform( + "timely_stateful_operator", + new TypeHint>() {}.getTypeInfo(), + timelyOperator).uid("CheckpointingTimelyStatefulOperator1") + .addSink(new AccumulatorCountingSink<>()); + + env + .addSource(parallelSource).uid("CheckpointingSource2") + .keyBy(0) + .flatMap(flatMap).startNewChain().uid("CheckpointingKeyedStateFlatMap2") + .keyBy(0) + .transform( + "timely_stateful_operator", + new TypeHint>() {}.getTypeInfo(), + timelyOperator).uid("CheckpointingTimelyStatefulOperator2") + .addSink(new AccumulatorCountingSink<>()); + + if (executionMode == ExecutionMode.PERFORM_SAVEPOINT) { + executeAndSavepoint( + env, + "src/test/resources/" + getSavepointPath(testMigrateVersion, testStateBackend), + new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS * 2)); + } else { + restoreAndExecute( + env, + getResourceFilename(getSavepointPath(testMigrateVersion, testStateBackend)), + new Tuple2<>(CheckingNonParallelSourceWithListState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1), + new Tuple2<>(CheckingParallelSourceWithUnionListState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, parallelism), + new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS * 2), + new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS * 2), + new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS * 2), + new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS * 2), + new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS * 2)); + } + } + + private String getSavepointPath(MigrationVersion savepointVersion, String backendType) { + switch (backendType) { + case AbstractStateBackend.ROCKSDB_STATE_BACKEND_NAME: + return "new-stateful-udf-migration-itcase-flink" + savepointVersion + "-rocksdb-savepoint"; + case AbstractStateBackend.MEMORY_STATE_BACKEND_NAME: + return "new-stateful-udf-migration-itcase-flink" + savepointVersion + "-savepoint"; + default: + throw new UnsupportedOperationException(); + } + } + + private static class CheckpointingNonParallelSourceWithListState + implements SourceFunction>, CheckpointedFunction { + + static final ListStateDescriptor STATE_DESCRIPTOR = + new ListStateDescriptor<>("source-state", StringSerializer.INSTANCE); + + static final String CHECKPOINTED_STRING = "Here be dragons!"; + static final String CHECKPOINTED_STRING_1 = "Here be more dragons!"; + static final String CHECKPOINTED_STRING_2 = "Here be yet more dragons!"; + static final String CHECKPOINTED_STRING_3 = "Here be the mostest dragons!"; + + private static final long serialVersionUID = 1L; + + private volatile boolean isRunning = true; + + private final int numElements; + + private transient ListState unionListState; + + CheckpointingNonParallelSourceWithListState(int numElements) { + this.numElements = numElements; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + unionListState.clear(); + unionListState.add(CHECKPOINTED_STRING); + unionListState.add(CHECKPOINTED_STRING_1); + unionListState.add(CHECKPOINTED_STRING_2); + unionListState.add(CHECKPOINTED_STRING_3); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + unionListState = context.getOperatorStateStore().getListState( + STATE_DESCRIPTOR); + } + + @Override + public void run(SourceContext> ctx) throws Exception { + + ctx.emitWatermark(new Watermark(0)); + + synchronized (ctx.getCheckpointLock()) { + for (long i = 0; i < numElements; i++) { + ctx.collect(new Tuple2<>(i, i)); + } + } + + // don't emit a final watermark so that we don't trigger the registered event-time + // timers + while (isRunning) { + Thread.sleep(20); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class CheckingNonParallelSourceWithListState + extends RichSourceFunction> implements CheckpointedFunction { + + private static final long serialVersionUID = 1L; + + static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingNonParallelSourceWithListState.class + "_RESTORE_CHECK"; + + private volatile boolean isRunning = true; + + private final int numElements; + + CheckingNonParallelSourceWithListState(int numElements) { + this.numElements = numElements; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + ListState unionListState = context.getOperatorStateStore().getListState( + CheckpointingNonParallelSourceWithListState.STATE_DESCRIPTOR); + + if (context.isRestored()) { + assertThat(unionListState.get(), + containsInAnyOrder( + CheckpointingNonParallelSourceWithListState.CHECKPOINTED_STRING, + CheckpointingNonParallelSourceWithListState.CHECKPOINTED_STRING_1, + CheckpointingNonParallelSourceWithListState.CHECKPOINTED_STRING_2, + CheckpointingNonParallelSourceWithListState.CHECKPOINTED_STRING_3)); + + getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter()); + getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1); + } else { + throw new RuntimeException( + "This source should always be restored because it's only used when restoring from a savepoint."); + } + } + + @Override + public void run(SourceContext> ctx) throws Exception { + + // immediately trigger any set timers + ctx.emitWatermark(new Watermark(1000)); + + synchronized (ctx.getCheckpointLock()) { + for (long i = 0; i < numElements; i++) { + ctx.collect(new Tuple2<>(i, i)); + } + } + + while (isRunning) { + Thread.sleep(20); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class CheckpointingParallelSourceWithUnionListState + extends RichSourceFunction> implements CheckpointedFunction { + + static final ListStateDescriptor STATE_DESCRIPTOR = + new ListStateDescriptor<>("source-state", StringSerializer.INSTANCE); + + static final String[] CHECKPOINTED_STRINGS = { + "Here be dragons!", + "Here be more dragons!", + "Here be yet more dragons!", + "Here be the mostest dragons!" }; + + private static final long serialVersionUID = 1L; + + private volatile boolean isRunning = true; + + private final int numElements; + + private transient ListState unionListState; + + CheckpointingParallelSourceWithUnionListState(int numElements) { + this.numElements = numElements; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + unionListState.clear(); + + for (String s : CHECKPOINTED_STRINGS) { + if (s.hashCode() % getRuntimeContext().getNumberOfParallelSubtasks() == getRuntimeContext().getIndexOfThisSubtask()) { + unionListState.add(s); + } + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + unionListState = context.getOperatorStateStore().getUnionListState( + STATE_DESCRIPTOR); + } + + @Override + public void run(SourceContext> ctx) throws Exception { + + ctx.emitWatermark(new Watermark(0)); + + synchronized (ctx.getCheckpointLock()) { + for (long i = 0; i < numElements; i++) { + if (i % getRuntimeContext().getNumberOfParallelSubtasks() == getRuntimeContext().getIndexOfThisSubtask()) { + ctx.collect(new Tuple2<>(i, i)); + } + } + } + + // don't emit a final watermark so that we don't trigger the registered event-time + // timers + while (isRunning) { + Thread.sleep(20); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class CheckingParallelSourceWithUnionListState + extends RichParallelSourceFunction> implements CheckpointedFunction { + + private static final long serialVersionUID = 1L; + + static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingParallelSourceWithUnionListState.class + "_RESTORE_CHECK"; + + private volatile boolean isRunning = true; + + private final int numElements; + + CheckingParallelSourceWithUnionListState(int numElements) { + this.numElements = numElements; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + ListState unionListState = context.getOperatorStateStore().getUnionListState( + CheckpointingNonParallelSourceWithListState.STATE_DESCRIPTOR); + + if (context.isRestored()) { + assertThat(unionListState.get(), + containsInAnyOrder(CheckpointingParallelSourceWithUnionListState.CHECKPOINTED_STRINGS)); + + getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter()); + getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1); + } else { + throw new RuntimeException( + "This source should always be restored because it's only used when restoring from a savepoint."); + } + } + + @Override + public void run(SourceContext> ctx) throws Exception { + + // immediately trigger any set timers + ctx.emitWatermark(new Watermark(1000)); + + synchronized (ctx.getCheckpointLock()) { + for (long i = 0; i < numElements; i++) { + if (i % getRuntimeContext().getNumberOfParallelSubtasks() == getRuntimeContext().getIndexOfThisSubtask()) { + ctx.collect(new Tuple2<>(i, i)); + } + } + } + + while (isRunning) { + Thread.sleep(20); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class CheckpointingKeyedStateFlatMap extends RichFlatMapFunction, Tuple2> { + + private static final long serialVersionUID = 1L; + + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state-name", LongSerializer.INSTANCE); + + @Override + public void flatMap(Tuple2 value, Collector> out) throws Exception { + out.collect(value); + + getRuntimeContext().getState(stateDescriptor).update(value.f1); + } + } + + private static class CheckingKeyedStateFlatMap extends RichFlatMapFunction, Tuple2> { + + private static final long serialVersionUID = 1L; + + static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingKeyedStateFlatMap.class + "_RESTORE_CHECK"; + + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state-name", LongSerializer.INSTANCE); + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter()); + } + + @Override + public void flatMap(Tuple2 value, Collector> out) throws Exception { + out.collect(value); + + ValueState state = getRuntimeContext().getState(stateDescriptor); + if (state == null) { + throw new RuntimeException("Missing key value state for " + value); + } + + assertEquals(value.f1, state.value()); + getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1); + } + } + + private static class CheckpointingTimelyStatefulOperator + extends AbstractStreamOperator> + implements OneInputStreamOperator, Tuple2>, Triggerable { + private static final long serialVersionUID = 1L; + + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state-name", LongSerializer.INSTANCE); + + private transient InternalTimerService timerService; + + @Override + public void open() throws Exception { + super.open(); + + timerService = getInternalTimerService( + "timer", + LongSerializer.INSTANCE, + this); + + } + + @Override + public void processElement(StreamRecord> element) throws Exception { + ValueState state = getKeyedStateBackend().getPartitionedState( + element.getValue().f0, + LongSerializer.INSTANCE, + stateDescriptor); + + state.update(element.getValue().f1); + + timerService.registerEventTimeTimer(element.getValue().f0, timerService.currentWatermark() + 10); + timerService.registerProcessingTimeTimer(element.getValue().f0, timerService.currentProcessingTime() + 30_000); + + output.collect(element); + } + + @Override + public void onEventTime(InternalTimer timer) { + + } + + @Override + public void onProcessingTime(InternalTimer timer) { + + } + + @Override + public void processWatermark(Watermark mark) { + output.emitWatermark(mark); + } + } + + private static class CheckingTimelyStatefulOperator + extends AbstractStreamOperator> + implements OneInputStreamOperator, Tuple2>, Triggerable { + private static final long serialVersionUID = 1L; + + static final String SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PROCESS_CHECKS"; + static final String SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_ET_CHECKS"; + static final String SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PT_CHECKS"; + + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state-name", LongSerializer.INSTANCE); + + @Override + public void open() throws Exception { + super.open(); + + // have to re-register to ensure that our onEventTime() is called + getInternalTimerService( + "timer", + LongSerializer.INSTANCE, + this); + + getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, new IntCounter()); + getRuntimeContext().addAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, new IntCounter()); + getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, new IntCounter()); + } + + @Override + public void processElement(StreamRecord> element) throws Exception { + ValueState state = getKeyedStateBackend().getPartitionedState( + element.getValue().f0, + LongSerializer.INSTANCE, + stateDescriptor); + + assertEquals(state.value(), element.getValue().f1); + getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR).add(1); + + output.collect(element); + } + + @Override + public void onEventTime(InternalTimer timer) throws Exception { + ValueState state = getKeyedStateBackend().getPartitionedState( + timer.getNamespace(), + LongSerializer.INSTANCE, + stateDescriptor); + + assertEquals(state.value(), timer.getNamespace()); + getRuntimeContext().getAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR).add(1); + } + + @Override + public void onProcessingTime(InternalTimer timer) throws Exception { + ValueState state = getKeyedStateBackend().getPartitionedState( + timer.getNamespace(), + LongSerializer.INSTANCE, + stateDescriptor); + + assertEquals(state.value(), timer.getNamespace()); + getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR).add(1); + } + } + + private static class AccumulatorCountingSink extends RichSinkFunction { + private static final long serialVersionUID = 1L; + + static final String NUM_ELEMENTS_ACCUMULATOR = AccumulatorCountingSink.class + "_NUM_ELEMENTS"; + + int count = 0; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + getRuntimeContext().addAccumulator(NUM_ELEMENTS_ACCUMULATOR, new IntCounter()); + } + + @Override + public void invoke(T value, Context context) throws Exception { + count++; + getRuntimeContext().getAccumulator(NUM_ELEMENTS_ACCUMULATOR).add(1); + } + } + +} diff --git a/flink-tests/src/test/resources/new-stateful-udf-migration-itcase-flink1.4-rocksdb-savepoint/_metadata b/flink-tests/src/test/resources/new-stateful-udf-migration-itcase-flink1.4-rocksdb-savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..d7c2783be323c96191c0c3629c9aa141a3f31bf8 GIT binary patch literal 44848 zcmeHQON<;x8SdHGiRJhZb1=$F*$9&eLGAXV`cX0n;t&xN0}+l85Fw?yyLxxR?94LL z>tuaE>sV0^hDRA(%7P*CUIQ0Fe2e7djX z=DH-GHz?`lrkAhYR{rz9{`vFsFTMKq%9s`60u zkK!x(9(taPv}1%Tl_O+oLJ`M05vi}U$T&e}OBdH3`1>y&{WgwuS8EJ(|DrmopDw5_ z+Bb*#Wn%k*>!dVwe#kJnYnLiv9+!bqAE&NU;gOGJ~LiTM=?~&%BZ$I$E>}&+sv1bSy#7A zdoG{9-?}bqnpV2j|9iO=-hUiFIa?0I*INRSdj~uu5=k$-<+0s2! zbMQDl<7`W}=VptnBN&)lG>-5Vmb%r0jxl2lJg7Q-(|3P-=)X_W&@RR$<{mY&l}FD# z$2$|7dA?$O)!K8iYuz^T@ybp>ZC-XE%)CXyyKWq}rZPLX-m~s4d?tR-Bssl?_m5TG zT5VN3S;sDL-|GClL9Z@nt`lv&*qiUhn{O;? z?VZuX$_}x=z$2m!35X7XF{=Vuc!BDTPyI@h(%;!4I^wX%W z2WYY3G`lMx!{%Fx+M5Sv9}XQ4rEPv-7X5zz>EciUqJ8zrJMRnt7#a}(FvbHIvPGu+ zSja%yLVKYSHnMH)xS1P~h!_M&QyT#!8hrHPkqUs6sUQrU#1*dZ_(Fy%5pfXNLdD8W zm6w_zjBzP}F9t}h9s{K9Uz#yM8gIQkLCnek3G?5!KyaFi8x{kkqIv5z$(F)s21vxP z^{t02`Nm{LD- zjrQCC(2TjIisWVJ#L%f=|HtDwnOnj2|bazQRX|YiG4r7ZG8agjz8b*F7{Tjzgy z{l}g|Iq^dDAxu_qM56%INbR2&cyVB2Ej(LBLi(ir^OO>yYbUm}1K&-6(60XLI_+P> z+DrO1QVv%rvDVdH{UKcXqw+aeXEzC*9p3ZXtg~y*u+FYF;b2N?NN=^Vy-R;%?HpWH z&g85an!IKR!i(hBDEV2-?pk-VQ0Rsd?HiMcF!BSi95NG;6DJ}{Rp{7>@-r<#KsqrN#tEI4&2X#uO9ig4j2+zpOHg;_LskW?ZZe%im z4S4%{bm2VEznCie9C{TfqOErwV2r~UXV+jHMlK`Vl_p>&O2~!MssLRnP75cop)$NA z{S)V9x?(KDPa+RY8h9D#eU}$k`#V7QmW8)Y{={%piU+am^@(epz>X0 z>xGew_GMyU#^k{^f#NI8LKtF?#4M8seM_wg03VPCm0};wzm{t9fN~B#Qsvh`h>rCI z8yJH+@S5@6&=r}4g@aUCDxw&+p-BWw1INpBX2aCrntPaCV0NLe%QL$$B7g&fAO=BY z7=TN6KxZL45DaAkLUu^Pu9OnB4neajSl8tb-BYp-FBP-{ZeP8pVk7AKv3x0rjqqUn zr@hz}o|n1AM%Z8@;EU=yTDgcz6NegcK-iRleJMCj$NET0EQnbUZ-pSH$$KL=R5oR} zsJ?3gBAeb2vk`quvkA01$c8g$cuFOq2xulu1p;bB zHry-_d<=nb*py-3Fxk-8qnT`AvSF(N>TPKdg)6~avs|LG4DFk7CLlv=VqjH(u8=q_ z=VxF4!tbuZ^<;)XAG)V7Nf_lPfihgNVaZ7Ih~l1+ zR0TW=U)%HKT`xZJn{U4NS0Db?TgT4h+W*UQ{Pz2wKWM~9j-5Gsp#91Jyn55v&rutY z)GdwXJDdcdRD358aRPS{8F&GFM#4xWzKY|}H<@-5{6L@MC(cJUE7ebRSM?W>F+zmJ zaCDLiPstF<76cR4VGs#8m&TDna14hdW3)Ny>I@rg_Q%C$Hiy}q?Pha0>gt6<(T@j+ zI5Ng%7a8L;KnkS-5Y+_E2EHhc=ccIwZKI+9>L>+65#@P^A}?cvkJpKpxEZP%1V|&% zCH)>j-}L|~@f>WqfPiGKG$}lHeP6_W3eVj{C5XHc+hGQ46pp&e>MmAyvAXM`>IXKy z9ok~1gwkvKC}0AnR7Jcj7pxq0b+tiAJZ&jq$N=fG14uPLR}7RIAkoRYin{7Yj*P<0 z2m{kv?6?!ak|)+M?9pE48$XGq3%{Zv=rodkBj{A~OZZxFOL6Rb+BRA^fsdGBiNbDG znSrV;^F0}aj-$v;n4_*TbYkel(21ebR___b(1~JCQrfQQ=?tANJLq(#>=9!{T@75@ zaU2(0_&5qtS7Q^4*fFpWlAdo9gvo}ZuHJL{@#XhDef#pl;UC^UedF0r;pfKFLK1qS zRFO~g2+&c2&A4;WeIp%jOYPwb*1c>q(ngZ!YR; zsFlQqzle*XT*O7m@GXvIia#-8v8I7`71TBCs@#6F;~;T5owlacm^ln z-n}FgPUL!lgWYP4onlJosHMv~Z3LXNl1Uw>T%k^pF+Di(3F#TK`hMA>Xt)YU`* z>_kwLnAo8mW__VuCljulING(94oO$SQCFE=7!YieUdr;GE*G8qC;dvUv{{zfg*B3@ zUoh?#A=~F*L1$j9c%WXu|j8M{?I)Y>o63lwaUL1_l@OCL2QICjjKHm8?D(0 z0U~|u1`}slBD4;Zzzxz+X}Gy^)K&JSU|$OMrMM_U=%3Y#DGamHj$7{R96?MIK1W^M zu^pv&I~;X&U}-i11Cq@}T?LmwamO-Y``QQN;29AcH-UR7yiir@D7dAvDZ`HSYrtq@ zQ-(!0WmsD!jHNRne7N_dGxv|`TJkk4u`0d{_(?R%!UMg|Qa3T8m+Ks{&8rBl zxa};R;P^sye5E>`BWnoP%>o<28`3PYg_O2}HH40Z(FVp4*z{ddoX{!y=_x$G&~McL z#~Y~s6#WQ3=cw!Vy4I0*F+rTjyU_FOY9E?QY_7(^yG_?j$7VY1>bXw4ck7`#W1)kB eI`@V;ciH6AeI+;7CHcHTNiR3OeD$`n&-@P|A0@c} literal 0 HcmV?d00001 diff --git a/flink-tests/src/test/resources/new-stateful-udf-migration-itcase-flink1.4-savepoint/_metadata b/flink-tests/src/test/resources/new-stateful-udf-migration-itcase-flink1.4-savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..ddc9bcdbd9a50f068886bd07e06da172e9534c49 GIT binary patch literal 44776 zcmeHQUyLM08SmM>YkmMoxL0G zmSNZXI+^L|uI}od>aXhi_3!&rNA6x?p6AWM2bZ_GmAC);_k8xXN7S89JaYZ>H(nKg z?WeH60iV6q8h?BLtrq+ZOT!f&=yz9pnPvS#6}CPHi%*qDd5x#4^>fwwbi-TFsmWy| z5+))mSdeEqQ;7&W!uc#t_FAglI%#`Q4K596 zpZj{=;Wtpzolx;cJ;$Kdwk^7ywZ|TO{MqKcx4H_0=_(wp%iWT@r{j2Y-iN2F<_^_U zE4Hw`w(a*v>%67g$y-ZhWf$07jf1z@o}Jfr-t^Q`x6{Aj%6i927YgdVSJ!*jY%xDl za$Q|gEL$k)7q(vvPIG_$*FXRM@-xrB{?HvKulVPe@Os4m#rEsAL{G>RndgAGxKi`cTcm zqj-;tZPi&?EO#A(j=4Sk2p;I`tG((*rx-EDZZy6B+DCtU<$oT=u3h#^7<QZWGHK6NaH)o5aH$e5iHvi>fkz7cBw&H$3W!o|m^HE1VQhj#$3!L(nKVP) zOmK1E5sGwGusIW;25+IgJt?qDA(U#Gdd zVT=Vg*z)CwRO+l%Dr!7x^|zwcoy0@)4_^EoGXdK*4N<`nJaQqB7M#ix$-p@`V?me) zzyyHbiPbu>i7}z#;Ma2LHN*svukMIAmvI7QUKx$1n9gsU01ggM1aO+AL%(w_#}q|5R?C$Q-1RuVW!%7RM0!5+qWy z#P|>d1L0kfl_CvP0L;OoP$w}5U)LG75QOw1nv13?tWr+=7`T2b^OQ3?sIwNOEKp zZYM^87=;lrCq`j{L5LWIiCHQ}T}P?`Y3Sv7TO$|#9~?H_7)^?GWW<@7V%^Y~-n#pZ zi~=sT=4`B&njI);qL(hF;(OQG7wq9^B{#7Rvv@~A~Vo;LHij?WGJ=e zkdZ^Bl_Y>1VOv5%M%cDGF zZA>JF%Mj`#1v+~Lx=D4emrrHHcVVDT;c21o!dLyv&pdhai5H(f?ov1f z{SNSh^_4yd6Qc^D(3==9fd!NOA5W8aed z`0Woqb7%*o=!b zPqg&m*pV<#*q+#|HTJ2VvdbA}c$SO6Fe5{V;?!vh_$Lx93xgz-IhQ`qC(eFgg;7 zg+>Q{F7r%9;0n759ZltGi}~iYA1dEKQVS6VDdcH|Y!^zwMWP6TTsT*G;sZNG8C9!^ zA1I?LWmE-_&}x#$33;5jc_l7vMg!CWTN%12-~8Dp;hD}ch0xLNLPw5$t2sF69%D<5 znOKU|{*ejH+M-Q^G=4lQ>HQ8CVtL#F?Y$==S>`T6^bXH?6HK{P3pv zYfgO>%BQnAx*Hq1HDH?{rG22nBw`Z$ssWV6h9xS4yRb6VHnot{71%7wsJd*)7m0k4 zpf9_*A!*LTGNbwSOx#JnNaTw&g`Iaga_Rho-cV~-%T%r`7?osWV_*VeP@iE0t}MQ? zz(Zt)unc6T@zxWT<_``Lbx;uFbaSm?35l>jCeVF&LxApVgJ6g`IA=$gkqRc@ zGi%@jngtg3z2PcQ!=ZXz9yI)gi~^oCH~@?z8IK`3yh@o!LIH8aQZcOppCy^$xgR9H z2_(=ivFF#Dm;p7t08G9r3YnHr93B_j7pnEoEgBGTBQ!O=DPbeO-9wh zIiO%M|H*6Bj#}Hoda!@0$O-gafEn8(m?}-MNLf(NJPVmh41`ZWf>lU&F1Yby@?9vv zezaPE2P*k4knh5fMYqkN+7Q`WMhKU8?4P{gvGsezPQpRBeHyWSzvmr#17`7KMXxop!~i>U zsJX!AY8<@H_Uyd2^QNbkx}E+FSJpdLx=>K>y}I7JW{dfelI!Y{V%b7Tzp(vcaGH~u F{{cbEQX~KX literal 0 HcmV?d00001 From 0d89a1c9f1a0b18650c226f1bea1ceba4288800c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 23 Feb 2018 17:31:44 +0100 Subject: [PATCH 274/367] [hotfix] Update docs version to Flink 1.4.1 --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 4c52b73627c1c..7740cf6fb6cb0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -27,7 +27,7 @@ # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) -version: "1.4.0" +version: "1.4.1" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version version_title: "1.4" From cb6a8a68c4f8fc34ca00ebc90b40640e441e7552 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 23 Feb 2018 19:25:06 +0800 Subject: [PATCH 275/367] [FLINK-8741] [kafka] Fix incorrect user code classloader in FlinkKafkaConsumer This commit fixes incorrectly using the parent of the user code class loader. Since Kafka 010 / 011 versions directly reuse 09 code, this fix fixes the issue for all versions. This commit also extends the Kafka010Example, so that is uses a custom watermark assigner. This allows our end-to-end tests to have caught this bug. --- .../kafka/internal/Kafka09Fetcher.java | 2 +- .../examples/kafka/Kafka010Example.java | 49 ++++++++++++++++++- .../test_streaming_kafka010.sh | 4 +- 3 files changed, 50 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index aca0da1ac709b..3c2042ffe23b7 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -93,7 +93,7 @@ public Kafka09Fetcher( watermarksPunctuated, processingTimeProvider, autoWatermarkInterval, - userCodeClassLoader.getParent(), + userCodeClassLoader, consumerMetricGroup, useMetrics); diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java index 3fbd2b462ccea..881aa6785f67d 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java @@ -21,16 +21,26 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import javax.annotation.Nullable; /** * An example that shows how to read from and write to Kafka. This will read String messages * from the input topic, prefix them by a configured prefix and output to the output topic. * + *

This example also demonstrates using a watermark assigner to generate per-partition + * watermarks directly in the Flink Kafka consumer. For demonstration purposes, it is assumed that + * the String messages are of formatted as a (message,timestamp) tuple. + * *

Example usage: * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer */ @@ -59,11 +69,15 @@ public static void main(String[] args) throws Exception { // make parameters available in the web interface env.getConfig().setGlobalJobParameters(parameterTool); + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + DataStream input = env - .addSource(new FlinkKafkaConsumer010<>( + .addSource( + new FlinkKafkaConsumer010<>( parameterTool.getRequired("input-topic"), new SimpleStringSchema(), - parameterTool.getProperties())) + parameterTool.getProperties()) + .assignTimestampsAndWatermarks(new CustomWatermarkExtractor())) .map(new PrefixingMapper(prefix)); input.addSink( @@ -76,6 +90,9 @@ public static void main(String[] args) throws Exception { } private static class PrefixingMapper implements MapFunction { + + private static final long serialVersionUID = 1180234853172462378L; + private final String prefix; public PrefixingMapper(String prefix) { @@ -87,4 +104,32 @@ public String map(String value) throws Exception { return prefix + value; } } + + /** + * A custom {@link AssignerWithPeriodicWatermarks}, that simply assumes that the input stream + * records are strictly ascending. + * + *

Flink also ships some built-in convenience assigners, such as the + * {@link BoundedOutOfOrdernessTimestampExtractor} and {@link AscendingTimestampExtractor} + */ + private static class CustomWatermarkExtractor implements AssignerWithPeriodicWatermarks { + + private static final long serialVersionUID = -742759155861320823L; + + private long currentTimestamp = Long.MIN_VALUE; + + @Override + public long extractTimestamp(String element, long previousElementTimestamp) { + // the inputs are assumed to be of format (message,timestamp) + long timestamp = Long.valueOf(element.substring(element.indexOf(",") + 1)); + this.currentTimestamp = timestamp; + return timestamp; + } + + @Nullable + @Override + public Watermark getCurrentWatermark() { + return new Watermark(currentTimestamp == Long.MIN_VALUE ? Long.MIN_VALUE : currentTimestamp - 1); + } + } } diff --git a/test-infra/end-to-end-test/test_streaming_kafka010.sh b/test-infra/end-to-end-test/test_streaming_kafka010.sh index dda2db566b853..51a570ab181d5 100755 --- a/test-infra/end-to-end-test/test_streaming_kafka010.sh +++ b/test-infra/end-to-end-test/test_streaming_kafka010.sh @@ -70,12 +70,12 @@ $FLINK_DIR/bin/flink run -d build-target/examples/streaming/Kafka010Example.jar --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer --auto.offset.reset earliest # send some data to Kafka -echo -e "hello\nwhats\nup" | $KAFKA_DIR/bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test-input +echo -e "hello,45218\nwhats,46213\nup,51348" | $KAFKA_DIR/bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test-input DATA_FROM_KAFKA=$($KAFKA_DIR/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic test-output --from-beginning --max-messages 3 2> /dev/null) # make sure we have actual newlines in the string, not "\n" -EXPECTED=$(printf "PREFIX:hello\nPREFIX:whats\nPREFIX:up") +EXPECTED=$(printf "PREFIX:hello,45218\nPREFIX:whats,46213\nPREFIX:up,51348") if [[ "$DATA_FROM_KAFKA" != "$EXPECTED" ]]; then echo "Output from Flink program does not match expected output." echo -e "EXPECTED: --$EXPECTED--" From 59169d0eeb2e8ebb1e7f830dd7286914e85f31e7 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 26 Feb 2018 16:11:04 +0800 Subject: [PATCH 276/367] [hotfix] [test] Make test-streaming-kafka010.sh more flexible for local execution --- test-infra/end-to-end-test/test_streaming_kafka010.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test-infra/end-to-end-test/test_streaming_kafka010.sh b/test-infra/end-to-end-test/test_streaming_kafka010.sh index 51a570ab181d5..c7564eab53c95 100755 --- a/test-infra/end-to-end-test/test_streaming_kafka010.sh +++ b/test-infra/end-to-end-test/test_streaming_kafka010.sh @@ -64,7 +64,7 @@ $KAFKA_DIR/bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication $KAFKA_DIR/bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic test-output # run the Flink job (detached mode) -$FLINK_DIR/bin/flink run -d build-target/examples/streaming/Kafka010Example.jar \ +$FLINK_DIR/bin/flink run -d $FLINK_DIR/examples/streaming/Kafka010Example.jar \ --input-topic test-input --output-topic test-output \ --prefix=PREFIX \ --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer --auto.offset.reset earliest From 0396fc8c03a64f1281f2a535b2702188c5234f5c Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 26 Feb 2018 19:01:32 +0800 Subject: [PATCH 277/367] [hotfix] [test] Also trap INT signal in Kafka end-to-end test This allows the test to perform the cleanup procedure (as well as printing any error logs) if an interruption occurred while waiting for the test data to be written to Kafka, therefore increasing visibility of reasons to why the test was stalling. This closes #5568. --- test-infra/end-to-end-test/test_streaming_kafka010.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/test-infra/end-to-end-test/test_streaming_kafka010.sh b/test-infra/end-to-end-test/test_streaming_kafka010.sh index c7564eab53c95..a6a9a8e166b11 100755 --- a/test-infra/end-to-end-test/test_streaming_kafka010.sh +++ b/test-infra/end-to-end-test/test_streaming_kafka010.sh @@ -51,6 +51,7 @@ function kafka_cleanup { # make sure to run regular cleanup as well cleanup } +trap kafka_cleanup INT trap kafka_cleanup EXIT # zookeeper outputs the "Node does not exist" bit to stderr From a0193f10af66c9efa0d9abbe975853f8447c4bf5 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Sat, 24 Feb 2018 21:52:44 +0800 Subject: [PATCH 278/367] [FLINK-8772] [kafka] Fix missing log parameter This closes #5574. --- .../streaming/connectors/kafka/FlinkKafkaConsumerBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 2645ddc35df2c..537e73da9f8fc 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -596,7 +596,7 @@ public void run() { while (running) { if (LOG.isDebugEnabled()) { - LOG.debug("Consumer subtask {} is trying to discover new partitions ..."); + LOG.debug("Consumer subtask {} is trying to discover new partitions ...", getRuntimeContext().getIndexOfThisSubtask()); } try { From b74c705157fef3e0d305fdd5bf1a006ae0a98666 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 22 Feb 2018 17:24:33 +0100 Subject: [PATCH 279/367] [FLINK-8543] Don't call super.close() in AvroKeyValueSinkWriter The call to keyValueWriter.close() in AvroKeyValueSinkWriter.close() will eventually call flush() on the wrapped stream which fails if we close it before(). Now we call flush ourselves before closing the KeyValyeWriter, which internally closes the wrapped stream eventually. --- .../connectors/fs/AvroKeyValueSinkWriter.java | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java index e9316333519b1..6b2f7d625a110 100644 --- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java +++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java @@ -150,17 +150,29 @@ private CodecFactory getCompressionCodec(Map conf) { public void open(FileSystem fs, Path path) throws IOException { super.open(fs, path); - CodecFactory compressionCodec = getCompressionCodec(properties); - Schema keySchema = Schema.parse(properties.get(CONF_OUTPUT_KEY_SCHEMA)); - Schema valueSchema = Schema.parse(properties.get(CONF_OUTPUT_VALUE_SCHEMA)); - keyValueWriter = new AvroKeyValueWriter(keySchema, valueSchema, compressionCodec, getStream()); + try { + CodecFactory compressionCodec = getCompressionCodec(properties); + Schema keySchema = Schema.parse(properties.get(CONF_OUTPUT_KEY_SCHEMA)); + Schema valueSchema = Schema.parse(properties.get(CONF_OUTPUT_VALUE_SCHEMA)); + keyValueWriter = new AvroKeyValueWriter( + keySchema, + valueSchema, + compressionCodec, + getStream()); + } finally { + if (keyValueWriter == null) { + close(); + } + } } @Override public void close() throws IOException { - super.close(); //the order is important since super.close flushes inside if (keyValueWriter != null) { keyValueWriter.close(); + } else { + // need to make sure we close this if we never created the Key/Value Writer. + super.close(); } } From 392cfaaed9380c5ea38b8593d23023925638cbe3 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 27 Feb 2018 17:09:19 +0100 Subject: [PATCH 280/367] [FLINK-8798] [core] Force 'commons-logging' to be parent-first loaded. --- .../main/java/org/apache/flink/configuration/CoreOptions.java | 2 +- .../org/apache/flink/configuration/ParentFirstPatternsTest.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index 27f39a4e816f0..1ac8bdc63a60b 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -36,7 +36,7 @@ public class CoreOptions { public static final ConfigOption ALWAYS_PARENT_FIRST_LOADER = ConfigOptions .key("classloader.parent-first-patterns") - .defaultValue("java.;scala.;org.apache.flink.;com.esotericsoftware.kryo;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback"); + .defaultValue("java.;scala.;org.apache.flink.;com.esotericsoftware.kryo;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging.log4j;org.apache.commons.logging;ch.qos.logback"); // ------------------------------------------------------------------------ diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java index 784d0998768b4..2773bd2b2def8 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java @@ -55,6 +55,7 @@ public void testLoggersParentFirst() { assertTrue(PARENT_FIRST_PACKAGES.contains("org.slf4j")); assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.log4j")); assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.logging.log4j")); + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.commons.logging")); assertTrue(PARENT_FIRST_PACKAGES.contains("ch.qos.logback")); } From 179cfdd4c6fd5f38ab01c2a55b17f251cc173dbc Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 21 Feb 2018 20:20:51 +0100 Subject: [PATCH 281/367] [FLINK-8762] [quickstarts] Make 'StreamingJob' the default main class and remove WordCount example from the quickstart. The packaged example jobs have been reported to not be terribly helpful and simply create noise in the initial project setup. --- .../resources/archetype-resources/pom.xml | 6 +- .../src/main/java/BatchJob.java | 20 +--- .../main/java/SocketTextStreamWordCount.java | 108 ------------------ .../src/main/java/StreamingJob.java | 21 +--- .../src/main/java/WordCount.java | 94 --------------- .../resources/archetype-resources/pom.xml | 6 +- .../src/main/scala/BatchJob.scala | 25 ++-- .../scala/SocketTextStreamWordCount.scala | 69 ----------- .../src/main/scala/StreamingJob.scala | 24 ++-- .../src/main/scala/WordCount.scala | 53 --------- 10 files changed, 27 insertions(+), 399 deletions(-) delete mode 100644 flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java delete mode 100644 flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/WordCount.java delete mode 100644 flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala delete mode 100644 flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCount.scala diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 71bc0a1006640..84584afd15a50 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -189,15 +189,11 @@ under the License. - - diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java index d0e68a4b265be..971192422b96d 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java @@ -23,22 +23,12 @@ /** * Skeleton for a Flink Batch Job. * - *

For a full example of a Flink Batch Job, see the WordCountJob.java file in the - * same package/directory or have a look at the website. + *

For a tutorial how to write a Flink batch application, check the + * tutorials and examples on the Flink Website. * - *

You can also generate a .jar file that you can submit on your Flink - * cluster. - * Just type - * mvn clean package - * in the projects root directory. - * You will find the jar in - * target/${artifactId}-${version}.jar - * From the CLI you can then run - * ./bin/flink run -c ${package}.BatchJob target/${artifactId}-${version}.jar - * - *

For more information on the CLI see: - * - *

http://flink.apache.org/docs/latest/apis/cli.html + *

To package your appliation into a JAR file for execution, + * change the main class in the POM.xml file to this class (simply search for 'mainClass') + * and run 'mvn clean package' on the command line. */ public class BatchJob { diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java deleted file mode 100644 index 97df489afbc42..0000000000000 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java +++ /dev/null @@ -1,108 +0,0 @@ -package ${package}; - -/* - * 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 org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.util.Collector; - -/** - * This example shows an implementation of WordCount with data from a text - * socket. To run the example make sure that the service providing the text data - * is already up and running. - * - *

To start an example socket text stream on your local machine run netcat from - * a command line: nc -lk 9999, where the parameter specifies the - * port number. - * - *

Usage: - * SocketTextStreamWordCount <hostname> <port> - *
- * - *

This example shows how to: - *

    - *
  • use StreamExecutionEnvironment.socketTextStream - *
  • write a simple Flink program - *
  • write and use user-defined functions - *
- * - * @see netcat - */ -public class SocketTextStreamWordCount { - - // - // Program - // - - public static void main(String[] args) throws Exception { - - if (args.length != 2){ - System.err.println("USAGE:\nSocketTextStreamWordCount "); - return; - } - - String hostName = args[0]; - Integer port = Integer.parseInt(args[1]); - - // set up the execution environment - final StreamExecutionEnvironment env = StreamExecutionEnvironment - .getExecutionEnvironment(); - - // get input data - DataStream text = env.socketTextStream(hostName, port); - - DataStream> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new LineSplitter()) - // group by the tuple field "0" and sum up tuple field "1" - .keyBy(0) - .sum(1); - - counts.print(); - - // execute program - env.execute("Java WordCount from SocketTextStream Example"); - } - - // - // User Functions - // - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into - * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>). - */ - public static final class LineSplitter implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } - } -} diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java index 45a67ae3b7bcf..6027e75165085 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java @@ -20,26 +20,17 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - /** * Skeleton for a Flink Streaming Job. * - *

For a full example of a Flink Streaming Job, see the SocketTextStreamWordCount.java - * file in the same package/directory or have a look at the website. - * - *

You can also generate a .jar file that you can submit on your Flink - * cluster. - * Just type - * mvn clean package - * in the projects root directory. - * You will find the jar in - * target/${artifactId}-${version}.jar - * From the CLI you can then run - * ./bin/flink run -c ${package}.StreamingJob target/${artifactId}-${version}.jar + *

For a tutorial how to write a Flink streaming application, check the + * tutorials and examples on the Flink Website. * - *

For more information on the CLI see: + *

To package your appliation into a JAR file for execution, run + * 'mvn clean package' on the command line. * - *

http://flink.apache.org/docs/latest/apis/cli.html + *

If you change the name of the main class (with the public static void main(String[] args)) + * method, change the respective entry in the POM.xml file (simply search for 'mainClass'). */ public class StreamingJob { diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/WordCount.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/WordCount.java deleted file mode 100644 index 6c953890ff356..0000000000000 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/WordCount.java +++ /dev/null @@ -1,94 +0,0 @@ -package ${package}; - -/** - * 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 org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.util.Collector; - -/** - * Implements the "WordCount" program that computes a simple word occurrence histogram - * over some sample data - * - *

This example shows how to: - *

    - *
  • write a simple Flink program. - *
  • use Tuple data types. - *
  • write and use user-defined functions. - *
- * - */ -public class WordCount { - - // - // Program - // - - public static void main(String[] args) throws Exception { - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet text = env.fromElements( - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles," - ); - - DataSet> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new LineSplitter()) - // group by the tuple field "0" and sum up tuple field "1" - .groupBy(0) - .sum(1); - - // execute and print result - counts.print(); - - } - - // - // User Functions - // - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into - * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>). - */ - public static final class LineSplitter implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } - } -} diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index b76f9b910c146..24b76f3085eec 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -201,15 +201,11 @@ under the License. - - diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala index 4ecfeed17bc4b..a533da90f22e0 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala @@ -23,26 +23,15 @@ import org.apache.flink.api.scala._ /** * Skeleton for a Flink Batch Job. * - * For a full example of a Flink Batch Job, see the WordCountJob.scala file in the - * same package/directory or have a look at the website. + * For a tutorial how to write a Flink batch application, check the + * tutorials and examples on the Flink Website. * - * You can also generate a .jar file that you can submit on your Flink - * cluster. Just type - * {{{ - * mvn clean package - * }}} - * in the projects root directory. You will find the jar in - * target/${artifactId}-${version}.jar - * From the CLI you can then run - * {{{ - * ./bin/flink run -c ${package}.BatchJob target/${artifactId}-${version}.jar - * }}} - * - * For more information on the CLI see: - * - * http://flink.apache.org/docs/latest/apis/cli.html + * To package your appliation into a JAR file for execution, + * change the main class in the POM.xml file to this class (simply search for 'mainClass') + * and run 'mvn clean package' on the command line. */ object BatchJob { + def main(args: Array[String]) { // set up the batch execution environment val env = ExecutionEnvironment.getExecutionEnvironment @@ -74,4 +63,4 @@ object BatchJob { // execute program env.execute("Flink Batch Scala API Skeleton") } -} \ No newline at end of file +} diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala deleted file mode 100644 index a6987acc613b9..0000000000000 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala +++ /dev/null @@ -1,69 +0,0 @@ -package ${package} - -/* - * 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 org.apache.flink.streaming.api.scala._ - -/** - * This example shows an implementation of WordCount with data from a text socket. - * To run the example make sure that the service providing the text data is already up and running. - * - * To start an example socket text stream on your local machine run netcat from a command line, - * where the parameter specifies the port number: - * - * {{{ - * nc -lk 9999 - * }}} - * - * Usage: - * {{{ - * SocketTextStreamWordCount - * }}} - * - * This example shows how to: - * - * - use StreamExecutionEnvironment.socketTextStream - * - write a simple Flink Streaming program in scala - * - write and use user-defined functions - */ -object SocketTextStreamWordCount { - - def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("USAGE:\nSocketTextStreamWordCount ") - return - } - - val hostName = args(0) - val port = args(1).toInt - - val env = StreamExecutionEnvironment.getExecutionEnvironment - - // create streams for names and ages by mapping the inputs to the corresponding objects - val text = env.socketTextStream(hostName, port) - val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } } - .map { (_, 1) } - .keyBy(0) - .sum(1) - - counts print - - env.execute("Scala WordCount from SocketTextStream Example") - } -} diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala index 7a45fc226661b..7c950b14aceac 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala @@ -23,24 +23,14 @@ import org.apache.flink.streaming.api.scala._ /** * Skeleton for a Flink Streaming Job. * - * For a full example of a Flink Streaming Job, see the SocketTextStreamWordCount.java - * file in the same package/directory or have a look at the website. + * For a tutorial how to write a Flink streaming application, check the + * tutorials and examples on the Flink Website. * - * You can also generate a .jar file that you can submit on your Flink - * cluster. Just type - * {{{ - * mvn clean package - * }}} - * in the projects root directory. You will find the jar in - * target/${artifactId}-${version}.jar - * From the CLI you can then run - * {{{ - * ./bin/flink run -c ${package}.StreamingJob target/${artifactId}-${version}.jar - * }}} + * To package your appliation into a JAR file for execution, run + * 'mvn clean package' on the command line. * - * For more information on the CLI see: - * - * http://flink.apache.org/docs/latest/apis/cli.html + * If you change the name of the main class (with the public static void main(String[] args)) + * method, change the respective entry in the POM.xml file (simply search for 'mainClass'). */ object StreamingJob { def main(args: Array[String]) { @@ -70,4 +60,4 @@ object StreamingJob { // execute program env.execute("Flink Streaming Scala API Skeleton") } -} \ No newline at end of file +} diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCount.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCount.scala deleted file mode 100644 index b88dcc6d2299b..0000000000000 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCount.scala +++ /dev/null @@ -1,53 +0,0 @@ -package ${package} - -/** - * 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 org.apache.flink.api.scala._ - -/** - * Implements the "WordCount" program that computes a simple word occurrence histogram - * over some sample data - * - * This example shows how to: - * - * - write a simple Flink program. - * - use Tuple data types. - * - write and use user-defined functions. - */ -object WordCount { - def main(args: Array[String]) { - - // set up the execution environment - val env = ExecutionEnvironment.getExecutionEnvironment - - // get input data - val text = env.fromElements("To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,") - - val counts = text.flatMap { _.toLowerCase.split("\\W+") } - .map { (_, 1) } - .groupBy(0) - .sum(1) - - // execute and print result - counts.print() - - } -} From 4a60fcb0f41e96221108a43fc7037e08ea047758 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 21 Feb 2018 20:30:35 +0100 Subject: [PATCH 282/367] [hotfix] [quickstarts] Fix header and package declaration order. --- .../archetype-resources/src/main/java/BatchJob.java | 6 +++--- .../archetype-resources/src/main/java/StreamingJob.java | 6 +++--- .../archetype-resources/src/main/scala/BatchJob.scala | 6 +++--- .../archetype-resources/src/main/scala/StreamingJob.scala | 6 +++--- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java index 971192422b96d..9515791af809d 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java @@ -1,6 +1,4 @@ -package ${package}; - -/** +/* * 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 @@ -18,6 +16,8 @@ * limitations under the License. */ +package ${package}; + import org.apache.flink.api.java.ExecutionEnvironment; /** diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java index 6027e75165085..40918894f873f 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java @@ -1,6 +1,4 @@ -package ${package}; - -/** +/* * 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 @@ -18,6 +16,8 @@ * limitations under the License. */ +package ${package}; + import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala index a533da90f22e0..46520b7dfcccd 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala @@ -1,6 +1,4 @@ -package ${package} - -/** +/* * 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 @@ -18,6 +16,8 @@ package ${package} * limitations under the License. */ +package ${package} + import org.apache.flink.api.scala._ /** diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala index 7c950b14aceac..20115a75a2532 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala @@ -1,6 +1,4 @@ -package ${package} - -/** +/* * 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 @@ -18,6 +16,8 @@ package ${package} * limitations under the License. */ +package ${package} + import org.apache.flink.streaming.api.scala._ /** From 6bd35a3c537eaa54c55312a6b3703208582b9229 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 21 Feb 2018 21:05:23 +0100 Subject: [PATCH 283/367] [hotfix] [quickstarts] Fix block comments in program stubs. --- .../resources/archetype-resources/src/main/java/BatchJob.java | 2 +- .../archetype-resources/src/main/java/StreamingJob.java | 2 +- .../resources/archetype-resources/src/main/scala/BatchJob.scala | 2 +- .../archetype-resources/src/main/scala/StreamingJob.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java index 9515791af809d..db2ee601c0234 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java @@ -36,7 +36,7 @@ public static void main(String[] args) throws Exception { // set up the batch execution environment final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - /** + /* * Here, you can start creating your execution plan for Flink. * * Start with getting some data from the environment, like diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java index 40918894f873f..5bcee21cf45d8 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java @@ -38,7 +38,7 @@ public static void main(String[] args) throws Exception { // set up the streaming execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - /** + /* * Here, you can start creating your execution plan for Flink. * * Start with getting some data from the environment, like diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala index 46520b7dfcccd..329e6c253a8f9 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala @@ -36,7 +36,7 @@ object BatchJob { // set up the batch execution environment val env = ExecutionEnvironment.getExecutionEnvironment - /** + /* * Here, you can start creating your execution plan for Flink. * * Start with getting some data from the environment, like diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala index 20115a75a2532..bab0bb97abf33 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala @@ -37,7 +37,7 @@ object StreamingJob { // set up the streaming execution environment val env = StreamExecutionEnvironment.getExecutionEnvironment - /** + /* * Here, you can start creating your execution plan for Flink. * * Start with getting some data from the environment, like From fa306385fe190450251fd902a544d5d3fa2ca664 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 23 Feb 2018 10:53:22 +0100 Subject: [PATCH 284/367] [FLINK-8764] [quickstarts] Make quickstarts work out of the box for IDE and JAR packaging - All Flink and Scala dependencies are properly set to provided - That way, Maven JAR packaging behaves correctly by default - Eclipse adds 'provided' dependencies to the classpath when running programs, so works out of the box - There is a profile that automatically activates in IntelliJ that adds the necessary dependencies in 'compile' scope to make it run out of the box. --- .../resources/archetype-resources/pom.xml | 231 +++++++--------- .../resources/archetype-resources/pom.xml | 246 ++++++++---------- 2 files changed, 207 insertions(+), 270 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 84584afd15a50..6ac05b0419b63 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -50,161 +50,53 @@ under the License. - - - - org.apache.flink - flink-core - ${flink.version} - + org.apache.flink flink-java ${flink.version} + provided - org.apache.flink - flink-clients_${scala.binary.version} + flink-streaming-java_${scala.binary.version} ${flink.version} + provided + + + + - + + org.slf4j slf4j-log4j12 ${slf4j.version} + runtime log4j log4j ${log4j.version} + runtime - - - - build-jar - - - false - - - - - org.apache.flink - flink-core - ${flink.version} - provided - - - org.apache.flink - flink-java - ${flink.version} - provided - - - org.apache.flink - flink-clients_${scala.binary.version} - ${flink.version} - provided - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${flink.version} - provided - - - org.slf4j - slf4j-log4j12 - ${slf4j.version} - provided - - - log4j - log4j - ${log4j.version} - provided - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.1 - - - - package - - shade - - - - - org.apache.flink:force-shading - com.google.code.findbugs:jsr305 - org.slf4j:* - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - ${package}.StreamingJob - - - - - - - - - - - + + org.apache.maven.plugins maven-compiler-plugin @@ -214,12 +106,57 @@ under the License. 1.8 + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + package + + shade + + + + + org.apache.flink:force-shading + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + ${package}.StreamingJob + + + + + + - - + + org.eclipse.m2e lifecycle-mapping @@ -246,10 +185,10 @@ under the License. org.apache.maven.plugins - maven-assembly-plugin - [2.4,) + maven-shade-plugin + [3.0.0,) - single + shade @@ -276,6 +215,36 @@ under the License. - --> + + + + + + + add-dependencies-for-IDEA + + + + idea.version + + + + + + org.apache.flink + flink-java + ${flink.version} + compile + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + compile + + + + + diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 24b76f3085eec..b318a1971508d 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -52,171 +52,104 @@ under the License. 2.11.11 - - - - org.apache.flink - flink-core - ${flink.version} - - - - org.apache.flink - flink-clients_${scala.binary.version} - ${flink.version} - + org.apache.flink flink-scala_${scala.binary.version} ${flink.version} + provided org.apache.flink flink-streaming-scala_${scala.binary.version} ${flink.version} + provided + org.scala-lang scala-library ${scala.version} + provided + + + + + - + + org.slf4j slf4j-log4j12 ${slf4j.version} + runtime log4j log4j ${log4j.version} + runtime - - - - build-jar - - false - - - - org.apache.flink - flink-core - ${flink.version} - provided - - - org.apache.flink - flink-clients_${scala.binary.version} - ${flink.version} - provided - - - org.apache.flink - flink-scala_${scala.binary.version} - ${flink.version} - provided - - - org.apache.flink - flink-streaming-scala_${scala.binary.version} - ${flink.version} - provided - - - org.scala-lang - scala-library - ${scala.version} - provided - - - org.slf4j - slf4j-log4j12 - ${slf4j.version} - provided - - - log4j - log4j - ${log4j.version} - provided - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.1 - - - - package - - shade - - - - - org.apache.flink:force-shading - com.google.code.findbugs:jsr305 - org.slf4j:* - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - ${package}.StreamingJob - - - - - - - - - - - + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + package + + shade + + + + + org.apache.flink:force-shading + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + ${package}.StreamingJob + + + + + + + + org.apache.maven.plugins maven-compiler-plugin @@ -226,6 +159,8 @@ under the License. 1.8 + + net.alchim31.maven scala-maven-plugin @@ -240,7 +175,7 @@ under the License. - + org.apache.maven.plugins maven-eclipse-plugin @@ -255,10 +190,8 @@ under the License. org.scala-ide.sdt.core.scalabuilder - org.scala-ide.sdt.launching.SCALA_CONTAINER - - org.eclipse.jdt.launching.JRE_CONTAINER - + org.scala-ide.sdt.launching.SCALA_CONTAINER + org.eclipse.jdt.launching.JRE_CONTAINER org.scala-lang:scala-library @@ -270,8 +203,6 @@ under the License. - - org.codehaus.mojo build-helper-maven-plugin @@ -307,4 +238,41 @@ under the License. + + + + + + + add-dependencies-for-IDEA + + + + idea.version + + + + + + org.apache.flink + flink-scala_${scala.binary.version} + ${flink.version} + compile + + + org.apache.flink + flink-streaming-scala_${scala.binary.version} + ${flink.version} + compile + + + org.scala-lang + scala-library + ${scala.version} + compile + + + + + From 11e19a56bfdedb4f41ecc3dcf1cd4120191cdb1a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 23 Feb 2018 11:10:43 +0100 Subject: [PATCH 285/367] [FLINK-8765] [quickstarts] Simplify quickstart properties This does not pull out the slf4j and log4j version into properties any more, making the quickstarts a bit simpler. Given that both versions are used only once, and only for the feature to have convenience logging in the IDE, the versions might as well be defined directly in the dependencies. --- .../src/main/resources/archetype-resources/pom.xml | 6 ++---- .../src/main/resources/archetype-resources/pom.xml | 6 ++---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 6ac05b0419b63..50b035192a9d4 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -31,8 +31,6 @@ under the License. UTF-8 @project.version@ - @slf4j.version@ - @log4j.version@ @scala.binary.version@ @@ -82,13 +80,13 @@ under the License. org.slf4j slf4j-log4j12 - ${slf4j.version} + 1.7.7 runtime log4j log4j - ${log4j.version} + 1.2.17 runtime diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index b318a1971508d..4a866c3805829 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -46,8 +46,6 @@ under the License. UTF-8 @project.version@ - @slf4j.version@ - @log4j.version@ 2.11 2.11.11 @@ -92,13 +90,13 @@ under the License. org.slf4j slf4j-log4j12 - ${slf4j.version} + 1.7.7 runtime log4j log4j - ${log4j.version} + 1.2.17 runtime From 23c37cf43cac94ecaa43641d264f7422bc1c57ed Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 23 Feb 2018 11:13:01 +0100 Subject: [PATCH 286/367] [FLINK-8766] [quickstarts] Pin scala runtime version for Java Quickstart Followup to FLINK-7414, which pinned the scala version for the Scala Quickstart --- .../src/main/resources/archetype-resources/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 50b035192a9d4..d53415a5f7ac5 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -31,7 +31,7 @@ under the License. UTF-8 @project.version@ - @scala.binary.version@ + 2.11 From 4eb0a0fb9f41b29f6b87ce77f5e985f2b9fa30a3 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 23 Feb 2018 11:18:36 +0100 Subject: [PATCH 287/367] [FLINK-8767] [quickstarts] Set the maven.compiler.source and .target properties for Java Quickstart Setting these properties helps properly pinning the Java version in IntelliJ. Without these properties, Java version keeps switching back to 1.5 in some setups. --- .../src/main/resources/archetype-resources/pom.xml | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index d53415a5f7ac5..0ca6eb925987e 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -31,7 +31,10 @@ under the License. UTF-8 @project.version@ + 1.8 2.11 + ${java.version} + ${java.version} @@ -100,8 +103,8 @@ under the License. maven-compiler-plugin 3.1 - 1.8 - 1.8 + ${java.version} + ${java.version} @@ -158,8 +161,8 @@ under the License. maven-compiler-plugin - 1.8 - 1.8 + ${java.version} + ${java.version} jdt From 0c848aa98b466764873f2372af3ffd64cd8f71ac Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 26 Feb 2018 12:19:00 +0100 Subject: [PATCH 288/367] [FLINK-8764] [docs] Adjust quickstart documentation --- docs/quickstart/java_api_quickstart.md | 119 +++++------------------- docs/quickstart/scala_api_quickstart.md | 95 ++++++------------- 2 files changed, 52 insertions(+), 162 deletions(-) diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md index baf14deb7c5cc..9a325911dd27b 100644 --- a/docs/quickstart/java_api_quickstart.md +++ b/docs/quickstart/java_api_quickstart.md @@ -1,6 +1,6 @@ --- -title: "Sample Project using the Java API" -nav-title: Sample Project in Java +title: "Project Template for Java" +nav-title: Project Template for Java nav-parent_id: start nav-pos: 0 --- @@ -86,120 +86,51 @@ quickstart/ │   └── myorg │   └── quickstart │   ├── BatchJob.java - │   ├── SocketTextStreamWordCount.java - │   ├── StreamingJob.java - │   └── WordCount.java + │   └── StreamingJob.java └── resources └── log4j.properties {% endhighlight %} -The sample project is a __Maven project__, which contains four classes. _StreamingJob_ and _BatchJob_ are basic skeleton programs, _SocketTextStreamWordCount_ is a working streaming example and _WordCountJob_ is a working batch example. Please note that the _main_ method of all classes allow you to start Flink in a development/testing mode. +The sample project is a __Maven project__, which contains two classes: _StreamingJob_ and _BatchJob_ are the basic skeleton programs for a *DataStream* and *DataSet* program. +The _main_ method is the entry point of the program, both for in-IDE testing/execution and for proper deployments. We recommend you __import this project into your IDE__ to develop and -test it. If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) +test it. IntelliJ IDEA supports Maven projects out of the box. +If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import). Some Eclipse bundles include that plugin by default, others require you -to install it manually. The IntelliJ IDE supports Maven projects out of -the box. +to install it manually. - -*A note to Mac OS X users*: The default JVM heapsize for Java is too +*A note to Mac OS X users*: The default JVM heapsize for Java mey be too small for Flink. You have to manually increase it. In Eclipse, choose `Run Configurations -> Arguments` and write into the `VM Arguments` box: `-Xmx800m`. ## Build Project -If you want to __build your project__, go to your project directory and -issue the `mvn clean install -Pbuild-jar` command. You will -__find a jar__ that runs on every Flink cluster with a compatible -version, __target/original-your-artifact-id-your-version.jar__. There -is also a fat-jar in __target/your-artifact-id-your-version.jar__ which, -additionally, contains all dependencies that were added to the Maven -project. +If you want to __build/package your project__, go to your project directory and +run the '`mvn clean package`' command. +You will __find a JAR file__ that contains your application, plus connectors and libraries +that you may have added as dependencoes to the application: `target/-.jar`. + +__Note:__ If you use a different class than *StreamingJob* as the application's main class / entry point, +we recommend you change the `mainClass` setting in the `pom.xml` file accordingly. That way, the Flink +can run time application from the JAR file without additionally specifying the main class. ## Next Steps Write your application! -The quickstart project contains a `WordCount` implementation, the -"Hello World" of Big Data processing systems. The goal of `WordCount` -is to determine the frequencies of words in a text, e.g., how often do -the terms "the" or "house" occur in all Wikipedia texts. - -__Sample Input__: - -~~~bash -big data is big -~~~ - -__Sample Output__: - -~~~bash -big 2 -data 1 -is 1 -~~~ - -The following code shows the `WordCount` implementation from the -Quickstart which processes some text lines with two operators (a FlatMap -and a Reduce operation via aggregating a sum), and prints the resulting -words and counts to std-out. - -~~~java -public class WordCount { - - public static void main(String[] args) throws Exception { - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet text = env.fromElements( - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles," - ); - - DataSet> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new LineSplitter()) - // group by the tuple field "0" and sum up tuple field "1" - .groupBy(0) - .sum(1); - - // execute and print result - counts.print(); - } -} -~~~ - -The operations are defined by specialized classes, here the LineSplitter class. - -~~~java -public static final class LineSplitter implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } -} -~~~ - -{% gh_link /flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java "Check GitHub" %} for the full example code. - -For a complete overview over our API, have a look at the +If you are writing a streaming application and you are looking for inspiration what to write, +take a look at the [Stream Processing Application Tutorial]({{ site.baseurl }}/quickstart/run_example_quickstart.html#writing-a-flink-program) + +If you are writing a batch processing application and you are looking for inspiration what to write, +take a look at the [Batch Application Examples]({{ site.baseurl }}/dev/batch/examples.html) + +For a complete overview over the APIa, have a look at the [DataStream API]({{ site.baseurl }}/dev/datastream_api.html) and [DataSet API]({{ site.baseurl }}/dev/batch/index.html) sections. + If you have any trouble, ask on our [Mailing List](http://mail-archives.apache.org/mod_mbox/flink-user/). We are happy to provide help. diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md index 40c02a9b8913a..a7b73e322c43b 100644 --- a/docs/quickstart/scala_api_quickstart.md +++ b/docs/quickstart/scala_api_quickstart.md @@ -1,6 +1,6 @@ --- -title: "Sample Project using the Scala API" -nav-title: Sample Project in Scala +title: "Project Template for Scala" +nav-title: Project Template for Scala nav-parent_id: start nav-pos: 1 --- @@ -173,14 +173,18 @@ quickstart/ └── myorg └── quickstart ├── BatchJob.scala - ├── SocketTextStreamWordCount.scala - ├── StreamingJob.scala - └── WordCount.scala + └── StreamingJob.scala {% endhighlight %} -The sample project is a __Maven project__, which contains four classes. _StreamingJob_ and _BatchJob_ are basic skeleton programs, _SocketTextStreamWordCount_ is a working streaming example and _WordCountJob_ is a working batch example. Please note that the _main_ method of all classes allow you to start Flink in a development/testing mode. +The sample project is a __Maven project__, which contains two classes: _StreamingJob_ and _BatchJob_ are the basic skeleton programs for a *DataStream* and *DataSet* program. +The _main_ method is the entry point of the program, both for in-IDE testing/execution and for proper deployments. -We recommend you __import this project into your IDE__. For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites: +We recommend you __import this project into your IDE__. + +IntelliJ IDEA supports Maven out of the box and offers a plugin for Scala development. +From our experience, IntelliJ provides the best experience for developing Flink applications. + +For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites: * _Eclipse 4.x_ * [Scala IDE](http://download.scala-ide.org/sdk/lithium/e44/scala211/stable/site) @@ -191,78 +195,33 @@ We recommend you __import this project into your IDE__. For Eclipse, you need th * [m2eclipse-scala](http://alchim31.free.fr/m2e-scala/update-site) * [Build Helper Maven Plugin](https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.14.0/N/0.14.0.201109282148/) -The IntelliJ IDE supports Maven out of the box and offers a plugin for -Scala development. +### Build Project +If you want to __build/package your project__, go to your project directory and +run the '`mvn clean package`' command. +You will __find a JAR file__ that contains your application, plus connectors and libraries +that you may have added as dependencoes to the application: `target/-.jar`. -### Build Project +__Note:__ If you use a different class than *StreamingJob* as the application's main class / entry point, +we recommend you change the `mainClass` setting in the `pom.xml` file accordingly. That way, the Flink +can run time application from the JAR file without additionally specifying the main class. -If you want to __build your project__, go to your project directory and -issue the `mvn clean package -Pbuild-jar` command. You will -__find a jar__ that runs on every Flink cluster with a compatible -version, __target/original-your-artifact-id-your-version.jar__. There -is also a fat-jar in __target/your-artifact-id-your-version.jar__ which, -additionally, contains all dependencies that were added to the Maven -project. ## Next Steps Write your application! -The quickstart project contains a `WordCount` implementation, the -"Hello World" of Big Data processing systems. The goal of `WordCount` -is to determine the frequencies of words in a text, e.g., how often do -the terms "the" or "house" occur in all Wikipedia texts. - -__Sample Input__: - -~~~bash -big data is big -~~~ +If you are writing a streaming application and you are looking for inspiration what to write, +take a look at the [Stream Processing Application Tutorial]({{ site.baseurl }}/quickstart/run_example_quickstart.html#writing-a-flink-program) -__Sample Output__: - -~~~bash -big 2 -data 1 -is 1 -~~~ - -The following code shows the `WordCount` implementation from the -Quickstart which processes some text lines with two operators (a FlatMap -and a Reduce operation via aggregating a sum), and prints the resulting -words and counts to std-out. - -~~~scala -object WordCountJob { - def main(args: Array[String]) { - - // set up the execution environment - val env = ExecutionEnvironment.getExecutionEnvironment - - // get input data - val text = env.fromElements("To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,") - - val counts = text.flatMap { _.toLowerCase.split("\\W+") } - .map { (_, 1) } - .groupBy(0) - .sum(1) - - // emit result and print result - counts.print() - } -} -~~~ +If you are writing a batch processing application and you are looking for inspiration what to write, +take a look at the [Batch Application Examples]({{ site.baseurl }}/dev/batch/examples.html) -{% gh_link flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala "Check GitHub" %} for the full example code. +For a complete overview over the APIa, have a look at the +[DataStream API]({{ site.baseurl }}/dev/datastream_api.html) and +[DataSet API]({{ site.baseurl }}/dev/batch/index.html) sections. -For a complete overview over our API, have a look at the -[DataStream API]({{ site.baseurl }}/dev/datastream_api.html), -[DataSet API]({{ site.baseurl }}/dev/batch/index.html), and -[Scala API Extensions]({{ site.baseurl }}/dev/scala_api_extensions.html) -sections. If you have any trouble, ask on our +If you have any trouble, ask on our [Mailing List](http://mail-archives.apache.org/mod_mbox/flink-user/). We are happy to provide help. From 51231c471bca5e400e1bf152d3cfbdb68c809567 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 26 Feb 2018 16:41:24 +0100 Subject: [PATCH 289/367] [FLINK-8791] [docs] Fix documentation about configuring dependencies --- docs/dev/linking.md | 96 ------- docs/dev/linking_with_flink.md | 146 ----------- docs/redirects/linking_with_flink.md | 25 ++ .../linking_with_optional_modules.md | 25 ++ docs/start/dependencies.md | 244 ++++++++++++++++++ 5 files changed, 294 insertions(+), 242 deletions(-) delete mode 100644 docs/dev/linking.md delete mode 100644 docs/dev/linking_with_flink.md create mode 100644 docs/redirects/linking_with_flink.md create mode 100644 docs/redirects/linking_with_optional_modules.md create mode 100644 docs/start/dependencies.md diff --git a/docs/dev/linking.md b/docs/dev/linking.md deleted file mode 100644 index 78ef54494831f..0000000000000 --- a/docs/dev/linking.md +++ /dev/null @@ -1,96 +0,0 @@ ---- -nav-title: "Linking with Optional Modules" -title: "Linking with modules not contained in the binary distribution" -nav-parent_id: start -nav-pos: 10 ---- - - -The binary distribution contains jar packages in the `lib` folder that are automatically -provided to the classpath of your distributed programs. Almost all of Flink classes are -located there with a few exceptions, for example the streaming connectors and some freshly -added modules. To run code depending on these modules you need to make them accessible -during runtime, for which we suggest two options: - -1. Either copy the required jar files to the `lib` folder onto all of your TaskManagers. -Note that you have to restart your TaskManagers after this. -2. Or package them with your code. - -The latter version is recommended as it respects the classloader management in Flink. - -### Packaging dependencies with your usercode with Maven - -To provide these dependencies not included by Flink we suggest two options with Maven. - -1. The maven assembly plugin builds a so-called uber-jar (executable jar) containing all your dependencies. -The assembly configuration is straight-forward, but the resulting jar might become bulky. -See [maven-assembly-plugin](http://maven.apache.org/plugins/maven-assembly-plugin/usage.html) for further information. -2. The maven unpack plugin unpacks the relevant parts of the dependencies and -then packages it with your code. - -Using the latter approach in order to bundle the Kafka connector, `flink-connector-kafka` -you would need to add the classes from both the connector and the Kafka API itself. Add -the following to your plugins section. - -~~~xml - - org.apache.maven.plugins - maven-dependency-plugin - 2.9 - - - unpack - - prepare-package - - unpack - - - - - - org.apache.flink - flink-connector-kafka - {{ site.version }} - jar - false - ${project.build.directory}/classes - org/apache/flink/** - - - - org.apache.kafka - kafka_ - - jar - false - ${project.build.directory}/classes - kafka/** - - - - - - -~~~ - -Now when running `mvn clean package` the produced jar includes the required dependencies. - -{% top %} diff --git a/docs/dev/linking_with_flink.md b/docs/dev/linking_with_flink.md deleted file mode 100644 index f2380b23a07f6..0000000000000 --- a/docs/dev/linking_with_flink.md +++ /dev/null @@ -1,146 +0,0 @@ ---- -title: "Linking with Flink" -nav-parent_id: start -nav-pos: 2 ---- - - -To write programs with Flink, you need to include the Flink library corresponding to -your programming language in your project. - -The simplest way to do this is to use one of the quickstart scripts: either for -[Java]({{ site.baseurl }}/quickstart/java_api_quickstart.html) or for [Scala]({{ site.baseurl }}/quickstart/scala_api_quickstart.html). They -create a blank project from a template (a Maven Archetype), which sets up everything for you. To -manually create the project, you can use the archetype and create a project by calling: - -
-
-{% highlight bash %} -mvn archetype:generate \ - -DarchetypeGroupId=org.apache.flink \ - -DarchetypeArtifactId=flink-quickstart-java \ - -DarchetypeVersion={{site.version }} -{% endhighlight %} -
-
-{% highlight bash %} -mvn archetype:generate \ - -DarchetypeGroupId=org.apache.flink \ - -DarchetypeArtifactId=flink-quickstart-scala \ - -DarchetypeVersion={{site.version }} -{% endhighlight %} -
-
- -The archetypes are working for stable releases and preview versions (`-SNAPSHOT`). - -If you want to add Flink to an existing Maven project, add the following entry to your -*dependencies* section in the *pom.xml* file of your project: - -
-
-{% highlight xml %} - - - org.apache.flink - flink-streaming-java{{ site.scala_version_suffix }} - {{site.version }} - - - - org.apache.flink - flink-java - {{site.version }} - - - org.apache.flink - flink-clients{{ site.scala_version_suffix }} - {{site.version }} - -{% endhighlight %} -
-
-{% highlight xml %} - - - org.apache.flink - flink-streaming-scala{{ site.scala_version_suffix }} - {{site.version }} - - - - org.apache.flink - flink-scala{{ site.scala_version_suffix }} - {{site.version }} - - - org.apache.flink - flink-clients{{ site.scala_version_suffix }} - {{site.version }} - -{% endhighlight %} - -**Important:** When working with the Scala API you must have one of these two imports: -{% highlight scala %} -import org.apache.flink.api.scala._ -{% endhighlight %} - -or - -{% highlight scala %} -import org.apache.flink.api.scala.createTypeInformation -{% endhighlight %} - -The reason is that Flink analyzes the types that are used in a program and generates serializers -and comparators for them. By having either of those imports you enable an implicit conversion -that creates the type information for Flink operations. - -If you would rather use SBT, see [here]({{ site.baseurl }}/quickstart/scala_api_quickstart.html#sbt). -
-
- -#### Scala Dependency Versions - -Because Scala 2.10 binary is not compatible with Scala 2.11 binary, we provide multiple artifacts -to support both Scala versions. - -Starting from the 0.10 line, we cross-build all Flink modules for both 2.10 and 2.11. If you want -to run your program on Flink with Scala 2.11, you need to add a `_2.11` suffix to the `artifactId` -values of the Flink modules in your dependencies section. - -If you are looking for building Flink with Scala 2.11, please check -[build guide]({{ site.baseurl }}/start/building.html#scala-versions). - -#### Hadoop Dependency Versions - -If you are using Flink together with Hadoop, the version of the dependency may vary depending on the -version of Hadoop (or more specifically, HDFS) that you want to use Flink with. Please refer to the -[downloads page](http://flink.apache.org/downloads.html) for a list of available versions, and instructions -on how to link with custom versions of Hadoop. - -In order to link against the latest SNAPSHOT versions of the code, please follow -[this guide](http://flink.apache.org/how-to-contribute.html#snapshots-nightly-builds). - -The *flink-clients* dependency is only necessary to invoke the Flink program locally (for example to -run it standalone for testing and debugging). If you intend to only export the program as a JAR -file and [run it on a cluster]({{ site.baseurl }}/dev/cluster_execution.html), you can skip that dependency. - -{% top %} - diff --git a/docs/redirects/linking_with_flink.md b/docs/redirects/linking_with_flink.md new file mode 100644 index 0000000000000..1289487c9ff22 --- /dev/null +++ b/docs/redirects/linking_with_flink.md @@ -0,0 +1,25 @@ +--- +title: "Linking with Flink" +layout: redirect +redirect: /start/dependencies.html +permalink: /dev/linking_with_flink.html +--- + + diff --git a/docs/redirects/linking_with_optional_modules.md b/docs/redirects/linking_with_optional_modules.md new file mode 100644 index 0000000000000..e494fbc54d082 --- /dev/null +++ b/docs/redirects/linking_with_optional_modules.md @@ -0,0 +1,25 @@ +--- +title: "Linking with Optional Modules" +layout: redirect +redirect: /start/dependencies.html +permalink: /dev/linking.html +--- + + diff --git a/docs/start/dependencies.md b/docs/start/dependencies.md new file mode 100644 index 0000000000000..1375c6f30f8ea --- /dev/null +++ b/docs/start/dependencies.md @@ -0,0 +1,244 @@ +--- +title: "Configuring Dependencies, Connectors, Libraries" +nav-parent_id: start +nav-pos: 2 +--- + + +Every Flink application depends on a set of Flink libraries. At the bare minimum, the application depends +on the Flink APIs. Many applications depend in addition on certain connector libraries (like Kafka, Cassandra, etc.). +When running Flink applications (either in a distributed deployment, or in the IDE for testing), the Flink +runtime library must be available as well. + + +## Flink Core and Application Dependencies + +As with most systems that run user-defined applications, there are two broad categories of dependencies and libraries in Flink: + + - **Flink Core Dependenies**: Flink itself consists of a set of classes and dependencies that are needed to run the system, for example + coordination, networking, checkpoints, failover, APIs, operations (such as windowing), resource management, etc. + The set of all these classes and dependencies forms the core of Flink's runtime and must be present when a Flink + application is started. + + These core classes and dependencies are packaged in the `flink-dist` jar. They are part of Flink's `lib` folder and + part of the basic Flink container images. Think of these dependencies as similar to Java's core library (`rt.jar`, `charsets.jar`, etc.), + which contains the classes like `String` and `List`. + + The Flink Core Dependencies do not contain any connectors or libraries (CEP, SQL, ML, etc.) in order to avoid having an excessive + number of dependencies and classes in the classpath by default. In fact, we try to keep the core dependencies as slim as possible + to keep the default classpath small and avoid dependency clashes. + + - The **User Application Dependencies** are all connectors, formats, or libraries that a specific user application needs. + + The user application is typically packaged into an *application jar*, which contains the application code and the required + connector and library dependencies. + + The user application dependencies explicitly do not include the Flink DataSet / DataStream APIs and runtime dependencies, + because those are already part of Flink's Core Dependencies. + + +## Setting up a Project: Basic Dependencies + +Every Flink application needs as the bare minimum the API dependencies, to develop against. +For Maven, you can use the [Java Project Template]({{ site.baseurl }}/quickstart/java_api_quickstart.html) +or [Scala Project Template]({{ site.baseurl }}/quickstart/scala_api_quickstart.html) to create +a program skeleton with these initial dependencies. + +When setting up a project manually, you need to add the following dependencies for the Java/Scala API +(here presented in Maven syntax, but the same dependencies apply to other build tools (Gradle, SBT, etc.) as well. + +
+
+{% highlight xml %} + + org.apache.flink + flink-java + {{site.version }} + provided + + + org.apache.flink + flink-streaming-java{{ site.scala_version_suffix }} + {{site.version }} + provided + +{% endhighlight %} +
+
+{% highlight xml %} + + org.apache.flink + flink-scala{{ site.scala_version_suffix }} + {{site.version }} + provided + + + org.apache.flink + flink-streaming-scala{{ site.scala_version_suffix }} + {{site.version }} + provided + +{% endhighlight %} +
+
+ +**Important:** Please note that all these dependencies have their scope set to *provided*. +That means that they are needed to compile against, but that they should not be packaged into the +project's resulting application jar file - these dependencies are Flink Core Dependencies, +which are already available in any setup. + +It is highly recommended to keep the dependencies in scope *provided*. If they are not set to *provided*, +the best case is that the resulting JAR becomes excessively large, because it also contains all Flink core +dependencies. The worst case is that the Flink core dependencies that are added to the application's jar file +clash with some of your own dependency versions (which is normally avoided through inverted classloading). + +**Note on IntelliJ:** To make the applications run within IntelliJ IDEA, the Flink dependencies need +to be declared in scope *compile* rather than *provided*. Otherwise IntelliJ will not add them to the classpath and +the in-IDE execution will fail with a `NoClassDefFountError`. To avoid having to declare the +dependency scope as *compile* (which is not recommended, see above), the above linked Java- and Scala +project templates use a trick: They add a profile that selectively activates when the application +is run in IntelliJ and only then promotes the dependencies to scope *compile*, without affecting +the packaging of the JAR files. + + +## Adding Connector and Library Dependencies + +Most applications need specific connectors or libraries to run, for example a connector to Kafka, Cassandra, etc. +These connectors are not part of Flink's core dependencies and must hence be added as dependencies to the application + +Below is an example adding the connector for Kafka 0.10 as a dependency (Maven syntax): +{% highlight xml %} + + org.apache.flink + flink-connector-kafka-0.10{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} + +We recommend to package the application code and all its required dependencies into one *jar-with-dependencies* which +we refer to as the *application jar*. The application jar can be submitted to an already running Flink cluster, +or added to a Flink application container image. + +Projects created from the [Java Project Template]({{ site.baseurl }}/quickstart/java_api_quickstart.html) or +[Scala Project Template]({{ site.baseurl }}/quickstart/scala_api_quickstart.html) are configured to automatically include +the application dependencies into the application jar when running `mvn clean package`. For projects that are +not set up from those templates, we recommend to add the Maven Shade Plugin (as listed in the Appendix below) +to build the application jar with all required dependencies. + +**Important:** For Maven (and other build tools) to correctly package the dependencies into the application jar, +these application dependencies must be specified in scope *compile* (unlike the core dependencies, which +must be specified in scope *provided*). + + +## Scala Versions + +Scala versions (2.10, 2.11, 2.12, etc.) are not binary compatible with one another. +For that reason, Flink for Scala 2.11 cannot be used with an application that uses +Scala 2.12. + +All Flink dependencies that (transitively) depend on Scala are suffixed with the +Scala version that they are built for, for example `flink-streaming-scala_2.11`. + +Developers that only use Java can pick any Scala version, Scala developers need to +pick the Scala version that matches their application's Scala version. + +Please refer to the [build guide]({{ site.baseurl }}/start/building.html#scala-versions) +for details on how to build Flink for a specific Scala version. + +**Note:** Because of major breaking changes in Scala 2.12, Flink 1.5 currently builds only for Scala 2.11. +We aim to add support for Scala 2.12 in the next versions. + + +## Hadoop Dependencies + +**General rule: It should never be necessary to add Hadoop dependencies directly to your application.** +*(The only exception being when using existing Hadoop input-/output formats with Flink's Hadoop compatibility wrappers)* + +If you want to use Flink with Hadoop, you need to have a Flink setup that includes the Hadoop dependencies, rather than +adding Hadoop as an application dependency. Please refer to the [Hadoop Setup Guide]({{ site.baseurl }}/ops/deployment/hadoop.html) +for details. + +There are two main reasons for that design: + + - Some Hadoop interaction happens in Flink's core, possibly before the user application is started, for example + setting up HDFS for checkpoints, authenticating via Hadoop's Kerberos tokens, or deployment on YARN. + + - Flink's inverted classloading approach hides many transitive dependencies from the core dependencies. That applies not only + to Flink's own core dependencies, but also to Hadoop's dependencies when present in the setup. + That way, applications can use different versions of the same dependencies without running into dependency conflicts (and + trust us, that's a big deal, because Hadoops dependency tree is huge.) + +If you need Hadoop dependencies during testing or development inside the IDE (for example for HDFS access), please configure +these dependencies similar to the scope of the dependencies to *test* or to *provided*. + + +## Appendix: Template for bulding a Jar with Dependencies + +To build an application JAR that contains all dependencies required for declared connectors and libraries, +you can use the following shade plugin definition: + +{% highlight xml %} + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + + + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + my.prorgams.main.clazz + + + + + + + + +{% endhighlight %} + +{% top %} + From 93f823fff7b1517cc3f2484fa0091ebdc9e546d0 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 22 Feb 2018 17:22:54 +0100 Subject: [PATCH 290/367] [FLINK-8451] [serializers] Make Scala tuple serializer deserialization more failure tolerant This closes #5567. --- .../TupleSerializerConfigSnapshot.java | 2 +- .../apache/flink/util/InstantiationUtil.java | 56 +++++++++-- .../flink-1.3.2-scala-types-serializer-data | Bin 0 -> 97 bytes ...link-1.3.2-scala-types-serializer-snapshot | Bin 0 -> 7634 bytes .../TupleSerializerCompatibilityTest.scala | 86 ++++++++++++++++ ...SerializerCompatibilityTestGenerator.scala | 94 ++++++++++++++++++ 6 files changed, 231 insertions(+), 7 deletions(-) create mode 100644 flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-data create mode 100644 flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-snapshot create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTest.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTestGenerator.scala diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java index 705099e9b2dbc..eac5200da9c6f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java @@ -61,7 +61,7 @@ public void read(DataInputView in) throws IOException { super.read(in); try (final DataInputViewStream inViewWrapper = new DataInputViewStream(in)) { - tupleClass = InstantiationUtil.deserializeObject(inViewWrapper, getUserCodeClassLoader()); + tupleClass = InstantiationUtil.deserializeObject(inViewWrapper, getUserCodeClassLoader(), true); } catch (ClassNotFoundException e) { throw new IOException("Could not find requested tuple class in classpath.", e); } diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index 7ffad55bb7619..a95bdf701d86f 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -113,7 +113,7 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas * *

This can be removed once 1.2 is no longer supported. */ - private static Set scalaSerializerClassnames = new HashSet<>(); + private static final Set scalaSerializerClassnames = new HashSet<>(); static { scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.TraversableSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.CaseClassSerializer"); @@ -121,10 +121,53 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.EnumValueSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.OptionSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.TrySerializer"); - scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.EitherSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.UnitSerializer"); } + /** + * The serialVersionUID might change between Scala versions and since those classes are + * part of the tuple serializer config snapshots we need to ignore them. + * + * @see FLINK-8451 + */ + private static final Set scalaTypes = new HashSet<>(); + static { + scalaTypes.add("scala.Tuple1"); + scalaTypes.add("scala.Tuple2"); + scalaTypes.add("scala.Tuple3"); + scalaTypes.add("scala.Tuple4"); + scalaTypes.add("scala.Tuple5"); + scalaTypes.add("scala.Tuple6"); + scalaTypes.add("scala.Tuple7"); + scalaTypes.add("scala.Tuple8"); + scalaTypes.add("scala.Tuple9"); + scalaTypes.add("scala.Tuple10"); + scalaTypes.add("scala.Tuple11"); + scalaTypes.add("scala.Tuple12"); + scalaTypes.add("scala.Tuple13"); + scalaTypes.add("scala.Tuple14"); + scalaTypes.add("scala.Tuple15"); + scalaTypes.add("scala.Tuple16"); + scalaTypes.add("scala.Tuple17"); + scalaTypes.add("scala.Tuple18"); + scalaTypes.add("scala.Tuple19"); + scalaTypes.add("scala.Tuple20"); + scalaTypes.add("scala.Tuple21"); + scalaTypes.add("scala.Tuple22"); + scalaTypes.add("scala.Tuple1$mcJ$sp"); + scalaTypes.add("scala.Tuple1$mcI$sp"); + scalaTypes.add("scala.Tuple1$mcD$sp"); + scalaTypes.add("scala.Tuple2$mcJJ$sp"); + scalaTypes.add("scala.Tuple2$mcJI$sp"); + scalaTypes.add("scala.Tuple2$mcJD$sp"); + scalaTypes.add("scala.Tuple2$mcIJ$sp"); + scalaTypes.add("scala.Tuple2$mcII$sp"); + scalaTypes.add("scala.Tuple2$mcID$sp"); + scalaTypes.add("scala.Tuple2$mcDJ$sp"); + scalaTypes.add("scala.Tuple2$mcDI$sp"); + scalaTypes.add("scala.Tuple2$mcDD$sp"); + } + /** * An {@link ObjectInputStream} that ignores serialVersionUID mismatches when deserializing objects of * anonymous classes or our Scala serializer classes and also replaces occurences of GenericData.Array @@ -158,12 +201,13 @@ protected ObjectStreamClass readClassDescriptor() throws IOException, ClassNotFo } } - Class localClass = resolveClass(streamClassDescriptor); - if (scalaSerializerClassnames.contains(localClass.getName()) || localClass.isAnonymousClass() + final Class localClass = resolveClass(streamClassDescriptor); + final String name = localClass.getName(); + if (scalaSerializerClassnames.contains(name) || scalaTypes.contains(name) || localClass.isAnonymousClass() // isAnonymousClass does not work for anonymous Scala classes; additionally check by classname - || localClass.getName().contains("$anon$") || localClass.getName().contains("$anonfun")) { + || name.contains("$anon$") || name.contains("$anonfun")) { - ObjectStreamClass localClassDescriptor = ObjectStreamClass.lookup(localClass); + final ObjectStreamClass localClassDescriptor = ObjectStreamClass.lookup(localClass); if (localClassDescriptor != null && localClassDescriptor.getSerialVersionUID() != streamClassDescriptor.getSerialVersionUID()) { LOG.warn("Ignoring serialVersionUID mismatch for anonymous class {}; was {}, now {}.", diff --git a/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-data b/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-data new file mode 100644 index 0000000000000000000000000000000000000000..ddd6ac01112405f127b5c3f3d9412bf060009d3f GIT binary patch literal 97 zcmZQzV9;V@^GMCf$!B0I&qyq>chE=!0|o{L9w^oV(ioB{K*1A0oC4L(2vwm2(f%I@ H8i3LO{3sa1 literal 0 HcmV?d00001 diff --git a/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-snapshot b/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..1ebdabb29fc94d31123061db091237561394f4f2 GIT binary patch literal 7634 zcmeHMO>7%Q6n?v@LzR-Gv=BiB5>!6)(p|-A%8!vM#r=_sC6FA1+5*+Yo!FDEcg^g$ z#1%i{0yo5=2jGejrQ*f`Rq6#eASzB=xKyf&%AZ8V1#tn9zM0*4H_60}z3a*dTUy(* z*1mb~dw<_N1AqbQHTosMBVXZHT#sJ<>=tBv4ljDVYLN!1)Tp)SF!wAjnB`Z9LoD8O z1*=nQzS(eSnQ}%PcAoNrS8ou(7MR0CYo7Yz7^P3_m!lMwnN;it#0KJlLJ~~0?9FR)=G{nfaL}*GAk=9|2BE3=tgheL0taH zw;ulF^3>n&OMI0)j0?T9Hb`1PK&_%HBD3SlahThheM-z)Bg@RKI#hUW>(h5GePg|P zL~8d6yrH6YRx37$+O|KeBTUpNk5qf}PlJA#eS6jfvjh_kU5u$M;9##_p5&}ri;0p- zj<;3KYgN?)Sr#0vjueh#DW#5~V`jAS)|>vk%VHWm^Qe`*c5&v%RS2b=Gack?l0Aj3 zCjM+#0H%Nb=lCxz^TgMy8_P;$s%Vww+#^@0%F|j^^bpN?o>^%L=f zA>z5EeU+)!d*8m>0dZ;Jj`rIggzBW$0lhW>*%I1Se4qKn$A=z#DsMuj2nIV0rkE~) zK?UqCsSRvd_U#~5r3XY#z(n{{;5)p#P2?DpD(_VV%7CI)YS|?Xzvc<_ zl#I$u=&|2$iT6gz*dspz7(*P#F9J+l0GQkhuqy}fz}4`X8(|L}%vzaJb1~qU7V8b^ z`5fV-P6g!-_5HQAJV0EbmIN#Glkr1{`M`GoBbDynWf|(!K>E^*W%*8{BgynL;J7x@ zHp%Ms-Cdj3n7-=uJBmM#-R5PyZmPWiZib=MO`X5101svcCxlBD+iM~6Nh2rqkRD9L zdx0s4@sSror2EM|v}|v)6U2I=+9nDSqZ=Og)^<--z_5NvN+*wJ0fA2MxR)vgPdz0i z>yv|ugk?v9s1TI$#*!TmCi_J~SG(0M)b%xVbPcb+q~dt}6lQX2ho#g#HJHUSDP9u$ zAybf(uGlS?K{hkV Date: Sat, 3 Mar 2018 09:34:56 +0100 Subject: [PATCH 291/367] [FLINK-8487] Verify ZooKeeper checkpoint store behaviour with ITCase --- .../ZooKeeperCompletedCheckpointStore.java | 2 +- .../flink/runtime/concurrent/FutureUtils.java | 77 +++++ .../ZooKeeperHighAvailabilityITCase.java | 323 ++++++++++++++++++ 3 files changed, 401 insertions(+), 1 deletion(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index 73598e628b657..46405519061c4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -209,7 +209,7 @@ public void recover() throws Exception { if (completedCheckpoints.isEmpty() && numberOfInitialCheckpoints > 0) { throw new FlinkException( - "Could not read any of the " + numberOfInitialCheckpoints + " from storage."); + "Could not read any of the " + numberOfInitialCheckpoints + " checkpoints from storage."); } else if (completedCheckpoints.size() != numberOfInitialCheckpoints) { LOG.warn( "Could only fetch {} of {} checkpoints from storage.", diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index c18068bdf6eb4..5403490c17905 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -37,9 +37,11 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Predicate; import java.util.function.Supplier; import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -189,6 +191,81 @@ private static void retryOperationWithDelay( } } + /** + * Retry the given operation with the given delay in between successful completions where the + * result does not match a given predicate. + * + * @param operation to retry + * @param retryDelay delay between retries + * @param deadline A deadline that specifies at what point we should stop retrying + * @param acceptancePredicate Predicate to test whether the result is acceptable + * @param scheduledExecutor executor to be used for the retry operation + * @param type of the result + * @return Future which retries the given operation a given amount of times and delays the retry + * in case the predicate isn't matched + */ + public static CompletableFuture retrySuccesfulWithDelay( + final Supplier> operation, + final Time retryDelay, + final Deadline deadline, + final Predicate acceptancePredicate, + final ScheduledExecutor scheduledExecutor) { + + final CompletableFuture resultFuture = new CompletableFuture<>(); + + retrySuccessfulOperationWithDelay( + resultFuture, + operation, + retryDelay, + deadline, + acceptancePredicate, + scheduledExecutor); + + return resultFuture; + } + + private static void retrySuccessfulOperationWithDelay( + final CompletableFuture resultFuture, + final Supplier> operation, + final Time retryDelay, + final Deadline deadline, + final Predicate acceptancePredicate, + final ScheduledExecutor scheduledExecutor) { + + if (!resultFuture.isDone()) { + final CompletableFuture operationResultFuture = operation.get(); + + operationResultFuture.whenComplete( + (t, throwable) -> { + if (throwable != null) { + if (throwable instanceof CancellationException) { + resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable)); + } else { + resultFuture.completeExceptionally(throwable); + } + } else { + if (acceptancePredicate.test(t)) { + resultFuture.complete(t); + } if (deadline.hasTimeLeft()) { + final ScheduledFuture scheduledFuture = scheduledExecutor.schedule( + () -> retrySuccessfulOperationWithDelay(resultFuture, operation, retryDelay, deadline, acceptancePredicate, scheduledExecutor), + retryDelay.toMilliseconds(), + TimeUnit.MILLISECONDS); + + resultFuture.whenComplete( + (innerT, innerThrowable) -> scheduledFuture.cancel(false)); + } else { + resultFuture.completeExceptionally( + new RetryException("Could not satisfy the predicate within the allowed time.")); + } + } + }); + + resultFuture.whenComplete( + (t, throwable) -> operationResultFuture.cancel(false)); + } + } + /** * Exception with which the returned future is completed if the {@link #retry(Supplier, int, Executor)} * operation fails. diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java new file mode 100644 index 0000000000000..4fd8ea9536c00 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -0,0 +1,323 @@ +/* + * 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.flink.test.checkpointing; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.test.checkpointing.utils.SavepointMigrationTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.curator.test.TestingServer; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.nio.file.Files; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Integration tests for {@link org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore}. + */ +public class ZooKeeperHighAvailabilityITCase extends TestBaseUtils { + + private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES); + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static File haStorageDir; + + private static TestingServer zkServer; + + private static OneShotLatch waitForCheckpointLatch = new OneShotLatch(); + private static OneShotLatch failInCheckpointLatch = new OneShotLatch(); + private static OneShotLatch successfulRestoreLatch = new OneShotLatch(); + + // in CheckpointBlockingFunction we verify that we only call initializeState() + // once with isRestored() == false. All other invocations must have isRestored() == true. This + // verifies that we don't restart a job from scratch in case the CompletedCheckpoints can't + // be read. + private static AtomicInteger allowedRestores = new AtomicInteger(1); + + private static final Logger LOG = LoggerFactory.getLogger(SavepointMigrationTestBase.class); + protected LocalFlinkMiniCluster cluster = null; + + @Before + public void setup() throws Exception { + + zkServer = new TestingServer(); + + // Flink configuration + final Configuration config = new Configuration(); + + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + + haStorageDir = temporaryFolder.newFolder(); + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haStorageDir.toString()); + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); + + cluster = TestBaseUtils.startCluster(config, false); + } + + @After + public void teardown() throws Exception { + stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); + + zkServer.stop(); + zkServer.close(); + } + + /** + * Verify that we don't start a job from scratch if we cannot restore any of the + * CompletedCheckpoints. + * + *

Synchronization for the different steps and things we want to observe happens via + * latches in the test method and the methods of {@link CheckpointBlockingFunction}. + * + *

The test follows these steps: + *

    + *
  1. Start job and block on a latch until we have done some checkpoints + *
  2. Block in the special function + *
  3. Move away the ZooKeeper HA directory to make restoring checkpoints impossible + *
  4. Unblock the special function, which now induces a failure + *
  5. Make sure that the job does not recover successfully + *
  6. Move back the HA directory + *
  7. Make sure that the job recovers, we use a latch to ensure that the operator + * restored successfully + *
+ */ + @Test(timeout = 60_000L) + public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { + final Deadline deadline = TEST_TIMEOUT.fromNow(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 100L)); + env.enableCheckpointing(10); // Flink doesn't allow lower than 10 ms + + File checkpointLocation = temporaryFolder.newFolder(); + env.setStateBackend(new FsStateBackend(checkpointLocation.toURI())); + + DataStreamSource source = env.addSource(new UnboundedSource()); + + source + .keyBy((str) -> str) + .map(new CheckpointBlockingFunction()); + + JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + JobID jobID = Preconditions.checkNotNull(jobGraph.getJobID()); + + // Retrieve the job manager + ActorGateway jobManager = Await.result(cluster.leaderGateway().future(), deadline.timeLeft()); + + JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph); + + LOG.info("Submitted job {} and waiting...", jobSubmissionResult.getJobID()); + + StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration()); + + // wait until we did some checkpoints + waitForCheckpointLatch.await(); + + // mess with the HA directory so that the job cannot restore + File movedCheckpointLocation = temporaryFolder.newFolder(); + // clean it out, we only want the name so that we can move the checkpoints directory + assertTrue(movedCheckpointLocation.delete()); + Files.move(haStorageDir.toPath(), movedCheckpointLocation.toPath()); + + failInCheckpointLatch.trigger(); + + Thread.sleep(2000); + + // Ensure that we see at least one cycle where the job tries to restart and fails. + CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(1), + deadline, + (jobStatus) -> jobStatus.equals(JobStatus.RESTARTING), + TestingUtils.defaultScheduledExecutor()); + assertEquals( + JobStatus.RESTARTING, + jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(1), + deadline, + (jobStatus) -> jobStatus.equals(JobStatus.FAILING), + TestingUtils.defaultScheduledExecutor()); + assertEquals( + JobStatus.FAILING, + jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + + // move back the HA directory so that the job can restore + Files.move(movedCheckpointLocation.toPath(), haStorageDir.toPath()); + + // now the job should be able to go to RUNNING again + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus.equals(JobStatus.RUNNING), + TestingUtils.defaultScheduledExecutor()); + + assertEquals( + JobStatus.RUNNING, + jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + + // make sure we saw a successful restore + successfulRestoreLatch.await(); + + clusterClient.cancel(jobID); + + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus.equals(JobStatus.CANCELED), + TestingUtils.defaultScheduledExecutor()); + + assertEquals( + JobStatus.CANCELED, + jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + } + + /** + * Requests the {@link JobStatus} of the job with the given {@link JobID}. + */ + public CompletableFuture getJobStatus( + ActorGateway jobManager, + JobID jobId, + FiniteDuration timeout) { + + Future response = jobManager.ask(JobManagerMessages.getRequestJobStatus(jobId), timeout); + + CompletableFuture javaFuture = FutureUtils.toJava(response); + + return javaFuture.thenApply((responseMessage) -> { + if (responseMessage instanceof JobManagerMessages.CurrentJobStatus) { + return ((JobManagerMessages.CurrentJobStatus) responseMessage).status(); + } else if (responseMessage instanceof JobManagerMessages.JobNotFound) { + throw new CompletionException( + new IllegalStateException("Could not find job with JobId " + jobId)); + } else { + throw new CompletionException( + new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); + } + }); + } + + + private static class UnboundedSource implements SourceFunction { + private boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + while (running) { + ctx.collect("hello"); + // don't overdo it ... ;-) + Thread.sleep(50); + } + } + + @Override + public void cancel() { + running = false; + } + } + + private static class CheckpointBlockingFunction + extends RichMapFunction + implements CheckpointedFunction { + + // also have some state to write to the checkpoint + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); + + @Override + public String map(String value) throws Exception { + getRuntimeContext().getState(stateDescriptor).update("42"); + return value; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (context.getCheckpointId() > 5) { + waitForCheckpointLatch.trigger(); + failInCheckpointLatch.await(); + throw new RuntimeException("Failing on purpose."); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) { + if (!context.isRestored()) { + int updatedValue = allowedRestores.decrementAndGet(); + if (updatedValue < 0) { + throw new RuntimeException("We are not allowed any more restores."); + } + } else { + successfulRestoreLatch.trigger(); + } + } + } +} From bddd910b526985f6c2b1a76ccc41539676468e6f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 5 Mar 2018 13:57:30 +0100 Subject: [PATCH 292/367] Add more through test verification --- .../ZooKeeperHighAvailabilityITCase.java | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java index 4fd8ea9536c00..947c42b70274a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -62,6 +62,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import scala.concurrent.Await; @@ -69,7 +70,9 @@ import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; +import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -94,7 +97,12 @@ public class ZooKeeperHighAvailabilityITCase extends TestBaseUtils { // once with isRestored() == false. All other invocations must have isRestored() == true. This // verifies that we don't restart a job from scratch in case the CompletedCheckpoints can't // be read. - private static AtomicInteger allowedRestores = new AtomicInteger(1); + private static AtomicInteger allowedInitializeCallsWithoutRestore = new AtomicInteger(1); + + // in CheckpointBlockingFunction we count when we see restores that are not allowed. We only + // allow restores once we messed with the HA directory and moved it back again + private static AtomicInteger illegalRestores = new AtomicInteger(0); + private static AtomicBoolean restoreAllowed = new AtomicBoolean(false); private static final Logger LOG = LoggerFactory.getLogger(SavepointMigrationTestBase.class); protected LocalFlinkMiniCluster cluster = null; @@ -151,7 +159,7 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 100L)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0)); env.enableCheckpointing(10); // Flink doesn't allow lower than 10 ms File checkpointLocation = temporaryFolder.newFolder(); @@ -186,8 +194,6 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { failInCheckpointLatch.trigger(); - Thread.sleep(2000); - // Ensure that we see at least one cycle where the job tries to restart and fails. CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), @@ -210,6 +216,7 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); // move back the HA directory so that the job can restore + restoreAllowed.set(true); Files.move(movedCheckpointLocation.toPath(), haStorageDir.toPath()); // now the job should be able to go to RUNNING again @@ -239,6 +246,8 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { assertEquals( JobStatus.CANCELED, jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + + assertThat("We saw illegal restores.", illegalRestores.get(), is(0)); } /** @@ -311,11 +320,14 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @Override public void initializeState(FunctionInitializationContext context) { if (!context.isRestored()) { - int updatedValue = allowedRestores.decrementAndGet(); + int updatedValue = allowedInitializeCallsWithoutRestore.decrementAndGet(); if (updatedValue < 0) { throw new RuntimeException("We are not allowed any more restores."); } } else { + if (!restoreAllowed.get()) { + illegalRestores.getAndIncrement(); + } successfulRestoreLatch.trigger(); } } From 64adf4bcfa387047036534059cfe361977439cca Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 2 Mar 2018 17:46:56 +0100 Subject: [PATCH 293/367] [FLINK-8807] Fix ZookeeperCompleted checkpoint store can get stuck in infinite loop Before, CompletedCheckpoint did not have proper equals()/hashCode(), which meant that the fixpoint condition in ZooKeeperCompletedCheckpointStore would never hold if at least on checkpoint became unreadable. We now compare the interesting fields of the checkpoints manually and extended the test to properly create new CompletedCheckpoints. Before, we were reusing the same CompletedCheckpoint instances, meaning that Objects.equals()/hashCode() would make the test succeed. --- .../checkpoint/CompletedCheckpoint.java | 26 +++++++++ .../ZooKeeperCompletedCheckpointStore.java | 2 +- ...ZooKeeperCompletedCheckpointStoreTest.java | 53 ++++++++++--------- 3 files changed, 56 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index d6d0827fb7a9d..e4f7da24dd0b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.SharedStateRegistry; @@ -34,7 +35,9 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -277,6 +280,29 @@ public String getExternalPointer() { return externalPointer; } + public static boolean checkpointsMatch( + Collection first, + Collection second) { + + Set> firstInterestingFields = + new HashSet<>(); + + for (CompletedCheckpoint checkpoint : first) { + firstInterestingFields.add( + new Tuple2<>(checkpoint.getCheckpointID(), checkpoint.getJobId())); + } + + Set> secondInterestingFields = + new HashSet<>(); + + for (CompletedCheckpoint checkpoint : second) { + secondInterestingFields.add( + new Tuple2<>(checkpoint.getCheckpointID(), checkpoint.getJobId())); + } + + return firstInterestingFields.equals(secondInterestingFields); + } + /** * Sets the callback for tracking when this checkpoint is discarded. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index 46405519061c4..f22127041d31b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -199,7 +199,7 @@ public void recover() throws Exception { } } while (retrievedCheckpoints.size() != numberOfInitialCheckpoints && - !lastTryRetrievedCheckpoints.equals(retrievedCheckpoints)); + !CompletedCheckpoint.checkpointsMatch(lastTryRetrievedCheckpoints, retrievedCheckpoints)); // Clear local handles in order to prevent duplicates on // recovery. The local handles should reflect the state diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index 7c19b19e0d28e..977de85b9f63e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -32,8 +32,10 @@ import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.api.ErrorListenerPathable; import org.apache.curator.utils.EnsurePath; + import org.junit.Test; import org.junit.runner.RunWith; + import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -80,31 +82,16 @@ public void testPathConversion() { /** * Tests that the completed checkpoint store can retrieve all checkpoints stored in ZooKeeper * and ignores those which cannot be retrieved via their state handles. + * + *

We have a timeout in case the ZooKeeper store get's into a deadlock/livelock situation. */ - @Test + @Test(timeout = 50000) public void testCheckpointRecovery() throws Exception { + final JobID jobID = new JobID(); + final long checkpoint1Id = 1L; + final long checkpoint2Id = 2; final List, String>> checkpointsInZooKeeper = new ArrayList<>(4); - final CompletedCheckpoint completedCheckpoint1 = new CompletedCheckpoint( - new JobID(), - 1L, - 1L, - 1L, - new HashMap<>(), - null, - CheckpointProperties.forStandardCheckpoint(), - null, null); - - final CompletedCheckpoint completedCheckpoint2 = new CompletedCheckpoint( - new JobID(), - 2L, - 2L, - 2L, - new HashMap<>(), - null, - CheckpointProperties.forStandardCheckpoint(), - null, null); - final Collection expectedCheckpointIds = new HashSet<>(2); expectedCheckpointIds.add(1L); expectedCheckpointIds.add(2L); @@ -113,10 +100,28 @@ public void testCheckpointRecovery() throws Exception { when(failingRetrievableStateHandle.retrieveState()).thenThrow(new IOException("Test exception")); final RetrievableStateHandle retrievableStateHandle1 = mock(RetrievableStateHandle.class); - when(retrievableStateHandle1.retrieveState()).thenReturn(completedCheckpoint1); + when(retrievableStateHandle1.retrieveState()).then( + (invocation) -> new CompletedCheckpoint( + jobID, + checkpoint1Id, + 1L, + 1L, + new HashMap<>(), + null, + CheckpointProperties.forStandardCheckpoint(), + null, null)); final RetrievableStateHandle retrievableStateHandle2 = mock(RetrievableStateHandle.class); - when(retrievableStateHandle2.retrieveState()).thenReturn(completedCheckpoint2); + when(retrievableStateHandle2.retrieveState()).then( + (invocation -> new CompletedCheckpoint( + jobID, + checkpoint2Id, + 1L, + 1L, + new HashMap<>(), + null, + CheckpointProperties.forStandardCheckpoint(), + null, null))); checkpointsInZooKeeper.add(Tuple2.of(retrievableStateHandle1, "/foobar1")); checkpointsInZooKeeper.add(Tuple2.of(failingRetrievableStateHandle, "/failing1")); @@ -182,7 +187,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { // check that we return the latest retrievable checkpoint // this should remove the latest checkpoint because it is broken - assertEquals(completedCheckpoint2.getCheckpointID(), latestCompletedCheckpoint.getCheckpointID()); + assertEquals(checkpoint2Id, latestCompletedCheckpoint.getCheckpointID()); // this should remove the second broken checkpoint because we're iterating over all checkpoints List completedCheckpoints = zooKeeperCompletedCheckpointStore.getAllCheckpoints(); From 4d16c282c45892c27b748ca179cfa1c9219cb68b Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 5 Mar 2018 15:45:36 +0100 Subject: [PATCH 294/367] Fix checkstyle in ZooKeeperCompletedCheckpointStoreTest --- ...ZooKeeperCompletedCheckpointStoreTest.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index 977de85b9f63e..7156cb5e1192d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -32,10 +32,8 @@ import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.api.ErrorListenerPathable; import org.apache.curator.utils.EnsurePath; - import org.junit.Test; import org.junit.runner.RunWith; - import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -66,6 +64,9 @@ import static org.powermock.api.mockito.PowerMockito.doThrow; import static org.powermock.api.mockito.PowerMockito.whenNew; +/** + * Tests for {@link ZooKeeperCompletedCheckpointStore}. + */ @RunWith(PowerMockRunner.class) @PrepareForTest(ZooKeeperCompletedCheckpointStore.class) public class ZooKeeperCompletedCheckpointStoreTest extends TestLogger { @@ -208,7 +209,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { // are subsumed should they be discarded. verify(failingRetrievableStateHandle, never()).discardState(); } - + /** * Tests that the checkpoint does not exist in the store when we fail to add * it into the store (i.e., there exists an exception thrown by the method). @@ -217,29 +218,29 @@ public Void answer(InvocationOnMock invocation) throws Throwable { public void testAddCheckpointWithFailedRemove() throws Exception { final CuratorFramework client = mock(CuratorFramework.class, Mockito.RETURNS_DEEP_STUBS); final RetrievableStateStorageHelper storageHelperMock = mock(RetrievableStateStorageHelper.class); - - ZooKeeperStateHandleStore zookeeperStateHandleStoreMock = + + ZooKeeperStateHandleStore zookeeperStateHandleStoreMock = spy(new ZooKeeperStateHandleStore<>(client, storageHelperMock, Executors.directExecutor())); whenNew(ZooKeeperStateHandleStore.class).withAnyArguments().thenReturn(zookeeperStateHandleStoreMock); - + doAnswer(new Answer>() { @Override public RetrievableStateHandle answer(InvocationOnMock invocationOnMock) throws Throwable { - CompletedCheckpoint checkpoint = (CompletedCheckpoint)invocationOnMock.getArguments()[1]; - + CompletedCheckpoint checkpoint = (CompletedCheckpoint) invocationOnMock.getArguments()[1]; + RetrievableStateHandle retrievableStateHandle = mock(RetrievableStateHandle.class); when(retrievableStateHandle.retrieveState()).thenReturn(checkpoint); - + return retrievableStateHandle; } }).when(zookeeperStateHandleStoreMock).addAndLock(anyString(), any(CompletedCheckpoint.class)); - + doThrow(new Exception()).when(zookeeperStateHandleStoreMock).releaseAndTryRemove(anyString(), any(ZooKeeperStateHandleStore.RemoveCallback.class)); - + final int numCheckpointsToRetain = 1; final String checkpointsPath = "foobar"; final RetrievableStateStorageHelper stateSotrage = mock(RetrievableStateStorageHelper.class); - + ZooKeeperCompletedCheckpointStore zooKeeperCompletedCheckpointStore = new ZooKeeperCompletedCheckpointStore( numCheckpointsToRetain, client, @@ -251,10 +252,10 @@ public RetrievableStateHandle answer(InvocationOnMock invoc CompletedCheckpoint checkpointToAdd = mock(CompletedCheckpoint.class); doReturn(i).when(checkpointToAdd).getCheckpointID(); doReturn(Collections.emptyMap()).when(checkpointToAdd).getOperatorStates(); - + try { zooKeeperCompletedCheckpointStore.addCheckpoint(checkpointToAdd); - + // The checkpoint should be in the store if we successfully add it into the store. List addedCheckpoints = zooKeeperCompletedCheckpointStore.getAllCheckpoints(); assertTrue(addedCheckpoints.contains(checkpointToAdd)); From f58ade2c435f4715a9d19fd57a28da5d3ef4bb97 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 1 Mar 2018 10:40:20 +0100 Subject: [PATCH 295/367] [hotfix][docs] Drop the incorrect parallel remark in windowAll This closes #5607. --- .../streaming/api/datastream/DataStream.java | 25 ++++++++----------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 83c11266ce00a..8ad2f54e2e6ed 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -675,9 +675,8 @@ public JoinedStreams join(DataStream otherStream) { * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic * set using * - *

Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)} * @@ -699,9 +698,8 @@ public AllWindowedStream timeWindowAll(Time size) { * set using * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)} * - *

Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param size The size of the window. */ @@ -716,9 +714,8 @@ public AllWindowedStream timeWindowAll(Time size, Time slide) { /** * Windows this {@code DataStream} into tumbling count windows. * - *

Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param size The size of the windows in number of elements. */ @@ -729,9 +726,8 @@ public AllWindowedStream countWindowAll(long size) { /** * Windows this {@code DataStream} into sliding count windows. * - *

Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param size The size of the windows in number of elements. * @param slide The slide interval in number of elements. @@ -752,9 +748,8 @@ public AllWindowedStream countWindowAll(long size, long slide) * when windows are evaluated. However, {@code WindowAssigners} have a default {@code Trigger} * that is used if a {@code Trigger} is not specified. * - *

Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param assigner The {@code WindowAssigner} that assigns elements to windows. * @return The trigger windows data stream. From 5aa481d5aa2a22755959fdc48e3449535f2b44e9 Mon Sep 17 00:00:00 2001 From: Ken Krugler Date: Sun, 4 Mar 2018 09:27:11 -0800 Subject: [PATCH 296/367] [FLINK-8849][docs] Fix links to chaining docs This closes #5630. --- docs/concepts/runtime.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/concepts/runtime.md b/docs/concepts/runtime.md index cb6d58f0c1e97..1c7c2816a0e14 100644 --- a/docs/concepts/runtime.md +++ b/docs/concepts/runtime.md @@ -31,7 +31,7 @@ under the License. For distributed execution, Flink *chains* operator subtasks together into *tasks*. Each task is executed by one thread. Chaining operators together into tasks is a useful optimization: it reduces the overhead of thread-to-thread handover and buffering, and increases overall throughput while decreasing latency. -The chaining behavior can be configured; see the [chaining docs](../dev/datastream_api.html#task-chaining-and-resource-groups) for details. +The chaining behavior can be configured; see the [chaining docs](../dev/stream/operators/#task-chaining-and-resource-groups) for details. The sample dataflow in the figure below is executed with five subtasks, and hence with five parallel threads. @@ -98,7 +98,7 @@ job. Allowing this *slot sharing* has two main benefits: TaskManagers with shared Task Slots -The APIs also include a *[resource group](../dev/datastream_api.html#task-chaining-and-resource-groups)* mechanism which can be used to prevent undesirable slot sharing. +The APIs also include a *[resource group](../dev/stream/operators/#task-chaining-and-resource-groups)* mechanism which can be used to prevent undesirable slot sharing. As a rule-of-thumb, a good default number of task slots would be the number of CPU cores. With hyper-threading, each slot then takes 2 or more hardware thread contexts. From 87af955b3a0d9e173c1901c5998cf5876cff91ea Mon Sep 17 00:00:00 2001 From: neoremind Date: Mon, 5 Mar 2018 16:50:37 +0800 Subject: [PATCH 297/367] [FLINK-8857][hbase] Remove redundant execute() call in hbase example This closes #5633. --- .../apache/flink/addons/hbase/example/HBaseReadExample.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java index 817ae090c4ef0..8475fb81ec0dd 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java @@ -86,9 +86,6 @@ public boolean filter(Tuple2 t) throws Exception { hbaseDs.print(); - // kick off execution. - env.execute(); - } } From 8e1987d8496ccbe1889fe8d44eaabf6d85e3dd34 Mon Sep 17 00:00:00 2001 From: Stephen Parente Date: Fri, 2 Mar 2018 14:20:10 -0800 Subject: [PATCH 298/367] [hotfix][docs] Remove reference to CheckpointedRestoring This closes #5627. --- docs/dev/stream/state/state.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index ce1ce825f4d52..7e38b0ced0edc 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -386,8 +386,7 @@ public class BufferingSink {% highlight scala %} class BufferingSink(threshold: Int = 0) extends SinkFunction[(String, Int)] - with CheckpointedFunction - with CheckpointedRestoring[List[(String, Int)]] { + with CheckpointedFunction { @transient private var checkpointedState: ListState[(String, Int)] = _ @@ -426,9 +425,6 @@ class BufferingSink(threshold: Int = 0) } } - override def restoreState(state: List[(String, Int)]): Unit = { - bufferedElements ++= state - } } {% endhighlight %} From 7f8b0b81699f6ddfdcc4d69710e47b58b4d0d79f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 5 Mar 2018 20:05:23 +0100 Subject: [PATCH 299/367] Revert "Add more through test verification" This reverts commit bddd910b526985f6c2b1a76ccc41539676468e6f which was added to the release-1.4 branch by mistake. --- .../ZooKeeperHighAvailabilityITCase.java | 22 +++++-------------- 1 file changed, 5 insertions(+), 17 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java index 947c42b70274a..4fd8ea9536c00 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -62,7 +62,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import scala.concurrent.Await; @@ -70,9 +69,7 @@ import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; -import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -97,12 +94,7 @@ public class ZooKeeperHighAvailabilityITCase extends TestBaseUtils { // once with isRestored() == false. All other invocations must have isRestored() == true. This // verifies that we don't restart a job from scratch in case the CompletedCheckpoints can't // be read. - private static AtomicInteger allowedInitializeCallsWithoutRestore = new AtomicInteger(1); - - // in CheckpointBlockingFunction we count when we see restores that are not allowed. We only - // allow restores once we messed with the HA directory and moved it back again - private static AtomicInteger illegalRestores = new AtomicInteger(0); - private static AtomicBoolean restoreAllowed = new AtomicBoolean(false); + private static AtomicInteger allowedRestores = new AtomicInteger(1); private static final Logger LOG = LoggerFactory.getLogger(SavepointMigrationTestBase.class); protected LocalFlinkMiniCluster cluster = null; @@ -159,7 +151,7 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 100L)); env.enableCheckpointing(10); // Flink doesn't allow lower than 10 ms File checkpointLocation = temporaryFolder.newFolder(); @@ -194,6 +186,8 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { failInCheckpointLatch.trigger(); + Thread.sleep(2000); + // Ensure that we see at least one cycle where the job tries to restart and fails. CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), @@ -216,7 +210,6 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); // move back the HA directory so that the job can restore - restoreAllowed.set(true); Files.move(movedCheckpointLocation.toPath(), haStorageDir.toPath()); // now the job should be able to go to RUNNING again @@ -246,8 +239,6 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { assertEquals( JobStatus.CANCELED, jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - - assertThat("We saw illegal restores.", illegalRestores.get(), is(0)); } /** @@ -320,14 +311,11 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @Override public void initializeState(FunctionInitializationContext context) { if (!context.isRestored()) { - int updatedValue = allowedInitializeCallsWithoutRestore.decrementAndGet(); + int updatedValue = allowedRestores.decrementAndGet(); if (updatedValue < 0) { throw new RuntimeException("We are not allowed any more restores."); } } else { - if (!restoreAllowed.get()) { - illegalRestores.getAndIncrement(); - } successfulRestoreLatch.trigger(); } } From 2b8fd0a5d14effb0804e5a9065a813a9a6c106f9 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 5 Mar 2018 20:05:46 +0100 Subject: [PATCH 300/367] Revert "[FLINK-8487] Verify ZooKeeper checkpoint store behaviour with ITCase" This reverts commit 901cd47395c1e9a440b03d40c6d28b8b1788f983, which was added to the release-1.4 branch by mistake. --- .../ZooKeeperCompletedCheckpointStore.java | 2 +- .../flink/runtime/concurrent/FutureUtils.java | 77 ----- .../ZooKeeperHighAvailabilityITCase.java | 323 ------------------ 3 files changed, 1 insertion(+), 401 deletions(-) delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index f22127041d31b..0cbd4fb6c9e9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -209,7 +209,7 @@ public void recover() throws Exception { if (completedCheckpoints.isEmpty() && numberOfInitialCheckpoints > 0) { throw new FlinkException( - "Could not read any of the " + numberOfInitialCheckpoints + " checkpoints from storage."); + "Could not read any of the " + numberOfInitialCheckpoints + " from storage."); } else if (completedCheckpoints.size() != numberOfInitialCheckpoints) { LOG.warn( "Could only fetch {} of {} checkpoints from storage.", diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index 5403490c17905..c18068bdf6eb4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -37,11 +37,9 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; -import java.util.function.Predicate; import java.util.function.Supplier; import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -191,81 +189,6 @@ private static void retryOperationWithDelay( } } - /** - * Retry the given operation with the given delay in between successful completions where the - * result does not match a given predicate. - * - * @param operation to retry - * @param retryDelay delay between retries - * @param deadline A deadline that specifies at what point we should stop retrying - * @param acceptancePredicate Predicate to test whether the result is acceptable - * @param scheduledExecutor executor to be used for the retry operation - * @param type of the result - * @return Future which retries the given operation a given amount of times and delays the retry - * in case the predicate isn't matched - */ - public static CompletableFuture retrySuccesfulWithDelay( - final Supplier> operation, - final Time retryDelay, - final Deadline deadline, - final Predicate acceptancePredicate, - final ScheduledExecutor scheduledExecutor) { - - final CompletableFuture resultFuture = new CompletableFuture<>(); - - retrySuccessfulOperationWithDelay( - resultFuture, - operation, - retryDelay, - deadline, - acceptancePredicate, - scheduledExecutor); - - return resultFuture; - } - - private static void retrySuccessfulOperationWithDelay( - final CompletableFuture resultFuture, - final Supplier> operation, - final Time retryDelay, - final Deadline deadline, - final Predicate acceptancePredicate, - final ScheduledExecutor scheduledExecutor) { - - if (!resultFuture.isDone()) { - final CompletableFuture operationResultFuture = operation.get(); - - operationResultFuture.whenComplete( - (t, throwable) -> { - if (throwable != null) { - if (throwable instanceof CancellationException) { - resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable)); - } else { - resultFuture.completeExceptionally(throwable); - } - } else { - if (acceptancePredicate.test(t)) { - resultFuture.complete(t); - } if (deadline.hasTimeLeft()) { - final ScheduledFuture scheduledFuture = scheduledExecutor.schedule( - () -> retrySuccessfulOperationWithDelay(resultFuture, operation, retryDelay, deadline, acceptancePredicate, scheduledExecutor), - retryDelay.toMilliseconds(), - TimeUnit.MILLISECONDS); - - resultFuture.whenComplete( - (innerT, innerThrowable) -> scheduledFuture.cancel(false)); - } else { - resultFuture.completeExceptionally( - new RetryException("Could not satisfy the predicate within the allowed time.")); - } - } - }); - - resultFuture.whenComplete( - (t, throwable) -> operationResultFuture.cancel(false)); - } - } - /** * Exception with which the returned future is completed if the {@link #retry(Supplier, int, Executor)} * operation fails. diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java deleted file mode 100644 index 4fd8ea9536c00..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ /dev/null @@ -1,323 +0,0 @@ -/* - * 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.flink.test.checkpointing; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.common.typeutils.base.StringSerializer; -import org.apache.flink.client.program.StandaloneClusterClient; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.runtime.state.filesystem.FsStateBackend; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.test.checkpointing.utils.SavepointMigrationTestBase; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.util.Preconditions; - -import org.apache.curator.test.TestingServer; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.nio.file.Files; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * Integration tests for {@link org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore}. - */ -public class ZooKeeperHighAvailabilityITCase extends TestBaseUtils { - - private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES); - - @ClassRule - public static TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private static File haStorageDir; - - private static TestingServer zkServer; - - private static OneShotLatch waitForCheckpointLatch = new OneShotLatch(); - private static OneShotLatch failInCheckpointLatch = new OneShotLatch(); - private static OneShotLatch successfulRestoreLatch = new OneShotLatch(); - - // in CheckpointBlockingFunction we verify that we only call initializeState() - // once with isRestored() == false. All other invocations must have isRestored() == true. This - // verifies that we don't restart a job from scratch in case the CompletedCheckpoints can't - // be read. - private static AtomicInteger allowedRestores = new AtomicInteger(1); - - private static final Logger LOG = LoggerFactory.getLogger(SavepointMigrationTestBase.class); - protected LocalFlinkMiniCluster cluster = null; - - @Before - public void setup() throws Exception { - - zkServer = new TestingServer(); - - // Flink configuration - final Configuration config = new Configuration(); - - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - - haStorageDir = temporaryFolder.newFolder(); - config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haStorageDir.toString()); - config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); - config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); - - cluster = TestBaseUtils.startCluster(config, false); - } - - @After - public void teardown() throws Exception { - stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); - - zkServer.stop(); - zkServer.close(); - } - - /** - * Verify that we don't start a job from scratch if we cannot restore any of the - * CompletedCheckpoints. - * - *

Synchronization for the different steps and things we want to observe happens via - * latches in the test method and the methods of {@link CheckpointBlockingFunction}. - * - *

The test follows these steps: - *

    - *
  1. Start job and block on a latch until we have done some checkpoints - *
  2. Block in the special function - *
  3. Move away the ZooKeeper HA directory to make restoring checkpoints impossible - *
  4. Unblock the special function, which now induces a failure - *
  5. Make sure that the job does not recover successfully - *
  6. Move back the HA directory - *
  7. Make sure that the job recovers, we use a latch to ensure that the operator - * restored successfully - *
- */ - @Test(timeout = 60_000L) - public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { - final Deadline deadline = TEST_TIMEOUT.fromNow(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 100L)); - env.enableCheckpointing(10); // Flink doesn't allow lower than 10 ms - - File checkpointLocation = temporaryFolder.newFolder(); - env.setStateBackend(new FsStateBackend(checkpointLocation.toURI())); - - DataStreamSource source = env.addSource(new UnboundedSource()); - - source - .keyBy((str) -> str) - .map(new CheckpointBlockingFunction()); - - JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - JobID jobID = Preconditions.checkNotNull(jobGraph.getJobID()); - - // Retrieve the job manager - ActorGateway jobManager = Await.result(cluster.leaderGateway().future(), deadline.timeLeft()); - - JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph); - - LOG.info("Submitted job {} and waiting...", jobSubmissionResult.getJobID()); - - StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration()); - - // wait until we did some checkpoints - waitForCheckpointLatch.await(); - - // mess with the HA directory so that the job cannot restore - File movedCheckpointLocation = temporaryFolder.newFolder(); - // clean it out, we only want the name so that we can move the checkpoints directory - assertTrue(movedCheckpointLocation.delete()); - Files.move(haStorageDir.toPath(), movedCheckpointLocation.toPath()); - - failInCheckpointLatch.trigger(); - - Thread.sleep(2000); - - // Ensure that we see at least one cycle where the job tries to restart and fails. - CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( - () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), - Time.milliseconds(1), - deadline, - (jobStatus) -> jobStatus.equals(JobStatus.RESTARTING), - TestingUtils.defaultScheduledExecutor()); - assertEquals( - JobStatus.RESTARTING, - jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - - jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( - () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), - Time.milliseconds(1), - deadline, - (jobStatus) -> jobStatus.equals(JobStatus.FAILING), - TestingUtils.defaultScheduledExecutor()); - assertEquals( - JobStatus.FAILING, - jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - - // move back the HA directory so that the job can restore - Files.move(movedCheckpointLocation.toPath(), haStorageDir.toPath()); - - // now the job should be able to go to RUNNING again - jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( - () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), - Time.milliseconds(50), - deadline, - (jobStatus) -> jobStatus.equals(JobStatus.RUNNING), - TestingUtils.defaultScheduledExecutor()); - - assertEquals( - JobStatus.RUNNING, - jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - - // make sure we saw a successful restore - successfulRestoreLatch.await(); - - clusterClient.cancel(jobID); - - jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( - () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), - Time.milliseconds(50), - deadline, - (jobStatus) -> jobStatus.equals(JobStatus.CANCELED), - TestingUtils.defaultScheduledExecutor()); - - assertEquals( - JobStatus.CANCELED, - jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - } - - /** - * Requests the {@link JobStatus} of the job with the given {@link JobID}. - */ - public CompletableFuture getJobStatus( - ActorGateway jobManager, - JobID jobId, - FiniteDuration timeout) { - - Future response = jobManager.ask(JobManagerMessages.getRequestJobStatus(jobId), timeout); - - CompletableFuture javaFuture = FutureUtils.toJava(response); - - return javaFuture.thenApply((responseMessage) -> { - if (responseMessage instanceof JobManagerMessages.CurrentJobStatus) { - return ((JobManagerMessages.CurrentJobStatus) responseMessage).status(); - } else if (responseMessage instanceof JobManagerMessages.JobNotFound) { - throw new CompletionException( - new IllegalStateException("Could not find job with JobId " + jobId)); - } else { - throw new CompletionException( - new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); - } - }); - } - - - private static class UnboundedSource implements SourceFunction { - private boolean running = true; - - @Override - public void run(SourceContext ctx) throws Exception { - while (running) { - ctx.collect("hello"); - // don't overdo it ... ;-) - Thread.sleep(50); - } - } - - @Override - public void cancel() { - running = false; - } - } - - private static class CheckpointBlockingFunction - extends RichMapFunction - implements CheckpointedFunction { - - // also have some state to write to the checkpoint - private final ValueStateDescriptor stateDescriptor = - new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); - - @Override - public String map(String value) throws Exception { - getRuntimeContext().getState(stateDescriptor).update("42"); - return value; - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - if (context.getCheckpointId() > 5) { - waitForCheckpointLatch.trigger(); - failInCheckpointLatch.await(); - throw new RuntimeException("Failing on purpose."); - } - } - - @Override - public void initializeState(FunctionInitializationContext context) { - if (!context.isRestored()) { - int updatedValue = allowedRestores.decrementAndGet(); - if (updatedValue < 0) { - throw new RuntimeException("We are not allowed any more restores."); - } - } else { - successfulRestoreLatch.trigger(); - } - } - } -} From 3d4ff17a6cad02475406943af87794d21c42128b Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 7 Mar 2018 11:58:07 +0100 Subject: [PATCH 301/367] [FLINK-8890] Compare checkpoints with order in CompletedCheckpoint.checkpointsMatch() This method is used, among other things, to check if a list of restored checkpoints is stable after several restore attempts in the ZooKeeper checkpoint store. The order of checkpoints is somewhat important because we want the latest checkpoint to stay the latest checkpoint. --- .../checkpoint/CompletedCheckpoint.java | 12 +- .../checkpoint/CompletedCheckpointTest.java | 136 +++++++++++++++++- 2 files changed, 141 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index e4f7da24dd0b5..bd6fdd1cad8ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -35,9 +35,8 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; +import java.util.List; import java.util.Map; -import java.util.Set; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -283,17 +282,18 @@ public String getExternalPointer() { public static boolean checkpointsMatch( Collection first, Collection second) { + if (first.size() != second.size()) { + return false; + } - Set> firstInterestingFields = - new HashSet<>(); + List> firstInterestingFields = new ArrayList<>(first.size()); for (CompletedCheckpoint checkpoint : first) { firstInterestingFields.add( new Tuple2<>(checkpoint.getCheckpointID(), checkpoint.getJobId())); } - Set> secondInterestingFields = - new HashSet<>(); + List> secondInterestingFields = new ArrayList<>(second.size()); for (CompletedCheckpoint checkpoint : second) { secondInterestingFields.add( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java index 293675c14e932..ea21e51400f53 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java @@ -26,17 +26,22 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.filesystem.FileStateHandle; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; import java.io.File; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -46,6 +51,135 @@ public class CompletedCheckpointTest { @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); + @Test + public void testCompareCheckpointsWithDifferentOrder() { + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + new JobID(), 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + new JobID(), 1, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + checkpoints1.add(checkpoint2); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint2); + checkpoints2.add(checkpoint1); + checkpoints2.add(checkpoint2); + + assertFalse(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + + @Test + public void testCompareCheckpointsWithSameOrder() { + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + new JobID(), 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + new JobID(), 1, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + checkpoints1.add(checkpoint2); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint1); + checkpoints2.add(checkpoint2); + checkpoints2.add(checkpoint1); + + assertTrue(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + + /** + * Verify that both JobID and checkpoint id are taken into account when comparing. + */ + @Test + public void testCompareCheckpointsWithSameJobID() { + JobID jobID = new JobID(); + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + jobID, 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + jobID, 1, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint2); + + assertFalse(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + + /** + * Verify that both JobID and checkpoint id are taken into account when comparing. + */ + @Test + public void testCompareCheckpointsWithSameCheckpointId() { + JobID jobID1 = new JobID(); + JobID jobID2 = new JobID(); + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + jobID1, 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + jobID2, 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forStandardCheckpoint(), + null, + null); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint2); + + assertFalse(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + /** * Tests that persistent checkpoints discard their header file. */ @@ -83,7 +217,7 @@ public void testCleanUpOnSubsume() throws Exception { boolean discardSubsumed = true; CheckpointProperties props = new CheckpointProperties(false, false, false, discardSubsumed, true, true, true, true); - + CompletedCheckpoint checkpoint = new CompletedCheckpoint( new JobID(), 0, 0, 1, operatorStates, From 72b3ddad1df86f383ea4f75e3dd597a07374df65 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Sat, 3 Mar 2018 09:34:56 +0100 Subject: [PATCH 302/367] [FLINK-8487] Verify ZooKeeper checkpoint store behaviour with ITCase --- .../ZooKeeperCompletedCheckpointStore.java | 2 +- .../flink/runtime/concurrent/FutureUtils.java | 77 ++++ .../ZooKeeperHighAvailabilityITCase.java | 340 ++++++++++++++++++ 3 files changed, 418 insertions(+), 1 deletion(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index 0cbd4fb6c9e9f..f22127041d31b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -209,7 +209,7 @@ public void recover() throws Exception { if (completedCheckpoints.isEmpty() && numberOfInitialCheckpoints > 0) { throw new FlinkException( - "Could not read any of the " + numberOfInitialCheckpoints + " from storage."); + "Could not read any of the " + numberOfInitialCheckpoints + " checkpoints from storage."); } else if (completedCheckpoints.size() != numberOfInitialCheckpoints) { LOG.warn( "Could only fetch {} of {} checkpoints from storage.", diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index c18068bdf6eb4..5403490c17905 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -37,9 +37,11 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Predicate; import java.util.function.Supplier; import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -189,6 +191,81 @@ private static void retryOperationWithDelay( } } + /** + * Retry the given operation with the given delay in between successful completions where the + * result does not match a given predicate. + * + * @param operation to retry + * @param retryDelay delay between retries + * @param deadline A deadline that specifies at what point we should stop retrying + * @param acceptancePredicate Predicate to test whether the result is acceptable + * @param scheduledExecutor executor to be used for the retry operation + * @param type of the result + * @return Future which retries the given operation a given amount of times and delays the retry + * in case the predicate isn't matched + */ + public static CompletableFuture retrySuccesfulWithDelay( + final Supplier> operation, + final Time retryDelay, + final Deadline deadline, + final Predicate acceptancePredicate, + final ScheduledExecutor scheduledExecutor) { + + final CompletableFuture resultFuture = new CompletableFuture<>(); + + retrySuccessfulOperationWithDelay( + resultFuture, + operation, + retryDelay, + deadline, + acceptancePredicate, + scheduledExecutor); + + return resultFuture; + } + + private static void retrySuccessfulOperationWithDelay( + final CompletableFuture resultFuture, + final Supplier> operation, + final Time retryDelay, + final Deadline deadline, + final Predicate acceptancePredicate, + final ScheduledExecutor scheduledExecutor) { + + if (!resultFuture.isDone()) { + final CompletableFuture operationResultFuture = operation.get(); + + operationResultFuture.whenComplete( + (t, throwable) -> { + if (throwable != null) { + if (throwable instanceof CancellationException) { + resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable)); + } else { + resultFuture.completeExceptionally(throwable); + } + } else { + if (acceptancePredicate.test(t)) { + resultFuture.complete(t); + } if (deadline.hasTimeLeft()) { + final ScheduledFuture scheduledFuture = scheduledExecutor.schedule( + () -> retrySuccessfulOperationWithDelay(resultFuture, operation, retryDelay, deadline, acceptancePredicate, scheduledExecutor), + retryDelay.toMilliseconds(), + TimeUnit.MILLISECONDS); + + resultFuture.whenComplete( + (innerT, innerThrowable) -> scheduledFuture.cancel(false)); + } else { + resultFuture.completeExceptionally( + new RetryException("Could not satisfy the predicate within the allowed time.")); + } + } + }); + + resultFuture.whenComplete( + (t, throwable) -> operationResultFuture.cancel(false)); + } + } + /** * Exception with which the returned future is completed if the {@link #retry(Supplier, int, Executor)} * operation fails. diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java new file mode 100644 index 0000000000000..5dcb912f8a37e --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -0,0 +1,340 @@ +/* + * 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.flink.test.checkpointing; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.curator.test.TestingServer; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Integration tests for {@link org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore}. + */ +public class ZooKeeperHighAvailabilityITCase extends TestBaseUtils { + + private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES); + + private static final int NUM_JMS = 1; + private static final int NUM_TMS = 1; + private static final int NUM_SLOTS_PER_TM = 1; + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static File haStorageDir; + + private static TestingServer zkServer; + + private static LocalFlinkMiniCluster cluster = null; + + private static OneShotLatch waitForCheckpointLatch = new OneShotLatch(); + private static OneShotLatch failInCheckpointLatch = new OneShotLatch(); + + @BeforeClass + public static void setup() throws Exception { + zkServer = new TestingServer(); + + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + + haStorageDir = TEMPORARY_FOLDER.newFolder(); + + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haStorageDir.toString()); + config.setString(HighAvailabilityOptions.HA_CLUSTER_ID, UUID.randomUUID().toString()); + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); + + cluster = TestBaseUtils.startCluster(config, false); + } + + @AfterClass + public static void tearDown() throws Exception { + stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); + + zkServer.stop(); + zkServer.close(); + } + + /** + * Verify that we don't start a job from scratch if we cannot restore any of the + * CompletedCheckpoints. + * + *

Synchronization for the different steps and things we want to observe happens via + * latches in the test method and the methods of {@link CheckpointBlockingFunction}. + * + *

The test follows these steps: + *

    + *
  1. Start job and block on a latch until we have done some checkpoints + *
  2. Block in the special function + *
  3. Move away the contents of the ZooKeeper HA directory to make restoring from + * checkpoints impossible + *
  4. Unblock the special function, which now induces a failure + *
  5. Make sure that the job does not recover successfully + *
  6. Move back the HA directory + *
  7. Make sure that the job recovers, we use a latch to ensure that the operator + * restored successfully + *
+ */ + @Test(timeout = 120_000L) + public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { + CheckpointBlockingFunction.allowedInitializeCallsWithoutRestore.set(1); + CheckpointBlockingFunction.successfulRestores.set(0); + CheckpointBlockingFunction.illegalRestores.set(0); + CheckpointBlockingFunction.afterMessWithZooKeeper.set(false); + CheckpointBlockingFunction.failedAlready.set(false); + + waitForCheckpointLatch = new OneShotLatch(); + failInCheckpointLatch = new OneShotLatch(); + + final Deadline deadline = TEST_TIMEOUT.fromNow(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0)); + env.enableCheckpointing(10); // Flink doesn't allow lower than 10 ms + + File checkpointLocation = TEMPORARY_FOLDER.newFolder(); + env.setStateBackend(new FsStateBackend(checkpointLocation.toURI())); + + DataStreamSource source = env.addSource(new UnboundedSource()); + + source + .keyBy((str) -> str) + .map(new CheckpointBlockingFunction()); + + JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + JobID jobID = Preconditions.checkNotNull(jobGraph.getJobID()); + + // Retrieve the job manager + ActorGateway jobManager = Await.result(cluster.leaderGateway().future(), deadline.timeLeft()); + cluster.submitJobDetached(jobGraph); + + // wait until we did some checkpoints + waitForCheckpointLatch.await(); + + // mess with the HA directory so that the job cannot restore + File movedCheckpointLocation = TEMPORARY_FOLDER.newFolder(); + int numCheckpoints = 0; + File[] files = haStorageDir.listFiles(); + assertNotNull(files); + for (File file : files) { + if (file.getName().startsWith("completedCheckpoint")) { + assertTrue(file.renameTo(new File(movedCheckpointLocation, file.getName()))); + numCheckpoints++; + } + } + assertTrue(numCheckpoints > 0); + + failInCheckpointLatch.trigger(); + + // Ensure that we see at least one cycle where the job tries to restart and fails. + CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(1), + deadline, + (jobStatus) -> jobStatus == JobStatus.RESTARTING, + TestingUtils.defaultScheduledExecutor()); + assertEquals(JobStatus.RESTARTING, jobStatusFuture.get()); + + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(1), + deadline, + (jobStatus) -> jobStatus == JobStatus.FAILING, + TestingUtils.defaultScheduledExecutor()); + assertEquals(JobStatus.FAILING, jobStatusFuture.get()); + + // move back the HA directory so that the job can restore + CheckpointBlockingFunction.afterMessWithZooKeeper.set(true); + + files = movedCheckpointLocation.listFiles(); + assertNotNull(files); + for (File file : files) { + if (file.getName().startsWith("completedCheckpoint")) { + assertTrue(file.renameTo(new File(haStorageDir, file.getName()))); + } + } + + // now the job should be able to go to RUNNING again and then eventually to FINISHED, + // which it only does if it could successfully restore + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> getJobStatus(jobManager, jobID, TEST_TIMEOUT), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus == JobStatus.FINISHED, + TestingUtils.defaultScheduledExecutor()); + assertEquals(JobStatus.FINISHED, jobStatusFuture.get()); + + assertThat("We saw illegal restores.", CheckpointBlockingFunction.illegalRestores.get(), is(0)); + } + + /** + * Requests the {@link JobStatus} of the job with the given {@link JobID}. + */ + private CompletableFuture getJobStatus( + ActorGateway jobManager, + JobID jobId, + FiniteDuration timeout) { + + Future response = jobManager.ask(JobManagerMessages.getRequestJobStatus(jobId), timeout); + + CompletableFuture javaFuture = FutureUtils.toJava(response); + + return javaFuture.thenApply((responseMessage) -> { + if (responseMessage instanceof JobManagerMessages.CurrentJobStatus) { + return ((JobManagerMessages.CurrentJobStatus) responseMessage).status(); + } else if (responseMessage instanceof JobManagerMessages.JobNotFound) { + throw new CompletionException( + new IllegalStateException("Could not find job with JobId " + jobId)); + } else { + throw new CompletionException( + new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); + } + }); + } + + private static class UnboundedSource implements SourceFunction { + private volatile boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + while (running && !CheckpointBlockingFunction.afterMessWithZooKeeper.get()) { + ctx.collect("hello"); + // don't overdo it ... ;-) + Thread.sleep(50); + } + } + + @Override + public void cancel() { + running = false; + } + } + + private static class CheckpointBlockingFunction + extends RichMapFunction + implements CheckpointedFunction { + + // verify that we only call initializeState() + // once with isRestored() == false. All other invocations must have isRestored() == true. This + // verifies that we don't restart a job from scratch in case the CompletedCheckpoints can't + // be read. + static AtomicInteger allowedInitializeCallsWithoutRestore = new AtomicInteger(1); + + // we count when we see restores that are not allowed. We only + // allow restores once we messed with the HA directory and moved it back again + static AtomicInteger illegalRestores = new AtomicInteger(0); + static AtomicInteger successfulRestores = new AtomicInteger(0); + + // whether we are after the phase where we messed with the ZooKeeper HA directory, i.e. + // whether it's now ok for a restore to happen + static AtomicBoolean afterMessWithZooKeeper = new AtomicBoolean(false); + + static AtomicBoolean failedAlready = new AtomicBoolean(false); + + // also have some state to write to the checkpoint + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); + + @Override + public String map(String value) throws Exception { + getRuntimeContext().getState(stateDescriptor).update("42"); + return value; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (context.getCheckpointId() > 5) { + waitForCheckpointLatch.trigger(); + failInCheckpointLatch.await(); + if (!failedAlready.getAndSet(true)) { + throw new RuntimeException("Failing on purpose."); + } + } + } + + @Override + public void initializeState(FunctionInitializationContext context) { + if (!context.isRestored()) { + int updatedValue = allowedInitializeCallsWithoutRestore.decrementAndGet(); + if (updatedValue < 0) { + illegalRestores.getAndIncrement(); + throw new RuntimeException("We are not allowed any more restores."); + } + } else { + if (!afterMessWithZooKeeper.get()) { + illegalRestores.getAndIncrement(); + } else if (successfulRestores.getAndIncrement() > 0) { + // already saw the one allowed successful restore + illegalRestores.getAndIncrement(); + } + } + } + } +} From d57d4e0af0c28c85d880a8261aac90e4a07eda32 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 15:18:33 +0100 Subject: [PATCH 303/367] [FLINK-8856] [TaskManager] Move all cancellation interrupt calls to TaskCanceller thread This cleans up the code and guards against a JVM bug where 'interrupt()' calls block/deadlock if the thread is engaged in certain I/O operations. In addition, this makes sure that the process really goes away when the cancellation timeout expires, rather than relying on the TaskManager to be able to properly handle the fatal error notification. --- .../flink/runtime/taskmanager/Task.java | 282 +++++++++++------- 1 file changed, 180 insertions(+), 102 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 2cb356c81f116..ae8cb47b95c18 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -69,6 +69,7 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.util.FatalExitExceptionHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -78,7 +79,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.IOException; import java.net.URL; @@ -93,11 +93,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -1080,15 +1080,51 @@ else if (current == ExecutionState.RUNNING) { invokable, executingThread, taskNameWithSubtask, - taskCancellationInterval, - taskCancellationTimeout, - taskManagerActions, producedPartitions, inputGates); - Thread cancelThread = new Thread(executingThread.getThreadGroup(), canceler, + + Thread cancelThread = new Thread( + executingThread.getThreadGroup(), + canceler, String.format("Canceler for %s (%s).", taskNameWithSubtask, executionId)); cancelThread.setDaemon(true); + cancelThread.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); cancelThread.start(); + + // the periodic interrupting thread - a different thread than the canceller, in case + // the application code does blocking stuff in its cancellation paths. + Runnable interrupter = new TaskInterrupter( + LOG, + executingThread, + taskNameWithSubtask, + taskCancellationInterval); + + Thread interruptingThread = new Thread( + executingThread.getThreadGroup(), + interrupter, + String.format("Canceler/Interrupts for %s (%s).", taskNameWithSubtask, executionId)); + interruptingThread.setDaemon(true); + interruptingThread.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); + interruptingThread.start(); + + // if a cancellation timeout is set, the watchdog thread kills the process + // if graceful cancellation does not succeed + if (taskCancellationTimeout > 0) { + Runnable cancelWatchdog = new TaskCancelerWatchDog( + executingThread, + taskManagerActions, + taskCancellationTimeout, + LOG); + + Thread watchDogThread = new Thread( + executingThread.getThreadGroup(), + cancelWatchdog, + String.format("Cancellation Watchdog for %s (%s).", + taskNameWithSubtask, executionId)); + watchDogThread.setDaemon(true); + watchDogThread.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); + watchDogThread.start(); + } } return; } @@ -1406,9 +1442,29 @@ public String toString() { return String.format("%s (%s) [%s]", taskNameWithSubtask, executionId, executionState); } + // ------------------------------------------------------------------------ + // Task cancellation + // + // The task cancellation uses in total three threads, as a safety net + // against various forms of user- and JVM bugs. + // + // - The first thread calls 'cancel()' on the invokable and closes + // the input and output connections, for fast thread termination + // - The second thread periodically interrupts the invokable in order + // to pull the thread out of blocking wait and I/O operations + // - The third thread (watchdog thread) waits until the cancellation + // timeout and then performs a hard cancel (kill process, or let + // the TaskManager know) + // + // Previously, thread two and three were in one thread, but we needed + // to separate this to make sure the watchdog thread does not call + // 'interrupt()'. This is a workaround for the following JVM bug + // https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8138622 + // ------------------------------------------------------------------------ + /** - * This runner calls cancel() on the invokable and periodically interrupts the - * thread until it has terminated. + * This runner calls cancel() on the invokable, closes input-/output resources, + * and initially interrupts the task thread. */ private static class TaskCanceler implements Runnable { @@ -1419,27 +1475,11 @@ private static class TaskCanceler implements Runnable { private final ResultPartition[] producedPartitions; private final SingleInputGate[] inputGates; - /** Interrupt interval. */ - private final long interruptInterval; - - /** Timeout after which a fatal error notification happens. */ - private final long interruptTimeout; - - /** TaskManager to notify about a timeout */ - private final TaskManagerActions taskManager; - - /** Watch Dog thread */ - @Nullable - private final Thread watchDogThread; - public TaskCanceler( Logger logger, AbstractInvokable invokable, Thread executer, String taskName, - long cancellationInterval, - long cancellationTimeout, - TaskManagerActions taskManager, ResultPartition[] producedPartitions, SingleInputGate[] inputGates) { @@ -1447,39 +1487,19 @@ public TaskCanceler( this.invokable = invokable; this.executer = executer; this.taskName = taskName; - this.interruptInterval = cancellationInterval; - this.interruptTimeout = cancellationTimeout; - this.taskManager = taskManager; this.producedPartitions = producedPartitions; this.inputGates = inputGates; - - if (cancellationTimeout > 0) { - // The watch dog repeatedly interrupts the executor until - // the cancellation timeout kicks in (at which point the - // task manager is notified about a fatal error) or the - // executor has terminated. - this.watchDogThread = new Thread( - executer.getThreadGroup(), - new TaskCancelerWatchDog(), - "WatchDog for " + taskName + " cancellation"); - this.watchDogThread.setDaemon(true); - } else { - this.watchDogThread = null; - } } @Override public void run() { try { - if (watchDogThread != null) { - watchDogThread.start(); - } - // the user-defined cancel method may throw errors. // we need do continue despite that try { invokable.cancel(); } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); logger.error("Error while canceling the task {}.", taskName, t); } @@ -1494,6 +1514,7 @@ public void run() { try { partition.destroyBufferPool(); } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); LOG.error("Failed to release result partition buffer pool for task {}.", taskName, t); } } @@ -1502,89 +1523,146 @@ public void run() { try { inputGate.releaseAllResources(); } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); LOG.error("Failed to release input gate for task {}.", taskName, t); } } - // interrupt the running thread initially + // send the initial interruption signal executer.interrupt(); - try { - executer.join(interruptInterval); - } - catch (InterruptedException e) { - // we can ignore this - } - - if (watchDogThread != null) { - watchDogThread.interrupt(); - watchDogThread.join(); - } - } catch (Throwable t) { + } + catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); logger.error("Error in the task canceler for task {}.", taskName, t); } } + } - /** - * Watchdog for the cancellation. If the task is stuck in cancellation, - * we notify the task manager about a fatal error. - */ - private class TaskCancelerWatchDog implements Runnable { + /** + * This thread sends the delayed, periodic interrupt calls to the executing thread. + */ + private static final class TaskInterrupter implements Runnable { - @Override - public void run() { - long intervalNanos = TimeUnit.NANOSECONDS.convert(interruptInterval, TimeUnit.MILLISECONDS); - long timeoutNanos = TimeUnit.NANOSECONDS.convert(interruptTimeout, TimeUnit.MILLISECONDS); - long deadline = System.nanoTime() + timeoutNanos; + /** The logger to report on the fatal condition. */ + private final Logger log; - try { - // Initial wait before interrupting periodically - Thread.sleep(interruptInterval); - } catch (InterruptedException ignored) { - } + /** The executing task thread that we wait for to terminate. */ + private final Thread executerThread; + + /** The name of the task, for logging purposes. */ + private final String taskName; + + /** The interval in which we interrupt. */ + private final long interruptIntervalMillis; + + TaskInterrupter( + Logger log, + Thread executerThread, + String taskName, + long interruptIntervalMillis) { - // It is possible that the user code does not react to the task canceller. - // for that reason, we spawn this separate thread that repeatedly interrupts - // the user code until it exits. If the suer user code does not exit within - // the timeout, we notify the job manager about a fatal error. - while (executer.isAlive()) { - long now = System.nanoTime(); + this.log = log; + this.executerThread = executerThread; + this.taskName = taskName; + this.interruptIntervalMillis = interruptIntervalMillis; + } + @Override + public void run() { + try { + // we initially wait for one interval + // in most cases, the threads go away immediately (by the cancellation thread) + // and we need not actually do anything + executerThread.join(interruptIntervalMillis); + + // log stack trace where the executing thread is stuck and + // interrupt the running thread periodically while it is still alive + while (executerThread.isAlive()) { // build the stack trace of where the thread is stuck, for the log + StackTraceElement[] stack = executerThread.getStackTrace(); StringBuilder bld = new StringBuilder(); - StackTraceElement[] stack = executer.getStackTrace(); for (StackTraceElement e : stack) { bld.append(e).append('\n'); } - if (now >= deadline) { - long duration = TimeUnit.SECONDS.convert(interruptInterval, TimeUnit.MILLISECONDS); - String msg = String.format("Task '%s' did not react to cancelling signal in " + - "the last %d seconds, but is stuck in method:\n %s", - taskName, - duration, - bld.toString()); + log.warn("Task '{}' did not react to cancelling signal for {} seconds, but is stuck in method:\n {}", + taskName, (interruptIntervalMillis / 1000), bld); - logger.info("Notifying TaskManager about fatal error. {}.", msg); + executerThread.interrupt(); + try { + executerThread.join(interruptIntervalMillis); + } + catch (InterruptedException e) { + // we ignore this and fall through the loop + } + } + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); + log.error("Error in the task canceler for task {}.", taskName, t); + } + } + } - taskManager.notifyFatalError(msg, null); + /** + * Watchdog for the cancellation. + * If the task thread does not go away gracefully within a certain time, we + * trigger a hard cancel action (notify TaskManager of fatal error, which in + * turn kills the process). + */ + private static class TaskCancelerWatchDog implements Runnable { - return; // done, don't forget to leave the loop - } else { - logger.warn("Task '{}' did not react to cancelling signal, but is stuck in method:\n {}", - taskName, bld.toString()); + /** The logger to report on the fatal condition. */ + private final Logger log; - executer.interrupt(); - try { - long timeLeftNanos = Math.min(intervalNanos, deadline - now); - long timeLeftMillis = TimeUnit.MILLISECONDS.convert(timeLeftNanos, TimeUnit.NANOSECONDS); + /** The executing task thread that we wait for to terminate. */ + private final Thread executerThread; - if (timeLeftMillis > 0) { - executer.join(timeLeftMillis); - } - } catch (InterruptedException ignored) { - } + /** The TaskManager to notify if cancellation does not happen in time. */ + private final TaskManagerActions taskManager; + + /** The timeout for cancellation. */ + private final long timeoutMillis; + + TaskCancelerWatchDog( + Thread executerThread, + TaskManagerActions taskManager, + long timeoutMillis, + Logger log) { + + checkArgument(timeoutMillis > 0); + + this.log = log; + this.executerThread = executerThread; + this.taskManager = taskManager; + this.timeoutMillis = timeoutMillis; + } + + @Override + public void run() { + try { + final long hardKillDeadline = System.nanoTime() + timeoutMillis * 1_000_000; + + long millisLeft; + while (executerThread.isAlive() + && (millisLeft = (hardKillDeadline - System.nanoTime()) / 1_000_000) > 0) { + + try { + executerThread.join(millisLeft); + } + catch (InterruptedException ignored) { + // we don't react to interrupted exceptions, simply fall through the loop } } + + if (executerThread.isAlive()) { + String msg = "Task did not exit gracefully within " + (timeoutMillis / 1000) + " + seconds."; + log.error(msg); + taskManager.notifyFatalError(msg, null); + } + } + catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); + log.error("Error in Task Cancellation Watch Dog", t); } } } From 64db78c68368b22eff087eb4e74b62063420bc63 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 8 Mar 2018 12:22:32 +0100 Subject: [PATCH 304/367] [FLINK-8896] [kafka08] remove all cancel MARKERs before trying to find partition leaders This guards us against #cancel() being called multiple times and then trying to look up an invalid topic/partition pair. --- .../streaming/connectors/kafka/internals/Kafka08Fetcher.java | 3 ++- .../streaming/connectors/kafka/FlinkKafkaConsumerBase.java | 5 ++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java index d8bdd22f50f74..af95267deb57b 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java @@ -187,7 +187,8 @@ public void runFetchLoop() throws Exception { // special marker into the queue List> partitionsToAssign = unassignedPartitionsQueue.getBatchBlocking(5000); - partitionsToAssign.remove(MARKER); + // note: if there are more markers, remove them all + partitionsToAssign.removeIf(MARKER::equals); if (!partitionsToAssign.isEmpty()) { LOG.info("Assigning {} partitions to broker threads", partitionsToAssign.size()); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 537e73da9f8fc..084f23d30a707 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -626,7 +626,10 @@ public void run() { discoveryLoopErrorRef.set(e); } finally { // calling cancel will also let the fetcher loop escape - cancel(); + // (if not running, cancel() was already called) + if (running) { + cancel(); + } } } }); From 430336f5746d2a2fdf68d83522b03558f05bc780 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 6 Mar 2018 10:45:25 +0100 Subject: [PATCH 305/367] [FLINK-8847][build] Always generate .class files for package-info.java This closes #5644. --- pom.xml | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index dd9faec7a381e..53f52593ad6a8 100644 --- a/pom.xml +++ b/pom.xml @@ -1091,9 +1091,13 @@ under the License. ${java.version} ${java.version} - - -Xlint:all + + + -Xlint:all + + -Xpkginfo:always + From 8d9f339a8baf29669b25f8f006971f9f03131309 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 6 Mar 2018 11:04:42 +0100 Subject: [PATCH 306/367] [hotfix][build] Enable incremental compilation --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index 53f52593ad6a8..8e643cb316abb 100644 --- a/pom.xml +++ b/pom.xml @@ -1091,6 +1091,8 @@ under the License. ${java.version} ${java.version} + + false From cb4e088e6bb2022130d2c4f27a17f7f2907f80a2 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 12 Mar 2018 19:25:50 +0800 Subject: [PATCH 307/367] [hotfix] Update docs version to Flink 1.4.2 --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 7740cf6fb6cb0..aa39885b0a3c0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -27,7 +27,7 @@ # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) -version: "1.4.1" +version: "1.4.2" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version version_title: "1.4" From 8b1376b1808d302574073ce180dc561244adc6f5 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 27 Feb 2018 17:23:20 +0100 Subject: [PATCH 308/367] [FLINK-8801][yarn/s3] fix Utils#setupLocalResource() relying on consistent read-after-write "Amazon S3 provides read-after-write consistency for PUTS of new objects in your S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD or GET request to the key name (to find if the object exists) before creating the object, Amazon S3 provides eventual consistency for read-after-write." https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel Some S3 file system implementations may actually execute such a request for the about-to-write object and thus the read-after-write is only eventually consistent. org.apache.flink.yarn.Utils#setupLocalResource() currently relies on a consistent read-after-write since it accesses the remote resource to get file size and modification timestamp. Since there we have access to the local resource, we can use this metadata directly instead and circumvent the problem. This closes #5602. --- .../java/org/apache/flink/yarn/Utils.java | 34 +++++++++++++++++-- 1 file changed, 32 insertions(+), 2 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 652afec370e43..cbf7808377edf 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -140,7 +140,8 @@ static Tuple2 setupLocalResource( Path homedir, String relativeTargetPath) throws IOException { - if (new File(localSrcPath.toUri().getPath()).isDirectory()) { + File localFile = new File(localSrcPath.toUri().getPath()); + if (localFile.isDirectory()) { throw new IllegalArgumentException("File to copy must not be a directory: " + localSrcPath); } @@ -158,11 +159,40 @@ static Tuple2 setupLocalResource( fs.copyFromLocalFile(false, true, localSrcPath, dst); + // Note: If we used registerLocalResource(FileSystem, Path) here, we would access the remote + // file once again which has problems with eventually consistent read-after-write file + // systems. Instead, we decide to preserve the modification time at the remote + // location because this and the size of the resource will be checked by YARN based on + // the values we provide to #registerLocalResource() below. + fs.setTimes(dst, localFile.lastModified(), -1); // now create the resource instance - LocalResource resource = registerLocalResource(fs, dst); + LocalResource resource = registerLocalResource(dst, localFile.length(), localFile.lastModified()); + return Tuple2.of(dst, resource); } + /** + * Creates a YARN resource for the remote object at the given location. + * + * @param remoteRsrcPath remote location of the resource + * @param resourceSize size of the resource + * @param resourceModificationTime last modification time of the resource + * + * @return YARN resource + */ + private static LocalResource registerLocalResource( + Path remoteRsrcPath, + long resourceSize, + long resourceModificationTime) { + LocalResource localResource = Records.newRecord(LocalResource.class); + localResource.setResource(ConverterUtils.getYarnUrlFromURI(remoteRsrcPath.toUri())); + localResource.setSize(resourceSize); + localResource.setTimestamp(resourceModificationTime); + localResource.setType(LocalResourceType.FILE); + localResource.setVisibility(LocalResourceVisibility.APPLICATION); + return localResource; + } + private static LocalResource registerLocalResource(FileSystem fs, Path remoteRsrcPath) throws IOException { LocalResource localResource = Records.newRecord(LocalResource.class); FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); From e3b95a70ac8541b0717bce3cf32bed44e21d4747 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Tue, 27 Mar 2018 14:28:07 +0800 Subject: [PATCH 309/367] [FLINK-9093] [docs] Ship jQuery library without external provider This closes #5770. --- docs/_layouts/base.html | 2 +- docs/page/js/jquery.min.js | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 docs/page/js/jquery.min.js diff --git a/docs/_layouts/base.html b/docs/_layouts/base.html index 691670c47fdc0..2138bf5260a2a 100644 --- a/docs/_layouts/base.html +++ b/docs/_layouts/base.html @@ -81,7 +81,7 @@ - + diff --git a/docs/page/js/jquery.min.js b/docs/page/js/jquery.min.js new file mode 100644 index 0000000000000..e6a051d0d1d32 --- /dev/null +++ b/docs/page/js/jquery.min.js @@ -0,0 +1,4 @@ +/*! jQuery v1.11.2 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.2",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b){return b.toUpperCase()};m.fn=m.prototype={jquery:l,constructor:m,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},m.extend=m.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||m.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return!m.isArray(a)&&a-parseFloat(a)+1>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==m.type(a)||a.nodeType||m.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(k.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=r(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=r(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),m.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=hb(),z=hb(),A=hb(),B=function(a,b){return a===b&&(l=!0),0},C=1<<31,D={}.hasOwnProperty,E=[],F=E.pop,G=E.push,H=E.push,I=E.slice,J=function(a,b){for(var c=0,d=a.length;d>c;c++)if(a[c]===b)return c;return-1},K="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",L="[\\x20\\t\\r\\n\\f]",M="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",N=M.replace("w","w#"),O="\\["+L+"*("+M+")(?:"+L+"*([*^$|!~]?=)"+L+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+N+"))|)"+L+"*\\]",P=":("+M+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+O+")*)|.*)\\)|)",Q=new RegExp(L+"+","g"),R=new RegExp("^"+L+"+|((?:^|[^\\\\])(?:\\\\.)*)"+L+"+$","g"),S=new RegExp("^"+L+"*,"+L+"*"),T=new RegExp("^"+L+"*([>+~]|"+L+")"+L+"*"),U=new RegExp("="+L+"*([^\\]'\"]*?)"+L+"*\\]","g"),V=new RegExp(P),W=new RegExp("^"+N+"$"),X={ID:new RegExp("^#("+M+")"),CLASS:new RegExp("^\\.("+M+")"),TAG:new RegExp("^("+M.replace("w","w*")+")"),ATTR:new RegExp("^"+O),PSEUDO:new RegExp("^"+P),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+L+"*(even|odd|(([+-]|)(\\d*)n|)"+L+"*(?:([+-]|)"+L+"*(\\d+)|))"+L+"*\\)|)","i"),bool:new RegExp("^(?:"+K+")$","i"),needsContext:new RegExp("^"+L+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+L+"*((?:-\\d)?\\d*)"+L+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+L+"?|("+L+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},eb=function(){m()};try{H.apply(E=I.call(v.childNodes),v.childNodes),E[v.childNodes.length].nodeType}catch(fb){H={apply:E.length?function(a,b){G.apply(a,I.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function gb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],k=b.nodeType,"string"!=typeof a||!a||1!==k&&9!==k&&11!==k)return d;if(!e&&p){if(11!==k&&(f=_.exec(a)))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return H.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName)return H.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=1!==k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+rb(o[l]);w=ab.test(a)&&pb(b.parentNode)||b,x=o.join(",")}if(x)try{return H.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function hb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ib(a){return a[u]=!0,a}function jb(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function kb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function lb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||C)-(~a.sourceIndex||C);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function nb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function ob(a){return ib(function(b){return b=+b,ib(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function pb(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=gb.support={},f=gb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=gb.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=g.documentElement,e=g.defaultView,e&&e!==e.top&&(e.addEventListener?e.addEventListener("unload",eb,!1):e.attachEvent&&e.attachEvent("onunload",eb)),p=!f(g),c.attributes=jb(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=jb(function(a){return a.appendChild(g.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(g.getElementsByClassName),c.getById=jb(function(a){return o.appendChild(a).id=u,!g.getElementsByName||!g.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(g.querySelectorAll))&&(jb(function(a){o.appendChild(a).innerHTML="",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+L+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+L+"*(?:value|"+K+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),jb(function(a){var b=g.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+L+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&jb(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",P)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===g||a.ownerDocument===v&&t(v,a)?-1:b===g||b.ownerDocument===v&&t(v,b)?1:k?J(k,a)-J(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,h=[a],i=[b];if(!e||!f)return a===g?-1:b===g?1:e?-1:f?1:k?J(k,a)-J(k,b):0;if(e===f)return lb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?lb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},g):n},gb.matches=function(a,b){return gb(a,null,null,b)},gb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return gb(b,n,null,[a]).length>0},gb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},gb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&D.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},gb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},gb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=gb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=gb.selectors={cacheLength:50,createPseudo:ib,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||gb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&gb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+L+")"+a+"("+L+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=gb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(Q," ")+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||gb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ib(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=J(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ib(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?ib(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ib(function(a){return function(b){return gb(a,b).length>0}}),contains:ib(function(a){return a=a.replace(cb,db),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ib(function(a){return W.test(a||"")||gb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:ob(function(){return[0]}),last:ob(function(a,b){return[b-1]}),eq:ob(function(a,b,c){return[0>c?c+b:c]}),even:ob(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:ob(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:ob(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:ob(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function sb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function tb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function ub(a,b,c){for(var d=0,e=b.length;e>d;d++)gb(a,b[d],c);return c}function vb(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function wb(a,b,c,d,e,f){return d&&!d[u]&&(d=wb(d)),e&&!e[u]&&(e=wb(e,f)),ib(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||ub(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:vb(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=vb(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?J(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=vb(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):H.apply(g,r)})}function xb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=sb(function(a){return a===b},h,!0),l=sb(function(a){return J(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];f>i;i++)if(c=d.relative[a[i].type])m=[sb(tb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return wb(i>1&&tb(m),i>1&&rb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&xb(a.slice(i,e)),f>e&&xb(a=a.slice(e)),f>e&&rb(a))}m.push(c)}return tb(m)}function yb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=F.call(i));s=vb(s)}H.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&gb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?ib(f):f}return h=gb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=xb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,yb(e,d)),f.selector=a}return f},i=gb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&pb(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&rb(j),!a)return H.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&pb(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=jb(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),jb(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||kb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&jb(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||kb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),jb(function(a){return null==a.getAttribute("disabled")})||kb(K,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),gb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e.reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){m.each(b,function(b,c){var d=m.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?m.extend(a,d):d}},e={};return d.pipe=d.then,m.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&m.isFunction(a.promise)?e:0,g=1===f?a:m.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&m.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m.fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1; +return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="
a",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.isWindow(d)){for(i=k.delegateType||p,$.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m._data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((m.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/

","
"],tr:[2,"","
"],col:[2,"","
"],td:[3,"","
"],_default:k.htmlSerialize?[0,"",""]:[1,"X
","
"]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push(d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.removeAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?""!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,m.cleanData(ub(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEval")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("