From 2cb7a73e252e79f1b9e7122c35a2164206a9501f Mon Sep 17 00:00:00 2001 From: Fred Ji Date: Thu, 30 Nov 2017 15:29:25 -0800 Subject: [PATCH 01/36] SAMZA-1407 upgrade junit version to 4.12 --- gradle/dependency-versions.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 4f467ab8d3..e5c93ab769 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -31,7 +31,7 @@ jerseyVersion = "2.22.1" jodaTimeVersion = "2.2" joptSimpleVersion = "3.2" - junitVersion = "4.8.1" + junitVersion = "4.12" kafkaVersion = "0.10.1.1" log4jVersion = "1.2.17" metricsVersion = "2.2.0" From 2490e602e9491fdc5323b29987d91af8b8fe5aaf Mon Sep 17 00:00:00 2001 From: Jacob Maes Date: Thu, 30 Nov 2017 17:37:40 -0800 Subject: [PATCH 02/36] SAMZA-1518: Add CPU utilization and file descriptor count to JvmMetrics Author: Jacob Maes Reviewers: Jagadish , Prateek Maheshwari Closes #372 from jmakes/samza-1518 --- .../versioned/container/metrics-table.html | 12 ++++++++ .../org/apache/samza/metrics/JvmMetrics.scala | 28 +++++++++++++++++-- 2 files changed, 37 insertions(+), 3 deletions(-) diff --git a/docs/learn/documentation/versioned/container/metrics-table.html b/docs/learn/documentation/versioned/container/metrics-table.html index e504fa3fbd..e2f4e48f9c 100644 --- a/docs/learn/documentation/versioned/container/metrics-table.html +++ b/docs/learn/documentation/versioned/container/metrics-table.html @@ -345,6 +345,18 @@

Samza Metrics Reference

threads-terminated Current number of terminated threads + + process-cpu-usage + Current CPU usage of the JVM process as a percentage from 0 to 100. The percentage represents the proportion of executed ticks by the JVM process to the total ticks across all CPUs. A negative number indicates the value was not available from the operating system. For more detail, see the JavaDoc for com.sun.management.OperatingSystemMXBean. + + + system-cpu-usage + Current CPU usage of the all processes in the whole system as a percentage from 0 to 100. The percentage represents the proportion of executed ticks by all processes to the total ticks across all CPUs. A negative number indicates the value was not available from the operating system. For more detail, see the JavaDoc for com.sun.management.OperatingSystemMXBean. + + + open-file-descriptor-count + Current number of open file descriptors + org.apache.samza.system.SystemConsumersMetrics diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala b/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala index 6692e35585..f26bd2c24d 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala @@ -19,12 +19,14 @@ package org.apache.samza.metrics -import java.lang.management.ManagementFactory import scala.collection._ import scala.collection.JavaConverters._ +import java.lang.management.ManagementFactory import java.lang.Thread.State._ import java.util.concurrent.Executors import java.util.concurrent.TimeUnit + +import com.sun.management.{OperatingSystemMXBean, UnixOperatingSystemMXBean} import org.apache.samza.util.Logging import org.apache.samza.util.DaemonThreadFactory @@ -40,10 +42,12 @@ object JvmMetrics { */ class JvmMetrics(val registry: MetricsRegistry) extends MetricsHelper with Runnable with Logging { final val M = 1024 * 1024.0F + final val PCT = 100.0 val memoryMXBean = ManagementFactory.getMemoryMXBean() val gcBeans = ManagementFactory.getGarbageCollectorMXBeans() val threadMXBean = ManagementFactory.getThreadMXBean() + val osMXBean = ManagementFactory.getOperatingSystemMXBean() var gcBeanCounters = Map[String, (Counter, Counter)]() val executor = Executors.newScheduledThreadPool(1, new DaemonThreadFactory(JvmMetrics.JVM_METRICS_THREAD_NAME_PREFIX)) @@ -63,6 +67,11 @@ class JvmMetrics(val registry: MetricsRegistry) extends MetricsHelper with Runna val cGcCount = newCounter("gc-count") val cGcTimeMillis = newCounter("gc-time-millis") + // Conditional metrics. Only emitted if the Operating System supports it. + val gProcessCpuUsage = if (osMXBean.isInstanceOf[OperatingSystemMXBean]) newGauge("process-cpu-usage", 0.0) else null + val gSystemCpuUsage = if (osMXBean.isInstanceOf[OperatingSystemMXBean]) newGauge("system-cpu-usage", 0.0) else null + val gOpenFileDescriptorCount = if (osMXBean.isInstanceOf[UnixOperatingSystemMXBean]) newGauge("open-file-descriptor-count", 0.0) else null + def start { executor.scheduleWithFixedDelay(this, 0, 5, TimeUnit.SECONDS) } @@ -73,11 +82,12 @@ class JvmMetrics(val registry: MetricsRegistry) extends MetricsHelper with Runna updateMemoryUsage updateGcUsage updateThreadUsage + updateOperatingSystemMetrics - debug("updated metrics to: [%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s]" format + debug("updated metrics to: [%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s]" format (gMemNonHeapUsedM, gMemNonHeapCommittedM, gMemNonHeapMaxM, gMemHeapUsedM, gMemHeapCommittedM,gMemHeapMaxM, gThreadsNew, gThreadsRunnable, gThreadsBlocked, gThreadsWaiting, gThreadsTimedWaiting, - gThreadsTerminated, cGcCount, cGcTimeMillis)) + gThreadsTerminated, cGcCount, cGcTimeMillis, gProcessCpuUsage, gSystemCpuUsage, gOpenFileDescriptorCount)) } def stop = executor.shutdown @@ -153,4 +163,16 @@ class JvmMetrics(val registry: MetricsRegistry) extends MetricsHelper with Runna gThreadsTimedWaiting.set(threadsTimedWaiting) gThreadsTerminated.set(threadsTerminated) } + + private def updateOperatingSystemMetrics { + if (osMXBean.isInstanceOf[OperatingSystemMXBean]) { + val operatingSystemMXBean = osMXBean.asInstanceOf[OperatingSystemMXBean] + gProcessCpuUsage.set(operatingSystemMXBean.getProcessCpuLoad * PCT) + gSystemCpuUsage.set(operatingSystemMXBean.getSystemCpuLoad * PCT) + + if (osMXBean.isInstanceOf[UnixOperatingSystemMXBean]) { + gOpenFileDescriptorCount.set(osMXBean.asInstanceOf[UnixOperatingSystemMXBean].getOpenFileDescriptorCount) + } + } + } } From 1ca9e3268a324f8c1c74798bbb8661d1b7c9722f Mon Sep 17 00:00:00 2001 From: navina Date: Fri, 1 Dec 2017 08:57:06 -0800 Subject: [PATCH 03/36] SAMZA-1519: Add release notes to website documentation Adding a versioned page for release/upgrade notes. We can start this process from the next major version release, aka 0.14.0. Please update this page as and when you add new features/configs/API or deprecate features/configs/API. Basically, anything that can be useful for Samza users trying to upgrade. Note: `site.version` is not necessarily same as samza release version. For now, I am using it as a placeholder. Hopefully, with the next generation of our website, it will be better defined. Author: navina Reviewers: Jacob Maes Closes #301 from navina/versioning --- docs/_config.yml | 2 +- docs/_docs/replace-versioned.sh | 3 + .../releases/versioned/release-notes.md | 75 +++++++++++++++++++ 3 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 docs/startup/releases/versioned/release-notes.md diff --git a/docs/_config.yml b/docs/_config.yml index d81ecac8d8..11850aa28f 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -20,7 +20,7 @@ highlighter: pygments markdown: redcarpet exclude: ['_notes'] redcarpet: - extensions: ['with_toc_data', 'smart'] + extensions: ['with_toc_data', 'smart', 'strikethrough'] exclude: [_docs] baseurl: http://samza.apache.org version: latest diff --git a/docs/_docs/replace-versioned.sh b/docs/_docs/replace-versioned.sh index f148781262..c005e6219d 100755 --- a/docs/_docs/replace-versioned.sh +++ b/docs/_docs/replace-versioned.sh @@ -37,5 +37,8 @@ mv -f $DIR/_site/learn/documentation/versioned $DIR/_site/learn/documentation/$v echo "replaced learn/tutorials/versioned to learn/tutorials/"$version mv -f $DIR/_site/learn/tutorials/versioned $DIR/_site/learn/tutorials/$version +echo "replaced learn/releases/versioned to learn/releases/"$version +mv -f $DIR/_site/startup/releases/versioned $DIR/_site/startup/releases/$version + echo "replaced startup/hello-samza/versioned to startup/hello-samza/"$version mv -f $DIR/_site/startup/hello-samza/versioned $DIR/_site/startup/hello-samza/$version diff --git a/docs/startup/releases/versioned/release-notes.md b/docs/startup/releases/versioned/release-notes.md new file mode 100644 index 0000000000..460cbda7b3 --- /dev/null +++ b/docs/startup/releases/versioned/release-notes.md @@ -0,0 +1,75 @@ +--- +layout: page +title: Release Notes +--- + + +{% if site.version != "latest" %} +1. [Download](#download) +2. [Release Notes for Samza-{{site.version}} version](#release-notes-for-{{site.version}}) +3. [Upgrade Notes](#upgrade-notes) + +{% else %} +1. [Release Notes for Samza-{{site.version}} version](#release-notes) +2. [Upgrade Notes](#upgrade-notes) +{% endif %} + +{% if site.version != "latest" %} +## Download + +## Source Release +[samza-sources-{{site.version}}.tgz](http://www.apache.org/dyn/closer.lua/samza/{{site.version}}.*) + + +{% endif %} + + + +## Release Notes + + + + +## Upgrade Notes + + + +### Configuration Changes + + +* Introduced a new **mandatory** configuration - `job.coordination.utils.factory`. Read more about it +[here](../../learn/{{site.version}}/configuration.html).
This config is applicable to all Samza +applications deployed using the `LocalApplicationRunner` (that is, non-yarn deployments). + +### API Changes + + +* The following APIs in `SystemAdmin` have been deprecated in the previous versions and hence, replaced with newer APIs. +If you have a custom **System** implementation, then you have to update to the newer APIs. + * ~~void createChangelogStream(String streamName, int numOfPartitions);~~ -> ``` boolean createStream(StreamSpec streamSpec); ``` + * ~~void createCoordinatorStream(String streamName);~~ -> ``` boolean createStream(StreamSpec streamSpec); ``` + * ~~void validateChangelogStream(String streamName, int numOfPartitions);~~ -> ``` void validateStream(StreamSpec streamSpec) throws StreamValidationException; ``` + + +* New API has been added to `SystemAdmin` that clear a stream.
+``` +boolean clearStream(StreamSpec streamSpec); +``` +
+Read more about it in the [API docs](). + From 9c9bbc45cfdbf4d14f27c84ab3dc3314169ade7c Mon Sep 17 00:00:00 2001 From: Abhishek Shivanna Date: Fri, 1 Dec 2017 15:23:12 -0800 Subject: [PATCH 04/36] SAMZA-1506: Fix for robust ContainerHeartbeatMonitor exception handling. The Fix includes the following changes: - Catch all exceptions inside the heartbeat thread and not just IOException. - A time based force kill when the heartbeat is invalid, this makes the monitor immune to threads that may keep the container stuck in the shutdown sequence. When the timeout occurs, a System.exit(1) is called. - Increasing number of retries for failed heartbeats from 3 to 6. This prevents short intermittent network failurs from causing the containers to be invalidated. Author: Abhishek Shivanna Reviewers: Jacob Maes Closes #375 from abhishekshivanna/container-heartbeat --- .../container/ContainerHeartbeatClient.java | 20 ++++++++++++------- .../container/ContainerHeartbeatMonitor.java | 11 +++++++++- .../samza/runtime/LocalContainerRunner.java | 9 +++++++-- 3 files changed, 30 insertions(+), 10 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatClient.java b/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatClient.java index f2c2651d68..7273d54f4b 100644 --- a/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatClient.java +++ b/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatClient.java @@ -49,7 +49,7 @@ */ public class ContainerHeartbeatClient { private static final Logger LOG = LoggerFactory.getLogger(ContainerHeartbeatClient.class); - private static final int NUM_RETRIES = 3; + private static final int NUM_RETRIES = 6; private static final int TIMEOUT_MS = 5000; private static final int BACKOFF_MULTIPLIER = 2; private final String heartbeatEndpoint; @@ -72,9 +72,10 @@ public ContainerHeartbeatResponse requestHeartbeat() { LOG.debug("Container Heartbeat got response {}", reply); response = mapper.readValue(reply, ContainerHeartbeatResponse.class); return response; - } catch (IOException e) { - LOG.error("Error in container heart beat protocol. Query url: {} response: {}", heartbeatEndpoint, reply); + } catch (Exception e) { + LOG.error("Error in container heartbeat to JobCoordinator.", e); } + LOG.error("Container heartbeat expired"); response = new ContainerHeartbeatResponse(false); return response; } @@ -82,10 +83,11 @@ public ContainerHeartbeatResponse requestHeartbeat() { String httpGet(URL url) throws IOException { HttpURLConnection conn; int delayMillis = 1000; - + BufferedReader br = null; for (int currentTry = 0; currentTry < NUM_RETRIES; currentTry++) { - conn = Util.getHttpConnection(url, TIMEOUT_MS); - try (BufferedReader br = new BufferedReader(new InputStreamReader(conn.getInputStream()))) { + try { + conn = Util.getHttpConnection(url, TIMEOUT_MS); + br = new BufferedReader(new InputStreamReader(conn.getInputStream())); if (conn.getResponseCode() != HttpURLConnection.HTTP_OK) { throw new IOException(String.format("HTTP error fetching url %s. Returned status code %d", url.toString(), conn.getResponseCode())); @@ -93,9 +95,13 @@ String httpGet(URL url) throws IOException { return br.lines().collect(Collectors.joining()); } } catch (Exception e) { - LOG.error("Error in heartbeat request", e); + LOG.error(String.format("Error in heartbeat request. Retrying [%d/%d].", currentTry + 1, NUM_RETRIES), e); sleepUninterruptibly(delayMillis); delayMillis = delayMillis * BACKOFF_MULTIPLIER; + } finally { + if (br != null) { + br.close(); + } } } throw new IOException(String.format("Error fetching url: %s. Tried %d time(s).", url.toString(), NUM_RETRIES)); diff --git a/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatMonitor.java b/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatMonitor.java index 940e80f02c..64e74501dc 100644 --- a/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatMonitor.java +++ b/samza-core/src/main/java/org/apache/samza/container/ContainerHeartbeatMonitor.java @@ -27,10 +27,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + public class ContainerHeartbeatMonitor { private static final Logger LOG = LoggerFactory.getLogger(ContainerHeartbeatMonitor.class); private static final ThreadFactory THREAD_FACTORY = new HeartbeatThreadFactory(); private static final int SCHEDULE_MS = 60000; + private static final int SHUTDOWN_TIMOUT_MS = 120000; private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(THREAD_FACTORY); private final Runnable onContainerExpired; private final ContainerHeartbeatClient containerHeartbeatClient; @@ -50,6 +52,11 @@ public void start() { scheduler.scheduleAtFixedRate(() -> { ContainerHeartbeatResponse response = containerHeartbeatClient.requestHeartbeat(); if (!response.isAlive()) { + scheduler.schedule(() -> { + // On timeout of container shutting down, force exit. + LOG.error("Graceful shutdown timeout expired. Force exiting."); + System.exit(1); + }, SHUTDOWN_TIMOUT_MS, TimeUnit.MILLISECONDS); onContainerExpired.run(); } }, 0, SCHEDULE_MS, TimeUnit.MILLISECONDS); @@ -69,7 +76,9 @@ private static class HeartbeatThreadFactory implements ThreadFactory { @Override public Thread newThread(Runnable runnable) { - return new Thread(runnable, PREFIX + INSTANCE_NUM.getAndIncrement()); + Thread t = new Thread(runnable, PREFIX + INSTANCE_NUM.getAndIncrement()); + t.setDaemon(true); + return t; } } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 50c8181daf..6750ccd2af 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -153,8 +153,13 @@ private void startContainerHeartbeatMonitor() { if (executionEnvContainerId != null) { log.info("Got execution environment container id: {}", executionEnvContainerId); containerHeartbeatMonitor = new ContainerHeartbeatMonitor(() -> { - container.shutdown(); - containerRunnerException = new SamzaException("Container shutdown due to expired heartbeat"); + try { + container.shutdown(); + containerRunnerException = new SamzaException("Container shutdown due to expired heartbeat"); + } catch (Exception e) { + log.error("Heartbeat monitor failed to shutdown the container gracefully. Exiting process.", e); + System.exit(1); + } }, new ContainerHeartbeatClient(coordinatorUrl, executionEnvContainerId)); containerHeartbeatMonitor.start(); } else { From 32d2af5dd10b2c2c1cf3f6e683967bafd5285cdf Mon Sep 17 00:00:00 2001 From: Prateek Maheshwari Date: Mon, 4 Dec 2017 14:21:21 -0800 Subject: [PATCH 05/36] Updated Serde related documentation and error messages for High Level API Updated and clarified the documentation and error messages related to Serdes for Input/Output/PartitionBy streams. Author: Prateek Maheshwari Reviewers: Jagadish Venkatraman Closes #376 from prateekm/documentation-cleanup --- .../apache/samza/operators/MessageStream.java | 18 +++++++--- .../apache/samza/operators/StreamGraph.java | 33 +++++++++++++++---- .../org/apache/samza/serializers/KVSerde.java | 8 +++-- .../samza/operators/MessageStreamImpl.java | 4 +++ .../samza/operators/StreamGraphImpl.java | 6 ++-- .../operators/impl/InputOperatorImpl.java | 2 +- .../operators/impl/OutputOperatorImpl.java | 2 +- .../impl/PartitionByOperatorImpl.java | 4 --- .../operators/spec/InputOperatorSpec.java | 10 +++--- .../operators/spec/OutputStreamImpl.java | 10 +++--- .../stream/IntermediateMessageStreamImpl.java | 13 ++++++++ .../operators/TestMessageStreamImpl.java | 2 ++ 12 files changed, 80 insertions(+), 32 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index dcce7c8499..acb2f33fc6 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -93,6 +93,9 @@ public interface MessageStream { /** * Allows sending messages in this {@link MessageStream} to an {@link OutputStream}. + *

+ * When sending messages to an {@code OutputStream>}, messages are partitioned using their serialized key. + * When sending messages to any other {@code OutputStream}, messages are partitioned using a null partition key. * * @param outputStream the output stream to send messages to */ @@ -185,8 +188,9 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, - * uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. - * If no default serde has been provided before calling this method, no-op serdes are used for keys and values. + * uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default + * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided + * before calling this method, a {@code KVSerde} is used. *

* The number of partitions for this intermediate stream is determined as follows: * If the stream is an eventual input to a {@link #join}, and the number of partitions for the other stream is known, @@ -200,8 +204,11 @@ static MessageStream mergeAll(Collection + * Unlike {@link #sendTo}, messages with a null key are all sent to partition 0. * - * @param keyExtractor the {@link Function} to extract the message and partition key from the input message + * @param keyExtractor the {@link Function} to extract the message and partition key from the input message. + * Messages with a null key are all sent to partition 0. * @param valueExtractor the {@link Function} to extract the value from the input message * @param serde the {@link KVSerde} to use for (de)serializing the key and value. * @param id the unique id of this operator in this application @@ -212,9 +219,12 @@ static MessageStream mergeAll(Collection MessageStream> partitionBy(Function keyExtractor, Function valueExtractor, KVSerde serde, String id); - /** * Same as calling {@link #partitionBy(Function, Function, KVSerde, String)} with a null KVSerde. + *

+ * Uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default + * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided + * before calling this method, a {@code KVSerde} is used. * * @param keyExtractor the {@link Function} to extract the message and partition key from the input message * @param valueExtractor the {@link Function} to extract the value from the input message diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java index 17223b1b60..49306316db 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java @@ -32,8 +32,10 @@ public interface StreamGraph { * Sets the default {@link Serde} to use for (de)serializing messages. *

. * If the default serde is set, it must be set before creating any input or output streams. - * If no explicit or default serdes are provided, a NoOpSerde is used for keys and values. This means that any - * streams created without explicit or default serdes should be cast to MessageStream<KV<Object, Object>>. + *

+ * If no explicit or default serdes are provided, a {@code KVSerde} is used. This means that + * any streams created without explicit or default serdes should be cast to {@code MessageStream>}. + *

* Providing an incompatible message type for the input/output streams that use the default serde will result in * {@link ClassCastException}s at runtime. * @@ -44,6 +46,13 @@ public interface StreamGraph { /** * Gets the input {@link MessageStream} corresponding to the {@code streamId}. *

+ * An input {@code MessageStream}, which can be obtained by calling this method with a {@code KVSerde}, + * can receive messages of type {@code KV}. An input {@code MessageStream} with any other {@code Serde} + * can receive messages of type M - the key in the incoming message is ignored. + *

+ * A {@code KVSerde} or {@code NoOpSerde} may be used if the {@code SystemConsumer} + * deserializes the incoming messages itself, and no further deserialization is required from the framework. + *

* Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. * * @param streamId the unique ID for the stream @@ -58,9 +67,10 @@ public interface StreamGraph { * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via * {@link #setDefaultSerde(Serde)} for deserializing input messages. *

- * If no default serde has been provided before calling this method, a no-op serde is used. - * Providing a message type {@code M} that is incompatible with the default Serde will result in + * If no default serde has been provided before calling this method, a {@code KVSerde} + * is used. Providing a message type {@code M} that is incompatible with the default Serde will result in * {@link ClassCastException}s at runtime. + *

* Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. * * @param streamId the unique ID for the stream @@ -73,6 +83,16 @@ public interface StreamGraph { /** * Gets the {@link OutputStream} corresponding to the {@code streamId}. *

+ * An {@code OutputStream>}, which can be obtained by calling this method with a {@code KVSerde}, + * can send messages of type {@code KV}. An {@code OutputStream} with any other {@code Serde} can + * send messages of type M without a key. + *

+ * A {@code KVSerde} or {@code NoOpSerde} may be used if the {@code SystemProducer} + * serializes the outgoing messages itself, and no prior serialization is required from the framework. + *

+ * When sending messages to an {@code OutputStream>}, messages are partitioned using their serialized key. + * When sending messages to any other {@code OutputStream}, messages are partitioned using a null partition key. + *

* Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. * * @param streamId the unique ID for the stream @@ -87,9 +107,10 @@ public interface StreamGraph { * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via * {@link #setDefaultSerde(Serde)} for serializing output messages. *

- * If no default serde has been provided before calling this method, a no-op serde is used. - * Providing a message type {@code M} that is incompatible with the default Serde will result in + * If no default serde has been provided before calling this method, a {@code KVSerde} + * is used. Providing a message type {@code M} that is incompatible with the default Serde will result in * {@link ClassCastException}s at runtime. + *

* Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. * * @param streamId the unique ID for the stream diff --git a/samza-api/src/main/java/org/apache/samza/serializers/KVSerde.java b/samza-api/src/main/java/org/apache/samza/serializers/KVSerde.java index 10f224ba6f..42a527d7a1 100644 --- a/samza-api/src/main/java/org/apache/samza/serializers/KVSerde.java +++ b/samza-api/src/main/java/org/apache/samza/serializers/KVSerde.java @@ -25,8 +25,12 @@ /** - * A marker serde class to indicate that messages are keyed and should be deserialized as K-V pairs. This class is - * intended for use cases where a single Serde parameter or configuration is required. + * A serializer that serves one of the following two purposes: + *

    + *
  1. When used for input/output streams, serves as an indication to the framework that messages are keyed and + * should be deserialized as K-V pairs.
  2. + *
  3. When used for a key or value of type KV, serializes the key and value pair using the provided serdes.
  4. + *
* * @param type of the key in the message * @param type of the value in the message diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index e6e711c01d..3f4e40d45e 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -152,6 +152,10 @@ public MessageStream> partitionBy(Function valueExtractor, KVSerde serde, String userDefinedId) { String opId = this.graph.getNextOpId(OpCode.PARTITION_BY, userDefinedId); IntermediateMessageStreamImpl> intermediateStream = this.graph.getIntermediateStream(opId, serde); + if (!intermediateStream.isKeyed()) { + // this can only happen when the default serde partitionBy variant is being used + throw new SamzaException("partitionBy can not be used with a default serde that is not a KVSerde."); + } PartitionByOperatorSpec partitionByOperatorSpec = OperatorSpecs.createPartitionByOperatorSpec( intermediateStream.getOutputStream(), keyExtractor, valueExtractor, opId); diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java index 5323c32342..d014cb9419 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java @@ -108,8 +108,7 @@ public MessageStream getInputStream(String streamId, Serde serde) { InputOperatorSpec inputOperatorSpec = OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed, this.getNextOpId(OpCode.INPUT, null)); - inputOperators.put(streamSpec, - inputOperatorSpec); + inputOperators.put(streamSpec, inputOperatorSpec); return new MessageStreamImpl<>(this, inputOperators.get(streamSpec)); } @@ -137,8 +136,7 @@ public OutputStream getOutputStream(String streamId, Serde serde) { } boolean isKeyed = serde instanceof KVSerde; - outputStreams.put(streamSpec, - new OutputStreamImpl<>(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); + outputStreams.put(streamSpec, new OutputStreamImpl<>(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); return outputStreams.get(streamSpec); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java index 9cc5370325..ef3c322d5e 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/InputOperatorImpl.java @@ -50,7 +50,7 @@ protected void handleInit(Config config, TaskContext context) { @Override public Collection handleMessage(KV pair, MessageCollector collector, TaskCoordinator coordinator) { - Object message = this.inputOpSpec.isKeyedInput() ? pair : pair.getValue(); + Object message = this.inputOpSpec.isKeyed() ? pair : pair.getValue(); return Collections.singletonList(message); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java index 7b7e49ce87..27bef877fd 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java @@ -57,7 +57,7 @@ protected void handleInit(Config config, TaskContext context) { public Collection handleMessage(M message, MessageCollector collector, TaskCoordinator coordinator) { Object key, value; - if (outputStream.isKeyedOutput()) { + if (outputStream.isKeyed()) { key = ((KV) message).getKey(); value = ((KV) message).getValue(); } else { diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java index b3fb4b2a82..9fc1e7c65f 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java @@ -18,7 +18,6 @@ */ package org.apache.samza.operators.impl; -import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.operators.KV; @@ -55,9 +54,6 @@ class PartitionByOperatorImpl extends OperatorImpl { PartitionByOperatorImpl(PartitionByOperatorSpec partitionByOpSpec, Config config, TaskContext context) { this.partitionByOpSpec = partitionByOpSpec; OutputStreamImpl> outputStream = partitionByOpSpec.getOutputStream(); - if (!outputStream.isKeyedOutput()) { - throw new SamzaException("Output stream for repartitioning must be a keyed stream."); - } this.systemStream = new SystemStream( outputStream.getStreamSpec().getSystemName(), outputStream.getStreamSpec().getPhysicalName()); diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java index 4c0687a3b3..2ad0597c0f 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java @@ -35,15 +35,15 @@ public class InputOperatorSpec extends OperatorSpec, Object> { // private final StreamSpec streamSpec; private final Serde keySerde; private final Serde valueSerde; - private final boolean isKeyedInput; + private final boolean isKeyed; public InputOperatorSpec(StreamSpec streamSpec, - Serde keySerde, Serde valueSerde, boolean isKeyedInput, String opId) { + Serde keySerde, Serde valueSerde, boolean isKeyed, String opId) { super(OpCode.INPUT, opId); this.streamSpec = streamSpec; this.keySerde = keySerde; this.valueSerde = valueSerde; - this.isKeyedInput = isKeyedInput; + this.isKeyed = isKeyed; } public StreamSpec getStreamSpec() { @@ -58,8 +58,8 @@ public Serde getValueSerde() { return valueSerde; } - public boolean isKeyedInput() { - return isKeyedInput; + public boolean isKeyed() { + return isKeyed; } @Override diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java index a793e0cb46..e439c4e178 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java @@ -28,14 +28,14 @@ public class OutputStreamImpl implements OutputStream { private final StreamSpec streamSpec; private final Serde keySerde; private final Serde valueSerde; - private final boolean isKeyedOutput; + private final boolean isKeyed; public OutputStreamImpl(StreamSpec streamSpec, - Serde keySerde, Serde valueSerde, boolean isKeyedOutput) { + Serde keySerde, Serde valueSerde, boolean isKeyed) { this.streamSpec = streamSpec; this.keySerde = keySerde; this.valueSerde = valueSerde; - this.isKeyedOutput = isKeyedOutput; + this.isKeyed = isKeyed; } public StreamSpec getStreamSpec() { @@ -50,7 +50,7 @@ public Serde getValueSerde() { return valueSerde; } - public boolean isKeyedOutput() { - return isKeyedOutput; + public boolean isKeyed() { + return isKeyed; } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java index 279cdd4844..5eeca99e6d 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java @@ -25,6 +25,8 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.system.StreamSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An intermediate stream is both an input and an output stream (e.g. a repartitioned stream). @@ -39,12 +41,19 @@ */ public class IntermediateMessageStreamImpl extends MessageStreamImpl implements OutputStream { + private static final Logger LOGGER = LoggerFactory.getLogger(IntermediateMessageStreamImpl.class); private final OutputStreamImpl outputStream; + private final boolean isKeyed; public IntermediateMessageStreamImpl(StreamGraphImpl graph, InputOperatorSpec inputOperatorSpec, OutputStreamImpl outputStream) { super(graph, (OperatorSpec) inputOperatorSpec); this.outputStream = outputStream; + if (inputOperatorSpec.isKeyed() != outputStream.isKeyed()) { + LOGGER.error("Input and output streams for intermediate stream {} aren't keyed consistently. Input: {}, Output: {}", + new Object[]{inputOperatorSpec.getStreamSpec().getId(), inputOperatorSpec.isKeyed(), outputStream.isKeyed()}); + } + this.isKeyed = inputOperatorSpec.isKeyed() && outputStream.isKeyed(); } public StreamSpec getStreamSpec() { @@ -54,4 +63,8 @@ public StreamSpec getStreamSpec() { public OutputStreamImpl getOutputStream() { return this.outputStream; } + + public boolean isKeyed() { + return isKeyed; + } } diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java index 5b3c3a0109..d97d4945fe 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java @@ -207,6 +207,7 @@ public void testRepartition() { .thenReturn(mockIntermediateStream); when(mockIntermediateStream.getOutputStream()) .thenReturn(mockOutputStreamImpl); + when(mockIntermediateStream.isKeyed()).thenReturn(true); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); Function mockKeyFunction = mock(Function.class); @@ -236,6 +237,7 @@ public void testRepartitionWithoutSerde() { .thenReturn(mockIntermediateStream); when(mockIntermediateStream.getOutputStream()) .thenReturn(mockOutputStreamImpl); + when(mockIntermediateStream.isKeyed()).thenReturn(true); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); Function mockKeyFunction = mock(Function.class); From e3efdf5c872cda681d5d012046524b9992ffda91 Mon Sep 17 00:00:00 2001 From: Bharath Kumarasubramanian Date: Mon, 4 Dec 2017 17:11:03 -0800 Subject: [PATCH 06/36] Fix compile errors with scala 2.12 and update release notes to use check all Author: Bharath Kumarasubramanian Reviewers: Xinyu Liu Closes #378 from bharathkk/master --- RELEASE.md | 4 ++-- .../samza/storage/kv/RocksDbKeyValueStore.scala | 12 ++++++------ .../apache/samza/storage/kv/AccessLoggedStore.scala | 4 ++-- .../samza/storage/kv/KeyValueStorageEngine.scala | 4 ++-- .../org/apache/samza/storage/kv/LoggedStore.scala | 4 ++-- .../samza/storage/kv/NullSafeKeyValueStore.scala | 4 ++-- .../samza/storage/kv/SerializedKeyValueStore.scala | 4 ++-- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/RELEASE.md b/RELEASE.md index 2f38c8d82a..cb83594a49 100644 --- a/RELEASE.md +++ b/RELEASE.md @@ -30,9 +30,9 @@ And before you proceed, do the following steps: * change the samza-test versions in samza-test/src/main/config/join/README to $VERSION w/o the suffix '-SNAPSHOT'. * push the changes to the $VERSION branch -Validate that all Samza source files have proper license information in their header. +Validate Samza using all our supported build matrix. - ./gradlew check + ./bin/check-all.sh To release to a local Maven repository: diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 2ae4bb0113..6aad45f573 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -122,7 +122,7 @@ class RocksDbKeyValueStore( found } - def getAll(keys: java.util.List[Array[Byte]]): java.util.Map[Array[Byte], Array[Byte]] = ifOpen { + override def getAll(keys: java.util.List[Array[Byte]]): java.util.Map[Array[Byte], Array[Byte]] = ifOpen { metrics.getAlls.inc require(keys != null, "Null keys not allowed.") val map = db.multiGet(keys) @@ -251,10 +251,10 @@ class RocksDbKeyValueStore( new Entry(key, value) } - // By virtue of how RocksdbIterator is implemented, the implementation of - // our iterator is slightly different from standard java iterator next will - // always point to the current element, when next is called, we return the - // current element we are pointing to and advance the iterator to the next + // By virtue of how RocksdbIterator is implemented, the implementation of + // our iterator is slightly different from standard java iterator next will + // always point to the current element, when next is called, we return the + // current element we are pointing to and advance the iterator to the next // location (The new location may or may not be valid - this will surface // when the next next() call is made, the isValid will fail) override def next(): Entry[Array[Byte], Array[Byte]] = ifOpen { @@ -280,7 +280,7 @@ class RocksDbKeyValueStore( } class RocksDbRangeIterator(iter: RocksIterator, from: Array[Byte], to: Array[Byte]) extends RocksDbIterator(iter) { - // RocksDB's JNI interface does not expose getters/setters that allow the + // RocksDB's JNI interface does not expose getters/setters that allow the // comparator to be pluggable, and the default is lexicographic, so it's // safe to just force lexicographic comparator here for now. val comparator: LexicographicComparator = lexicographic diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/AccessLoggedStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/AccessLoggedStore.scala index c21c9a6527..879a144850 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/AccessLoggedStore.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/AccessLoggedStore.scala @@ -56,7 +56,7 @@ class AccessLoggedStore[K, V]( logAccess(DBOperation.READ, list, store.get(key)) } - def getAll(keys: util.List[K]): util.Map[K, V] = { + override def getAll(keys: util.List[K]): util.Map[K, V] = { logAccess(DBOperation.READ, serializeKeys(keys), store.getAll(keys)) } @@ -76,7 +76,7 @@ class AccessLoggedStore[K, V]( logAccess(DBOperation.DELETE, list, store.delete(key)) } - def deleteAll(keys: util.List[K]): Unit = { + override def deleteAll(keys: util.List[K]): Unit = { logAccess(DBOperation.DELETE, serializeKeys(keys), store.deleteAll(keys)) } diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala index 8385c4a8cc..646b60608c 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala @@ -51,7 +51,7 @@ class KeyValueStorageEngine[K, V]( } } - def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { + override def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { metrics.gets.inc(keys.size) wrapperStore.getAll(keys) } @@ -75,7 +75,7 @@ class KeyValueStorageEngine[K, V]( } } - def deleteAll(keys: java.util.List[K]) = { + override def deleteAll(keys: java.util.List[K]) = { metrics.deletes.inc(keys.size) wrapperStore.deleteAll(keys) } diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala index 33ff41b231..e0c7a31628 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala @@ -41,7 +41,7 @@ class LoggedStore[K, V]( store.get(key) } - def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { + override def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { metrics.gets.inc(keys.size) store.getAll(keys) } @@ -90,7 +90,7 @@ class LoggedStore[K, V]( /** * Perform the local deletes and log them out to the changelog */ - def deleteAll(keys: java.util.List[K]) = { + override def deleteAll(keys: java.util.List[K]) = { metrics.deletes.inc(keys.size) val keysIterator = keys.iterator while (keysIterator.hasNext) { diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/NullSafeKeyValueStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/NullSafeKeyValueStore.scala index 9e67fc8b6b..3cc35d3688 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/NullSafeKeyValueStore.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/NullSafeKeyValueStore.scala @@ -37,7 +37,7 @@ class NullSafeKeyValueStore[K, V](store: KeyValueStore[K, V]) extends KeyValueSt store.get(key) } - def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { + override def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { notNull(keys, NullKeysErrorMessage) keys.asScala.foreach(key => notNull(key, NullKeyErrorMessage)) store.getAll(keys) @@ -62,7 +62,7 @@ class NullSafeKeyValueStore[K, V](store: KeyValueStore[K, V]) extends KeyValueSt store.delete(key) } - def deleteAll(keys: java.util.List[K]) = { + override def deleteAll(keys: java.util.List[K]) = { notNull(keys, NullKeysErrorMessage) keys.asScala.foreach(key => notNull(key, NullKeyErrorMessage)) store.deleteAll(keys) diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala index c8939b7451..16dd980678 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala @@ -38,7 +38,7 @@ class SerializedKeyValueStore[K, V]( fromBytesOrNull(found, msgSerde) } - def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { + override def getAll(keys: java.util.List[K]): java.util.Map[K, V] = { metrics.gets.inc(keys.size) val mapBytes = store.getAll(serializeKeys(keys)) if (mapBytes != null) { @@ -80,7 +80,7 @@ class SerializedKeyValueStore[K, V]( store.delete(keyBytes) } - def deleteAll(keys: java.util.List[K]) = { + override def deleteAll(keys: java.util.List[K]) = { metrics.deletes.inc(keys.size) store.deleteAll(serializeKeys(keys)) } From e74998c5e58c891c0744ce1be2b13f7c5861656a Mon Sep 17 00:00:00 2001 From: Wei Song Date: Tue, 5 Dec 2017 13:23:44 -0800 Subject: [PATCH 07/36] Initial version of Table API Initial version of table API, it includes - Core table API (Table, TableDescriptor, TableSpec) - Local table implementation for in-memory and RocksDb - The writeTo() and stream-table join operators nickpan47 xinyuiscool prateekm could you help review? Author: Wei Song Reviewers: Yi Pan , Christopher Pettitt Closes #349 from weisong44/table-api-14 --- build.gradle | 9 + .../java/org/apache/samza/operators/KV.java | 2 +- .../apache/samza/operators/MessageStream.java | 51 ++- .../apache/samza/operators/StreamGraph.java | 15 + .../samza/operators/TableDescriptor.java | 73 +++++ .../functions/StreamTableJoinFunction.java | 59 ++++ .../table/LocalStoreBackedTableProvider.java | 37 +++ .../apache/samza/table/ReadWriteTable.java | 75 +++++ .../org/apache/samza/table/ReadableTable.java | 61 ++++ .../java/org/apache/samza/table/Table.java | 31 ++ .../org/apache/samza/table/TableProvider.java | 57 ++++ .../samza/table/TableProviderFactory.java | 35 ++ .../org/apache/samza/table/TableSpec.java | 125 +++++++ .../org/apache/samza/task/TaskContext.java | 6 +- .../apache/samza/config/JavaTableConfig.java | 87 +++++ .../samza/container/TaskContextImpl.java | 24 +- .../samza/execution/ExecutionPlanner.java | 5 + .../org/apache/samza/execution/JobGraph.java | 16 + .../execution/JobGraphJsonGenerator.java | 50 +++ .../org/apache/samza/execution/JobNode.java | 54 +++- .../samza/operators/BaseTableDescriptor.java | 94 ++++++ .../samza/operators/MessageStreamImpl.java | 31 +- .../samza/operators/StreamGraphImpl.java | 40 ++- .../org/apache/samza/operators/TableImpl.java | 40 +++ .../operators/impl/OperatorImplGraph.java | 30 +- .../impl/SendToTableOperatorImpl.java | 71 ++++ .../impl/StreamTableJoinOperatorImpl.java | 82 +++++ .../samza/operators/spec/OperatorSpec.java | 12 +- .../samza/operators/spec/OperatorSpecs.java | 44 ++- .../spec/SendToTableOperatorSpec.java | 65 ++++ .../spec/StreamTableJoinOperatorSpec.java | 67 ++++ .../org/apache/samza/table/TableManager.java | 153 +++++++++ .../samza/container/SamzaContainer.scala | 75 ++--- .../apache/samza/container/TaskInstance.scala | 40 ++- .../samza/config/TestJavaTableConfig.java | 58 ++++ .../operators/TestMessageStreamImpl.java | 68 +++- .../samza/operators/TestStreamGraphImpl.java | 25 +- .../impl/TestStreamTableJoinOperatorImpl.java | 101 ++++++ .../apache/samza/table/TestTableManager.java | 176 ++++++++++ .../apache/samza/task/TestAsyncRunLoop.java | 14 +- .../kv/inmemory/InMemoryTableDescriptor.java | 59 ++++ .../kv/inmemory/InMemoryTableProvider.java | 65 ++++ .../InMemoryTableProviderFactory.java | 33 ++ .../inmemory/TestInMemoryTableDescriptor.java | 48 +++ .../inmemory/TestInMemoryTableProvider.java | 65 ++++ .../storage/kv/RocksDbTableDescriptor.java | 232 +++++++++++++ .../storage/kv/RocksDbTableProvider.java | 64 ++++ .../kv/RocksDbTableProviderFactory.java | 31 ++ .../kv/TestRocksDbTableDescriptor.java | 87 +++++ .../storage/kv/TestRocksDbTableProvider.java | 66 ++++ .../BaseLocalStoreBackedTableDescriptor.java | 56 ++++ .../kv/BaseLocalStoreBackedTableProvider.java | 92 ++++++ .../kv/LocalStoreBackedReadWriteTable.java | 68 ++++ .../kv/LocalStoreBackedReadableTable.java | 61 ++++ ...TestLocalBaseStoreBackedTableProvider.java | 77 +++++ .../samza/test/table/TestLocalTable.java | 304 ++++++++++++++++++ .../samza/test/table/TestTableData.java | 200 ++++++++++++ .../samza/test/util/ArraySystemConsumer.java | 4 +- .../samza/test/util/SimpleSystemAdmin.java | 26 +- 59 files changed, 3668 insertions(+), 128 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/ReadWriteTable.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/ReadableTable.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/Table.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/TableProvider.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java create mode 100644 samza-api/src/main/java/org/apache/samza/table/TableSpec.java create mode 100644 samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java create mode 100644 samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java create mode 100644 samza-core/src/main/java/org/apache/samza/operators/TableImpl.java create mode 100644 samza-core/src/main/java/org/apache/samza/operators/impl/SendToTableOperatorImpl.java create mode 100644 samza-core/src/main/java/org/apache/samza/operators/impl/StreamTableJoinOperatorImpl.java create mode 100644 samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java create mode 100644 samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java create mode 100644 samza-core/src/main/java/org/apache/samza/table/TableManager.java create mode 100644 samza-core/src/test/java/org/apache/samza/config/TestJavaTableConfig.java create mode 100644 samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamTableJoinOperatorImpl.java create mode 100644 samza-core/src/test/java/org/apache/samza/table/TestTableManager.java create mode 100644 samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java create mode 100644 samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java create mode 100644 samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java create mode 100644 samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java create mode 100644 samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java create mode 100644 samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java create mode 100644 samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java create mode 100644 samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java create mode 100644 samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java create mode 100644 samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java create mode 100644 samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java create mode 100644 samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java create mode 100644 samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java create mode 100644 samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java create mode 100644 samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java create mode 100644 samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java create mode 100644 samza-test/src/test/java/org/apache/samza/test/table/TestTableData.java diff --git a/build.gradle b/build.gradle index 50cc5e0bdc..be1baf7b8d 100644 --- a/build.gradle +++ b/build.gradle @@ -565,6 +565,15 @@ project(":samza-kv_$scalaVersion") { project(":samza-kv-inmemory_$scalaVersion") { apply plugin: 'scala' + // Force scala joint compilation + sourceSets.main.scala.srcDir "src/main/java" + sourceSets.test.scala.srcDir "src/test/java" + + // Disable the Javac compiler by forcing joint compilation by scalac. This is equivalent to setting + // tasks.compileTestJava.enabled = false + sourceSets.main.java.srcDirs = [] + sourceSets.test.java.srcDirs = [] + dependencies { compile project(':samza-api') compile project(":samza-core_$scalaVersion") diff --git a/samza-api/src/main/java/org/apache/samza/operators/KV.java b/samza-api/src/main/java/org/apache/samza/operators/KV.java index 0bed3b9cd5..824bcb49f1 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/KV.java +++ b/samza-api/src/main/java/org/apache/samza/operators/KV.java @@ -25,7 +25,7 @@ * @param type of the key * @param type of the value */ -public class KV { +public final class KV { public final K key; public final V value; diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index acb2f33fc6..f0a5526a2b 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -18,21 +18,23 @@ */ package org.apache.samza.operators; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.function.Function; + import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.functions.StreamTableJoinFunction; import org.apache.samza.operators.windows.Window; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.function.Function; +import org.apache.samza.table.Table; /** @@ -153,6 +155,34 @@ MessageStream join(MessageStream otherStream, Serde keySerde, Serde messageSerde, Serde otherMessageSerde, Duration ttl, String id); + /** + * Joins this {@link MessageStream} with another {@link Table} using the provided + * pairwise {@link StreamTableJoinFunction}. + *

+ * The type of input message is expected to be {@link KV}. + *

+ * Records are looked up from the joined table using the join key, join function + * is applied and join results are emitted as matches are found. + *

+ * The join function allows implementation of both inner and left outer join. A null will be + * passed to the join function, if no record matching the join key is found in the table. + * The join function can choose to return an instance of JM (outer left join) or null + * (inner join); if null is returned, it won't be processed further. + *

+ * Both the input stream and table being joined must have the same number of partitions, + * and should be partitioned by the same join key. + *

+ * + * @param table the table being joined + * @param joinFn the join function + * @param the type of join key + * @param the type of table record + * @param the type of messages resulting from the {@code joinFn} + * @return the joined {@link MessageStream} + */ + MessageStream join(Table table, + StreamTableJoinFunction joinFn); + /** * Merges all {@code otherStreams} with this {@link MessageStream}. *

@@ -235,4 +265,15 @@ MessageStream> partitionBy(Function keyE */ MessageStream> partitionBy(Function keyExtractor, Function valueExtractor, String id); + + /** + * Sends messages in this {@link MessageStream} to a {@link Table}. The type of input message is expected + * to be {@link KV}, otherwise a {@link ClassCastException} will be thrown. + * + * @param table the table to write messages to + * @param the type of key in the table + * @param the type of record value in the table + */ + void sendTo(Table> table); + } diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java index 49306316db..6871bc76d4 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java @@ -20,6 +20,7 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; /** @@ -120,6 +121,20 @@ public interface StreamGraph { */ OutputStream getOutputStream(String streamId); + /** + * Gets the {@link Table} corresponding to the {@link TableDescriptor}. + *

+ * Multiple invocations of this method with the same {@link TableDescriptor} will throw an + * {@link IllegalStateException}. + * + * @param tableDesc the {@link TableDescriptor} + * @param the type of the key + * @param the type of the value + * @return the {@link Table} corresponding to the {@code tableDesc} + * @throws IllegalStateException when invoked multiple times with the same {@link TableDescriptor} + */ + Table> getTable(TableDescriptor tableDesc); + /** * Sets the {@link ContextManager} for this {@link StreamGraph}. *

diff --git a/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java new file mode 100644 index 0000000000..a60b6a9ecb --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.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.samza.operators; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.serializers.KVSerde; + +/** + * User facing class to collect metadata that fully describes a + * Samza table. This interface should be implemented by concrete table implementations. + *

+ * Typical user code should look like the following, notice withConfig() + * is defined in this class and the rest in subclasses. + * + *

+ * {@code
+ * TableDescriptor tableDesc = new RocksDbTableDescriptor("tbl")
+ *     .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde("UTF-8")))
+ *     .withBlockSize(1024)
+ *     .withConfig("some-key", "some-value");
+ * }
+ * 
+ + * Once constructed, a table descriptor can be registered with the system. Internally, + * the table descriptor is then converted to a {@link org.apache.samza.table.TableSpec}, + * which is used to track tables internally. + * + * @param the type of the key in this table + * @param the type of the value in this table + * @param the type of the concrete table descriptor + */ +@InterfaceStability.Unstable +public interface TableDescriptor> { + + /** + * Get the Id of the table + * @return Id of the table + */ + String getTableId(); + + /** + * Set the Serde for this table + * @param serde the serde + * @return this table descriptor instance + * @throws IllegalArgumentException if null is provided + */ + D withSerde(KVSerde serde); + + /** + * Add a configuration entry for the table + * @param key the key + * @param value the value + * @return this table descriptor instance + */ + D withConfig(String key, String value); + +} \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java new file mode 100644 index 0000000000..6afcf67ef1 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.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.samza.operators.functions; + +import org.apache.samza.annotation.InterfaceStability; + + +/** + * Joins incoming messages with records from a table by the join key. + * + * @param the type of join key + * @param type of input message + * @param type of the table record + * @param type of join results + */ +@InterfaceStability.Unstable +public interface StreamTableJoinFunction extends InitableFunction, ClosableFunction { + + /** + * Joins the provided messages and table record, returns the joined message. + * + * @param message the input message + * @param record the table record value + * @return the join result + */ + JM apply(M message, R record); + + /** + * Retrieve the join key from incoming messages + * + * @param message incoming message + * @return the join key + */ + K getMessageKey(M message); + + /** + * Retrieve the join key from table record + * + * @param record table record + * @return the join key + */ + K getRecordKey(R record); +} diff --git a/samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java b/samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java new file mode 100644 index 0000000000..21630ab10f --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java @@ -0,0 +1,37 @@ +/* + * 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.samza.table; + +import org.apache.samza.storage.StorageEngine; + + +/** + * Interface for tables backed by Samza local stores. The backing stores are + * injected during initialization of the table. Since the lifecycle + * of the underlying stores are already managed by Samza container, + * the table provider will not manage the lifecycle of the backing + * stores. + */ +public interface LocalStoreBackedTableProvider extends TableProvider { + /** + * Initializes the table provider with the backing store + * @param store the backing store + */ + void init(StorageEngine store); +} diff --git a/samza-api/src/main/java/org/apache/samza/table/ReadWriteTable.java b/samza-api/src/main/java/org/apache/samza/table/ReadWriteTable.java new file mode 100644 index 0000000000..d617153cbf --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/ReadWriteTable.java @@ -0,0 +1,75 @@ +/* + * 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.samza.table; + +import java.util.List; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.storage.kv.Entry; + +/** + * + * A table that supports get, put and delete by one or more keys + * + * @param the type of the key in this table + * @param the type of the value in this table + */ +@InterfaceStability.Unstable +public interface ReadWriteTable extends ReadableTable { + + /** + * Updates the mapping of the specified key-value pair; Associates the specified {@code key} with the specified {@code value}. + * + * @param key the key with which the specified {@code value} is to be associated. + * @param value the value with which the specified {@code key} is to be associated. + * @throws NullPointerException if the specified {@code key} or {@code value} is {@code null}. + */ + void put(K key, V value); + + /** + * Updates the mappings of the specified key-value {@code entries}. + * + * @param entries the updated mappings to put into this table. + * @throws NullPointerException if any of the specified {@code entries} has {@code null} as key or value. + */ + void putAll(List> entries); + + /** + * Deletes the mapping for the specified {@code key} from this table (if such mapping exists). + * + * @param key the key for which the mapping is to be deleted. + * @throws NullPointerException if the specified {@code key} is {@code null}. + */ + void delete(K key); + + /** + * Deletes the mappings for the specified {@code keys} from this table. + * + * @param keys the keys for which the mappings are to be deleted. + * @throws NullPointerException if the specified {@code keys} list, or any of the keys, is {@code null}. + */ + void deleteAll(List keys); + + + /** + * Flushes the underlying store of this table, if applicable. + */ + void flush(); + +} diff --git a/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java b/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java new file mode 100644 index 0000000000..5ad6e0f627 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.table; + +import java.util.List; +import java.util.Map; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.KV; + + +/** + * + * A table that supports get by one or more keys + * + * @param the type of the record key in this table + * @param the type of the record value in this table + */ +@InterfaceStability.Unstable +public interface ReadableTable extends Table> { + + /** + * Gets the value associated with the specified {@code key}. + * + * @param key the key with which the associated value is to be fetched. + * @return if found, the value associated with the specified {@code key}; otherwise, {@code null}. + * @throws NullPointerException if the specified {@code key} is {@code null}. + */ + V get(K key); + + /** + * Gets the values with which the specified {@code keys} are associated. + * + * @param keys the keys with which the associated values are to be fetched. + * @return a map of the keys that were found and their respective values. + * @throws NullPointerException if the specified {@code keys} list, or any of the keys, is {@code null}. + */ + Map getAll(List keys); + + /** + * Close the table and release any resources acquired + */ + void close(); + +} diff --git a/samza-api/src/main/java/org/apache/samza/table/Table.java b/samza-api/src/main/java/org/apache/samza/table/Table.java new file mode 100644 index 0000000000..767e17697f --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/Table.java @@ -0,0 +1,31 @@ +/* + * 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.samza.table; + +import org.apache.samza.annotation.InterfaceStability; + +/** + * + * Marker interface for a table. + * + * @param the type of records in the table + */ +@InterfaceStability.Unstable +public interface Table { +} diff --git a/samza-api/src/main/java/org/apache/samza/table/TableProvider.java b/samza-api/src/main/java/org/apache/samza/table/TableProvider.java new file mode 100644 index 0000000000..54c6f5d088 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/TableProvider.java @@ -0,0 +1,57 @@ +/* + * 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.samza.table; + +import java.util.Map; + +import org.apache.samza.annotation.InterfaceStability; + + +/** + * A table provider provides the implementation for a table. It ensures a table is + * properly constructed and also manages its lifecycle. + */ +@InterfaceStability.Unstable +public interface TableProvider { + /** + * Get an instance of the table for read/write operations + * @return the underlying table + */ + Table getTable(); + + /** + * Generate any configuration for this table, the generated configuration + * is used by Samza container to construct this table and any components + * necessary + * . + * @param config the current configuration + * @return configuration for this table + */ + Map generateConfig(Map config); + + /** + * Start the underlying table + */ + void start(); + + /** + * Stop the underlying table + */ + void stop(); +} diff --git a/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java b/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java new file mode 100644 index 0000000000..1bb0196717 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/TableProviderFactory.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.table; + +import org.apache.samza.annotation.InterfaceStability; + + +/** + * Factory of a table provider object + */ +@InterfaceStability.Unstable +public interface TableProviderFactory { + /** + * Constructs an instances of the table provider based on a given table spec + * @param tableSpec the table spec + * @return the table provider + */ + TableProvider getTableProvider(TableSpec tableSpec); +} diff --git a/samza-api/src/main/java/org/apache/samza/table/TableSpec.java b/samza-api/src/main/java/org/apache/samza/table/TableSpec.java new file mode 100644 index 0000000000..68043f98d8 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/table/TableSpec.java @@ -0,0 +1,125 @@ +/* + * 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.samza.table; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.serializers.KVSerde; + + +/** + * TableSpec is a blueprint for creating, validating, or simply describing a table in the runtime environment. + * + * It is typically created indirectly by constructing an instance of {@link org.apache.samza.operators.TableDescriptor}, + * and then invoke BaseTableDescriptor.getTableSpec(). + * + * It has specific attributes for common behaviors that Samza uses. + * + * It has the table provider factory, which provides the actual table implementation. + * + * It also includes a map of configurations which may be implementation-specific. + * + * It is immutable by design. + */ +@InterfaceStability.Unstable +public class TableSpec { + + private final String id; + private final KVSerde serde; + private final String tableProviderFactoryClassName; + private final Map config = new HashMap<>(); + + /** + * Default constructor + */ + public TableSpec() { + this.id = null; + this.serde = null; + this.tableProviderFactoryClassName = null; + } + + /** + * Constructs a {@link TableSpec} + * + * @param tableId Id of the table + * @param tableProviderFactoryClassName table provider factory + * @param serde the serde + * @param config implementation specific configuration + */ + public TableSpec(String tableId, KVSerde serde, String tableProviderFactoryClassName, + Map config) { + this.id = tableId; + this.serde = serde; + this.tableProviderFactoryClassName = tableProviderFactoryClassName; + this.config.putAll(config); + } + + /** + * Get the Id of the table + * @return Id of the table + */ + public String getId() { + return id; + } + + /** + * Get the serde + * @param the type of the key + * @param the type of the value + * @return the key serde + */ + public KVSerde getSerde() { + return serde; + } + + /** + * Get the class name of the table provider factory + * @return class name of the table provider factory + */ + public String getTableProviderFactoryClassName() { + return tableProviderFactoryClassName; + } + + /** + * Get implementation configuration for the table + * @return configuration for the table + */ + public Map getConfig() { + return Collections.unmodifiableMap(config); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || !getClass().equals(o.getClass())) { + return false; + } + return id.equals(((TableSpec) o).id); + } + + @Override + public int hashCode() { + return id.hashCode(); + } +} diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java index 4ef3d301b3..11ffacc35f 100644 --- a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java +++ b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java @@ -19,12 +19,14 @@ package org.apache.samza.task; +import java.util.Set; + import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.container.TaskName; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.table.Table; -import java.util.Set; /** * A TaskContext provides resources about the {@link org.apache.samza.task.StreamTask}, particularly during @@ -37,6 +39,8 @@ public interface TaskContext { Object getStore(String name); + Table getTable(String tableId); + TaskName getTaskName(); SamzaContainerContext getSamzaContainerContext(); diff --git a/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java b/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.java new file mode 100644 index 0000000000..6cc39869e9 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/config/JavaTableConfig.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.samza.config; + +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A helper class for handling table configuration + */ +public class JavaTableConfig extends MapConfig { + + // Prefix + public static final String TABLES_PREFIX = "tables."; + public static final String TABLE_ID_PREFIX = TABLES_PREFIX + "%s"; + + // Suffix + public static final String TABLE_PROVIDER_FACTORY_SUFFIX = ".provider.factory"; + + // Config keys + public static final String TABLE_PROVIDER_FACTORY = String.format("%s.provider.factory", TABLE_ID_PREFIX); + public static final String TABLE_KEY_SERDE = String.format("%s.key.serde", TABLE_ID_PREFIX); + public static final String TABLE_VALUE_SERDE = String.format("%s.value.serde", TABLE_ID_PREFIX); + + + public JavaTableConfig(Config config) { + super(config); + } + + /** + * Get Id's of all tables + * @return list of table Id's + */ + public List getTableIds() { + Config subConfig = subset(TABLES_PREFIX, true); + Set tableNames = subConfig.keySet().stream() + .filter(k -> k.endsWith(TABLE_PROVIDER_FACTORY_SUFFIX)) + .map(k -> k.substring(0, k.indexOf("."))) + .collect(Collectors.toSet()); + return new LinkedList<>(tableNames); + } + + /** + * Get the {@link org.apache.samza.table.TableProviderFactory} class for a table + * @param tableId Id of the table + * @return the {@link org.apache.samza.table.TableProviderFactory} class name + */ + public String getTableProviderFactory(String tableId) { + return get(String.format(TABLE_PROVIDER_FACTORY, tableId), null); + } + + /** + * Get registry keys of key serde for this table + * @param tableId Id of the table + * @return serde retistry key + */ + public String getKeySerde(String tableId) { + return get(String.format(TABLE_KEY_SERDE, tableId), null); + } + + /** + * Get registry keys of value serde for this table + * @param tableId Id of the table + * @return serde retistry key + */ + public String getValueSerde(String tableId) { + return get(String.format(TABLE_VALUE_SERDE, tableId), null); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java b/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java index aa622a362f..0248486431 100644 --- a/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java +++ b/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java @@ -19,7 +19,10 @@ package org.apache.samza.container; -import com.google.common.collect.ImmutableSet; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + import org.apache.samza.checkpoint.OffsetManager; import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.ReadableMetricsRegistry; @@ -27,13 +30,13 @@ import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableManager; import org.apache.samza.task.TaskContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; +import com.google.common.collect.ImmutableSet; public class TaskContextImpl implements TaskContext { private static final Logger LOG = LoggerFactory.getLogger(TaskContextImpl.class); @@ -44,6 +47,7 @@ public class TaskContextImpl implements TaskContext { private final Set systemStreamPartitions; private final OffsetManager offsetManager; private final TaskStorageManager storageManager; + private final TableManager tableManager; private final JobModel jobModel; private final StreamMetadataCache streamMetadataCache; private final Map objectRegistry = new HashMap<>(); @@ -56,6 +60,7 @@ public TaskContextImpl(TaskName taskName, Set systemStreamPartitions, OffsetManager offsetManager, TaskStorageManager storageManager, + TableManager tableManager, JobModel jobModel, StreamMetadataCache streamMetadataCache) { this.taskName = taskName; @@ -64,6 +69,7 @@ public TaskContextImpl(TaskName taskName, this.systemStreamPartitions = ImmutableSet.copyOf(systemStreamPartitions); this.offsetManager = offsetManager; this.storageManager = storageManager; + this.tableManager = tableManager; this.jobModel = jobModel; this.streamMetadataCache = streamMetadataCache; } @@ -88,6 +94,16 @@ public KeyValueStore getStore(String storeName) { } } + @Override + public Table getTable(String tableId) { + if (tableManager != null) { + return tableManager.getTable(tableId); + } else { + LOG.warn("No table manager found"); + return null; + } + } + @Override public TaskName getTaskName() { return taskName; diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 468aab97f0..e2c122a426 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -39,6 +39,7 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; +import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,6 +96,7 @@ private void validateConfig() { Set sourceStreams = new HashSet<>(streamGraph.getInputOperators().keySet()); Set sinkStreams = new HashSet<>(streamGraph.getOutputStreams().keySet()); Set intStreams = new HashSet<>(sourceStreams); + Set tables = new HashSet<>(streamGraph.getTables().keySet()); intStreams.retainAll(sinkStreams); sourceStreams.removeAll(intStreams); sinkStreams.removeAll(intStreams); @@ -113,6 +115,9 @@ private void validateConfig() { // add intermediate streams intStreams.forEach(spec -> jobGraph.addIntermediateStream(spec, node, node)); + // add tables + tables.forEach(spec -> jobGraph.addTable(spec, node)); + jobGraph.validate(); return jobGraph; diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 2a09e90833..4a09260387 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -30,11 +30,13 @@ import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; + import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.StreamGraphImpl; import org.apache.samza.system.StreamSpec; +import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,6 +57,7 @@ private final Set sources = new HashSet<>(); private final Set sinks = new HashSet<>(); private final Set intermediateStreams = new HashSet<>(); + private final Set tables = new HashSet<>(); private final Config config; private final JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); @@ -86,6 +89,11 @@ public List getIntermediateStreams() { .collect(Collectors.toList()); } + void addTable(TableSpec tableSpec, JobNode node) { + tables.add(tableSpec); + node.addTable(tableSpec); + } + @Override public String getPlanAsJson() throws Exception { return jsonGenerator.toJson(this); @@ -210,6 +218,14 @@ Set getSinks() { return Collections.unmodifiableSet(sinks); } + /** + * Return the tables in the graph + * @return unmodifiable set of {@link TableSpec} + */ + Set getTables() { + return Collections.unmodifiableSet(tables); + } + /** * Return the intermediate streams in the graph * @return unmodifiable set of {@link StreamEdge} diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 03845e3854..2729fa3598 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -28,12 +28,15 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; + import org.apache.samza.config.ApplicationConfig; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputOperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; +import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; +import org.apache.samza.table.TableSpec; import org.codehaus.jackson.annotate.JsonProperty; import org.codehaus.jackson.map.ObjectMapper; @@ -54,6 +57,15 @@ static final class StreamSpecJson { int partitionCount; } + static final class TableSpecJson { + @JsonProperty("id") + String id; + @JsonProperty("tableProviderFactory") + String tableProviderFactory; + @JsonProperty("config") + Map config; + } + static final class StreamEdgeJson { @JsonProperty("streamSpec") StreamSpecJson streamSpec; @@ -97,6 +109,8 @@ static final class JobGraphJson { Map sinkStreams; @JsonProperty("intermediateStreams") Map intermediateStreams; + @JsonProperty("tables") + Map tables; @JsonProperty("applicationName") String applicationName; @JsonProperty("applicationId") @@ -119,9 +133,11 @@ static final class JobGraphJson { jobGraphJson.sourceStreams = new HashMap<>(); jobGraphJson.sinkStreams = new HashMap<>(); jobGraphJson.intermediateStreams = new HashMap<>(); + jobGraphJson.tables = new HashMap<>(); jobGraph.getSources().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.sourceStreams)); jobGraph.getSinks().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.sinkStreams)); jobGraph.getIntermediateStreamEdges().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.intermediateStreams)); + jobGraph.getTables().forEach(t -> buildTableJson(t, jobGraphJson.tables)); jobGraphJson.jobs = jobGraph.getJobNodes().stream() .map(jobNode -> buildJobNodeJson(jobNode)) @@ -206,6 +222,11 @@ private Map operatorToMap(OperatorSpec spec) { map.put("outputStreamId", outputStream.getStreamSpec().getId()); } + if (spec instanceof StreamTableJoinOperatorSpec) { + TableSpec tableSpec = ((StreamTableJoinOperatorSpec) spec).getTableSpec(); + map.put("tableId", tableSpec.getId()); + } + if (spec instanceof JoinOperatorSpec) { map.put("ttlMs", ((JoinOperatorSpec) spec).getTtlMs()); } @@ -247,4 +268,33 @@ private StreamEdgeJson buildStreamEdgeJson(StreamEdge edge, Map tableSpecs) { + String tableId = tableSpec.getId(); + TableSpecJson tableSpecJson = tableSpecs.get(tableId); + if (tableSpecJson == null) { + tableSpecJson = buildTableJson(tableSpec); + tableSpecs.put(tableId, tableSpecJson); + } + return tableSpecJson; + } + + /** + * Create the JSON POJO for a {@link TableSpec} + * @param tableSpec the {@link TableSpec} + * @return JSON representation of the {@link TableSpec} + */ + private TableSpecJson buildTableJson(TableSpec tableSpec) { + TableSpecJson tableSpecJson = new TableSpecJson(); + tableSpecJson.id = tableSpec.getId(); + tableSpecJson.tableProviderFactory = tableSpec.getTableProviderFactoryClassName(); + tableSpecJson.config = tableSpec.getConfig(); + return tableSpecJson; + } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index 2e89292829..4e337d9637 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -19,7 +19,6 @@ package org.apache.samza.execution; -import com.google.common.base.Joiner; import java.util.ArrayList; import java.util.Base64; import java.util.Collection; @@ -29,7 +28,9 @@ import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; + import org.apache.samza.config.Config; +import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.SerializerConfig; @@ -47,10 +48,16 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerializableSerde; import org.apache.samza.system.StreamSpec; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableProviderFactory; +import org.apache.samza.table.TableSpec; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Joiner; + + /** * A JobNode is a physical execution unit. In RemoteExecutionEnvironment, it's a job that will be submitted * to remote cluster. In LocalExecutionEnvironment, it's a set of StreamProcessors for local execution. @@ -67,6 +74,7 @@ public class JobNode { private final StreamGraphImpl streamGraph; private final List inEdges = new ArrayList<>(); private final List outEdges = new ArrayList<>(); + private final List tables = new ArrayList<>(); private final Config config; JobNode(String jobName, String jobId, StreamGraphImpl streamGraph, Config config) { @@ -109,6 +117,10 @@ List getOutEdges() { return outEdges; } + void addTable(TableSpec tableSpec) { + tables.add(tableSpec); + } + /** * Generate the configs for a job * @param executionPlanJson JSON representation of the execution plan @@ -147,6 +159,19 @@ public JobConfig generateConfig(String executionPlanJson) { // write serialized serde instances and stream serde configs to configs addSerdeConfigs(configs); + tables.forEach(tableSpec -> { + // Table provider factory + configs.put(String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, tableSpec.getId()), + tableSpec.getTableProviderFactoryClassName()); + + // Note: no need to generate config for Serde's, as they are already produced by addSerdeConfigs() + + // Generate additional configuration + TableProviderFactory tableProviderFactory = Util.getObj(tableSpec.getTableProviderFactoryClassName()); + TableProvider tableProvider = tableProviderFactory.getTableProvider(tableSpec); + configs.putAll(tableProvider.generateConfig(configs)); + }); + log.info("Job {} has generated configs {}", jobName, configs); String configPrefix = String.format(CONFIG_JOB_PREFIX, jobName); @@ -209,11 +234,21 @@ void addSerdeConfigs(Map configs) { } }); + // collect all key and msg serde instances for tables + Map tableKeySerdes = new HashMap<>(); + Map tableValueSerdes = new HashMap<>(); + tables.forEach(tableSpec -> { + tableKeySerdes.put(tableSpec.getId(), tableSpec.getSerde().getKeySerde()); + tableValueSerdes.put(tableSpec.getId(), tableSpec.getSerde().getValueSerde()); + }); + // for each unique stream or store serde instance, generate a unique name and serialize to config HashSet serdes = new HashSet<>(streamKeySerdes.values()); serdes.addAll(streamMsgSerdes.values()); serdes.addAll(storeKeySerdes.values()); serdes.addAll(storeMsgSerdes.values()); + serdes.addAll(tableKeySerdes.values()); + serdes.addAll(tableValueSerdes.values()); SerializableSerde serializableSerde = new SerializableSerde<>(); Base64.Encoder base64Encoder = Base64.getEncoder(); Map serdeUUIDs = new HashMap<>(); @@ -247,6 +282,17 @@ void addSerdeConfigs(Map configs) { String msgSerdeConfigKey = String.format(StorageConfig.MSG_SERDE(), storeName); configs.put(msgSerdeConfigKey, serdeUUIDs.get(serde)); }); + + // set key and msg serdes for tables to the serde names generated above + tableKeySerdes.forEach((tableId, serde) -> { + String keySerdeConfigKey = String.format(JavaTableConfig.TABLE_KEY_SERDE, tableId); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); + + tableValueSerdes.forEach((tableId, serde) -> { + String valueSerdeConfigKey = String.format(JavaTableConfig.TABLE_VALUE_SERDE, tableId); + configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); + }); } /** @@ -264,10 +310,14 @@ private long computeTriggerInterval() { // Filter out the join operators, and obtain a list of their ttl values List joinTtlIntervals = operatorSpecs.stream() - .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.JOIN) + .filter(spec -> spec instanceof JoinOperatorSpec) .map(spec -> ((JoinOperatorSpec) spec).getTtlMs()) .collect(Collectors.toList()); + if (joinTtlIntervals.isEmpty()) { + return -1; + } + // Combine both the above lists List candidateTimerIntervals = new ArrayList<>(joinTtlIntervals); candidateTimerIntervals.addAll(windowTimerIntervals); diff --git a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java new file mode 100644 index 0000000000..b875c2e52a --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.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.samza.operators; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.NoOpSerde; +import org.apache.samza.table.TableSpec; + + +/** + * Base class for all table descriptor implementations. + * + * @param the type of the key in this table + * @param the type of the value in this table + * @param the type of the concrete table descriptor + */ +abstract public class BaseTableDescriptor> + implements TableDescriptor { + + protected final String tableId; + + protected KVSerde serde = KVSerde.of(new NoOpSerde(), new NoOpSerde()); + + protected final Map config = new HashMap<>(); + + /** + * Constructs a table descriptor instance + * @param tableId Id of the table + */ + protected BaseTableDescriptor(String tableId) { + this.tableId = tableId; + } + + @Override + public D withConfig(String key, String value) { + config.put(key, value); + return (D) this; + } + + @Override + public D withSerde(KVSerde serde) { + if (serde == null) { + throw new IllegalArgumentException("Serde cannot be null"); + } + this.serde = serde; + return (D) this; + } + + @Override + public String getTableId() { + return tableId; + } + + /** + * Generate config for {@link TableSpec}; this method is used internally. + * @param tableSpecConfig configuration for the {@link TableSpec} + */ + protected void generateTableSpecConfig(Map tableSpecConfig) { + tableSpecConfig.putAll(config); + } + + /** + * Validate that this table descriptor is constructed properly; this method is used internally. + */ + protected void validate() { + } + + /** + * Create a {@link TableSpec} from this table descriptor; this method is used internally. + * + * @return the {@link TableSpec} + */ + abstract public TableSpec getTableSpec(); +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index 3f4e40d45e..07af54f6ac 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -19,12 +19,17 @@ package org.apache.samza.operators; +import java.time.Duration; +import java.util.Collection; +import java.util.function.Function; + import org.apache.samza.SamzaException; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.functions.StreamTableJoinFunction; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OperatorSpec.OpCode; @@ -32,18 +37,18 @@ import org.apache.samza.operators.spec.OutputOperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.spec.PartitionByOperatorSpec; +import org.apache.samza.operators.spec.SendToTableOperatorSpec; import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; +import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; import org.apache.samza.operators.windows.Window; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.internal.WindowInternal; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; - -import java.time.Duration; -import java.util.Collection; -import java.util.function.Function; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableSpec; /** @@ -137,6 +142,16 @@ public MessageStream join(MessageStream otherStream, return new MessageStreamImpl<>(this.graph, joinOpSpec); } + @Override + public MessageStream join(Table table, + StreamTableJoinFunction joinFn) { + TableSpec tableSpec = ((TableImpl) table).getTableSpec(); + StreamTableJoinOperatorSpec joinOpSpec = OperatorSpecs.createStreamTableJoinOperatorSpec( + tableSpec, (StreamTableJoinFunction) joinFn, this.graph.getNextOpId(OpCode.JOIN)); + this.operatorSpec.registerNextOperatorSpec(joinOpSpec); + return new MessageStreamImpl<>(this.graph, joinOpSpec); + } + @Override public MessageStream merge(Collection> otherStreams) { if (otherStreams.isEmpty()) return this; @@ -176,4 +191,12 @@ public MessageStream> partitionBy(Function getOperatorSpec() { return this.operatorSpec; } + + @Override + public void sendTo(Table> table) { + SendToTableOperatorSpec op = OperatorSpecs.createSendToTableOperatorSpec( + this.operatorSpec, ((TableImpl) table).getTableSpec(), this.graph.getNextOpId(OpCode.SEND_TO)); + this.operatorSpec.registerNextOperatorSpec(op); + } + } diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java index d014cb9419..b607c62f93 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java @@ -18,7 +18,15 @@ */ package org.apache.samza.operators; -import com.google.common.base.Preconditions; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; @@ -34,17 +42,12 @@ import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.system.StreamSpec; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; -import java.util.regex.Pattern; -import java.util.stream.Collectors; +import com.google.common.base.Preconditions; /** * A {@link StreamGraph} that provides APIs for accessing {@link MessageStream}s to be used to @@ -57,6 +60,7 @@ public class StreamGraphImpl implements StreamGraph { // We use a LHM for deterministic order in initializing and closing operators. private final Map inputOperators = new LinkedHashMap<>(); private final Map outputStreams = new LinkedHashMap<>(); + private final Map tables = new LinkedHashMap<>(); private final ApplicationRunner runner; private final Config config; @@ -145,6 +149,18 @@ public OutputStream getOutputStream(String streamId) { return (OutputStream) getOutputStream(streamId, defaultSerde); } + @Override + public Table> getTable(TableDescriptor tableDesc) { + TableSpec tableSpec = ((BaseTableDescriptor) tableDesc).getTableSpec(); + if (tables.containsKey(tableSpec)) { + throw new IllegalStateException(String.format( + "getTable() invoked multiple times with the same tableId: %s", + tableDesc.getTableId())); + } + tables.put(tableSpec, new TableImpl(tableSpec)); + return tables.get(tableSpec); + } + @Override public StreamGraph withContextManager(ContextManager contextManager) { this.contextManager = contextManager; @@ -163,7 +179,7 @@ public StreamGraph withContextManager(ContextManager contextManager) { */ IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serde serde) { StreamSpec streamSpec = runner.getStreamSpec(streamId); - + Preconditions.checkState(!inputOperators.containsKey(streamSpec) && !outputStreams.containsKey(streamSpec), "getIntermediateStream must not be called multiple times with the same streamId: " + streamId); @@ -190,6 +206,10 @@ public Map getOutputStreams() { return Collections.unmodifiableMap(outputStreams); } + public Map getTables() { + return Collections.unmodifiableMap(tables); + } + public ContextManager getContextManager() { return this.contextManager; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java b/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java new file mode 100644 index 0000000000..e671534c4b --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/operators/TableImpl.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.samza.operators; + +import org.apache.samza.table.Table; +import org.apache.samza.table.TableSpec; + + +/** + * This class is the holder of a {@link TableSpec} + */ +public class TableImpl implements Table { + + private final TableSpec tableSpec; + + public TableImpl(TableSpec tableSpec) { + this.tableSpec = tableSpec; + } + + public TableSpec getTableSpec() { + return tableSpec; + } + +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index 0bb12d23fd..ea278c16f4 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -18,10 +18,15 @@ */ package org.apache.samza.operators.impl; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; import java.util.stream.Collectors; + import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.job.model.JobModel; @@ -29,6 +34,7 @@ import org.apache.samza.operators.StreamGraphImpl; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.PartialJoinFunction; +import org.apache.samza.operators.impl.store.TimestampedValue; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; @@ -36,8 +42,9 @@ import org.apache.samza.operators.spec.PartitionByOperatorSpec; import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; +import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.operators.spec.SendToTableOperatorSpec; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.system.SystemStream; import org.apache.samza.task.TaskContext; @@ -45,14 +52,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; /** * The DAG of {@link OperatorImpl}s corresponding to the DAG of {@link OperatorSpec}s. @@ -212,6 +214,10 @@ OperatorImpl createOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec oper return new WindowOperatorImpl((WindowOperatorSpec) operatorSpec, clock); } else if (operatorSpec instanceof JoinOperatorSpec) { return createPartialJoinOperatorImpl(prevOperatorSpec, (JoinOperatorSpec) operatorSpec, config, context, clock); + } else if (operatorSpec instanceof StreamTableJoinOperatorSpec) { + return new StreamTableJoinOperatorImpl((StreamTableJoinOperatorSpec) operatorSpec, config, context); + } else if (operatorSpec instanceof SendToTableOperatorSpec) { + return new SendToTableOperatorImpl((SendToTableOperatorSpec) operatorSpec, config, context); } throw new IllegalArgumentException( String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName())); diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/SendToTableOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/SendToTableOperatorImpl.java new file mode 100644 index 0000000000..5ce13286c3 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/SendToTableOperatorImpl.java @@ -0,0 +1,71 @@ +/* + * 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.samza.operators.impl; + +import java.util.Collection; +import java.util.Collections; + +import org.apache.samza.config.Config; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.operators.spec.SendToTableOperatorSpec; +import org.apache.samza.table.ReadWriteTable; +import org.apache.samza.task.MessageCollector; +import org.apache.samza.task.TaskContext; +import org.apache.samza.task.TaskCoordinator; + + +/** + * Implementation of a send-stream-to-table operator that stores the record + * in the table. + * + * @param the type of the record key + * @param the type of the record value + */ +public class SendToTableOperatorImpl extends OperatorImpl, Void> { + + private final SendToTableOperatorSpec sendToTableOpSpec; + private final ReadWriteTable table; + + SendToTableOperatorImpl(SendToTableOperatorSpec sendToTableOpSpec, Config config, TaskContext context) { + this.sendToTableOpSpec = sendToTableOpSpec; + this.table = (ReadWriteTable) context.getTable(sendToTableOpSpec.getTableSpec().getId()); + } + + @Override + protected void handleInit(Config config, TaskContext context) { + } + + @Override + protected Collection handleMessage(KV message, MessageCollector collector, TaskCoordinator coordinator) { + table.put(message.getKey(), message.getValue()); + // there should be no further chained operators since this is a terminal operator. + return Collections.emptyList(); + } + + @Override + protected void handleClose() { + table.close(); + } + + @Override + protected OperatorSpec, Void> getOperatorSpec() { + return sendToTableOpSpec; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/StreamTableJoinOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/StreamTableJoinOperatorImpl.java new file mode 100644 index 0000000000..54a577088f --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/StreamTableJoinOperatorImpl.java @@ -0,0 +1,82 @@ +/* + * 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.samza.operators.impl; + +import java.util.Collection; +import java.util.Collections; + +import org.apache.samza.config.Config; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; +import org.apache.samza.table.ReadableTable; +import org.apache.samza.task.MessageCollector; +import org.apache.samza.task.TaskContext; +import org.apache.samza.task.TaskCoordinator; + + +/** + * Implementation of a stream-table join operator that first retrieve the value of + * the message key from incoming message, and then apply the join function. + * + * @param type of the join key + * @param type of input messages + * @param type of the table record + * @param type of the join result + */ +class StreamTableJoinOperatorImpl extends OperatorImpl { + + private final StreamTableJoinOperatorSpec joinOpSpec; + private final ReadableTable table; + + StreamTableJoinOperatorImpl(StreamTableJoinOperatorSpec joinOpSpec, + Config config, TaskContext context) { + this.joinOpSpec = joinOpSpec; + this.table = (ReadableTable) context.getTable(joinOpSpec.getTableSpec().getId()); + } + + @Override + protected void handleInit(Config config, TaskContext context) { + this.joinOpSpec.getJoinFn().init(config, context); + } + + @Override + public Collection handleMessage(M message, MessageCollector collector, TaskCoordinator coordinator) { + K key = joinOpSpec.getJoinFn().getMessageKey(message); + Object recordValue = table.get(key); + R record = recordValue != null ? (R) KV.of(key, recordValue) : null; + JM output = joinOpSpec.getJoinFn().apply(message, record); + + // The support for inner and outer join will be provided in the jonFn. For inner join, the joinFn might + // return null, when the corresponding record is absent in the table. + return output != null ? + Collections.singletonList(output) + : Collections.emptyList(); + } + + @Override + protected void handleClose() { + this.joinOpSpec.getJoinFn().close(); + } + + protected OperatorSpec getOperatorSpec() { + return joinOpSpec; + } + +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index 17f1b492cf..2a5991c6c7 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -18,17 +18,17 @@ */ package org.apache.samza.operators.spec; -import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.operators.functions.WatermarkFunction; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.MessageStreamImpl; - import java.util.Collection; import java.util.LinkedHashSet; import java.util.Set; +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.MessageStreamImpl; +import org.apache.samza.operators.functions.WatermarkFunction; + /** - * A stream operator specification that holds all the information required to transform + * A stream operator specification that holds all the information required to transform * the input {@link org.apache.samza.operators.MessageStreamImpl} and produce the output * {@link org.apache.samza.operators.MessageStreamImpl}. * diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java index 1b3b8aa63b..c752fe2057 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java @@ -19,6 +19,10 @@ package org.apache.samza.operators.spec; +import java.util.ArrayList; +import java.util.Collection; +import java.util.function.Function; + import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.functions.FilterFunction; @@ -26,15 +30,13 @@ import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.functions.StreamTableJoinFunction; import org.apache.samza.operators.windows.internal.WindowInternal; import org.apache.samza.serializers.Serde; import org.apache.samza.system.StreamSpec; +import org.apache.samza.table.TableSpec; import org.apache.samza.task.TaskContext; -import java.util.ArrayList; -import java.util.Collection; -import java.util.function.Function; - /** * Factory methods for creating {@link OperatorSpec} instances. @@ -242,4 +244,38 @@ public static StreamOperatorSpec createMergeOperatorSpec(String opId) }, OperatorSpec.OpCode.MERGE, opId); } + + /** + * Creates a {@link StreamTableJoinOperatorSpec} with a join function. + * + * @param tableSpec the table spec for the table on the right side of the join + * @param joinFn the user-defined join function to get join keys and results + * @param opId the unique ID of the operator + * @param the type of join key + * @param the type of input messages + * @param the type of table record + * @param the type of the join result + * @return the {@link StreamTableJoinOperatorSpec} + */ + public static StreamTableJoinOperatorSpec createStreamTableJoinOperatorSpec( + TableSpec tableSpec, StreamTableJoinFunction joinFn, String opId) { + return new StreamTableJoinOperatorSpec(tableSpec, joinFn, opId); + } + + /** + * Creates a {@link SendToTableOperatorSpec} with a key extractor and a value extractor function, + * the type of incoming message is expected to be KV<K, V>. + * + * @param inputOpSpec the operator spec for the input stream + * @param tableSpec the table spec for the underlying table + * @param opId the unique ID of the operator + * @param the type of the table record key + * @param the type of the table record value + * @return the {@link SendToTableOperatorSpec} + */ + public static SendToTableOperatorSpec createSendToTableOperatorSpec( + OperatorSpec> inputOpSpec, TableSpec tableSpec, String opId) { + return new SendToTableOperatorSpec(inputOpSpec, tableSpec, opId); + } + } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java new file mode 100644 index 0000000000..9084be2fad --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java @@ -0,0 +1,65 @@ +/* + * 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.samza.operators.spec; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.functions.WatermarkFunction; +import org.apache.samza.table.TableSpec; + + +/** + * The spec for operator that writes a stream to a table by extracting keys and values + * from the incoming messages. + * + * @param the type of the table record key + * @param the type of the table record value + */ +@InterfaceStability.Unstable +public class SendToTableOperatorSpec extends OperatorSpec, Void> { + + private final OperatorSpec> inputOpSpec; + private final TableSpec tableSpec; + + /** + * Constructor for a {@link SendToTableOperatorSpec}. + * + * @param inputOpSpec the operator spec of the input stream + * @param tableSpec the table spec of the table written to + * @param opId the unique ID for this operator + */ + SendToTableOperatorSpec(OperatorSpec> inputOpSpec, TableSpec tableSpec, String opId) { + super(OpCode.SEND_TO, opId); + this.inputOpSpec = inputOpSpec; + this.tableSpec = tableSpec; + } + + public OperatorSpec> getInputOpSpec() { + return inputOpSpec; + } + + public TableSpec getTableSpec() { + return tableSpec; + } + + @Override + public WatermarkFunction getWatermarkFn() { + return null; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java new file mode 100644 index 0000000000..730913a3de --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java @@ -0,0 +1,67 @@ +/* + * 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.samza.operators.spec; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.functions.StreamTableJoinFunction; +import org.apache.samza.operators.functions.WatermarkFunction; +import org.apache.samza.table.TableSpec; + + +/** + * The spec for stream-table join operator that retrieves a record from the table using key + * derived from the incoming message and joins with the incoming message. + * + * @param the type of input messages + * @param the type of table record + * @param the type of join result + */ +@InterfaceStability.Unstable +public class StreamTableJoinOperatorSpec extends OperatorSpec { + + private final TableSpec tableSpec; + private final StreamTableJoinFunction joinFn; + + /** + * Constructor for {@link StreamTableJoinOperatorSpec}. + * + * @param tableSpec the table spec for the table on the right side of the join + * @param joinFn the user-defined join function to get join keys and results + * @param opId the unique ID for this operator + */ + StreamTableJoinOperatorSpec(TableSpec tableSpec, StreamTableJoinFunction joinFn, String opId) { + super(OpCode.JOIN, opId); + this.tableSpec = tableSpec; + this.joinFn = joinFn; + } + + public TableSpec getTableSpec() { + return tableSpec; + } + + public StreamTableJoinFunction getJoinFn() { + return this.joinFn; + } + + @Override + public WatermarkFunction getWatermarkFn() { + return joinFn instanceof WatermarkFunction ? (WatermarkFunction) joinFn : null; + } + +} diff --git a/samza-core/src/main/java/org/apache/samza/table/TableManager.java b/samza-core/src/main/java/org/apache/samza/table/TableManager.java new file mode 100644 index 0000000000..c3555f325c --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/TableManager.java @@ -0,0 +1,153 @@ +/* + * 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.samza.table; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.Serde; +import org.apache.samza.storage.StorageEngine; +import org.apache.samza.util.Util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link TableManager} manages tables within a Samza task. For each table, it maintains + * the {@link TableSpec} and the {@link TableProvider}. It is used at execution for + * {@link org.apache.samza.container.TaskInstance} to retrieve table instances for + * read/write operations. + * + * A {@link TableManager} is constructed from job configuration, the {@link TableSpec} + * and {@link TableProvider} are constructed by processing the job configuration. + * + * After a {@link TableManager} is constructed, local tables are associated with + * local store instances created during {@link org.apache.samza.container.SamzaContainer} + * initialization. + * + * Method {@link TableManager#getTable(String)} will throw {@link IllegalStateException}, + * if it's called before initialization. + * + * For store backed tables, the list of stores must be injected into the constructor. + */ +public class TableManager { + + static public class TableCtx { + private TableSpec tableSpec; + private TableProvider tableProvider; + } + + private final Logger logger = LoggerFactory.getLogger(TableManager.class.getName()); + + // tableId -> TableCtx + private final Map tables = new HashMap<>(); + + private boolean localTablesInitialized; + + /** + * Construct a table manager instance + * @param config the job configuration + * @param serdes Serde instances for tables + */ + public TableManager(Config config, Map> serdes) { + + new JavaTableConfig(config).getTableIds().forEach(tableId -> { + + // Construct the table provider + String tableProviderFactory = config.get(String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, tableId)); + + // Construct the KVSerde + JavaTableConfig tableConfig = new JavaTableConfig(config); + KVSerde serde = KVSerde.of( + serdes.get(tableConfig.getKeySerde(tableId)), + serdes.get(tableConfig.getValueSerde(tableId))); + + TableSpec tableSpec = new TableSpec(tableId, serde, tableProviderFactory, + config.subset(String.format(JavaTableConfig.TABLE_ID_PREFIX, tableId) + ".")); + + addTable(tableSpec); + + logger.info("Added table " + tableSpec.getId()); + }); + } + + /** + * Initialize all local table + * @param stores stores created locally + */ + public void initLocalTables(Map stores) { + tables.values().forEach(ctx -> { + if (ctx.tableProvider instanceof LocalStoreBackedTableProvider) { + StorageEngine store = stores.get(ctx.tableSpec.getId()); + if (store == null) { + throw new SamzaException(String.format( + "Backing store for table %s was not injected by SamzaContainer", + ctx.tableSpec.getId())); + } + ((LocalStoreBackedTableProvider) ctx.tableProvider).init(store); + } + }); + + localTablesInitialized = true; + } + + /** + * Add a table to the table manager + * @param tableSpec the table spec + */ + private void addTable(TableSpec tableSpec) { + if (tables.containsKey(tableSpec.getId())) { + throw new SamzaException("Table " + tableSpec.getId() + " already exists"); + } + TableCtx ctx = new TableCtx(); + TableProviderFactory tableProviderFactory = Util.getObj(tableSpec.getTableProviderFactoryClassName()); + ctx.tableProvider = tableProviderFactory.getTableProvider(tableSpec); + ctx.tableSpec = tableSpec; + tables.put(tableSpec.getId(), ctx); + } + + /** + * Start the table manager, internally it starts all tables + */ + public void start() { + tables.values().forEach(ctx -> ctx.tableProvider.start()); + } + + /** + * Shutdown the table manager, internally it shuts down all tables + */ + public void shutdown() { + tables.values().forEach(ctx -> ctx.tableProvider.stop()); + } + + /** + * Get a table instance + * @param tableId Id of the table + * @return table instance + */ + public Table getTable(String tableId) { + if (!localTablesInitialized) { + throw new IllegalStateException("Local tables in TableManager not initialized."); + } + return tables.get(tableId).tableProvider.getTable(); + } +} diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 412e9dc5cd..f465bfce88 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -20,65 +20,37 @@ package org.apache.samza.container import java.io.File +import java.net.{URL, UnknownHostException} import java.nio.file.Path import java.util -import java.util.concurrent.{ExecutorService, Executors, TimeUnit} -import java.net.{URL, UnknownHostException} import java.util.Base64 +import java.util.concurrent.{ExecutorService, Executors, TimeUnit} -import org.apache.samza.{SamzaContainerStatus, SamzaException} import org.apache.samza.checkpoint.{CheckpointListener, CheckpointManagerFactory, OffsetManager, OffsetManagerMetrics} import org.apache.samza.config.JobConfig.Config2Job import org.apache.samza.config.MetricsConfig.Config2Metrics import org.apache.samza.config.SerializerConfig.Config2Serializer -import org.apache.samza.config._ import org.apache.samza.config.StorageConfig.Config2Storage import org.apache.samza.config.StreamConfig.Config2Stream import org.apache.samza.config.SystemConfig.Config2System import org.apache.samza.config.TaskConfig.Config2Task -import org.apache.samza.container.disk.DiskQuotaPolicyFactory -import org.apache.samza.container.disk.DiskSpaceMonitor +import org.apache.samza.config._ import org.apache.samza.container.disk.DiskSpaceMonitor.Listener -import org.apache.samza.container.disk.NoThrottlingDiskQuotaPolicyFactory -import org.apache.samza.container.disk.PollingScanDiskSpaceMonitor +import org.apache.samza.container.disk.{DiskQuotaPolicyFactory, DiskSpaceMonitor, NoThrottlingDiskQuotaPolicyFactory, PollingScanDiskSpaceMonitor} import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStatistics, SystemStatisticsMonitor} import org.apache.samza.coordinator.stream.CoordinatorStreamSystemFactory -import org.apache.samza.job.model.ContainerModel import org.apache.samza.job.model.JobModel -import org.apache.samza.metrics.JmxServer -import org.apache.samza.metrics.JvmMetrics -import org.apache.samza.metrics.MetricsRegistryMap -import org.apache.samza.metrics.MetricsReporter -import org.apache.samza.serializers.IntermediateMessageSerde -import org.apache.samza.serializers.NoOpSerde -import org.apache.samza.serializers.SerializableSerde -import org.apache.samza.serializers.Serde -import org.apache.samza.serializers.SerdeFactory -import org.apache.samza.serializers.SerdeManager -import org.apache.samza.serializers.StringSerde +import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter} +import org.apache.samza.serializers._ import org.apache.samza.serializers.model.SamzaObjectMapper -import org.apache.samza.storage.StorageEngineFactory -import org.apache.samza.storage.TaskStorageManager -import org.apache.samza.system.StreamMetadataCache -import org.apache.samza.system.SystemConsumers -import org.apache.samza.system.SystemConsumersMetrics -import org.apache.samza.system.SystemFactory -import org.apache.samza.system.SystemProducers -import org.apache.samza.system.SystemProducersMetrics -import org.apache.samza.system.SystemStream -import org.apache.samza.system.SystemStreamPartition -import org.apache.samza.system.chooser.DefaultChooser -import org.apache.samza.system.chooser.MessageChooserFactory -import org.apache.samza.system.chooser.RoundRobinChooserFactory +import org.apache.samza.storage.{StorageEngineFactory, TaskStorageManager} +import org.apache.samza.system._ +import org.apache.samza.system.chooser.{DefaultChooser, MessageChooserFactory, RoundRobinChooserFactory} +import org.apache.samza.table.TableManager import org.apache.samza.task._ -import org.apache.samza.util.HighResolutionClock -import org.apache.samza.util.ExponentialSleepStrategy -import org.apache.samza.util.Logging -import org.apache.samza.util.Throttleable -import org.apache.samza.util.MetricsReporterLoader -import org.apache.samza.util.SystemClock -import org.apache.samza.util.Util import org.apache.samza.util.Util.asScalaClock +import org.apache.samza.util._ +import org.apache.samza.{SamzaContainerStatus, SamzaException} import scala.collection.JavaConverters._ @@ -568,6 +540,11 @@ object SamzaContainer extends Logging { new StorageConfig(config).getChangeLogDeleteRetentionsInMs, new SystemClock) + val tableManager = new TableManager(config, serdes.asJava) + tableManager.initLocalTables(taskStores.asJava) + + info("Got table manager"); + val systemStreamPartitions = taskModel .getSystemStreamPartitions .asScala @@ -586,6 +563,7 @@ object SamzaContainer extends Logging { containerContext = containerContext, offsetManager = offsetManager, storageManager = storageManager, + tableManager = tableManager, reporters = reporters, systemStreamPartitions = systemStreamPartitions, exceptionHandler = TaskInstanceExceptionHandler(taskInstanceMetrics, config), @@ -711,6 +689,7 @@ class SamzaContainer( startOffsetManager startLocalityManager startStores + startTableManager startDiskSpaceMonitor startHostStatisticsMonitor startProducers @@ -745,6 +724,7 @@ class SamzaContainer( shutdownConsumers shutdownTask + shutdownTableManager shutdownStores shutdownDiskSpaceMonitor shutdownHostStatisticsMonitor @@ -885,9 +865,9 @@ class SamzaContainer( } def startStores { - info("Starting task instance stores.") taskInstances.values.foreach(taskInstance => { val startTime = System.currentTimeMillis() + info("Starting stores in task instance %s" format taskInstance.taskName) taskInstance.startStores // Measuring the time to restore the stores val timeToRestore = System.currentTimeMillis() - startTime @@ -898,6 +878,13 @@ class SamzaContainer( }) } + def startTableManager: Unit = { + taskInstances.values.foreach(taskInstance => { + info("Starting table manager in task instance %s" format taskInstance.taskName) + taskInstance.startTableManager + }) + } + def startTask { info("Initializing stream tasks.") @@ -1003,6 +990,12 @@ class SamzaContainer( taskInstances.values.foreach(_.shutdownStores) } + def shutdownTableManager: Unit = { + info("Shutting down task instance table manager.") + + taskInstances.values.foreach(_.shutdownTableManager) + } + def shutdownLocalityManager { if(localityManager != null) { info("Shutting down locality manager.") diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala index acec365deb..f2a50743e7 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala @@ -27,20 +27,9 @@ import org.apache.samza.config.StreamConfig.Config2Stream import org.apache.samza.job.model.JobModel import org.apache.samza.metrics.MetricsReporter import org.apache.samza.storage.TaskStorageManager -import org.apache.samza.system.IncomingMessageEnvelope -import org.apache.samza.system.StreamMetadataCache -import org.apache.samza.system.SystemAdmin -import org.apache.samza.system.SystemConsumers -import org.apache.samza.system.SystemStreamPartition -import org.apache.samza.task.AsyncStreamTask -import org.apache.samza.task.ClosableTask -import org.apache.samza.task.EndOfStreamListenerTask -import org.apache.samza.task.InitableTask -import org.apache.samza.task.ReadableCoordinator -import org.apache.samza.task.StreamTask -import org.apache.samza.task.TaskCallbackFactory -import org.apache.samza.task.TaskInstanceCollector -import org.apache.samza.task.WindowableTask +import org.apache.samza.system._ +import org.apache.samza.table.TableManager +import org.apache.samza.task._ import org.apache.samza.util.Logging import scala.collection.JavaConverters._ @@ -56,6 +45,7 @@ class TaskInstance( containerContext: SamzaContainerContext, val offsetManager: OffsetManager = new OffsetManager, storageManager: TaskStorageManager = null, + tableManager: TableManager = null, reporters: Map[String, MetricsReporter] = Map(), val systemStreamPartitions: Set[SystemStreamPartition] = Set(), val exceptionHandler: TaskInstanceExceptionHandler = new TaskInstanceExceptionHandler, @@ -68,7 +58,7 @@ class TaskInstance( val isAsyncTask = task.isInstanceOf[AsyncStreamTask] val context = new TaskContextImpl(taskName,metrics, containerContext, systemStreamPartitions.asJava, offsetManager, - storageManager, jobModel, streamMetadataCache) + storageManager, tableManager, jobModel, streamMetadataCache) // store the (ssp -> if this ssp is catched up) mapping. "catched up" // means the same ssp in other taskInstances have the same offset as @@ -101,6 +91,16 @@ class TaskInstance( } } + def startTableManager { + if (tableManager != null) { + debug("Starting table manager for taskName: %s" format taskName) + + tableManager.start + } else { + debug("Skipping table manager initialization for taskName: %s" format taskName) + } + } + def initTask { if (isInitableTask) { debug("Initializing task for taskName: %s" format taskName) @@ -225,6 +225,16 @@ class TaskInstance( } } + def shutdownTableManager { + if (tableManager != null) { + debug("Shutting down table manager for taskName: %s" format taskName) + + tableManager.shutdown + } else { + debug("Skipping table manager shutdown for taskName: %s" format taskName) + } + } + override def toString() = "TaskInstance for class %s and taskName %s." format (task.getClass.getName, taskName) def toDetailedString() = "TaskInstance [taskName = %s, windowable=%s, closable=%s endofstreamlistener=%s]" format diff --git a/samza-core/src/test/java/org/apache/samza/config/TestJavaTableConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestJavaTableConfig.java new file mode 100644 index 0000000000..2775ca77bf --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/config/TestJavaTableConfig.java @@ -0,0 +1,58 @@ +/* + * 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.samza.config; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.junit.Test; + +import com.google.common.collect.Sets; + +import static junit.framework.Assert.assertTrue; +import static org.junit.Assert.assertEquals; + + +public class TestJavaTableConfig { + @Test + public void testGetTableIds() { + Set ids = Sets.newHashSet("t1", "t2"); + Map map = ids.stream() + .map(id -> String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, id)) + .collect(Collectors.toMap(key -> key, key -> key + "-provider-factory")); + JavaTableConfig tableConfig = new JavaTableConfig(new MapConfig(map)); + + assertEquals(2, tableConfig.getTableIds().size()); + + ids.removeAll(tableConfig.getTableIds()); + assertTrue(ids.isEmpty()); + } + + @Test + public void testGetTableProperties() { + Map map = new HashMap<>(); + map.put("tables.t1.spec", "t1-spec"); + map.put("tables.t1.provider.factory", "t1-provider-factory"); + JavaTableConfig tableConfig = new JavaTableConfig(new MapConfig(map)); + assertEquals("t1-provider-factory", tableConfig.getTableProviderFactory("t1")); + } + +} diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java index d97d4945fe..96e234e604 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java @@ -18,7 +18,12 @@ */ package org.apache.samza.operators; -import com.google.common.collect.ImmutableList; +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.function.Function; +import java.util.function.Supplier; + import org.apache.samza.operators.data.TestMessageEnvelope; import org.apache.samza.operators.data.TestOutputMessageEnvelope; import org.apache.samza.operators.functions.FilterFunction; @@ -27,14 +32,17 @@ import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.functions.StreamTableJoinFunction; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputOperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.spec.PartitionByOperatorSpec; +import org.apache.samza.operators.spec.SendToTableOperatorSpec; import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; +import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; import org.apache.samza.operators.windows.Window; @@ -42,14 +50,11 @@ import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; +import org.apache.samza.table.TableSpec; import org.junit.Test; import org.mockito.ArgumentCaptor; -import java.time.Duration; -import java.util.Collection; -import java.util.Collections; -import java.util.function.Function; -import java.util.function.Supplier; +import com.google.common.collect.ImmutableList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -312,6 +317,57 @@ public void testJoin() { assertEquals(rightInputOpSpec, ((JoinOperatorSpec) leftRegisteredOpSpec).getRightInputOpSpec()); } + @Test + public void testSendToTable() { + StreamGraphImpl mockGraph = mock(StreamGraphImpl.class); + OperatorSpec inputOpSpec = mock(OperatorSpec.class); + MessageStreamImpl source = new MessageStreamImpl<>(mockGraph, inputOpSpec); + + TableSpec tableSpec = new TableSpec(); + TableImpl table = new TableImpl(tableSpec); + + source.sendTo(table); + + ArgumentCaptor registeredOpCaptor = ArgumentCaptor.forClass(OperatorSpec.class); + verify(inputOpSpec).registerNextOperatorSpec(registeredOpCaptor.capture()); + OperatorSpec registeredOpSpec = registeredOpCaptor.getValue(); + + assertTrue(registeredOpSpec instanceof SendToTableOperatorSpec); + SendToTableOperatorSpec sendToTableOperatorSpec = (SendToTableOperatorSpec) registeredOpSpec; + + assertEquals(OpCode.SEND_TO, sendToTableOperatorSpec.getOpCode()); + assertEquals(inputOpSpec, sendToTableOperatorSpec.getInputOpSpec()); + assertEquals(tableSpec, sendToTableOperatorSpec.getTableSpec()); + } + + @Test + public void testStreamTableJoin() { + StreamGraphImpl mockGraph = mock(StreamGraphImpl.class); + OperatorSpec leftInputOpSpec = mock(OperatorSpec.class); + MessageStreamImpl> source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec); + OperatorSpec rightInputOpSpec = mock(OperatorSpec.class); + MessageStreamImpl source2 = new MessageStreamImpl<>(mockGraph, rightInputOpSpec); + + TableSpec tableSpec = new TableSpec(); + TableImpl table = new TableImpl(tableSpec); + + source2.sendTo(table); + + StreamTableJoinFunction, KV, TestOutputMessageEnvelope> + mockJoinFn = mock(StreamTableJoinFunction.class); + source1.join(table, mockJoinFn); + + ArgumentCaptor leftRegisteredOpCaptor = ArgumentCaptor.forClass(OperatorSpec.class); + verify(leftInputOpSpec).registerNextOperatorSpec(leftRegisteredOpCaptor.capture()); + OperatorSpec leftRegisteredOpSpec = leftRegisteredOpCaptor.getValue(); + + assertTrue(leftRegisteredOpSpec instanceof StreamTableJoinOperatorSpec); + StreamTableJoinOperatorSpec joinOpSpec = (StreamTableJoinOperatorSpec) leftRegisteredOpSpec; + assertEquals(OpCode.JOIN, joinOpSpec.getOpCode()); + assertEquals(mockJoinFn, joinOpSpec.getJoinFn()); + assertEquals(tableSpec, joinOpSpec.getTableSpec()); + } + @Test public void testMerge() { StreamGraphImpl mockGraph = mock(StreamGraphImpl.class); diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java index cf0a198cdd..3bb44b55d7 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java @@ -12,14 +12,16 @@ * 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 + * KIND, either express or implied. See the License for THE * specific language governing permissions and limitations * under the License. */ package org.apache.samza.operators; -import com.google.common.collect.ImmutableList; -import junit.framework.Assert; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -33,10 +35,11 @@ import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.system.StreamSpec; +import org.apache.samza.table.TableSpec; import org.junit.Test; -import java.util.ArrayList; -import java.util.List; +import com.google.common.collect.ImmutableList; +import junit.framework.Assert; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -583,4 +586,16 @@ public void testGetInputStreamPreservesInsertionOrder() { Assert.assertEquals(inputSpecs.get(1).getStreamSpec(), testStreamSpec2); Assert.assertEquals(inputSpecs.get(2).getStreamSpec(), testStreamSpec3); } + + @Test + public void testGetTable() { + ApplicationRunner mockRunner = mock(ApplicationRunner.class); + Config mockConfig = mock(Config.class); + StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig); + + BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); + when(mockTableDescriptor.getTableSpec()).thenReturn( + new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>())); + Assert.assertNotNull(graph.getTable(mockTableDescriptor)); + } } diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamTableJoinOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamTableJoinOperatorImpl.java new file mode 100644 index 0000000000..d8b2e8d868 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamTableJoinOperatorImpl.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.samza.operators.impl; + +import java.util.Collection; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.data.TestMessageEnvelope; +import org.apache.samza.operators.functions.StreamTableJoinFunction; +import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; +import org.apache.samza.table.ReadableTable; +import org.apache.samza.table.TableSpec; +import org.apache.samza.task.MessageCollector; +import org.apache.samza.task.TaskContext; +import org.apache.samza.task.TaskCoordinator; +import org.junit.Test; + +import junit.framework.Assert; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +public class TestStreamTableJoinOperatorImpl { + @Test + public void testHandleMessage() { + + String tableId = "t1"; + TableSpec tableSpec = mock(TableSpec.class); + when(tableSpec.getId()).thenReturn(tableId); + + StreamTableJoinOperatorSpec mockJoinOpSpec = mock(StreamTableJoinOperatorSpec.class); + when(mockJoinOpSpec.getTableSpec()).thenReturn(tableSpec); + when(mockJoinOpSpec.getJoinFn()).thenReturn( + new StreamTableJoinFunction, KV, String>() { + @Override + public String apply(KV message, KV record) { + if ("1".equals(message.getKey())) { + Assert.assertEquals("m1", message.getValue()); + Assert.assertEquals("r1", record.getValue()); + return "m1r1"; + } else if ("2".equals(message.getKey())) { + Assert.assertEquals("m2", message.getValue()); + Assert.assertNull(record); + return null; + } + throw new SamzaException("Should never reach here!"); + } + + @Override + public String getMessageKey(KV message) { + return message.getKey(); + } + + @Override + public String getRecordKey(KV record) { + return record.getKey(); + } + }); + Config config = mock(Config.class); + ReadableTable table = mock(ReadableTable.class); + when(table.get("1")).thenReturn("r1"); + when(table.get("2")).thenReturn(null); + TaskContext mockTaskContext = mock(TaskContext.class); + when(mockTaskContext.getTable(tableId)).thenReturn(table); + + MessageCollector mockMessageCollector = mock(MessageCollector.class); + TaskCoordinator mockTaskCoordinator = mock(TaskCoordinator.class); + + StreamTableJoinOperatorImpl streamTableJoinOperator = new StreamTableJoinOperatorImpl( + mockJoinOpSpec, config, mockTaskContext); + + // Table has the key + Collection result; + result = streamTableJoinOperator.handleMessage(KV.of("1", "m1"), mockMessageCollector, mockTaskCoordinator); + Assert.assertEquals(1, result.size()); + Assert.assertEquals("m1r1", result.iterator().next()); + // Table doesn't have the key + result = streamTableJoinOperator.handleMessage(KV.of("2", "m2"), mockMessageCollector, mockTaskCoordinator); + Assert.assertEquals(0, result.size()); + } + +} diff --git a/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java b/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java new file mode 100644 index 0000000000..df5b9e5746 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java @@ -0,0 +1,176 @@ +/* + * 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.samza.table; + +import java.lang.reflect.Field; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.SerializerConfig; +import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerializableSerde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.storage.StorageEngine; +import org.junit.Test; + +import junit.framework.Assert; + +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +public class TestTableManager { + + private static final String TABLE_ID = "t1"; + + public static class DummyTableProviderFactory implements TableProviderFactory { + + static Table table; + static LocalStoreBackedTableProvider tableProvider; + + @Override + public TableProvider getTableProvider(TableSpec tableSpec) { + table = mock(Table.class); + tableProvider = mock(LocalStoreBackedTableProvider.class); + when(tableProvider.getTable()).thenReturn(table); + return tableProvider; + } + } + + @Test + public void testInitByConfig() { + Map map = new HashMap<>(); + map.put(String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, TABLE_ID), DummyTableProviderFactory.class.getName()); + map.put(String.format("tables.%s.some.config", TABLE_ID), "xyz"); + addKeySerde(map); + addValueSerde(map); + doTestInit(map); + } + + @Test(expected = Exception.class) + public void testInitFailsWithoutProviderFactory() { + Map map = new HashMap<>(); + addKeySerde(map); + addValueSerde(map); + doTestInit(map); + } + + @Test(expected = Exception.class) + public void testInitFailsWithoutKeySerde() { + Map map = new HashMap<>(); + map.put(String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, TABLE_ID), DummyTableProviderFactory.class.getName()); + addValueSerde(map); + doTestInit(map); + } + + @Test(expected = Exception.class) + public void testInitFailsWithoutValueSerde() { + Map map = new HashMap<>(); + map.put(String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, TABLE_ID), DummyTableProviderFactory.class.getName()); + addValueSerde(map); + doTestInit(map); + } + + @Test(expected = IllegalStateException.class) + public void testInitFailsWithoutInitializingLocalTables() { + TableManager tableManager = new TableManager(new MapConfig(new HashMap<>()), new HashMap<>()); + tableManager.getTable("dummy"); + } + + private void doTestInit(Map map) { + Map storageEngines = new HashMap<>(); + storageEngines.put(TABLE_ID, mock(StorageEngine.class)); + + Map> serdeMap = new HashMap<>(); + SerializableSerde serializableSerde = new SerializableSerde(); + map.keySet().stream() + .filter(k -> k.endsWith(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX())) + .forEach(k -> { + String serdeName = k + .replace(String.format(SerializerConfig.SERIALIZER_PREFIX(), ""), "") + .replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""); + String serializedSerde = map.get(k); + byte[] bytes = Base64.getDecoder().decode(serializedSerde); + Serde serde = serializableSerde.fromBytes(bytes); + serdeMap.put(serdeName, serde); + }); + + TableManager tableManager = new TableManager(new MapConfig(map), serdeMap); + tableManager.initLocalTables(storageEngines); + + Table table = tableManager.getTable(TABLE_ID); + verify(DummyTableProviderFactory.tableProvider, times(1)).init(anyObject()); + Assert.assertEquals(DummyTableProviderFactory.table, table); + + Map ctxMap = getFieldValue(tableManager, "tables"); + TableManager.TableCtx ctx = ctxMap.get(TABLE_ID); + + TableSpec tableSpec = getFieldValue(ctx, "tableSpec"); + Assert.assertEquals(TABLE_ID, tableSpec.getId()); + Assert.assertEquals(DummyTableProviderFactory.class.getName(), tableSpec.getTableProviderFactoryClassName()); + Assert.assertEquals(IntegerSerde.class, tableSpec.getSerde().getKeySerde().getClass()); + Assert.assertEquals(StringSerde.class, tableSpec.getSerde().getValueSerde().getClass()); + Assert.assertEquals("xyz", tableSpec.getConfig().get("some.config")); + + TableProvider tableProvider = getFieldValue(ctx, "tableProvider"); + Assert.assertNotNull(tableProvider); + } + + private void addKeySerde(Map map) { + String serdeId = "key-serde"; + map.put(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeId), + serializeSerde(new IntegerSerde())); + map.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, TABLE_ID), serdeId); + } + + private void addValueSerde(Map map) { + String serdeId = "value-serde"; + map.put(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeId), + serializeSerde(new StringSerde("UTF-8"))); + map.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, TABLE_ID), serdeId); + } + + private String serializeSerde(Serde serde) { + return Base64.getEncoder().encodeToString(new SerializableSerde().toBytes(serde)); + } + + private T getFieldValue(Object object, String fieldName) { + Field field = null; + try { + field = object.getClass().getDeclaredField(fieldName); + field.setAccessible(true); + return (T) field.get(object); + } catch (NoSuchFieldException | IllegalAccessException ex) { + throw new SamzaException(ex); + } finally { + if (field != null) { + field.setAccessible(false); + } + } + } + +} diff --git a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java index b399f5fae2..28a4f8b513 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java @@ -30,6 +30,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.samza.Partition; import org.apache.samza.checkpoint.OffsetManager; import org.apache.samza.config.Config; @@ -47,16 +48,15 @@ import org.apache.samza.system.TestSystemConsumers; import org.junit.Before; import org.junit.Test; + import scala.Option; import scala.collection.JavaConverters; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.*; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.*; // TODO(spvenkat) SAMZA-1183: Fix all commented out tests. public class TestAsyncRunLoop { @@ -86,7 +86,7 @@ TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemS scala.collection.immutable.Set sspSet = JavaConverters.asScalaSetConverter(Collections.singleton(ssp)).asScala().toSet(); return new TaskInstance(task, taskName, mock(Config.class), taskInstanceMetrics, null, consumers, mock(TaskInstanceCollector.class), mock(SamzaContainerContext.class), - manager, null, null, sspSet, new TaskInstanceExceptionHandler(taskInstanceMetrics, new scala.collection.immutable.HashSet()), null, null); + manager, null, null, null, sspSet, new TaskInstanceExceptionHandler(taskInstanceMetrics, new scala.collection.immutable.HashSet()), null, null); } TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemStreamPartition ssp) { diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java new file mode 100644 index 0000000000..2681fb336f --- /dev/null +++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.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.samza.storage.kv.inmemory; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.storage.kv.BaseLocalStoreBackedTableDescriptor; +import org.apache.samza.table.TableSpec; + + +/** + * Table descriptor for in-memory tables + * + * @param the type of the key + * @param the type of the value + */ +public class InMemoryTableDescriptor extends BaseLocalStoreBackedTableDescriptor> { + + public InMemoryTableDescriptor(String tableId) { + super(tableId); + } + + @Override + protected void generateTableSpecConfig(Map tableSpecConfig) { + super.generateTableSpecConfig(tableSpecConfig); + } + + @Override + public TableSpec getTableSpec() { + + validate(); + + Map tableSpecConfig = new HashMap<>(); + generateTableSpecConfig(tableSpecConfig); + + return new TableSpec(tableId, serde, InMemoryTableProviderFactory.class.getName(), tableSpecConfig); + } + + private void addInMemoryConfig(Map map, String key, String value) { + map.put("inmemory." + key, value); + } +} diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java new file mode 100644 index 0000000000..c1c2f1ca5c --- /dev/null +++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java @@ -0,0 +1,65 @@ +/* + * 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.samza.storage.kv.inmemory; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.storage.kv.BaseLocalStoreBackedTableProvider; +import org.apache.samza.table.TableSpec; + + +/** + * Table provider of an in-memory table + */ +public class InMemoryTableProvider extends BaseLocalStoreBackedTableProvider { + + public InMemoryTableProvider(TableSpec tableSpec) { + super(tableSpec); + } + + @Override + public Map generateConfig(Map config) { + + Map tableConfig = new HashMap<>(); + + // Store factory configuration + tableConfig.put(String.format( + StorageConfig.FACTORY(), tableSpec.getId()), + InMemoryKeyValueStorageEngineFactory.class.getName()); + + // Common store configuration + tableConfig.putAll(generateCommonStoreConfig(config)); + + // Rest of the configuration + tableSpec.getConfig().forEach((k, v) -> { + String realKey = k.startsWith("inmemory.") ? + String.format("stores.%s", tableSpec.getId()) + "." + k.substring("inmemory.".length()) + : String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; + tableConfig.put(realKey, v); + }); + + logger.info("Generated configuration for table " + tableSpec.getId()); + + return tableConfig; + } + +} diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java new file mode 100644 index 0000000000..f05982a8f8 --- /dev/null +++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProviderFactory.java @@ -0,0 +1,33 @@ +/* + * 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.samza.storage.kv.inmemory; + +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableProviderFactory; +import org.apache.samza.table.TableSpec; + +/** + * Factory class for an in-memory table provider + */ +public class InMemoryTableProviderFactory implements TableProviderFactory { + @Override + public TableProvider getTableProvider(TableSpec tableSpec) { + return new InMemoryTableProvider(tableSpec); + } +} diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java new file mode 100644 index 0000000000..840fb70243 --- /dev/null +++ b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java @@ -0,0 +1,48 @@ +/* + * 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.samza.storage.kv.inmemory; + +import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.table.TableSpec; +import org.junit.Assert; +import org.junit.Test; + + +public class TestInMemoryTableDescriptor { + @Test + public void testTableSpec() { + + TableSpec tableSpec = new InMemoryTableDescriptor("1") + .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde())) + .withConfig("inmemory.abc", "xyz") + .getTableSpec(); + + Assert.assertNotNull(tableSpec.getSerde()); + Assert.assertNotNull(tableSpec.getSerde().getKeySerde()); + Assert.assertNotNull(tableSpec.getSerde().getValueSerde()); + Assert.assertEquals("xyz", getConfig(tableSpec, "abc")); + } + + private String getConfig(TableSpec tableSpec, String key) { + return tableSpec.getConfig().get("inmemory." + key); + } +} diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java new file mode 100644 index 0000000000..76b7a666ba --- /dev/null +++ b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java @@ -0,0 +1,65 @@ +/* + * 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.samza.storage.kv.inmemory; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableSpec; +import org.junit.Test; + +import junit.framework.Assert; + + +public class TestInMemoryTableProvider { + @Test + public void testGenerateConfig() { + Map tableSpecConfig = new HashMap<>(); + tableSpecConfig.put("inmemory.c1", "c1-value"); + tableSpecConfig.put("inmemory.c2", "c2-value"); + tableSpecConfig.put("c3", "c3-value"); + tableSpecConfig.put("c4", "c4-value"); + + TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()), + "my-table-provider-factory", tableSpecConfig); + + Map config = new HashMap<>(); + config.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); + config.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); + + TableProvider tableProvider = new InMemoryTableProvider(tableSpec); + Map tableConfig = tableProvider.generateConfig(config); + + Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); + Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); + Assert.assertEquals( + InMemoryKeyValueStorageEngineFactory.class.getName(), + tableConfig.get(String.format(StorageConfig.FACTORY(), "t1"))); + Assert.assertEquals("c1-value", tableConfig.get("stores.t1.c1")); + Assert.assertEquals("c2-value", tableConfig.get("stores.t1.c2")); + Assert.assertEquals("c3-value", tableConfig.get("tables.t1.c3")); + Assert.assertEquals("c4-value", tableConfig.get("tables.t1.c4")); + } +} diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java new file mode 100644 index 0000000000..2c6215943f --- /dev/null +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java @@ -0,0 +1,232 @@ +/* + * 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.samza.storage.kv; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.table.TableSpec; + + +/** + * Table descriptor for RocksDb backed tables + * + * @param the type of the key + * @param the type of the value + */ +public class RocksDbTableDescriptor extends BaseLocalStoreBackedTableDescriptor> { + + static final public String WRITE_BATCH_SIZE = "write.batch.size"; + static final public String OBJECT_CACHE_SIZE = "object.cache.size"; + static final public String CONTAINER_CACHE_SIZE_BYTES = "container.cache.size.bytes"; + static final public String CONTAINER_WRITE_BUFFER_SIZE_BYTES = "container.write.buffer.size.bytes"; + static final public String ROCKSDB_COMPRESSION = "rocksdb.compression"; + static final public String ROCKSDB_BLOCK_SIZE_BYTES = "rocksdb.block.size.bytes"; + static final public String ROCKSDB_TTL_MS = "rocksdb.ttl.ms"; + static final public String ROCKSDB_COMPACTION_STYLE = "rocksdb.compaction.style"; + static final public String ROCKSDB_NUM_WRITE_BUFFERS = "rocksdb.num.write.buffers"; + static final public String ROCKSDB_MAX_LOG_FILE_SIZE_BYTES = "rocksdb.max.log.file.size.bytes"; + static final public String ROCKSDB_KEEP_LOG_FILE_NUM = "rocksdb.keep.log.file.num"; + + protected Integer writeBatchSize; + protected Integer objectCacheSize; + private Integer cacheSize; + private Integer writeBufferSize; + private Integer blockSize; + private Integer ttl; + private Integer numWriteBuffers; + private Integer maxLogFileSize; + private Integer numLogFilesToKeep; + private String compressionType; + private String compactionStyle; + + public RocksDbTableDescriptor(String tableId) { + super(tableId); + } + + /** + * Refer to stores.store-name.write.batch.size in Samza configuration guide + * @param writeBatchSize write batch size + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withWriteBatchSize(int writeBatchSize) { + this.writeBatchSize = writeBatchSize; + return this; + } + + /** + * Refer to stores.store-name.object.cache.size in Samza configuration guide + * @param objectCacheSize the object cache size + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withObjectCacheSize(int objectCacheSize) { + this.objectCacheSize = objectCacheSize; + return this; + } + + /** + * Refer to stores.store-name.container.cache.size.bytes in Samza configuration guide + * @param cacheSize the cache size in bytes + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withCacheSize(int cacheSize) { + this.cacheSize = cacheSize; + return this; + } + + /** + * Refer to stores.store-name.container.write.buffer.size.bytes in Samza configuration guide + * @param writeBufferSize the write buffer size in bytes + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withWriteBufferSize(int writeBufferSize) { + this.writeBufferSize = writeBufferSize; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.compression in Samza configuration guide + * @param compressionType the compression type + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withCompressionType(String compressionType) { + this.compressionType = compressionType; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.block.size.bytes in Samza configuration guide + * @param blockSize the block size in bytes + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withBlockSize(int blockSize) { + this.blockSize = blockSize; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.ttl.ms in Samza configuration guide + * @param ttl the time to live in milliseconds + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withTtl(int ttl) { + this.ttl = ttl; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.compaction.style in Samza configuration guide + * @param compactionStyle the compaction style + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withCompactionStyle(String compactionStyle) { + this.compactionStyle = compactionStyle; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.num.write.buffers in Samza configuration guide + * @param numWriteBuffers the number of write buffers + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withNumWriteBuffers(int numWriteBuffers) { + this.numWriteBuffers = numWriteBuffers; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.max.log.file.size.bytes in Samza configuration guide + * @param maxLogFileSize the maximal log file size in bytes + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withMaxLogFileSize(int maxLogFileSize) { + this.maxLogFileSize = maxLogFileSize; + return this; + } + + /** + * Refer to stores.store-name.rocksdb.num.write.buffers in Samza configuration guide + * @param numLogFilesToKeep the number of log files to keep + * @return this table descriptor instance + */ + public RocksDbTableDescriptor withNumLogFilesToKeep(int numLogFilesToKeep) { + this.numLogFilesToKeep = numLogFilesToKeep; + return this; + } + + /** + * Create a table spec based on this table description + * @return the table spec + */ + @Override + public TableSpec getTableSpec() { + + validate(); + + Map tableSpecConfig = new HashMap<>(); + generateTableSpecConfig(tableSpecConfig); + + return new TableSpec(tableId, serde, RocksDbTableProviderFactory.class.getName(), tableSpecConfig); + } + + @Override + protected void generateTableSpecConfig(Map tableSpecConfig) { + + super.generateTableSpecConfig(tableSpecConfig); + + if (writeBatchSize != null) { + addRocksDbConfig(tableSpecConfig, WRITE_BATCH_SIZE, writeBatchSize.toString()); + } + if (objectCacheSize != null) { + addRocksDbConfig(tableSpecConfig, OBJECT_CACHE_SIZE, objectCacheSize.toString()); + } + if (cacheSize != null) { + addRocksDbConfig(tableSpecConfig, CONTAINER_CACHE_SIZE_BYTES, cacheSize.toString()); + } + if (writeBufferSize != null) { + addRocksDbConfig(tableSpecConfig, CONTAINER_WRITE_BUFFER_SIZE_BYTES, writeBufferSize.toString()); + } + if (compressionType != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_COMPRESSION, compressionType); + } + if (blockSize != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_BLOCK_SIZE_BYTES, blockSize.toString()); + } + if (ttl != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_TTL_MS, ttl.toString()); + } + if (compactionStyle != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_COMPACTION_STYLE, compactionStyle); + } + if (numWriteBuffers != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_NUM_WRITE_BUFFERS, numWriteBuffers.toString()); + } + if (maxLogFileSize != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_MAX_LOG_FILE_SIZE_BYTES, maxLogFileSize.toString()); + } + if (numLogFilesToKeep != null) { + addRocksDbConfig(tableSpecConfig, ROCKSDB_KEEP_LOG_FILE_NUM, numLogFilesToKeep.toString()); + } + } + + private void addRocksDbConfig(Map map, String key, String value) { + map.put("rocksdb." + key, value); + } + +} diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java new file mode 100644 index 0000000000..eb8188f06d --- /dev/null +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.storage.kv; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.table.TableSpec; + + +/** + * Table provider for tables backed by RocksDb. + */ +public class RocksDbTableProvider extends BaseLocalStoreBackedTableProvider { + + public RocksDbTableProvider(TableSpec tableSpec) { + super(tableSpec); + } + + @Override + public Map generateConfig(Map config) { + + Map tableConfig = new HashMap<>(); + + // Store factory configuration + tableConfig.put(String.format( + StorageConfig.FACTORY(), tableSpec.getId()), + RocksDbKeyValueStorageEngineFactory.class.getName()); + + // Common store configuration + tableConfig.putAll(generateCommonStoreConfig(config)); + + // Rest of the configuration + tableSpec.getConfig().forEach((k, v) -> { + String realKey = k.startsWith("rocksdb.") ? + String.format("stores.%s", tableSpec.getId()) + "." + k.substring("rocksdb.".length()) + : String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; + tableConfig.put(realKey, v); + }); + + logger.info("Generated configuration for table " + tableSpec.getId()); + + return tableConfig; + } + +} diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java new file mode 100644 index 0000000000..dbe0f97a3c --- /dev/null +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProviderFactory.java @@ -0,0 +1,31 @@ +/* + * 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.samza.storage.kv; + +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableProviderFactory; +import org.apache.samza.table.TableSpec; + + +public class RocksDbTableProviderFactory implements TableProviderFactory { + @Override + public TableProvider getTableProvider(TableSpec tableSpec) { + return new RocksDbTableProvider(tableSpec); + } +} diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java new file mode 100644 index 0000000000..49fe6eb975 --- /dev/null +++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.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.samza.storage.kv; + +import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.table.TableSpec; +import org.junit.Test; + +import junit.framework.Assert; + + +public class TestRocksDbTableDescriptor { + + @Test + public void testMinimal() { + new RocksDbTableDescriptor("1") + .validate(); + } + + @Test + public void testSerde() { + TableSpec tableSpec = new RocksDbTableDescriptor("1") + .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde())) + .getTableSpec(); + Assert.assertNotNull(tableSpec.getSerde()); + Assert.assertEquals(tableSpec.getSerde().getKeySerde().getClass(), IntegerSerde.class); + Assert.assertEquals(tableSpec.getSerde().getValueSerde().getClass(), StringSerde.class); + } + + @Test + public void testTableSpec() { + + TableSpec tableSpec = new RocksDbTableDescriptor("1") + .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde())) + .withBlockSize(1) + .withCacheSize(2) + .withCompactionStyle("fifo") + .withCompressionType("snappy") + .withMaxLogFileSize(3) + .withNumLogFilesToKeep(4) + .withNumWriteBuffers(5) + .withObjectCacheSize(6) + .withTtl(7) + .withWriteBatchSize(8) + .withWriteBufferSize(9) + .withConfig("rocksdb.abc", "xyz") + .getTableSpec(); + + Assert.assertNotNull(tableSpec.getSerde()); + Assert.assertNotNull(tableSpec.getSerde().getKeySerde()); + Assert.assertNotNull(tableSpec.getSerde().getValueSerde()); + Assert.assertEquals("1", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_BLOCK_SIZE_BYTES)); + Assert.assertEquals("2", getConfig(tableSpec, RocksDbTableDescriptor.CONTAINER_CACHE_SIZE_BYTES)); + Assert.assertEquals("3", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_MAX_LOG_FILE_SIZE_BYTES)); + Assert.assertEquals("4", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_KEEP_LOG_FILE_NUM)); + Assert.assertEquals("5", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_NUM_WRITE_BUFFERS)); + Assert.assertEquals("6", getConfig(tableSpec, RocksDbTableDescriptor.OBJECT_CACHE_SIZE)); + Assert.assertEquals("7", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_TTL_MS)); + Assert.assertEquals("8", getConfig(tableSpec, RocksDbTableDescriptor.WRITE_BATCH_SIZE)); + Assert.assertEquals("9", getConfig(tableSpec, RocksDbTableDescriptor.CONTAINER_WRITE_BUFFER_SIZE_BYTES)); + Assert.assertEquals("snappy", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPRESSION)); + Assert.assertEquals("fifo", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPACTION_STYLE)); + Assert.assertEquals("xyz", getConfig(tableSpec, "abc")); + } + + private String getConfig(TableSpec tableSpec, String key) { + return tableSpec.getConfig().get("rocksdb." + key); + } +} diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java new file mode 100644 index 0000000000..beda5da49c --- /dev/null +++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java @@ -0,0 +1,66 @@ +/* + * 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.samza.storage.kv; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableSpec; +import org.junit.Test; + +import junit.framework.Assert; + + +public class TestRocksDbTableProvider { + @Test + public void testGenerateConfig() { + + Map tableSpecConfig = new HashMap<>(); + tableSpecConfig.put("rocksdb.c1", "c1-value"); + tableSpecConfig.put("rocksdb.c2", "c2-value"); + tableSpecConfig.put("c3", "c3-value"); + tableSpecConfig.put("c4", "c4-value"); + + TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()), + "my-table-provider-factory", tableSpecConfig); + + Map config = new HashMap<>(); + config.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); + config.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); + + TableProvider tableProvider = new RocksDbTableProvider(tableSpec); + Map tableConfig = tableProvider.generateConfig(config); + + Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); + Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); + Assert.assertEquals( + RocksDbKeyValueStorageEngineFactory.class.getName(), + tableConfig.get(String.format(StorageConfig.FACTORY(), "t1"))); + Assert.assertEquals("c1-value", tableConfig.get("stores.t1.c1")); + Assert.assertEquals("c2-value", tableConfig.get("stores.t1.c2")); + Assert.assertEquals("c3-value", tableConfig.get("tables.t1.c3")); + Assert.assertEquals("c4-value", tableConfig.get("tables.t1.c4")); + } +} diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java new file mode 100644 index 0000000000..1f9b57b854 --- /dev/null +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java @@ -0,0 +1,56 @@ +/* + * 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.samza.storage.kv; + +import java.util.Map; + +import org.apache.samza.operators.BaseTableDescriptor; + + +/** + * Table descriptor for store backed tables. + * + * @param the type of the key in this table + * @param the type of the value in this table + * @param the type of the concrete table descriptor + */ +abstract public class BaseLocalStoreBackedTableDescriptor> + extends BaseTableDescriptor { + + /** + * Constructs a table descriptor instance + * @param tableId Id of the table + */ + public BaseLocalStoreBackedTableDescriptor(String tableId) { + super(tableId); + } + + @Override + protected void generateTableSpecConfig(Map tableSpecConfig) { + super.generateTableSpecConfig(tableSpecConfig); + } + + /** + * Validate that this table descriptor is constructed properly + */ + protected void validate() { + super.validate(); + } + +} diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java new file mode 100644 index 0000000000..4af0f1dc78 --- /dev/null +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.storage.kv; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.storage.StorageEngine; +import org.apache.samza.table.LocalStoreBackedTableProvider; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Base class for tables backed by Samza stores, see {@link LocalStoreBackedTableProvider}. + */ +abstract public class BaseLocalStoreBackedTableProvider implements LocalStoreBackedTableProvider { + + protected final Logger logger = LoggerFactory.getLogger(getClass()); + + protected final TableSpec tableSpec; + + protected KeyValueStore kvStore; + + public BaseLocalStoreBackedTableProvider(TableSpec tableSpec) { + this.tableSpec = tableSpec; + } + + @Override + public void init(StorageEngine store) { + kvStore = (KeyValueStore) store; + logger.info("Initialized backing store for table " + tableSpec.getId()); + } + + @Override + public Table getTable() { + if (kvStore == null) { + throw new SamzaException("Store not initialized for table " + tableSpec.getId()); + } + return new LocalStoreBackedReadWriteTable(kvStore); + } + + @Override + public void start() { + logger.info("Starting table provider for table " + tableSpec.getId()); + } + + @Override + public void stop() { + logger.info("Stopping table provider for table " + tableSpec.getId()); + } + + protected Map generateCommonStoreConfig(Map config) { + + Map storeConfig = new HashMap<>(); + + // We assume the configuration for serde are already generated for this table, + // so we simply carry them over to store configuration. + // + JavaTableConfig tableConfig = new JavaTableConfig(new MapConfig(config)); + + String keySerde = tableConfig.getKeySerde(tableSpec.getId()); + storeConfig.put(String.format(StorageConfig.KEY_SERDE(), tableSpec.getId()), keySerde); + + String valueSerde = tableConfig.getValueSerde(tableSpec.getId()); + storeConfig.put(String.format(StorageConfig.MSG_SERDE(), tableSpec.getId()), valueSerde); + + return storeConfig; + } +} diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java new file mode 100644 index 0000000000..3149c86eaf --- /dev/null +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java @@ -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.samza.storage.kv; + +import java.util.List; + +import org.apache.samza.table.ReadWriteTable; + + +/** + * A store backed readable and writable table + * + * @param the type of the key in this table + * @param the type of the value in this table + */ +public class LocalStoreBackedReadWriteTable extends LocalStoreBackedReadableTable + implements ReadWriteTable { + + /** + * Constructs an instance of {@link LocalStoreBackedReadWriteTable} + * @param kvStore the backing store + */ + public LocalStoreBackedReadWriteTable(KeyValueStore kvStore) { + super(kvStore); + } + + @Override + public void put(K key, V value) { + kvStore.put(key, value); + } + + @Override + public void putAll(List> entries) { + entries.forEach(e -> kvStore.put(e.getKey(), e.getValue())); + } + + @Override + public void delete(K key) { + kvStore.delete(key); + } + + @Override + public void deleteAll(List keys) { + keys.forEach(k -> kvStore.delete(k)); + } + + @Override + public void flush() { + kvStore.flush(); + } + +} diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java new file mode 100644 index 0000000000..fead086348 --- /dev/null +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.storage.kv; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.samza.table.ReadableTable; + + +/** + * A store backed readable table + * + * @param the type of the key in this table + * @param the type of the value in this table + */ +public class LocalStoreBackedReadableTable implements ReadableTable { + + protected KeyValueStore kvStore; + + /** + * Constructs an instance of {@link LocalStoreBackedReadableTable} + * @param kvStore the backing store + */ + public LocalStoreBackedReadableTable(KeyValueStore kvStore) { + this.kvStore = kvStore; + } + + @Override + public V get(K key) { + return kvStore.get(key); + } + + @Override + public Map getAll(List keys) { + return keys.stream().collect(Collectors.toMap(k -> k, k -> kvStore.get(k))); + } + + @Override + public void close() { + // The KV store is not closed here as it may still be needed by downstream operators, + // it will be closed by the SamzaContainer + } +} diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java new file mode 100644 index 0000000000..9c95637834 --- /dev/null +++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.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.samza.storage.kv; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.storage.StorageEngine; +import org.apache.samza.table.TableSpec; +import org.junit.Before; +import org.junit.Test; + +import junit.framework.Assert; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +public class TestLocalBaseStoreBackedTableProvider { + + private BaseLocalStoreBackedTableProvider tableProvider; + + @Before + public void prepare() { + TableSpec tableSpec = mock(TableSpec.class); + when(tableSpec.getId()).thenReturn("t1"); + tableProvider = new BaseLocalStoreBackedTableProvider(tableSpec) { + @Override + public Map generateConfig(Map config) { + return generateCommonStoreConfig(config); + } + }; + } + + @Test + public void testInit() { + StorageEngine store = mock(KeyValueStorageEngine.class); + tableProvider.init(store); + Assert.assertNotNull(tableProvider.getTable()); + } + + @Test(expected = SamzaException.class) + public void testInitFail() { + Assert.assertNotNull(tableProvider.getTable()); + } + + @Test + public void testGenerateCommonStoreConfig() { + Map config = new HashMap<>(); + config.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); + config.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); + + Map tableConfig = tableProvider.generateConfig(config); + Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); + Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); + } +} diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java new file mode 100644 index 0000000000..8f7eb5d3fd --- /dev/null +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java @@ -0,0 +1,304 @@ +/* + * 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.samza.test.table; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.functions.MapFunction; +import org.apache.samza.operators.functions.StreamTableJoinFunction; +import org.apache.samza.runtime.LocalApplicationRunner; +import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.NoOpSerde; +import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory; +import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; +import org.apache.samza.storage.kv.inmemory.InMemoryTableDescriptor; +import org.apache.samza.table.ReadableTable; +import org.apache.samza.table.Table; +import org.apache.samza.task.TaskContext; +import org.apache.samza.test.harness.AbstractIntegrationTestHarness; +import org.apache.samza.test.table.TestTableData.EnrichedPageView; +import org.apache.samza.test.table.TestTableData.PageView; +import org.apache.samza.test.table.TestTableData.PageViewJsonSerde; +import org.apache.samza.test.table.TestTableData.PageViewJsonSerdeFactory; +import org.apache.samza.test.table.TestTableData.Profile; +import org.apache.samza.test.table.TestTableData.ProfileJsonSerde; +import org.apache.samza.test.util.ArraySystemFactory; +import org.apache.samza.test.util.Base64Serializer; +import org.junit.Assert; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * This test class tests sendTo() and join() for local tables + */ +public class TestLocalTable extends AbstractIntegrationTestHarness { + + @Test + public void testSendTo() throws Exception { + + int count = 10; + Profile[] profiles = TestTableData.generateProfiles(count); + + int partitionCount = 4; + Map configs = getBaseJobConfig(); + + configs.put("streams.Profile.samza.system", "test"); + configs.put("streams.Profile.source", Base64Serializer.serialize(profiles)); + configs.put("streams.Profile.partitionCount", String.valueOf(partitionCount)); + + MyMapFunction mapFn = new MyMapFunction(); + + final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); + final StreamApplication app = (streamGraph, cfg) -> { + + Table> table = streamGraph.getTable(new InMemoryTableDescriptor("t1") + .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))); + + streamGraph.getInputStream("Profile", new NoOpSerde()) + .map(mapFn) + .sendTo(table); + }; + + runner.run(app); + runner.waitForFinish(); + + assertEquals(count * partitionCount, mapFn.received.size()); + assertEquals(count, new HashSet(mapFn.received).size()); + mapFn.received.forEach(p -> Assert.assertTrue(mapFn.table.get(p.getMemberId()) != null)); + } + + @Test + public void testStreamTableJoin() throws Exception { + + List received = new LinkedList<>(); + List joined = new LinkedList<>(); + + int count = 10; + PageView[] pageViews = TestTableData.generatePageViews(count); + Profile[] profiles = TestTableData.generateProfiles(count); + + int partitionCount = 4; + Map configs = getBaseJobConfig(); + + configs.put("streams.PageView.samza.system", "test"); + configs.put("streams.PageView.source", Base64Serializer.serialize(pageViews)); + configs.put("streams.PageView.partitionCount", String.valueOf(partitionCount)); + + configs.put("streams.Profile.samza.system", "test"); + configs.put("streams.Profile.samza.bootstrap", "true"); + configs.put("streams.Profile.source", Base64Serializer.serialize(profiles)); + configs.put("streams.Profile.partitionCount", String.valueOf(partitionCount)); + + final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); + final StreamApplication app = (streamGraph, cfg) -> { + + Table> table = streamGraph.getTable(new InMemoryTableDescriptor("t1") + .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))); + + streamGraph.getInputStream("Profile", new NoOpSerde()) + .map(m -> new KV(m.getMemberId(), m)) + .sendTo(table); + + streamGraph.getInputStream("PageView", new NoOpSerde()) + .map(pv -> { + received.add(pv); + return pv; + }) + .partitionBy(PageView::getMemberId, v -> v, "p1") + .join(table, new PageViewToProfileJoinFunction()) + .sink((m, collector, coordinator) -> joined.add(m)); + }; + + runner.run(app); + runner.waitForFinish(); + + assertEquals(count * partitionCount, received.size()); + assertEquals(count * partitionCount, joined.size()); + assertTrue(joined.get(0) instanceof EnrichedPageView); + } + + @Test + public void testDualStreamTableJoin() throws Exception { + + List sentToProfileTable1 = new LinkedList<>(); + List sentToProfileTable2 = new LinkedList<>(); + List joinedPageViews1 = new LinkedList<>(); + List joinedPageViews2 = new LinkedList<>(); + + KVSerde profileKVSerde = KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()); + KVSerde pageViewKVSerde = KVSerde.of(new IntegerSerde(), new PageViewJsonSerde()); + + PageViewToProfileJoinFunction joinFn1 = new PageViewToProfileJoinFunction(); + PageViewToProfileJoinFunction joinFn2 = new PageViewToProfileJoinFunction(); + + int count = 10; + PageView[] pageViews = TestTableData.generatePageViews(count); + Profile[] profiles = TestTableData.generateProfiles(count); + + int partitionCount = 4; + Map configs = getBaseJobConfig(); + + configs.put("streams.Profile1.samza.system", "test"); + configs.put("streams.Profile1.source", Base64Serializer.serialize(profiles)); + configs.put("streams.Profile1.samza.bootstrap", "true"); + configs.put("streams.Profile1.partitionCount", String.valueOf(partitionCount)); + + configs.put("streams.Profile2.samza.system", "test"); + configs.put("streams.Profile2.source", Base64Serializer.serialize(profiles)); + configs.put("streams.Profile2.samza.bootstrap", "true"); + configs.put("streams.Profile2.partitionCount", String.valueOf(partitionCount)); + + configs.put("streams.PageView1.samza.system", "test"); + configs.put("streams.PageView1.source", Base64Serializer.serialize(pageViews)); + configs.put("streams.PageView1.partitionCount", String.valueOf(partitionCount)); + + configs.put("streams.PageView2.samza.system", "test"); + configs.put("streams.PageView2.source", Base64Serializer.serialize(pageViews)); + configs.put("streams.PageView2.partitionCount", String.valueOf(partitionCount)); + + final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); + final StreamApplication app = (streamGraph, cfg) -> { + + Table> profileTable = streamGraph.getTable(new InMemoryTableDescriptor("t1") + .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))); + + MessageStream profileStream1 = streamGraph.getInputStream("Profile1", new NoOpSerde()); + MessageStream profileStream2 = streamGraph.getInputStream("Profile2", new NoOpSerde()); + + profileStream1 + .map(m -> { + sentToProfileTable1.add(m); + return new KV(m.getMemberId(), m); + }) + .sendTo(profileTable); + profileStream2 + .map(m -> { + sentToProfileTable2.add(m); + return new KV(m.getMemberId(), m); + }) + .sendTo(profileTable); + + MessageStream pageViewStream1 = streamGraph.getInputStream("PageView1", new NoOpSerde()); + MessageStream pageViewStream2 = streamGraph.getInputStream("PageView2", new NoOpSerde()); + + pageViewStream1 + .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p1") + .join(profileTable, joinFn1) + .sink((m, collector, coordinator) -> joinedPageViews1.add(m)); + + pageViewStream2 + .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p2") + .join(profileTable, joinFn2) + .sink((m, collector, coordinator) -> joinedPageViews2.add(m)); + }; + + runner.run(app); + runner.waitForFinish(); + + assertEquals(count * partitionCount, sentToProfileTable1.size()); + assertEquals(count * partitionCount, sentToProfileTable2.size()); + assertEquals(count * partitionCount, joinFn1.count); + assertEquals(count * partitionCount, joinFn2.count); + assertEquals(count * partitionCount, joinedPageViews1.size()); + assertEquals(count * partitionCount, joinedPageViews2.size()); + assertTrue(joinedPageViews1.get(0) instanceof EnrichedPageView); + assertTrue(joinedPageViews2.get(0) instanceof EnrichedPageView); + } + + private Map getBaseJobConfig() { + Map configs = new HashMap<>(); + configs.put("systems.test.samza.factory", ArraySystemFactory.class.getName()); + + configs.put(JobConfig.JOB_NAME(), "test-table-job"); + configs.put(JobConfig.PROCESSOR_ID(), "1"); + configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName()); + configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName()); + configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName()); + + // For intermediate streams + configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory"); + configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl()); + configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect()); + configs.put("systems.kafka.samza.key.serde", "int"); + configs.put("systems.kafka.samza.msg.serde", "json"); + configs.put("systems.kafka.default.stream.replication.factor", "1"); + configs.put("job.default.system", "kafka"); + + configs.put("serializers.registry.int.class", "org.apache.samza.serializers.IntegerSerdeFactory"); + configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName()); + + return configs; + } + + private class MyMapFunction implements MapFunction> { + + private List received = new ArrayList<>(); + private ReadableTable table; + + @Override + public void init(Config config, TaskContext context) { + table = (ReadableTable) context.getTable("t1"); + } + + @Override + public KV apply(Profile profile) { + received.add(profile); + return new KV(profile.getMemberId(), profile); + } + } + + private class PageViewToProfileJoinFunction implements StreamTableJoinFunction + , KV, EnrichedPageView> { + private int count; + @Override + public EnrichedPageView apply(KV m, KV r) { + ++count; + return r == null ? null : + new EnrichedPageView(m.getValue().getPageKey(), m.getKey(), r.getValue().getCompany()); + } + + @Override + public Integer getMessageKey(KV message) { + return message.getKey(); + } + + @Override + public Integer getRecordKey(KV record) { + return record.getKey(); + } + } +} diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestTableData.java b/samza-test/src/test/java/org/apache/samza/test/table/TestTableData.java new file mode 100644 index 0000000000..dfd0d1ba77 --- /dev/null +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestTableData.java @@ -0,0 +1,200 @@ +/* + * 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.samza.test.table; + +import java.io.Serializable; +import java.util.Random; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerdeFactory; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.type.TypeReference; + + +public class TestTableData { + + public static class PageView implements Serializable { + @JsonProperty("pageKey") + final String pageKey; + @JsonProperty("memberId") + final int memberId; + + @JsonProperty("pageKey") + public String getPageKey() { + return pageKey; + } + + @JsonProperty("memberId") + public int getMemberId() { + return memberId; + } + + @JsonCreator + public PageView(@JsonProperty("pageKey") String pageKey, @JsonProperty("memberId") int memberId) { + this.pageKey = pageKey; + this.memberId = memberId; + } + } + + public static class Profile implements Serializable { + @JsonProperty("memberId") + final int memberId; + + @JsonProperty("company") + final String company; + + @JsonProperty("memberId") + public int getMemberId() { + return memberId; + } + + @JsonProperty("company") + public String getCompany() { + return company; + } + + @JsonCreator + public Profile(@JsonProperty("memberId") int memberId, @JsonProperty("company") String company) { + this.memberId = memberId; + this.company = company; + } + + @Override + public boolean equals(Object o) { + if (o == null || !(o instanceof Profile)) { + return false; + } + return ((Profile) o).getMemberId() == memberId; + } + + @Override + public int hashCode() { + return memberId; + } + } + + public static class EnrichedPageView extends PageView { + + @JsonProperty("company") + final String company; + + @JsonProperty("company") + public String getCompany() { + return company; + } + + @JsonCreator + public EnrichedPageView( + @JsonProperty("pageKey") String pageKey, + @JsonProperty("memberId") int memberId, + @JsonProperty("company") String company) { + super(pageKey, memberId); + this.company = company; + } + } + + public static class PageViewJsonSerdeFactory implements SerdeFactory { + @Override public Serde getSerde(String name, Config config) { + return new PageViewJsonSerde(); + } + } + + public static class ProfileJsonSerdeFactory implements SerdeFactory { + @Override public Serde getSerde(String name, Config config) { + return new ProfileJsonSerde(); + } + } + + public static class PageViewJsonSerde implements Serde { + + @Override + public PageView fromBytes(byte[] bytes) { + try { + ObjectMapper mapper = new ObjectMapper(); + return mapper.readValue(new String(bytes, "UTF-8"), new TypeReference() { }); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public byte[] toBytes(PageView pv) { + try { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(pv).getBytes("UTF-8"); + } catch (Exception e) { + throw new SamzaException(e); + } + } + } + + public static class ProfileJsonSerde implements Serde { + + @Override + public Profile fromBytes(byte[] bytes) { + try { + ObjectMapper mapper = new ObjectMapper(); + return mapper.readValue(new String(bytes, "UTF-8"), new TypeReference() { }); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public byte[] toBytes(Profile p) { + try { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(p).getBytes("UTF-8"); + } catch (Exception e) { + throw new SamzaException(e); + } + } + } + + private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"}; + + static public PageView[] generatePageViews(int count) { + Random random = new Random(); + PageView[] pageviews = new PageView[count]; + for (int i = 0; i < count; i++) { + String pagekey = PAGEKEYS[random.nextInt(PAGEKEYS.length - 1)]; + int memberId = random.nextInt(10); + pageviews[i] = new PageView(pagekey, memberId); + } + return pageviews; + } + + private static final String[] COMPANIES = {"MSFT", "LKND", "GOOG", "FB", "AMZN", "CSCO"}; + + static public Profile[] generateProfiles(int count) { + Random random = new Random(); + Profile[] profiles = new Profile[count]; + for (int i = 0; i < count; i++) { + String company = COMPANIES[random.nextInt(COMPANIES.length - 1)]; + profiles[i] = new Profile(i, company); + } + return profiles; + } + +} diff --git a/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java index 832457b121..6ba28ae50c 100644 --- a/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java +++ b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.samza.config.Config; import org.apache.samza.system.IncomingMessageEnvelope; @@ -58,9 +59,10 @@ public void register(SystemStreamPartition systemStreamPartition, String s) { public Map> poll(Set set, long l) throws InterruptedException { if (!done) { Map> envelopeMap = new HashMap<>(); + final AtomicInteger offset = new AtomicInteger(0); set.forEach(ssp -> { List envelopes = Arrays.stream(getArrayObjects(ssp.getSystemStream().getStream(), config)) - .map(object -> new IncomingMessageEnvelope(ssp, null, null, object)).collect(Collectors.toList()); + .map(object -> new IncomingMessageEnvelope(ssp, String.valueOf(offset.incrementAndGet()), null, object)).collect(Collectors.toList()); envelopes.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp)); envelopeMap.put(ssp, envelopes); }); diff --git a/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java b/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java index 8890a2f066..c735c74f82 100644 --- a/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java +++ b/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java @@ -24,7 +24,9 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; + import org.apache.samza.Partition; +import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStreamMetadata; @@ -50,13 +52,17 @@ public Map getOffsetsAfter(Map getSystemStreamMetadata(Set streamNames) { return streamNames.stream() - .collect(Collectors.toMap( - Function.identity(), - streamName -> { + .collect(Collectors.toMap(Function.identity(), streamName -> { + int messageCount = isBootstrapStream(streamName) ? getMessageCount(streamName) : -1; + String oldestOffset = messageCount < 0 ? null : "0"; + String newestOffset = messageCount < 0 ? null : String.valueOf(messageCount - 1); + String upcomingOffset = messageCount < 0 ? null : String.valueOf(messageCount); Map metadataMap = new HashMap<>(); int partitionCount = config.getInt("streams." + streamName + ".partitionCount", 1); for (int i = 0; i < partitionCount; i++) { - metadataMap.put(new Partition(i), new SystemStreamMetadata.SystemStreamPartitionMetadata(null, null, null)); + metadataMap.put(new Partition(i), new SystemStreamMetadata.SystemStreamPartitionMetadata( + oldestOffset, newestOffset, upcomingOffset + )); } return new SystemStreamMetadata(streamName, metadataMap); })); @@ -71,5 +77,17 @@ public Integer offsetComparator(String offset1, String offset2) { } return offset1.compareTo(offset2); } + + private int getMessageCount(String streamName) { + try { + return Base64Serializer.deserialize(config.get("streams." + streamName + ".source"), Object[].class).length; + } catch (Exception e) { + throw new SamzaException(e); + } + } + + private boolean isBootstrapStream(String streamName) { + return "true".equalsIgnoreCase(config.get("streams." + streamName + ".samza.bootstrap", "false")); + } } From 1d420e750eb8dc7f0715f7e11e01af1e9ce61506 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 11 Dec 2017 16:45:44 -0800 Subject: [PATCH 08/36] SAMZA-1532; Eventhub connector fix Key fixes vjagadish1989 lhaiesp srinipunuru - Switched Producer source vs destination assumptions in `send`, `register` - Check `OME.key` if `OME.partitionId` is null for to get partitionId - Upcoming offset changed the `END_OF_STREAM` rather than `newestOffset` + 1, eventHub returns an error if the offset does not exist in the system - Made the NewestOffset+1 as upcoming offset, consumer checks if the offset is valid on startup - Differentiated between streamNames and streamIds in configs, consumer, producer - Checkpoint table named after job name - Checkpoint prints better message for invalid key on write QOL - How to ignore integration tests - Improved logging EDIT: - Also added Round Robin producer partitioning Author: Daniel Chen <29577458+dxichen@users.noreply.github.com> Reviewers: Jagadish Closes #377 from dxichen/eventhub-connector-fix --- build.gradle | 5 - .../azure/AzureCheckpointManager.java | 31 +++++- .../azure/AzureCheckpointManagerFactory.java | 4 +- .../samza/system/eventhub/EventHubConfig.java | 71 ++++++++++-- .../eventhub/SamzaEventHubClientManager.java | 4 +- .../eventhub/admin/EventHubSystemAdmin.java | 17 +-- .../consumer/EventHubSystemConsumer.java | 77 +++++++++---- .../producer/EventHubSystemProducer.java | 103 ++++++++++-------- .../azure/ITestAzureCheckpointManager.java | 2 +- .../MockEventHubClientManagerFactory.java | 18 +++ .../admin/TestEventHubSystemAdmin.java | 15 +-- .../consumer/ITestEventHubSystemConsumer.java | 5 +- .../consumer/TestEventHubSystemConsumer.java | 19 +++- .../producer/ITestEventHubSystemProducer.java | 2 + .../producer/TestEventHubSystemProducer.java | 34 +++--- 15 files changed, 272 insertions(+), 135 deletions(-) diff --git a/build.gradle b/build.gradle index be1baf7b8d..330ff0b5e3 100644 --- a/build.gradle +++ b/build.gradle @@ -213,11 +213,6 @@ project(':samza-azure') { configFile = new File(rootDir, "checkstyle/checkstyle.xml") toolVersion = "$checkstyleVersion" } - test { - // Exclude integration tests that require connection to EventHub - exclude 'org/apache/samza/system/eventhub/producer/*ITest*' - exclude 'org/apache/samza/system/eventhub/consumer/*ITest*' - } } project(':samza-aws') { diff --git a/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java b/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java index df3e4904ce..2cad3bd6ee 100644 --- a/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java +++ b/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java @@ -34,6 +34,7 @@ import org.apache.samza.system.SystemStreamPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Option; import java.net.URISyntaxException; import java.util.HashMap; @@ -60,12 +61,16 @@ public class AzureCheckpointManager implements CheckpointManager { private static final Logger LOG = LoggerFactory.getLogger(AzureCheckpointManager.class.getName()); private static final String PARTITION_KEY = "PartitionKey"; + // Invalid characters in key field on Azure Table + public static final String REGEX_INVALID_KEY = ".*[#?/\\\\].*"; + public static final String REGEX_TABLE_NAME = "[^A-Za-z0-9]"; + public static final int MAX_WRITE_BATCH_SIZE = 100; - public static final String CHECKPOINT_MANAGER_TABLE_NAME = "SamzaTaskCheckpoints"; public static final String SYSTEM_PROP_NAME = "system"; public static final String STREAM_PROP_NAME = "stream"; public static final String PARTITION_PROP_NAME = "partition"; + private final String jobTableName; private final String storageConnectionString; private final AzureClient azureClient; private CloudTable cloudTable; @@ -73,7 +78,12 @@ public class AzureCheckpointManager implements CheckpointManager { private final Set taskNames = new HashSet<>(); private final JsonSerdeV2> jsonSerde = new JsonSerdeV2<>(); - AzureCheckpointManager(AzureConfig azureConfig) { + AzureCheckpointManager(AzureConfig azureConfig, Option jobName) { + if (!jobName.isDefined()) { + throw new AzureException("Jobs must have a name to use Azure Checkpoint Manager"); + } + // Remove invalid characters + jobTableName = jobName.get().replaceAll(REGEX_TABLE_NAME, ""); storageConnectionString = azureConfig.getAzureConnectionString(); azureClient = new AzureClient(storageConnectionString); } @@ -82,7 +92,7 @@ public class AzureCheckpointManager implements CheckpointManager { public void start() { try { // Create the table if it doesn't exist. - cloudTable = azureClient.getTableClient().getTableReference(CHECKPOINT_MANAGER_TABLE_NAME); + cloudTable = azureClient.getTableClient().getTableReference(jobTableName); cloudTable.createIfNotExists(); } catch (URISyntaxException e) { @@ -115,9 +125,13 @@ public void writeCheckpoint(TaskName taskName, Checkpoint checkpoint) { SystemStreamPartition ssp = entry.getKey(); String offset = entry.getValue(); + String partitionKey = taskName.toString(); + checkValidKey(partitionKey, "Taskname"); + String rowKey = serializeSystemStreamPartition(ssp); + checkValidKey(rowKey, "SystemStreamPartition"); + // Create table entity - TaskCheckpointEntity taskCheckpoint = new TaskCheckpointEntity(taskName.toString(), - serializeSystemStreamPartition(ssp), offset); + TaskCheckpointEntity taskCheckpoint = new TaskCheckpointEntity(partitionKey, rowKey, offset); // Add to batch operation batchOperation.insertOrReplace(taskCheckpoint); @@ -135,6 +149,13 @@ public void writeCheckpoint(TaskName taskName, Checkpoint checkpoint) { } } + private void checkValidKey(String key, String fieldUsed) { + if (key == null || key.matches(REGEX_INVALID_KEY)) { + throw new AzureException(String.format("Cannot insert to Azure Checkpoint Manager; %s %s contains invalid characters [*, /, \\\\, ?]", + fieldUsed, key)); + } + } + private String serializeSystemStreamPartition(SystemStreamPartition ssp) { // Create the Json string for SystemStreamPartition Map sspMap = new HashMap<>(); diff --git a/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManagerFactory.java b/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManagerFactory.java index 3c5d62a120..95728e39e8 100644 --- a/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManagerFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManagerFactory.java @@ -23,11 +23,13 @@ import org.apache.samza.checkpoint.CheckpointManagerFactory; import org.apache.samza.config.AzureConfig; import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.metrics.MetricsRegistry; public class AzureCheckpointManagerFactory implements CheckpointManagerFactory { @Override public CheckpointManager getCheckpointManager(Config config, MetricsRegistry registry) { - return new AzureCheckpointManager(new AzureConfig(config)); + JobConfig jobConfig = new JobConfig(config); + return new AzureCheckpointManager(new AzureConfig(config), jobConfig.getName()); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index 3bc04f885b..5d8391103e 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -20,10 +20,15 @@ package org.apache.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventHubClient; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StreamConfig; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; +import scala.collection.JavaConversions; import java.time.Duration; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -47,7 +52,7 @@ public class EventHubConfig extends MapConfig { .PartitioningMethod.EVENT_HUB_HASHING.name(); public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "systems.%s.eventhubs.send.key"; - public static final Boolean DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = false; + public static final Boolean DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = true; public static final String CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = "systems.%s.eventhubs.runtime.info.timeout"; public static final long DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); @@ -55,9 +60,47 @@ public class EventHubConfig extends MapConfig { public static final String CONFIG_CONSUMER_BUFFER_CAPACITY = "systems.%s.eventhubs.receive.queue.size"; public static final int DEFAULT_CONFIG_CONSUMER_BUFFER_CAPACITY = 100; + private final Map physcialToId = new HashMap<>(); - public EventHubConfig(Map config) { + public EventHubConfig(Config config) { super(config); + + // Build reverse index for streamName -> streamId + StreamConfig streamConfig = new StreamConfig(config); + JavaConversions.asJavaCollection(streamConfig.getStreamIds()) + .forEach((streamId) -> physcialToId.put(streamConfig.getPhysicalName(streamId), streamId)); + } + + private String getFromStreamIdOrName(String configName, String systemName, String streamName, String defaultString) { + String result = getFromStreamIdOrName(configName, systemName, streamName); + if (result == null) { + return defaultString; + } + return result; + } + + private String getFromStreamIdOrName(String configName, String systemName, String streamName) { + String streamId = getStreamId(streamName); + return get(String.format(configName, systemName, streamId), + streamId.equals(streamName) ? null : get(String.format(configName, systemName, streamName))); + } + + private String validateRequiredConfig(String value, String fieldName, String systemName, String streamName) { + if (value == null) { + throw new SamzaException(String.format("Missing %s configuration for system: %s, stream: %s", + fieldName, systemName, streamName)); + } + return value; + } + + /** + * Get the streamId for the specified streamName + * + * @param streamName the physical identifier of a stream + * @return the streamId identifier for the stream or the queried streamName if it is not found. + */ + public String getStreamId(String streamName) { + return physcialToId.getOrDefault(streamName, streamName); } /** @@ -75,55 +118,59 @@ public List getStreams(String systemName) { * Get the EventHubs namespace for the stream * * @param systemName name of the system - * @param streamName name of stream + * @param streamName name of stream (physical or streamId) * @return EventHubs namespace */ public String getStreamNamespace(String systemName, String streamName) { - return get(String.format(CONFIG_STREAM_NAMESPACE, systemName, streamName)); + return validateRequiredConfig(getFromStreamIdOrName(CONFIG_STREAM_NAMESPACE, systemName, streamName), + "Namespace", systemName, streamName); } /** * Get the EventHubs entity path (topic name) for the stream * * @param systemName name of the system - * @param streamName name of stream + * @param streamName name of stream (physical or streamId) * @return EventHubs entity path */ public String getStreamEntityPath(String systemName, String streamName) { - return get(String.format(CONFIG_STREAM_ENTITYPATH, systemName, streamName)); + return validateRequiredConfig(getFromStreamIdOrName(CONFIG_STREAM_ENTITYPATH, systemName, streamName), + "EntityPath", systemName, streamName); } /** * Get the EventHubs SAS (Shared Access Signature) key name for the stream * * @param systemName name of the system - * @param streamName name of stream + * @param streamName name of stream (physical or streamId) * @return EventHubs SAS key name */ public String getStreamSasKeyName(String systemName, String streamName) { - return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName)); + return validateRequiredConfig(getFromStreamIdOrName(CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), + "SASKeyName", systemName, streamName); } /** * Get the EventHubs SAS (Shared Access Signature) token for the stream * * @param systemName name of the system - * @param streamName name of stream + * @param streamName name of stream (physical or streamId) * @return EventHubs SAS token */ public String getStreamSasToken(String systemName, String streamName) { - return get(String.format(CONFIG_STREAM_SAS_TOKEN, systemName, streamName)); + return validateRequiredConfig(getFromStreamIdOrName(CONFIG_STREAM_SAS_TOKEN, systemName, streamName), + "SASToken", systemName, streamName); } /** * Get the EventHubs consumer group used for consumption for the stream * * @param systemName name of the system - * @param streamName name of stream + * @param streamName name of stream (physical or streamId) * @return EventHubs consumer group */ public String getStreamConsumerGroup(String systemName, String streamName) { - return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, systemName, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); + return getFromStreamIdOrName(CONFIG_STREAM_CONSUMER_GROUP, systemName, streamName, DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); } /** diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java index ada5694123..977e25237a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java @@ -75,8 +75,8 @@ public void init() { eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString(), retryPolicy); } catch (IOException | ServiceBusException e) { - String msg = String.format("Creation of EventHub client failed for eventHub %s %s %s %s on remote host %s:%d", - entityPath, eventHubNamespace, sasKeyName, sasKey, remoteHost, ClientConstants.AMQPS_PORT); + String msg = String.format("Creation of EventHub client failed for eventHub EntityPath: %s on remote host %s:%d", + entityPath, remoteHost, ClientConstants.AMQPS_PORT); LOG.error(msg, e); throw new SamzaException(msg, e); } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 11998a4734..91d504ceb5 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -63,8 +63,8 @@ public EventHubSystemAdmin(String systemName, EventHubConfig eventHubConfig, private String getNextOffset(String currentOffset) { // EventHub will return the first message AFTER the offset // that was specified in the fetch request. - return currentOffset.equals(EventHubSystemConsumer.END_OF_STREAM) ? currentOffset : - String.valueOf(Long.parseLong(currentOffset) + 1); + // If no such offset exists Eventhub will return an error. + return String.valueOf(Long.parseLong(currentOffset) + 1); } @Override @@ -158,6 +158,7 @@ private Map getPartitionMetadata(Strin long timeoutMs = eventHubConfig.getRuntimeInfoWaitTimeMS(systemName); EventHubPartitionRuntimeInformation ehPartitionInfo = ehPartitionRuntimeInfo.get(timeoutMs, TimeUnit.MILLISECONDS); + // Set offsets String startingOffset = EventHubSystemConsumer.START_OF_STREAM; String newestOffset = ehPartitionInfo.getLastEnqueuedOffset(); String upcomingOffset = getNextOffset(newestOffset); @@ -181,12 +182,14 @@ public static Integer compareOffsets(String offset1, String offset2) { if (offset1 == null || offset2 == null) { return null; } + // Should NOT be able to compare with END_OF_STREAM to allow new offsets to be + // considered caught up if stream started at END_OF_STREAM offset + if (EventHubSystemConsumer.END_OF_STREAM.equals(offset1) || + EventHubSystemConsumer.END_OF_STREAM.equals(offset2)) { + return null; + } try { - if (offset1.equals(EventHubSystemConsumer.END_OF_STREAM)) { - return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? 0 : 1; - } - return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? -1 : - Long.compare(Long.parseLong(offset1), Long.parseLong(offset2)); + return Long.compare(Long.parseLong(offset1), Long.parseLong(offset2)); } catch (NumberFormatException exception) { return null; } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 4de34de7d8..90c73dc22a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -20,6 +20,7 @@ package org.apache.samza.system.eventhub.consumer; import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubPartitionRuntimeInformation; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; import com.microsoft.azure.eventhubs.PartitionReceiver; import com.microsoft.azure.servicebus.ServiceBusException; @@ -34,6 +35,7 @@ import org.apache.samza.system.eventhub.Interceptor; import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import org.apache.samza.util.BlockingEnvelopeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -140,23 +142,23 @@ public EventHubSystemConsumer(EventHubConfig config, String systemName, this.config = config; this.systemName = systemName; this.interceptors = interceptors; - List streamNames = config.getStreams(systemName); + List streamIds = config.getStreams(systemName); // Create and initiate connections to Event Hubs - for (String streamName : streamNames) { + for (String streamId : streamIds) { EventHubClientManager eventHubClientManager = eventHubClientManagerFactory - .getEventHubClientManager(systemName, streamName, config); - streamEventHubManagers.put(streamName, eventHubClientManager); + .getEventHubClientManager(systemName, streamId, config); + streamEventHubManagers.put(streamId, eventHubClientManager); eventHubClientManager.init(); } // Initiate metrics - eventReadRates = streamNames.stream() + eventReadRates = streamIds.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); - eventByteReadRates = streamNames.stream() + eventByteReadRates = streamIds.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); - readLatencies = streamNames.stream() + readLatencies = streamIds.stream() .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); - readErrors = streamNames.stream() + readErrors = streamIds.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. @@ -174,20 +176,41 @@ public EventHubSystemConsumer(EventHubConfig config, String systemName, public void register(SystemStreamPartition systemStreamPartition, String offset) { super.register(systemStreamPartition, offset); + LOG.info(String.format("Eventhub consumer trying to register ssp %s, offset %s", systemStreamPartition, offset)); if (isStarted) { throw new SamzaException("Trying to add partition when the connection has already started."); } if (streamPartitionOffsets.containsKey(systemStreamPartition)) { + // Only update if new offset is lower than previous offset + if (END_OF_STREAM.equals(offset)) return; String prevOffset = streamPartitionOffsets.get(systemStreamPartition); - if (EventHubSystemAdmin.compareOffsets(offset, prevOffset) > -1) { - // Only update if new offset is lower than previous offset + if (!END_OF_STREAM.equals(prevOffset) && EventHubSystemAdmin.compareOffsets(offset, prevOffset) > -1) { return; } } streamPartitionOffsets.put(systemStreamPartition, offset); } + private String getNewestEventHubOffset(EventHubClientManager eventHubClientManager, String streamName, Integer partitionId) { + CompletableFuture partitionRuntimeInfoFuture = eventHubClientManager + .getEventHubClient() + .getPartitionRuntimeInformation(partitionId.toString()); + try { + long timeoutMs = config.getRuntimeInfoWaitTimeMS(systemName); + + EventHubPartitionRuntimeInformation partitionRuntimeInformation = partitionRuntimeInfoFuture + .get(timeoutMs, TimeUnit.MILLISECONDS); + + return partitionRuntimeInformation.getLastEnqueuedOffset(); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + String msg = String.format( + "Error while fetching EventHubPartitionRuntimeInfo for System:%s, Stream:%s, Partition:%s", + systemName, streamName, partitionId); + throw new SamzaException(msg); + } + } + @Override public void start() { isStarted = true; @@ -196,27 +219,36 @@ public void start() { SystemStreamPartition ssp = entry.getKey(); String streamName = ssp.getStream(); + String streamId = config.getStreamId(ssp.getStream()); Integer partitionId = ssp.getPartition().getPartitionId(); String offset = entry.getValue(); - String consumerGroup = config.getStreamConsumerGroup(systemName, streamName); - String namespace = config.getStreamNamespace(systemName, streamName); - String entityPath = config.getStreamEntityPath(systemName, streamName); - EventHubClientManager eventHubClientManager = streamEventHubManagers.get(streamName); + String consumerGroup = config.getStreamConsumerGroup(systemName, streamId); + String namespace = config.getStreamNamespace(systemName, streamId); + String entityPath = config.getStreamEntityPath(systemName, streamId); + EventHubClientManager eventHubClientManager = streamEventHubManagers.get(streamId); try { + // Fetch the newest offset + String newestEventHubOffset = getNewestEventHubOffset(eventHubClientManager, streamName, partitionId); PartitionReceiver receiver; - if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { + if (END_OF_STREAM.equals(offset) || EventHubSystemAdmin.compareOffsets(newestEventHubOffset, offset) == -1) { + // If the offset is greater than the newest offset, use the use current Instant as + // offset to fetch in Eventhub. receiver = eventHubClientManager.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), Instant.now()); } else { + // If the offset is less or equal to the newest offset in the system, it can be + // used as the starting offset to receive from. EventHub will return the first + // message AFTER the offset that was specified in the fetch request. + // If no such offset exists Eventhub will return an error. receiver = eventHubClientManager.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), offset, !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); } - PartitionReceiveHandler handler = new PartitionReceiverHandlerImpl(ssp, eventReadRates.get(streamName), - eventByteReadRates.get(streamName), readLatencies.get(streamName), readErrors.get(streamName), - interceptors.getOrDefault(streamName, null)); + PartitionReceiveHandler handler = new PartitionReceiverHandlerImpl(ssp, eventReadRates.get(streamId), + eventByteReadRates.get(streamId), readLatencies.get(streamId), readErrors.get(streamId), + interceptors.getOrDefault(streamId, null)); // Timeout for EventHubClient receive @@ -261,11 +293,11 @@ public Map> poll(Set events) { } String offset = event.getSystemProperties().getOffset(); Object partitionKey = event.getSystemProperties().getPartitionKey(); + if (partitionKey == null) { + partitionKey = event.getProperties().get(EventHubSystemProducer.KEY); + } try { updateMetrics(event); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 505421cb03..f294751714 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -58,6 +58,7 @@ public class EventHubSystemProducer implements SystemProducer { private static final long DEFAULT_FLUSH_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; + public static final String KEY = "key"; // Metrics recording private static final String EVENT_WRITE_RATE = "eventWriteRate"; @@ -75,8 +76,9 @@ public class EventHubSystemProducer implements SystemProducer { private static final Object AGGREGATE_METRICS_LOCK = new Object(); public enum PartitioningMethod { + ROUND_ROBIN, EVENT_HUB_HASHING, - PARTITION_KEY_AS_PARTITION, + PARTITION_KEY_AS_PARTITION } private final HashMap eventWriteRate = new HashMap<>(); @@ -85,7 +87,6 @@ public enum PartitioningMethod { private final HashMap sendCallbackLatency = new HashMap<>(); private final HashMap sendErrors = new HashMap<>(); - private final EventHubClientManagerFactory eventHubClientManagerFactory; private final EventHubConfig config; private final MetricsRegistry registry; private final PartitioningMethod partitioningMethod; @@ -109,36 +110,35 @@ public EventHubSystemProducer(EventHubConfig config, String systemName, this.registry = registry; this.systemName = systemName; this.partitioningMethod = config.getPartitioningMethod(systemName); - this.eventHubClientManagerFactory = eventHubClientManagerFactory; this.interceptors = interceptors; + + // Fetches the stream ids + List streamIds = config.getStreams(systemName); + + // Create and initiate connections to Event Hubs + for (String streamId : streamIds) { + EventHubClientManager ehClient = eventHubClientManagerFactory.getEventHubClientManager(systemName, streamId, config); + eventHubClients.put(streamId, ehClient); + ehClient.init(); + } } @Override - public synchronized void register(String streamName) { - LOG.debug("Trying to register {}.", streamName); + public synchronized void register(String source) { if (isStarted) { String msg = "Cannot register once the producer is started."; throw new SamzaException(msg); } - - if (eventHubClients.containsKey(streamName)) { - LOG.warn("Already registered stream {}.", streamName); - return; - } - - EventHubClientManager ehClient = eventHubClientManagerFactory.getEventHubClientManager(systemName, streamName, config); - - ehClient.init(); - eventHubClients.put(streamName, ehClient); } @Override public synchronized void start() { LOG.debug("Starting system producer."); + // Create partition senders if required if (PartitioningMethod.PARTITION_KEY_AS_PARTITION.equals(partitioningMethod)) { // Create all partition senders - eventHubClients.forEach((streamName, samzaEventHubClient) -> { + eventHubClients.forEach((streamId, samzaEventHubClient) -> { EventHubClient ehClient = samzaEventHubClient.getEventHubClient(); try { @@ -153,7 +153,7 @@ public synchronized void start() { partitionSenders.put(i, partitionSender); } - streamPartitionSenders.put(streamName, partitionSenders); + streamPartitionSenders.put(streamId, partitionSenders); } catch (InterruptedException | ExecutionException | TimeoutException e) { String msg = "Failed to fetch number of Event Hub partitions for partition sender creation"; throw new SamzaException(msg, e); @@ -164,15 +164,16 @@ public synchronized void start() { }); } - for (String eventHub : eventHubClients.keySet()) { - eventWriteRate.put(eventHub, registry.newCounter(eventHub, EVENT_WRITE_RATE)); - eventByteWriteRate.put(eventHub, registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); - sendLatency.put(eventHub, new SamzaHistogram(registry, eventHub, SEND_LATENCY)); - sendCallbackLatency.put(eventHub, new SamzaHistogram(registry, eventHub, SEND_CALLBACK_LATENCY)); - sendErrors.put(eventHub, registry.newCounter(eventHub, SEND_ERRORS)); - } + // Initiate metrics + eventHubClients.keySet().forEach((streamId) -> { + eventWriteRate.put(streamId, registry.newCounter(streamId, EVENT_WRITE_RATE)); + eventByteWriteRate.put(streamId, registry.newCounter(streamId, EVENT_BYTE_WRITE_RATE)); + sendLatency.put(streamId, new SamzaHistogram(registry, streamId, SEND_LATENCY)); + sendCallbackLatency.put(streamId, new SamzaHistogram(registry, streamId, SEND_CALLBACK_LATENCY)); + sendErrors.put(streamId, registry.newCounter(streamId, SEND_ERRORS)); + }); - // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. + // Locking to ensure that these aggregated metrics will be created only once across multiple system producers. synchronized (AGGREGATE_METRICS_LOCK) { if (aggEventWriteRate == null) { aggEventWriteRate = registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); @@ -187,35 +188,38 @@ public synchronized void start() { } @Override - public synchronized void send(String destination, OutgoingMessageEnvelope envelope) { + public synchronized void send(String source, OutgoingMessageEnvelope envelope) { + LOG.debug(String.format("Trying to send %s", envelope)); if (!isStarted) { throw new SamzaException("Trying to call send before the producer is started."); } - if (!eventHubClients.containsKey(destination)) { - String msg = String.format("Trying to send event to a destination {%s} that is not registered.", destination); + String streamId = config.getStreamId(envelope.getSystemStream().getStream()); + + if (!eventHubClients.containsKey(streamId)) { + String msg = String.format("Trying to send event to a destination {%s} that is not registered.", streamId); throw new SamzaException(msg); } checkCallbackThrowable("Received exception on message send"); - EventData eventData = createEventData(destination, envelope); + EventData eventData = createEventData(streamId, envelope); int eventDataLength = eventData.getBytes() == null ? 0 : eventData.getBytes().length; - eventWriteRate.get(destination).inc(); + eventWriteRate.get(streamId).inc(); aggEventWriteRate.inc(); - eventByteWriteRate.get(destination).inc(eventDataLength); + eventByteWriteRate.get(streamId).inc(eventDataLength); aggEventByteWriteRate.inc(eventDataLength); - EventHubClientManager ehClient = eventHubClients.get(destination); + EventHubClientManager ehClient = eventHubClients.get(streamId); long beforeSendTimeMs = System.currentTimeMillis(); // Async send call - CompletableFuture sendResult = sendToEventHub(destination, eventData, getEnvelopePartitionId(envelope), + CompletableFuture sendResult = sendToEventHub(streamId, eventData, getEnvelopePartitionId(envelope), ehClient.getEventHubClient()); long afterSendTimeMs = System.currentTimeMillis(); long latencyMs = afterSendTimeMs - beforeSendTimeMs; - sendLatency.get(destination).update(latencyMs); + sendLatency.get(streamId).update(latencyMs); aggSendLatency.update(latencyMs); pendingFutures.add(sendResult); @@ -223,32 +227,37 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo // Auto update the metrics and possible throwable when futures are complete. sendResult.handle((aVoid, throwable) -> { long callbackLatencyMs = System.currentTimeMillis() - afterSendTimeMs; - sendCallbackLatency.get(destination).update(callbackLatencyMs); + sendCallbackLatency.get(streamId).update(callbackLatencyMs); aggSendCallbackLatency.update(callbackLatencyMs); if (throwable != null) { - sendErrors.get(destination).inc(); + sendErrors.get(streamId).inc(); aggSendErrors.inc(); - LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); + LOG.error("Send message to event hub: {} failed with exception: ", streamId, throwable); sendExceptionOnCallback.compareAndSet(null, throwable); } return aVoid; }); } - private CompletableFuture sendToEventHub(String streamName, EventData eventData, Object partitionKey, + private CompletableFuture sendToEventHub(String streamId, EventData eventData, Object partitionKey, EventHubClient eventHubClient) { - if (partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { + if (PartitioningMethod.ROUND_ROBIN.equals(partitioningMethod)) { + return eventHubClient.send(eventData); + } else if (PartitioningMethod.EVENT_HUB_HASHING.equals(partitioningMethod)) { + if (partitionKey == null) { + throw new SamzaException("Partition key cannot be null for EventHub hashing"); + } return eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); - } else if (partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { + } else if (PartitioningMethod.PARTITION_KEY_AS_PARTITION.equals(partitioningMethod)) { if (!(partitionKey instanceof Integer)) { String msg = "Partition key should be of type Integer"; throw new SamzaException(msg); } - Integer numPartition = streamPartitionSenders.get(streamName).size(); + Integer numPartition = streamPartitionSenders.get(streamId).size(); Integer destinationPartition = (Integer) partitionKey % numPartition; - PartitionSender sender = streamPartitionSenders.get(streamName).get(destinationPartition); + PartitionSender sender = streamPartitionSenders.get(streamId).get(destinationPartition); return sender.send(eventData); } else { throw new SamzaException("Unknown partitioning method " + partitioningMethod); @@ -256,7 +265,7 @@ private CompletableFuture sendToEventHub(String streamName, EventData even } protected Object getEnvelopePartitionId(OutgoingMessageEnvelope envelope) { - return envelope.getPartitionKey(); + return envelope.getPartitionKey() == null ? envelope.getKey() : envelope.getPartitionKey(); } private String convertPartitionKeyToString(Object partitionKey) { @@ -267,12 +276,12 @@ private String convertPartitionKeyToString(Object partitionKey) { } else if (partitionKey instanceof byte[]) { return new String((byte[]) partitionKey, Charset.defaultCharset()); } else { - throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); + throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); } } - protected EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { - Optional interceptor = Optional.ofNullable(interceptors.getOrDefault(streamName, null)); + protected EventData createEventData(String streamId, OutgoingMessageEnvelope envelope) { + Optional interceptor = Optional.ofNullable(interceptors.getOrDefault(streamId, null)); byte[] eventValue = (byte[]) envelope.getMessage(); if (interceptor.isPresent()) { eventValue = interceptor.get().intercept(eventValue); @@ -288,7 +297,7 @@ protected EventData createEventData(String streamName, OutgoingMessageEnvelope e keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) : envelope.getKey().toString(); } - eventData.getProperties().put("key", keyValue); + eventData.getProperties().put(KEY, keyValue); } return eventData; } diff --git a/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java b/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java index 3e5ead008f..4560b114d1 100644 --- a/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java +++ b/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java @@ -34,7 +34,7 @@ import java.util.HashMap; import java.util.Map; -@Ignore("Intergration Test") +@Ignore("Requires Azure account credentials") public class ITestAzureCheckpointManager { private static String storageConnectionString = ""; diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java index 1f06f7d310..368087a952 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java @@ -101,6 +101,10 @@ private class MockEventHubClientManager implements EventHubClientManager { } return null; }); + EventHubPartitionRuntimeInformation mockPartitionRuntimeInfo = PowerMockito.mock(EventHubPartitionRuntimeInformation.class); + PowerMockito.when(mockPartitionRuntimeInfo.getLastEnqueuedOffset()) + .thenReturn(EventHubSystemConsumer.START_OF_STREAM); + CompletableFuture partitionFuture = new MockPartitionFuture(mockPartitionRuntimeInfo); // Producer mocks PartitionSender mockPartitionSender0 = PowerMockito.mock(PartitionSender.class); @@ -137,6 +141,7 @@ private class MockEventHubClientManager implements EventHubClientManager { startingOffsets.put(partitionId, offset); return mockPartitionReceiver; }); + PowerMockito.when(mockEventHubClient.getPartitionRuntimeInformation(anyString())).thenReturn(partitionFuture); // Producer calls PowerMockito.when(mockEventHubClient.createPartitionSenderSync("0")).thenReturn(mockPartitionSender0); @@ -191,6 +196,19 @@ public EventHubRuntimeInformation get(long timeout, TimeUnit unit) { } } + private class MockPartitionFuture extends CompletableFuture { + EventHubPartitionRuntimeInformation runtimeInformation; + + MockPartitionFuture(EventHubPartitionRuntimeInformation runtimeInformation) { + this.runtimeInformation = runtimeInformation; + } + + @Override + public EventHubPartitionRuntimeInformation get(long timeout, TimeUnit unit) { + return runtimeInformation; + } + } + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index c49e68eb70..8861152568 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -49,14 +49,8 @@ public void testOffsetComparison() { Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); Assert.assertNull(eventHubSystemAdmin.offsetComparator("1", "a")); - Assert.assertEquals(-1, eventHubSystemAdmin - .offsetComparator("100", EventHubSystemConsumer.END_OF_STREAM).intValue()); - Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator(EventHubSystemConsumer.END_OF_STREAM, - EventHubSystemConsumer.END_OF_STREAM).intValue()); - Assert.assertEquals(1, eventHubSystemAdmin - .offsetComparator(EventHubSystemConsumer.END_OF_STREAM, "100").intValue()); - Assert.assertEquals(-1, eventHubSystemAdmin - .offsetComparator(EventHubSystemConsumer.START_OF_STREAM, "10").intValue()); + Assert.assertNull(eventHubSystemAdmin.offsetComparator("100", EventHubSystemConsumer.END_OF_STREAM)); + Assert.assertNull(eventHubSystemAdmin.offsetComparator(EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); } @Test @@ -66,16 +60,13 @@ public void testGetNextOffset() { MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.EVENT_HUB_HASHING)); Map offsets = new HashMap<>(); SystemStreamPartition ssp0 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(0)); - SystemStreamPartition ssp1 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(1)); SystemStreamPartition ssp2 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(2)); offsets.put(ssp0, Integer.toString(0)); - offsets.put(ssp1, EventHubSystemConsumer.END_OF_STREAM); offsets.put(ssp2, EventHubSystemConsumer.START_OF_STREAM); Map updatedOffsets = eventHubSystemAdmin.getOffsetsAfter(offsets); Assert.assertEquals(offsets.size(), updatedOffsets.size()); Assert.assertEquals("1", updatedOffsets.get(ssp0)); - Assert.assertEquals("-2", updatedOffsets.get(ssp1)); Assert.assertEquals("0", updatedOffsets.get(ssp2)); } @@ -102,8 +93,6 @@ public void testGetStreamMetadata() { partitionMetadataMap.forEach((partition, metadata) -> { Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, metadata.getOldestOffset()); Assert.assertNotSame(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); - Assert.assertTrue(Long.parseLong(EventHubSystemConsumer.END_OF_STREAM) - <= Long.parseLong(metadata.getNewestOffset())); String expectedUpcomingOffset = String.valueOf(Long.parseLong(metadata.getNewestOffset()) + 1); Assert.assertEquals(expectedUpcomingOffset, metadata.getUpcomingOffset()); }); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java index dbf8807f86..cfd82170d9 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java @@ -29,14 +29,15 @@ import org.apache.samza.system.eventhub.TestMetricsRegistry; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; -import java.util.Collections; -import java.util.List; +import java.util.*; import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.STREAM_NAME1; import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.SYSTEM_NAME; +@Ignore("Requires Azure account credentials") public class ITestEventHubSystemConsumer { private Config createEventHubConfig() { diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index a25a3b61e7..865a248fc5 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -22,6 +22,7 @@ import com.microsoft.azure.eventhubs.*; import org.apache.samza.Partition; +import org.apache.samza.config.MapConfig; import org.apache.samza.metrics.Counter; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; @@ -41,7 +42,8 @@ import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; @RunWith(PowerMockRunner.class) -@PrepareForTest({EventHubRuntimeInformation.class, EventHubClient.class, PartitionReceiver.class, PartitionSender.class}) +@PrepareForTest({EventHubRuntimeInformation.class, EventHubPartitionRuntimeInformation.class, + EventHubClient.class, PartitionReceiver.class, PartitionSender.class}) public class TestEventHubSystemConsumer { private static final String MOCK_ENTITY_1 = "mocktopic1"; private static final String MOCK_ENTITY_2 = "mocktopic2"; @@ -85,11 +87,12 @@ public void testMultipleRegistersToSameSSP() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptors, + new EventHubSystemConsumer(new EventHubConfig(config), systemName, eventHubClientWrapperFactory, interceptors, testMetrics); consumer.register(ssp, "1"); consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); @@ -124,11 +127,12 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptors, + new EventHubSystemConsumer(new EventHubConfig(config), systemName, eventHubClientWrapperFactory, interceptors, testMetrics); consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); consumer.start(); @@ -173,11 +177,12 @@ public void testSinglePartitionConsumptionInterceptor() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptors, + new EventHubSystemConsumer(new EventHubConfig(config), systemName, eventHubClientWrapperFactory, interceptors, testMetrics); consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); consumer.start(); @@ -224,11 +229,12 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName), EVENTHUB_NAMESPACE); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptor, + new EventHubSystemConsumer(new EventHubConfig(config), systemName, eventHubClientWrapperFactory, interceptor, testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); @@ -284,11 +290,12 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName2), EVENTHUB_NAMESPACE); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName2), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName2), EVENTHUB_KEY); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptor, + new EventHubSystemConsumer(new EventHubConfig(config), systemName, eventHubClientWrapperFactory, interceptor, testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java index cc401984ce..32b1604ad3 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java @@ -30,12 +30,14 @@ import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.util.NoOpMetricsRegistry; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; +@Ignore("Requires Azure account credentials") public class ITestEventHubSystemProducer { private static final Logger LOG = LoggerFactory.getLogger(ITestEventHubSystemProducer.class.getName()); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index 10016ecea5..ef73775efc 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -20,6 +20,7 @@ package org.apache.samza.system.eventhub.producer; import com.microsoft.azure.eventhubs.*; +import org.apache.samza.config.MapConfig; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemStream; import org.apache.samza.system.eventhub.EventHubConfig; @@ -40,9 +41,12 @@ import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; @RunWith(PowerMockRunner.class) -@PrepareForTest({EventHubRuntimeInformation.class, EventHubClient.class, PartitionReceiver.class, PartitionSender.class}) +@PrepareForTest({EventHubRuntimeInformation.class, EventHubPartitionRuntimeInformation.class, + EventHubClient.class, PartitionReceiver.class, PartitionSender.class}) public class TestEventHubSystemProducer { + private static final String SOURCE = "TestEventHubSystemProducer"; + private static List generateMessages(int numMsg) { Random rand = new Random(System.currentTimeMillis()); List messages = new ArrayList<>(); @@ -76,20 +80,21 @@ public void testSendingToSpecificPartitions() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.PARTITION_KEY_AS_PARTITION.toString()); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = - new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, interceptor, testMetrics); + new EventHubSystemProducer(new EventHubConfig(config), systemName, factory, interceptor, testMetrics); SystemStream systemStream = new SystemStream(systemName, streamName); - producer.register(streamName); + producer.register(SOURCE); producer.start(); outgoingMessagesP0.forEach(message -> - producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); + producer.send(SOURCE, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); outgoingMessagesP1.forEach(message -> - producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); + producer.send(SOURCE, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); // Retrieve sent data List receivedData0 = factory.getSentData(systemName, streamName, partitionId0) @@ -126,20 +131,21 @@ public void testSendingToSpecificPartitionsWithInterceptor() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.PARTITION_KEY_AS_PARTITION.toString()); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = - new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, interceptors, testMetrics); + new EventHubSystemProducer(new EventHubConfig(config), systemName, factory, interceptors, testMetrics); SystemStream systemStream = new SystemStream(systemName, streamName); - producer.register(streamName); + producer.register(SOURCE); producer.start(); outgoingMessagesP0.forEach(message -> - producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); + producer.send(SOURCE, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); outgoingMessagesP1.forEach(message -> - producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); + producer.send(SOURCE, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); // Retrieve sent data List receivedData0 = factory.getSentData(systemName, streamName, partitionId0) @@ -180,23 +186,25 @@ public void testSendingToEventHubHashing() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); + // mod 2 on the partitionid to simulate consistent hashing configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.EVENT_HUB_HASHING.toString()); + MapConfig config = new MapConfig(configMap); MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = - new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, interceptor, testMetrics); + new EventHubSystemProducer(new EventHubConfig(config), systemName, factory, interceptor, testMetrics); SystemStream systemStream = new SystemStream(systemName, streamName); - producer.register(streamName); + producer.register(SOURCE); producer.start(); outgoingMessagesP0.forEach(message -> - producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); + producer.send(SOURCE, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); outgoingMessagesP1.forEach(message -> - producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); + producer.send(SOURCE, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); // Retrieve sent data List receivedData0 = factory.getSentData(systemName, streamName, 0) From 42bac30ad1832afa92767a5ea492bfd6c3de664a Mon Sep 17 00:00:00 2001 From: Srinivasulu Punuru Date: Tue, 12 Dec 2017 10:46:01 -0800 Subject: [PATCH 09/36] Documentation for Samza SQL **Samza tools** : Contains the following tools that can be used for playing with Samza sql or any other samza job. 1. Generate kafka events : Tool used to generate avro serialized kafka events 2. Event hub consumer : Tool used to consume events from event hubs topic. This can be used if the samza job writes events to event hubs. 3. Samza sql console : Tool used to execute SQL using samza sql. Adds documentation on how to use Samza SQL on a local machine and on a yarn environment and their associated Samza tooling. https://issues.apache.org/jira/browse/SAMZA-1526 Author: Srinivasulu Punuru Reviewers: Yi Pan, Jagadish Closes #374 from srinipunuru/docs.1 --- build.gradle | 28 +++ docs/README.md | 4 +- docs/learn/tutorials/versioned/index.md | 3 + docs/learn/tutorials/versioned/samza-sql.md | 123 +++++++++++ docs/learn/tutorials/versioned/samza-tools.md | 109 ++++++++++ docs/startup/download/index.md | 7 + gradle/dependency-versions.gradle | 1 + .../samza/sql/avro/AvroRelConverter.java | 6 +- samza-tools/config/eh-consumer-log4j.xml | 35 +++ .../config/generate-kafka-events-log4j.xml | 35 +++ .../config/samza-sql-console-log4j.xml | 35 +++ samza-tools/scripts/eh-consumer.sh | 34 +++ samza-tools/scripts/generate-kafka-events.sh | 34 +++ samza-tools/scripts/samza-sql-console.sh | 34 +++ .../apache/samza/tools/CommandLineHelper.java | 42 ++++ .../tools/ConsoleLoggingSystemFactory.java | 126 +++++++++++ .../samza/tools/EventHubConsoleConsumer.java | 120 ++++++++++ .../samza/tools/GenerateKafkaEvents.java | 205 ++++++++++++++++++ .../samza/tools/RandomValueGenerator.java | 87 ++++++++ .../apache/samza/tools/SamzaSqlConsole.java | 188 ++++++++++++++++ .../tools/avro/AvroSchemaGenRelConverter.java | 94 ++++++++ .../AvroSchemaGenRelConverterFactory.java | 43 ++++ .../samza/tools/avro/AvroSerDeFactory.java | 96 ++++++++ .../tools/json/JsonRelConverterFactory.java | 93 ++++++++ .../samza/tools/schemas/PageViewEvent.avsc | 51 +++++ .../samza/tools/schemas/PageViewEvent.java | 60 +++++ .../tools/schemas/ProfileChangeEvent.avsc | 51 +++++ .../tools/schemas/ProfileChangeEvent.java | 60 +++++ .../apache/samza/tools/udf/RegexMatchUdf.java | 40 ++++ samza-tools/src/main/resources/log4j.xml | 43 ++++ settings.gradle | 3 +- 31 files changed, 1886 insertions(+), 4 deletions(-) create mode 100644 docs/learn/tutorials/versioned/samza-sql.md create mode 100644 docs/learn/tutorials/versioned/samza-tools.md create mode 100644 samza-tools/config/eh-consumer-log4j.xml create mode 100644 samza-tools/config/generate-kafka-events-log4j.xml create mode 100644 samza-tools/config/samza-sql-console-log4j.xml create mode 100755 samza-tools/scripts/eh-consumer.sh create mode 100755 samza-tools/scripts/generate-kafka-events.sh create mode 100755 samza-tools/scripts/samza-sql-console.sh create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/CommandLineHelper.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/ConsoleLoggingSystemFactory.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/EventHubConsoleConsumer.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/GenerateKafkaEvents.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/RandomValueGenerator.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverter.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverterFactory.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSerDeFactory.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/json/JsonRelConverterFactory.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.avsc create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.avsc create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.java create mode 100644 samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.java create mode 100644 samza-tools/src/main/resources/log4j.xml diff --git a/build.gradle b/build.gradle index 330ff0b5e3..04b42f261d 100644 --- a/build.gradle +++ b/build.gradle @@ -323,6 +323,34 @@ project(':samza-sql') { } } +project(':samza-tools') { + apply plugin: 'java' + + dependencies { + compile project(':samza-sql') + compile project(':samza-api') + compile project(':samza-azure') + compile "log4j:log4j:$log4jVersion" + compile "org.slf4j:slf4j-api:$slf4jVersion" + compile "org.slf4j:slf4j-log4j12:$slf4jVersion" + compile "commons-cli:commons-cli:$commonsCliVersion" + compile "org.apache.avro:avro:$avroVersion" + compile "org.apache.commons:commons-lang3:$commonsLang3Version" + compile "org.apache.kafka:kafka-clients:$kafkaVersion" + } + + tasks.create(name: "releaseToolsTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + into "samza-tools-${version}" + compression = Compression.GZIP + from(project.file("./scripts")) { into "scripts/" } + from(project.file("./config")) { into "config/" } + from(project(':samza-shell').file("src/main/bash/run-class.sh")) { into "scripts/" } + from(configurations.runtime) { into("lib/") } + from(configurations.archives.artifacts.files) { into("lib/") } + duplicatesStrategy 'exclude' + } +} + project(":samza-kafka_$scalaVersion") { apply plugin: 'scala' diff --git a/docs/README.md b/docs/README.md index 21a49915fc..3de2a781da 100644 --- a/docs/README.md +++ b/docs/README.md @@ -83,8 +83,8 @@ Following can be done when updating the gradle.properties file * if this is a major release, add the x.x.x release to Archive category in docs/_layouts/default.html and x.x.x release part in docs/archive/index.html - * update the download page to use x.x.x release - * docs/startup/download/index.md + * update the download page (docs/startup/download/index.md) to use x.x.x release + * Add an entry to the Sources releases and Samza Tools section to use the new x.x.x release * update the version number in "tar -xvf ./target/hello-samza-y.y.y-dist.tar.gz -C deploy/samza" in each of the tutorials (and search for other uses of version x.x.x which may need to be replaced with y.y.y) * docs/startup/hello-samza/versioned/index.md diff --git a/docs/learn/tutorials/versioned/index.md b/docs/learn/tutorials/versioned/index.md index a9ac6a7441..d155b82076 100644 --- a/docs/learn/tutorials/versioned/index.md +++ b/docs/learn/tutorials/versioned/index.md @@ -42,6 +42,9 @@ title: Tutorials [Samza Async API and Multithreading User Guide](samza-async-user-guide.html) +[Samza SQL User Guide](samza-sql.html) + + + +There are couple of ways to use Samza SQL + +1. Run Samza SQL on your local machine. +2. Run Samza SQL on YARN. + +# Running Samza SQL on your local machine + + +Samza SQL console tool documented [here](samza-tools.html) uses Samza standalone to run the Samza SQL on your local machine. This is the quickest way to play with Samza SQL. Please follow the instructions [here](samza-tools.html) to get access to the Samza tools on your machine. + +## Start the Kafka server + +Please follow the instructions from the [Kafka quickstart](http://kafka.apache.org/quickstart) to start the zookeeper and Kafka server. + +## Create ProfileChangeStream Kafka topic + +The below sql statements requires a topic named ProfileChangeStream to be created on the Kafka broker. You can follow the instructions in the [Kafka quick start guide](http://kafka.apache.org/quickstart) to create a topic named "ProfileChangeStream". + +{% highlight bash %} +./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic ProfileChangeStream +{% endhighlight %} + +## Generate events into ProfileChangeStream topic + +Use generate-kafka-events from [Samza tools](samza-tools.html) to generate events into the ProfileChangeStream + +{% highlight bash %} +cd samza-tools- +./scripts/generate-kafka-events.sh -t ProfileChangeStream -e ProfileChange +{% endhighlight %} + +## Using Samza SQL Console to run Samza sql on your local machine + +Below are some of the sql queries that you can execute using the samza-sql-console tool from [Samza tools](samza-tools.html) package. + +{% highlight bash %} +# This command just prints out all the events in the Kafka topic ProfileChangeStream into console output as a json serialized payload. +./scripts/samza-sql-console.sh --sql "insert into log.consoleoutput select * from kafka.ProfileChangeStream" + +# This command prints out the fields that are selected into the console output as a json serialized payload. +./scripts/samza-sql-console.sh --sql "insert into log.consoleoutput select Name, OldCompany, NewCompany from kafka.ProfileChangeStream" + +# This command showcases the RegexMatch udf and filtering capabilities. +./scripts/samza-sql-console.sh --sql "insert into log.consoleoutput select Name as __key__, Name, NewCompany, RegexMatch('.*soft', OldCompany) from kafka.ProfileChangeStream where NewCompany = 'LinkedIn'" +{% endhighlight %} + + +# Running Samza SQL on YARN + +The [hello-samza](https://github.com/apache/samza-hello-samza) project is an example project designed to help you run your first Samza application. It has examples of applications using the low level task API, high level API as well as Samza SQL. + +This tutorial demonstrates a simple Samza application that uses SQL to perform stream processing. + +## Get the hello-samza Code and Start the grid + +Please follow the instructions from [hello-samza-high-level-yarn](hello-samza-high-level-yarn.html) on how to build the hello-samza repository and start the yarn grid. + +## Create the topic and generate Kafka events + +Please follow the steps in the section "Create ProfileChangeStream Kafka topic" and "Generate events into ProfileChangeStream topic" above. + +## Build a Samza Application Package + +Before you can run a Samza application, you need to build a package for it. Please follow the instructions from [hello-samza-high-level-yarn](hello-samza-high-level-yarn.html) on how to build the hello-samza application package. + +## Run a Samza Application + +After you've built your Samza package, you can start the app on the grid using the run-app.sh script. + +{% highlight bash %} +./deploy/samza/bin/run-app.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/page-view-filter-sql.properties +{% endhighlight %} + +The app executes the following SQL command : +{% highlight sql %} +insert into kafka.NewLinkedInEmployees select Name from ProfileChangeStream where NewCompany = 'LinkedIn' +{% endhighlight %} + +This SQL performs the following + +1. Consumes the Kafka topic ProfileChangeStreamStream which contains the avro serialized ProfileChangeEvent(s) +2. Deserializes the events and filters out only the profile change events where NewCompany = 'LinkedIn' i.e. Members who have moved to LinkedIn. +3. Writes the Avro serialized event that contains the Id and Name of those profiles to Kafka topic NewLinkedInEmployees. + + +Give the job a minute to startup, and then tail the Kafka topic: + +{% highlight bash %} +./deploy/kafka/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic NewLinkedInEmployees +{% endhighlight %} + + +Congratulations! You've now setup a local grid that includes YARN, Kafka, and ZooKeeper, and run a Samza SQL application on it. + +## Shutdown and cleanup + +To shutdown the app, use the same _run-app.sh_ script with an extra _--operation=kill_ argument +{% highlight bash %} +./deploy/samza/bin/run-app.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/page-view-filter-sql.properties --operation=kill +{% endhighlight %} + +Please follow the instructions from [Hello Samza High Level API - YARN Deployment](hello-samza-high-level-yarn.html) on how to shutdown and cleanup the app. diff --git a/docs/learn/tutorials/versioned/samza-tools.md b/docs/learn/tutorials/versioned/samza-tools.md new file mode 100644 index 0000000000..6e4ee71a05 --- /dev/null +++ b/docs/learn/tutorials/versioned/samza-tools.md @@ -0,0 +1,109 @@ +--- +layout: page +title: How to use Samza tools +--- + + + + +# Get Samza tools + +Please visit the [Download page] (/startup/download) to download the Samza tools package + +{% highlight bash %} +tar -xvzf samza-tools-*.tgz +cd samza-tools- +{% endhighlight %} + + +# Using Samza tools + + +## Generate kafka events + + +Generate kafka events tool is used to insert avro serialized events into kafka topics. Right now it can insert two types of events [PageViewEvent](https://github.com/apache/samza/blob/master/samza-tools/src/main/java/org/com/linkedin/samza/tools/schemas/PageViewEvent.avsc) and [ProfileChangeEvent](https://github.com/apache/samza/blob/master/samza-tools/src/main/java/org/com/linkedin/samza/tools/schemas/ProfileChangeEvent.avsc) + +Before you can generate kafka events, Please follow instructions [here](http://kafka.apache.org/quickstart) to start the zookeeper and kafka server on your local machine. + +You can follow below instructions on how to use Generate kafka events tool. + +{% highlight bash %} + +# Usage of the tool + +./scripts/generate-kafka-events.sh +usage: Error: Missing required options: t, e + generate-kafka-events.sh + -b,--broker Kafka broker endpoint Default (localhost:9092). + -n,--numEvents Number of events to be produced, + Default - Produces events continuously every second. + -p,--partitions Number of partitions in the topic, + Default (4). + -t,--topic Name of the topic to write events to. + -e,--eventtype Type of the event values can be (PageView|ProfileChange). + + +# Example command to generate 100 events of type PageViewEvent into topic named PageViewStream + + ./scripts/generate-kafka-events.sh -t PageViewStream -e PageView -n 100 + + +# Example command to generate ProfileChange events continuously into topic named ProfileChangeStream + + ./scripts/generate-kafka-events.sh -t ProfileChangeStream -e ProfileChange + +{% endhighlight %} + +## Samza SQL console tool + +Once you generated the events into the kafka topic. Now you can use samza-sql-console tool to perform processing on the events published into the kafka topic. + +There are two ways to use the tool - + +1. You can either pass the sql statement directly as an argument to the tool. +2. You can write the sql statement(s) into a file and pass the sql file as an argument to the tool. + +Second option allows you to execute multiple sql statements, whereas the first one lets you execute one at a time. + +Samza SQL needs all the events in the topic to be uniform schema. And it also needs access to the schema corresponding to the events in a topic. Typically in an organization, there is a deployment of schema registry which maps topics to schemas. + +In the absence of schema registry, Samza SQL console tool uses the convention to identify the schemas associated with the topic. If the topic name has string "page" it assumes the topic has PageViewEvents else ProfileChangeEvents. + +{% highlight bash %} + +# Usage of the tool + + ./scripts/samza-sql-console.sh +usage: Error: One of the (f or s) options needs to be set + samza-sql-console.sh + -f,--file Path to the SQL file to execute. + -s,--sql SQL statement to execute. + +# Example command to filter out all the users who have moved to LinkedIn + +./scripts/samza-sql-console.sh --sql "Insert into log.consoleOutput select Name, OldCompany from kafka.ProfileChangeStream where NewCompany = 'LinkedIn'" + +{% endhighlight %} + +You can run below sql commands using Samza sql console. Please make sure you are running generate-kafka-events tool to generate events into ProfileChangeStream before running the below command. + +{% highlight bash %} +./scripts/samza-sql-console.sh --sql "Insert into log.consoleOutput select Name, OldCompany from kafka.ProfileChangeStream where NewCompany = 'LinkedIn'" + +{% endhighlight %} diff --git a/docs/startup/download/index.md b/docs/startup/download/index.md index b39f90fbee..5dcf38022b 100644 --- a/docs/startup/download/index.md +++ b/docs/startup/download/index.md @@ -27,6 +27,12 @@ If you just want to play around with Samza for the first time, go to [Hello Samz Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this mailing list thread](http://mail-archives.apache.org/mod_mbox/samza-dev/201610.mbox/%3CCAHUevGGnOQD_VmLWEdpFNq3Lv%2B6gQQmw_JKx9jDr5Cw%2BxFfGtQ%40mail.gmail.com%3E) for details on this decision. +### Samza Tools + + Samza tools package contains command line tools that user can run to use Samza and it's input/output systems. + + * [samza-tools-0.14.0.tgz](tbd) + ### Source Releases * [samza-sources-0.13.1.tgz](http://www.apache.org/dyn/closer.lua/samza/0.13.1) @@ -40,6 +46,7 @@ Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this * [samza-sources-0.8.0-incubating.tgz](https://archive.apache.org/dist/incubator/samza/0.8.0-incubating) * [samza-sources-0.7.0-incubating.tgz](https://archive.apache.org/dist/incubator/samza/0.7.0-incubating) + ### Maven All Samza JARs are published through [Apache's Maven repository](https://repository.apache.org/content/groups/public/org/apache/samza/). diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 4f467ab8d3..20a1d56efd 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -20,6 +20,7 @@ apacheCommonsCollections4Version = "4.0" avroVersion = "1.7.0" calciteVersion = "1.14.0" + commonsCliVersion = "1.2" commonsCodecVersion = "1.9" commonsCollectionVersion = "3.2.1" commonsHttpClientVersion = "3.1" diff --git a/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java b/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java index 1c172950b4..ab46a9870c 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java @@ -167,7 +167,11 @@ public SamzaSqlRelMessage convertToRelMessage(KV samzaMessage) { @Override public KV convertToSamzaMessage(SamzaSqlRelMessage relMessage) { - GenericRecord record = new GenericData.Record(this.avroSchema); + return convertToSamzaMessage(relMessage, this.avroSchema); + } + + protected KV convertToSamzaMessage(SamzaSqlRelMessage relMessage, Schema avroSchema) { + GenericRecord record = new GenericData.Record(avroSchema); List fieldNames = relMessage.getFieldNames(); List values = relMessage.getFieldValues(); for (int index = 0; index < fieldNames.size(); index++) { diff --git a/samza-tools/config/eh-consumer-log4j.xml b/samza-tools/config/eh-consumer-log4j.xml new file mode 100644 index 0000000000..d971512d39 --- /dev/null +++ b/samza-tools/config/eh-consumer-log4j.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/samza-tools/config/generate-kafka-events-log4j.xml b/samza-tools/config/generate-kafka-events-log4j.xml new file mode 100644 index 0000000000..98c7a28e76 --- /dev/null +++ b/samza-tools/config/generate-kafka-events-log4j.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/samza-tools/config/samza-sql-console-log4j.xml b/samza-tools/config/samza-sql-console-log4j.xml new file mode 100644 index 0000000000..6b0038146c --- /dev/null +++ b/samza-tools/config/samza-sql-console-log4j.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/samza-tools/scripts/eh-consumer.sh b/samza-tools/scripts/eh-consumer.sh new file mode 100755 index 0000000000..363e028e44 --- /dev/null +++ b/samza-tools/scripts/eh-consumer.sh @@ -0,0 +1,34 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +if [ `uname` == 'Linux' ]; +then + base_dir=$(readlink -f $(dirname $0)) +else + base_dir=$(realpath $(dirname $0)) +fi + +if [ "x$LOG4J_OPTS" = "x" ]; then + export LOG4J_OPTS="-Dlog4j.configuration=file://$base_dir/../config/eh-consumer-log4j.xml" +fi + +if [ "x$HEAP_OPTS" = "x" ]; then + export HEAP_OPTS="-Xmx1G -Xms1G" +fi + +exec $base_dir/run-class.sh org.apache.samza.tools.EventHubConsoleConsumer "$@" diff --git a/samza-tools/scripts/generate-kafka-events.sh b/samza-tools/scripts/generate-kafka-events.sh new file mode 100755 index 0000000000..858ade646e --- /dev/null +++ b/samza-tools/scripts/generate-kafka-events.sh @@ -0,0 +1,34 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +if [ `uname` == 'Linux' ]; +then + base_dir=$(readlink -f $(dirname $0)) +else + base_dir=$(realpath $(dirname $0)) +fi + +if [ "x$LOG4J_OPTS" = "x" ]; then + export LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/generate-kafka-events-log4j.xml" +fi + +if [ "x$HEAP_OPTS" = "x" ]; then + export HEAP_OPTS="-Xmx1G -Xms1G" +fi + +exec $base_dir/run-class.sh org.apache.samza.tools.GenerateKafkaEvents "$@" diff --git a/samza-tools/scripts/samza-sql-console.sh b/samza-tools/scripts/samza-sql-console.sh new file mode 100755 index 0000000000..39d6930966 --- /dev/null +++ b/samza-tools/scripts/samza-sql-console.sh @@ -0,0 +1,34 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +if [ `uname` == 'Linux' ]; +then + base_dir=$(readlink -f $(dirname $0)) +else + base_dir=$(realpath $(dirname $0)) +fi + +if [ "x$LOG4J_OPTS" = "x" ]; then + export LOG4J_OPTS="-Dlog4j.configuration=file://$base_dir/../config/samza-sql-console-log4j.xml" +fi + +if [ "x$HEAP_OPTS" = "x" ]; then + export HEAP_OPTS="-Xmx1G -Xms1G" +fi + +exec $base_dir/run-class.sh org.apache.samza.tools.SamzaSqlConsole "$@" diff --git a/samza-tools/src/main/java/org/apache/samza/tools/CommandLineHelper.java b/samza-tools/src/main/java/org/apache/samza/tools/CommandLineHelper.java new file mode 100644 index 0000000000..3cabce374f --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/CommandLineHelper.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.samza.tools; + +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.lang3.StringUtils; + + +/** + * Simple command line helper util. + */ +public class CommandLineHelper { + public static Option createOption(String shortOpt, String longOpt, String argName, boolean required, + String description) { + OptionBuilder optionBuilder = OptionBuilder.withLongOpt(longOpt).withDescription(description).isRequired(required); + + if (!StringUtils.isEmpty(argName)) { + + optionBuilder = optionBuilder.withArgName(argName).hasArg(); + } + + return optionBuilder.create(shortOpt); + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/ConsoleLoggingSystemFactory.java b/samza-tools/src/main/java/org/apache/samza/tools/ConsoleLoggingSystemFactory.java new file mode 100644 index 0000000000..87abc76918 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/ConsoleLoggingSystemFactory.java @@ -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.samza.tools; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang.NotImplementedException; +import org.apache.samza.Partition; +import org.apache.samza.config.Config; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemConsumer; +import org.apache.samza.system.SystemFactory; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Console logging System factory that just writes the messages to the console output. + * This system factory is useful when the user wants to print the output of the stream processing to console. + */ +public class ConsoleLoggingSystemFactory implements SystemFactory { + + private static final Logger LOG = LoggerFactory.getLogger(ConsoleLoggingSystemFactory.class); + + @Override + public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { + throw new NotImplementedException(); + } + + @Override + public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { + return new LoggingSystemProducer(); + } + + @Override + public SystemAdmin getAdmin(String systemName, Config config) { + return new SimpleSystemAdmin(config); + } + + private class LoggingSystemProducer implements SystemProducer { + @Override + public void start() { + } + + @Override + public void stop() { + } + + @Override + public void register(String source) { + LOG.info("Registering source" + source); + } + + @Override + public void send(String source, OutgoingMessageEnvelope envelope) { + String msg = String.format("OutputStream:%s Key:%s Value:%s", envelope.getSystemStream(), envelope.getKey(), + new String((byte[]) envelope.getMessage())); + LOG.info(msg); + + if (envelope.getKey() != null) { + System.out.println(String.format("Key:%s Value:%s", envelope.getKey(), + new String((byte[]) envelope.getMessage()))); + } else { + System.out.println(new String((byte[]) envelope.getMessage())); + } + } + + @Override + public void flush(String source) { + } + } + + public static class SimpleSystemAdmin implements SystemAdmin { + + public SimpleSystemAdmin(Config config) { + } + + @Override + public Map getOffsetsAfter(Map offsets) { + return offsets.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, null)); + } + + @Override + public Map getSystemStreamMetadata(Set streamNames) { + return streamNames.stream() + .collect(Collectors.toMap(Function.identity(), streamName -> new SystemStreamMetadata(streamName, + Collections.singletonMap(new Partition(0), + new SystemStreamMetadata.SystemStreamPartitionMetadata(null, null, null))))); + } + + @Override + public Integer offsetComparator(String offset1, String offset2) { + if (offset1 == null) { + return offset2 == null ? 0 : -1; + } else if (offset2 == null) { + return 1; + } + return offset1.compareTo(offset2); + } + } +} \ No newline at end of file diff --git a/samza-tools/src/main/java/org/apache/samza/tools/EventHubConsoleConsumer.java b/samza-tools/src/main/java/org/apache/samza/tools/EventHubConsoleConsumer.java new file mode 100644 index 0000000000..096e12d490 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/EventHubConsoleConsumer.java @@ -0,0 +1,120 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package org.apache.samza.tools; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.EventHubRuntimeInformation; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; + +/** + * Tool to read events from Microsoft Azure event hubs. + */ +public class EventHubConsoleConsumer { + + private static final String OPT_SHORT_EVENTHUB_NAME = "e"; + private static final String OPT_LONG_EVENTHUB_NAME = "ehname"; + private static final String OPT_ARG_EVENTHUB_NAME = "EVENTHUB_NAME"; + private static final String OPT_DESC_EVENTHUB_NAME = "Name of the event hub."; + + private static final String OPT_SHORT_NAMESPACE = "n"; + private static final String OPT_LONG_NAMESPACE = "namespace"; + private static final String OPT_ARG_NAMESPACE = "EVENTHUB_NAMESPACE"; + private static final String OPT_DESC_NAMESPACE = "Namespace of the event hub."; + + private static final String OPT_SHORT_KEY_NAME = "k"; + private static final String OPT_LONG_KEY_NAME = "key"; + private static final String OPT_ARG_KEY_NAME = "KEY_NAME"; + private static final String OPT_DESC_KEY_NAME = "Name of the key."; + + private static final String OPT_SHORT_TOKEN = "t"; + private static final String OPT_LONG_TOKEN = "token"; + private static final String OPT_ARG_TOKEN = "TOKEN"; + private static final String OPT_DESC_TOKEN = "Token corresponding to the key."; + + public static void main(String[] args) + throws ServiceBusException, IOException, ExecutionException, InterruptedException { + Options options = new Options(); + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_EVENTHUB_NAME, OPT_LONG_EVENTHUB_NAME, OPT_ARG_EVENTHUB_NAME, true, + OPT_DESC_EVENTHUB_NAME)); + + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_NAMESPACE, OPT_LONG_NAMESPACE, OPT_ARG_NAMESPACE, true, OPT_DESC_NAMESPACE)); + + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_KEY_NAME, OPT_LONG_KEY_NAME, OPT_ARG_KEY_NAME, true, OPT_DESC_KEY_NAME)); + + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_TOKEN, OPT_LONG_TOKEN, OPT_ARG_TOKEN, true, OPT_DESC_TOKEN)); + + CommandLineParser parser = new BasicParser(); + CommandLine cmd; + try { + cmd = parser.parse(options, args); + } catch (Exception e) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(String.format("Error: %s%neh-console-consumer.sh", e.getMessage()), options); + return; + } + + String ehName = cmd.getOptionValue(OPT_SHORT_EVENTHUB_NAME); + String namespace = cmd.getOptionValue(OPT_SHORT_NAMESPACE); + String keyName = cmd.getOptionValue(OPT_SHORT_KEY_NAME); + String token = cmd.getOptionValue(OPT_SHORT_TOKEN); + + consumeEvents(ehName, namespace, keyName, token); + } + + private static void consumeEvents(String ehName, String namespace, String keyName, String token) + throws ServiceBusException, IOException, ExecutionException, InterruptedException { + ConnectionStringBuilder connStr = new ConnectionStringBuilder(namespace, ehName, keyName, token); + + EventHubClient client = EventHubClient.createFromConnectionStringSync(connStr.toString()); + + EventHubRuntimeInformation runTimeInfo = client.getRuntimeInformation().get(); + int numPartitions = runTimeInfo.getPartitionCount(); + for (int partition = 0; partition < numPartitions; partition++) { + PartitionReceiver receiver = + client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, String.valueOf(partition), + PartitionReceiver.START_OF_STREAM); + receiver.receive(10).handle((records, throwable) -> handleComplete(receiver, records, throwable)); + } + } + + private static Object handleComplete(PartitionReceiver receiver, Iterable records, Throwable throwable) { + for (EventData record : records) { + System.out.println( + String.format("Partition %s, Event %s", receiver.getPartitionId(), new String(record.getBytes()))); + } + + receiver.receive(10).handle((r, t) -> handleComplete(receiver, r, t)); + return null; + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/GenerateKafkaEvents.java b/samza-tools/src/main/java/org/apache/samza/tools/GenerateKafkaEvents.java new file mode 100644 index 0000000000..6c30eee876 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/GenerateKafkaEvents.java @@ -0,0 +1,205 @@ +/* +* 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.samza.tools; + +import org.apache.samza.tools.schemas.PageViewEvent; +import org.apache.samza.tools.schemas.ProfileChangeEvent; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.Properties; +import java.util.function.Function; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class GenerateKafkaEvents { + + private static final String OPT_SHORT_TOPIC_NAME = "t"; + private static final String OPT_LONG_TOPIC_NAME = "topic"; + private static final String OPT_ARG_TOPIC_NAME = "TOPIC_NAME"; + private static final String OPT_DESC_TOPIC_NAME = "Name of the topic to write events to."; + + private static final String OPT_SHORT_BROKER = "b"; + private static final String OPT_LONG_BROKER = "broker"; + private static final String OPT_ARG_BROKER = "BROKER"; + private static final String OPT_DESC_BROKER = "Kafka broker endpoint."; + private static final String DEFAULT_BROKER = "localhost:9092"; + + private static final String OPT_SHORT_NUM_EVENTS = "n"; + private static final String OPT_LONG_NUM_EVENTS = "numEvents"; + private static final String OPT_ARG_NUM_EVENTS = "NUM_EVENTS"; + private static final String OPT_DESC_NUM_EVENTS = "Number of events to be produced."; + + private static final String OPT_SHORT_EVENT_TYPE = "e"; + private static final String OPT_LONG_EVENT_TYPE = "eventtype"; + private static final String OPT_ARG_EVENT_TYPE = "EVENT_TYPE"; + private static final String OPT_DESC_EVENT_TYPE = + "Type of the event (PageView|ProfileChange) Default(ProfileChange)."; + + private static final Logger LOG = LoggerFactory.getLogger(GenerateKafkaEvents.class); + private static RandomValueGenerator randValueGenerator; + + private static String[] companies = + new String[]{"Microsoft", "LinkedIn", "Google", "Facebook", "Amazon", "Apple", "Twitter", "Snap"}; + + private static final String PAGEVIEW_EVENTTYPE = "pageview"; + + public static void main(String[] args) throws UnsupportedEncodingException, InterruptedException { + randValueGenerator = new RandomValueGenerator(System.currentTimeMillis()); + Options options = new Options(); + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_TOPIC_NAME, OPT_LONG_TOPIC_NAME, OPT_ARG_TOPIC_NAME, true, OPT_DESC_TOPIC_NAME)); + + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_BROKER, OPT_LONG_BROKER, OPT_ARG_BROKER, false, OPT_DESC_BROKER)); + + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_NUM_EVENTS, OPT_LONG_NUM_EVENTS, OPT_ARG_NUM_EVENTS, false, OPT_DESC_NUM_EVENTS)); + + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_EVENT_TYPE, OPT_LONG_EVENT_TYPE, OPT_ARG_EVENT_TYPE, false, OPT_DESC_EVENT_TYPE)); + + CommandLineParser parser = new BasicParser(); + CommandLine cmd; + try { + cmd = parser.parse(options, args); + } catch (Exception e) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(String.format("Error: %s%ngenerate-events.sh", e.getMessage()), options); + return; + } + + String topicName = cmd.getOptionValue(OPT_SHORT_TOPIC_NAME); + String broker = cmd.getOptionValue(OPT_SHORT_BROKER, DEFAULT_BROKER); + long numEvents = Long.parseLong(cmd.getOptionValue(OPT_SHORT_NUM_EVENTS, String.valueOf(Long.MAX_VALUE))); + String eventType = cmd.getOptionValue(OPT_SHORT_EVENT_TYPE); + generateEvents(broker, topicName, eventType, numEvents); + } + + private static void generateEvents(String brokers, String topicName, String eventType, long numEvents) + throws UnsupportedEncodingException, InterruptedException { + Properties props = new Properties(); + props.put("bootstrap.servers", brokers); + props.put("retries", 100); + props.put("batch.size", 16384); + props.put("key.serializer", ByteArraySerializer.class.getCanonicalName()); + props.put("value.serializer", ByteArraySerializer.class.getCanonicalName()); + + Function> eventGenerator; + if (eventType.toLowerCase().contains(PAGEVIEW_EVENTTYPE)) { + eventGenerator = GenerateKafkaEvents::generatePageViewEvent; + } else { + eventGenerator = GenerateKafkaEvents::generateProfileChangeEvent; + } + + boolean doSleep = false; + // sleep only when the events have to be produced continuously. + if (numEvents == Long.MAX_VALUE) { + doSleep = true; + } + + try (Producer producer = new KafkaProducer<>(props)) { + for (int index = 0; index < numEvents; index++) { + final int finalIndex = 0; + Pair record = eventGenerator.apply(index); + producer.send(new ProducerRecord<>(topicName, record.getLeft().getBytes("UTF-8"), record.getRight()), + (metadata, exception) -> { + if (exception == null) { + LOG.info("send completed for event {} at offset {}", finalIndex, metadata.offset()); + } else { + throw new RuntimeException("Failed to send message.", exception); + } + }); + System.out.println(String.format("Published event %d to topic %s", index, topicName)); + if (doSleep) { + Thread.sleep(1000); + } + } + + producer.flush(); + } + } + + private static Pair generateProfileChangeEvent(Integer index) { + ProfileChangeEvent event = new ProfileChangeEvent(); + String name = randValueGenerator.getNextString(10, 20); + event.Name = name; + event.NewCompany = companies[randValueGenerator.getNextInt(0, companies.length - 1)]; + event.OldCompany = companies[randValueGenerator.getNextInt(0, companies.length - 1)]; + event.ProfileChangeTimestamp = System.currentTimeMillis(); + byte[] value; + try { + value = encodeAvroSpecificRecord(ProfileChangeEvent.class, event); + } catch (IOException e) { + throw new RuntimeException(e); + } + + return new ImmutablePair<>(name, value); + } + + /** + * Encode an Avro record into byte array + * + * @param clazz The class type of the Avro record + * @param record the instance of the avro record + * @param The type of the avro record. + * @return encoded bytes + * @throws java.io.IOException + */ + public static byte[] encodeAvroSpecificRecord(Class clazz, T record) throws IOException { + DatumWriter msgDatumWriter = new SpecificDatumWriter<>(clazz); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + Encoder encoder = EncoderFactory.get().binaryEncoder(os, null); + msgDatumWriter.write(record, encoder); + encoder.flush(); + return os.toByteArray(); + } + + private static Pair generatePageViewEvent(int index) { + PageViewEvent event = new PageViewEvent(); + String name = randValueGenerator.getNextString(10, 20); + event.id = randValueGenerator.getNextInt(); + event.Name = name; + event.ViewerName = randValueGenerator.getNextString(10, 20); + event.ProfileViewTimestamp = System.currentTimeMillis(); + byte[] value; + try { + value = encodeAvroSpecificRecord(PageViewEvent.class, event); + } catch (IOException e) { + throw new RuntimeException(e); + } + return new ImmutablePair<>(name, value); + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/RandomValueGenerator.java b/samza-tools/src/main/java/org/apache/samza/tools/RandomValueGenerator.java new file mode 100644 index 0000000000..18e0316559 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/RandomValueGenerator.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.samza.tools; + +import java.util.Random; + + +/** + * Simple utility to generate random values. + */ +public class RandomValueGenerator { + + private String validChars = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; + private Random rand; + + // to help reproducibility of failed tests, seed is always required + public RandomValueGenerator(long seed) { + rand = new Random(seed); + } + + public int getNextInt() { + return rand.nextInt(); + } + + // to make it inclusive of min and max for the range, add 1 to the difference + public int getNextInt(int min, int max) { + if (max == min) { + return min; + } + // assert(max > min); + + return (rand.nextInt(max - min + 1) + min); + } + + public String getNextString(int min, int max) { + int length = getNextInt(min, max); + + StringBuilder strbld = new StringBuilder(); + for (int i = 0; i < length; i++) { + char ch = validChars.charAt(rand.nextInt(validChars.length())); + strbld.append(ch); + } + + return strbld.toString(); + } + + public double getNextDouble() { + return rand.nextDouble(); + } + + public float getNextFloat() { + return rand.nextFloat(); + } + + public long getNextLong() { + long randomLong = rand.nextLong(); + + return randomLong == Long.MIN_VALUE ? 0 : Math.abs(randomLong); + } + + public boolean getNextBoolean() { + return rand.nextBoolean(); + } + + public byte[] getNextBytes(int maxBytesLength) { + byte[] bytes = new byte[this.getNextInt(0, maxBytesLength)]; + rand.nextBytes(bytes); + return bytes; + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java new file mode 100644 index 0000000000..9803117144 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java @@ -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.samza.tools; + +import com.google.common.base.Joiner; +import org.apache.samza.tools.avro.AvroSchemaGenRelConverterFactory; +import org.apache.samza.tools.avro.AvroSerDeFactory; +import org.apache.samza.tools.json.JsonRelConverterFactory; +import org.apache.samza.tools.schemas.PageViewEvent; +import org.apache.samza.tools.schemas.ProfileChangeEvent; +import org.apache.samza.tools.udf.RegexMatchUdf; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory; +import org.apache.samza.serializers.StringSerdeFactory; +import org.apache.samza.sql.avro.ConfigBasedAvroRelSchemaProviderFactory; +import org.apache.samza.sql.fn.FlattenUdf; +import org.apache.samza.sql.impl.ConfigBasedSourceResolverFactory; +import org.apache.samza.sql.impl.ConfigBasedUdfResolver; +import org.apache.samza.sql.interfaces.SqlSystemStreamConfig; +import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; +import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; +import org.apache.samza.sql.testutil.JsonUtil; +import org.apache.samza.sql.testutil.SqlFileParser; +import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; +import org.apache.samza.system.kafka.KafkaSystemFactory; + + +public class SamzaSqlConsole { + + private static final String OPT_SHORT_SQL_FILE = "f"; + private static final String OPT_LONG_SQL_FILE = "file"; + private static final String OPT_ARG_SQL_FILE = "SQL_FILE"; + private static final String OPT_DESC_SQL_FILE = "Path to the SQL file to execute."; + + private static final String OPT_SHORT_SQL_STMT = "s"; + private static final String OPT_LONG_SQL_STMT = "sql"; + private static final String OPT_ARG_SQL_STMT = "SQL_STMT"; + private static final String OPT_DESC_SQL_STMT = "SQL statement to execute."; + + private static final String SAMZA_SYSTEM_KAFKA = "kafka"; + private static final String SAMZA_SYSTEM_LOG = "log"; + + public static void main(String[] args) { + Options options = new Options(); + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_SQL_FILE, OPT_LONG_SQL_FILE, OPT_ARG_SQL_FILE, false, OPT_DESC_SQL_FILE)); + options.addOption( + CommandLineHelper.createOption(OPT_SHORT_SQL_STMT, OPT_LONG_SQL_STMT, OPT_ARG_SQL_STMT, false, OPT_DESC_SQL_STMT)); + + CommandLineParser parser = new BasicParser(); + CommandLine cmd; + try { + cmd = parser.parse(options, args); + if (!cmd.hasOption(OPT_SHORT_SQL_STMT) && !cmd.hasOption(OPT_SHORT_SQL_FILE)) { + throw new Exception( + String.format("One of the (%s or %s) options needs to be set", OPT_SHORT_SQL_FILE, OPT_SHORT_SQL_STMT)); + } + } catch (Exception e) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(String.format("Error: %s%nsamza-sql-console.sh", e.getMessage()), options); + return; + } + + List sqlStmts; + + if (cmd.hasOption(OPT_SHORT_SQL_FILE)) { + String sqlFile = cmd.getOptionValue(OPT_SHORT_SQL_FILE); + sqlStmts = SqlFileParser.parseSqlFile(sqlFile); + } else { + String sql = cmd.getOptionValue(OPT_SHORT_SQL_STMT); + System.out.println("Executing sql " + sql); + sqlStmts = Collections.singletonList(sql); + } + + executeSql(sqlStmts); + } + + public static void executeSql(List sqlStmts) { + Map staticConfigs = fetchSamzaSqlConfig(); + staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + runner.runAndWaitForFinish(); + } + + public static Map fetchSamzaSqlConfig() { + HashMap staticConfigs = new HashMap<>(); + + staticConfigs.put(JobConfig.JOB_NAME(), "sql-job"); + staticConfigs.put(JobConfig.PROCESSOR_ID(), "1"); + staticConfigs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName()); + staticConfigs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName()); + + staticConfigs.put(SamzaSqlApplicationConfig.CFG_SOURCE_RESOLVER, "config"); + String configSourceResolverDomain = + String.format(SamzaSqlApplicationConfig.CFG_FMT_SOURCE_RESOLVER_DOMAIN, "config"); + staticConfigs.put(configSourceResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY, + ConfigBasedSourceResolverFactory.class.getName()); + + staticConfigs.put(SamzaSqlApplicationConfig.CFG_UDF_RESOLVER, "config"); + String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config"); + staticConfigs.put(configUdfResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY, + ConfigBasedUdfResolver.class.getName()); + staticConfigs.put(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES, + Joiner.on(",").join(RegexMatchUdf.class.getName(), FlattenUdf.class.getName())); + + staticConfigs.put("serializers.registry.string.class", StringSerdeFactory.class.getName()); + staticConfigs.put("serializers.registry.avro.class", AvroSerDeFactory.class.getName()); + staticConfigs.put(AvroSerDeFactory.CFG_AVRO_SCHEMA, ProfileChangeEvent.SCHEMA$.toString()); + + String kafkaSystemConfigPrefix = + String.format(ConfigBasedSourceResolverFactory.CFG_FMT_SAMZA_PREFIX, SAMZA_SYSTEM_KAFKA); + String avroSamzaSqlConfigPrefix = configSourceResolverDomain + String.format("%s.", SAMZA_SYSTEM_KAFKA); + staticConfigs.put(kafkaSystemConfigPrefix + "samza.factory", KafkaSystemFactory.class.getName()); + staticConfigs.put(kafkaSystemConfigPrefix + "samza.key.serde", "string"); + staticConfigs.put(kafkaSystemConfigPrefix + "samza.msg.serde", "avro"); + staticConfigs.put(kafkaSystemConfigPrefix + "consumer.zookeeper.connect", "localhost:2181"); + staticConfigs.put(kafkaSystemConfigPrefix + "producer.bootstrap.servers", "localhost:9092"); + + staticConfigs.put(kafkaSystemConfigPrefix + "samza.offset.reset", "true"); + staticConfigs.put(kafkaSystemConfigPrefix + "samza.offset.default", "oldest"); + + staticConfigs.put(avroSamzaSqlConfigPrefix + SqlSystemStreamConfig.CFG_SAMZA_REL_CONVERTER, "avro"); + staticConfigs.put(avroSamzaSqlConfigPrefix + SqlSystemStreamConfig.CFG_REL_SCHEMA_PROVIDER, "config"); + + String logSystemConfigPrefix = + String.format(ConfigBasedSourceResolverFactory.CFG_FMT_SAMZA_PREFIX, SAMZA_SYSTEM_LOG); + String logSamzaSqlConfigPrefix = configSourceResolverDomain + String.format("%s.", SAMZA_SYSTEM_LOG); + staticConfigs.put(logSystemConfigPrefix + "samza.factory", ConsoleLoggingSystemFactory.class.getName()); + staticConfigs.put(logSamzaSqlConfigPrefix + SqlSystemStreamConfig.CFG_SAMZA_REL_CONVERTER, "json"); + staticConfigs.put(logSamzaSqlConfigPrefix + SqlSystemStreamConfig.CFG_REL_SCHEMA_PROVIDER, "config"); + + String avroSamzaToRelMsgConverterDomain = + String.format(SamzaSqlApplicationConfig.CFG_FMT_SAMZA_REL_CONVERTER_DOMAIN, "avro"); + + staticConfigs.put(avroSamzaToRelMsgConverterDomain + SamzaSqlApplicationConfig.CFG_FACTORY, + AvroSchemaGenRelConverterFactory.class.getName()); + + String jsonSamzaToRelMsgConverterDomain = + String.format(SamzaSqlApplicationConfig.CFG_FMT_SAMZA_REL_CONVERTER_DOMAIN, "json"); + + staticConfigs.put(jsonSamzaToRelMsgConverterDomain + SamzaSqlApplicationConfig.CFG_FACTORY, + JsonRelConverterFactory.class.getName()); + + String configAvroRelSchemaProviderDomain = + String.format(SamzaSqlApplicationConfig.CFG_FMT_REL_SCHEMA_PROVIDER_DOMAIN, "config"); + staticConfigs.put(configAvroRelSchemaProviderDomain + SamzaSqlApplicationConfig.CFG_FACTORY, + ConfigBasedAvroRelSchemaProviderFactory.class.getName()); + + staticConfigs.put( + configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA, + "kafka", "PageViewStream"), PageViewEvent.SCHEMA$.toString()); + + staticConfigs.put( + configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA, + "kafka", "ProfileChangeStream"), ProfileChangeEvent.SCHEMA$.toString()); + + return staticConfigs; + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverter.java b/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverter.java new file mode 100644 index 0000000000..198b84bbab --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverter.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.samza.tools.avro; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.reflect.ReflectData; +import org.apache.samza.config.Config; +import org.apache.samza.operators.KV; +import org.apache.samza.sql.avro.AvroRelConverter; +import org.apache.samza.sql.avro.AvroRelSchemaProvider; +import org.apache.samza.sql.data.SamzaSqlRelMessage; +import org.apache.samza.system.SystemStream; + + +/** + * Special form for AvroRelConverter that generates the avro schema on the output based on the + * fields in {@link SamzaSqlRelMessage} and uses the schema to serialize the output. + * This is useful to test out the SQL quickly when the destination system supports Avro serialized data, + * without having to manually author the avro schemas for various SQL queries. + */ +public class AvroSchemaGenRelConverter extends AvroRelConverter { + + private final String streamName; + private Map schemas = new HashMap<>(); + + public AvroSchemaGenRelConverter(SystemStream systemStream, AvroRelSchemaProvider schemaProvider, Config config) { + super(systemStream, schemaProvider, config); + streamName = systemStream.getStream(); + } + + @Override + public KV convertToSamzaMessage(SamzaSqlRelMessage relMessage) { + Schema schema = computeSchema(streamName, relMessage); + return convertToSamzaMessage(relMessage, schema); + } + + private Schema computeSchema(String streamName, SamzaSqlRelMessage relMessage) { + List keyFields = new ArrayList<>(); + List fieldNames = relMessage.getFieldNames(); + List values = relMessage.getFieldValues(); + + for (int index = 0; index < fieldNames.size(); index++) { + if (fieldNames.get(index).equals(SamzaSqlRelMessage.KEY_NAME) || values.get(index) == null) { + continue; + } + + Object value = values.get(index); + Schema avroType; + if (value instanceof GenericData.Record) { + avroType = ((GenericData.Record) value).getSchema(); + } else { + avroType = ReflectData.get().getSchema(value.getClass()); + } + keyFields.add(new Schema.Field(fieldNames.get(index), avroType, "", null)); + } + + Schema ks = Schema.createRecord(streamName, "", streamName + "_namespace", false); + ks.setFields(keyFields); + String schemaStr = ks.toString(); + Schema schema; + // See whether we have a schema object corresponding to the schemaValue and reuse it. + // CachedSchemaRegistryClient doesn't like if we recreate schema objects. + if (schemas.containsKey(schemaStr)) { + schema = schemas.get(schemaStr); + } else { + schema = Schema.parse(schemaStr); + schemas.put(schemaStr, schema); + } + + return schema; + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverterFactory.java b/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverterFactory.java new file mode 100644 index 0000000000..cf8c5687b8 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSchemaGenRelConverterFactory.java @@ -0,0 +1,43 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package org.apache.samza.tools.avro; + +import java.util.HashMap; +import org.apache.samza.config.Config; +import org.apache.samza.sql.avro.AvroRelSchemaProvider; +import org.apache.samza.sql.interfaces.RelSchemaProvider; +import org.apache.samza.sql.interfaces.SamzaRelConverter; +import org.apache.samza.sql.interfaces.SamzaRelConverterFactory; +import org.apache.samza.system.SystemStream; + + +/** + * Factory for {@link AvroSchemaGenRelConverter} + */ +public class AvroSchemaGenRelConverterFactory implements SamzaRelConverterFactory { + + private final HashMap relConverters = new HashMap<>(); + + @Override + public SamzaRelConverter create(SystemStream systemStream, RelSchemaProvider relSchemaProvider, Config config) { + return relConverters.computeIfAbsent(systemStream, + ss -> new AvroSchemaGenRelConverter(ss, (AvroRelSchemaProvider) relSchemaProvider, config)); + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSerDeFactory.java b/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSerDeFactory.java new file mode 100644 index 0000000000..a0523060ac --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/avro/AvroSerDeFactory.java @@ -0,0 +1,96 @@ +/* +* 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.samza.tools.avro; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.commons.lang.NotImplementedException; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerdeFactory; + + +/** + * Avro SerDe that can be used to serialize or deserialize the avro {@link GenericRecord}. + */ +public class AvroSerDeFactory implements SerdeFactory { + + public static String CFG_AVRO_SCHEMA = "serializers.avro.schema"; + + @Override + public Serde getSerde(String name, Config config) { + return new AvroSerDe(config); + } + + private class AvroSerDe implements Serde { + private final Schema schema; + + public AvroSerDe(Config config) { + schema = Schema.parse(config.get(CFG_AVRO_SCHEMA)); + } + + @Override + public Object fromBytes(byte[] bytes) { + GenericRecord record; + try { + record = genericRecordFromBytes(bytes, schema); + } catch (IOException e) { + throw new SamzaException("Unable to deserialize the record", e); + } + return record; + } + + @Override + public byte[] toBytes(Object o) { + GenericRecord record = (GenericRecord) o; + try { + return encodeAvroGenericRecord(schema, record); + } catch (IOException e) { + throw new SamzaException("Unable to serialize the record", e); + } + } + } + + private byte[] encodeAvroGenericRecord(Schema schema, GenericRecord record) throws IOException { + DatumWriter msgDatumWriter = new GenericDatumWriter<>(schema); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + Encoder encoder = EncoderFactory.get().binaryEncoder(os, null); + msgDatumWriter.write(record, encoder); + encoder.flush(); + return os.toByteArray(); + } + + private static T genericRecordFromBytes(byte[] bytes, Schema schema) throws IOException { + BinaryDecoder binDecoder = DecoderFactory.defaultFactory().createBinaryDecoder(bytes, null); + GenericDatumReader reader = new GenericDatumReader<>(schema); + return reader.read(null, binDecoder); + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/json/JsonRelConverterFactory.java b/samza-tools/src/main/java/org/apache/samza/tools/json/JsonRelConverterFactory.java new file mode 100644 index 0000000000..de8dec8e06 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/json/JsonRelConverterFactory.java @@ -0,0 +1,93 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package org.apache.samza.tools.json; + +import java.io.IOException; +import java.util.List; +import org.apache.commons.lang.NotImplementedException; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.operators.KV; +import org.apache.samza.sql.data.SamzaSqlRelMessage; +import org.apache.samza.sql.interfaces.RelSchemaProvider; +import org.apache.samza.sql.interfaces.SamzaRelConverter; +import org.apache.samza.sql.interfaces.SamzaRelConverterFactory; +import org.apache.samza.system.SystemStream; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.node.ObjectNode; + + +/** + * SamzaRelConverter that can convert {@link SamzaSqlRelMessage} to json string byte array. + */ +public class JsonRelConverterFactory implements SamzaRelConverterFactory { + + ObjectMapper mapper = new ObjectMapper(); + + @Override + public SamzaRelConverter create(SystemStream systemStream, RelSchemaProvider relSchemaProvider, Config config) { + return new JsonRelConverter(); + } + + public class JsonRelConverter implements SamzaRelConverter { + + @Override + public SamzaSqlRelMessage convertToRelMessage(KV kv) { + throw new NotImplementedException(); + } + + @Override + public KV convertToSamzaMessage(SamzaSqlRelMessage relMessage) { + + String jsonValue; + ObjectNode node = mapper.createObjectNode(); + + List fieldNames = relMessage.getFieldNames(); + List values = relMessage.getFieldValues(); + + for (int index = 0; index < fieldNames.size(); index++) { + Object value = values.get(index); + if (value == null) { + continue; + } + + // TODO limited support right now. + if (Long.class.isAssignableFrom(value.getClass())) { + node.put(fieldNames.get(index), (Long) value); + } else if (Integer.class.isAssignableFrom(value.getClass())) { + node.put(fieldNames.get(index), (Integer) value); + } else if (Double.class.isAssignableFrom(value.getClass())) { + node.put(fieldNames.get(index), (Double) value); + } else if (String.class.isAssignableFrom(value.getClass())) { + node.put(fieldNames.get(index), (String) value); + } else { + node.put(fieldNames.get(index), value.toString()); + } + } + try { + jsonValue = mapper.writeValueAsString(node); + } catch (IOException e) { + throw new SamzaException("Error json serializing object", e); + } + + return new KV<>(relMessage.getKey(), jsonValue.getBytes()); + } + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.avsc b/samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.avsc new file mode 100644 index 0000000000..dea6a122b7 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.avsc @@ -0,0 +1,51 @@ +/* +* 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. +*/ + +{ + "name": "PageViewEvent", + "version" : 1, + "namespace": "com.linkedin.samza.tools.avro", + "type": "record", + "fields": [ + { + "name": "id", + "doc": "Record id.", + "type": ["null", "int"], + "default":null + }, + { + "name": "Name", + "doc": "Name of the profile.", + "type": ["null", "string"], + "default":null + }, + { + "name": "ViewerName", + "doc": "Name of the person who viewed the profile.", + "type": ["null", "string"], + "default":null + }, + { + "name": "ProfileViewTimestamp", + "doc": "Time at which the profile was viewed.", + "type": ["null", "long"], + "default":null + } + ] +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.java b/samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.java new file mode 100644 index 0000000000..7cfbcb9fb7 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/schemas/PageViewEvent.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. +*/ + +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.samza.tools.schemas; + +@SuppressWarnings("all") +public class PageViewEvent extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"PageViewEvent\",\"namespace\":\"com.linkedin.samza.tools.avro\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"],\"doc\":\"Record id.\",\"default\":null},{\"name\":\"Name\",\"type\":[\"null\",\"string\"],\"doc\":\"Name of the profile.\",\"default\":null},{\"name\":\"ViewerName\",\"type\":[\"null\",\"string\"],\"doc\":\"Name of the person who viewed the profile.\",\"default\":null},{\"name\":\"ProfileViewTimestamp\",\"type\":[\"null\",\"long\"],\"doc\":\"Time at which the profile was viewed.\",\"default\":null}]}"); + /** Record id. */ + public java.lang.Integer id; + /** Name of the profile. */ + public java.lang.CharSequence Name; + /** Name of the person who viewed the profile. */ + public java.lang.CharSequence ViewerName; + /** Time at which the profile was viewed. */ + public java.lang.Long ProfileViewTimestamp; + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return id; + case 1: return Name; + case 2: return ViewerName; + case 3: return ProfileViewTimestamp; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: id = (java.lang.Integer)value$; break; + case 1: Name = (java.lang.CharSequence)value$; break; + case 2: ViewerName = (java.lang.CharSequence)value$; break; + case 3: ProfileViewTimestamp = (java.lang.Long)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.avsc b/samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.avsc new file mode 100644 index 0000000000..5c1e49d447 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.avsc @@ -0,0 +1,51 @@ +/* +* 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. +*/ + +{ + "name": "ProfileChangeEvent", + "version" : 1, + "namespace": "com.linkedin.samza.tools.avro", + "type": "record", + "fields": [ + { + "name": "Name", + "doc": "Name of the profile.", + "type": ["null", "string"], + "default":null + }, + { + "name": "NewCompany", + "doc": "Name of the new company the person joined.", + "type": ["null", "string"], + "default":null + }, + { + "name": "OldCompany", + "doc": "Name of the old company the person was working.", + "type": ["null", "string"], + "default":null + }, + { + "name": "ProfileChangeTimestamp", + "doc": "Time at which the profile was changed.", + "type": ["null", "long"], + "default":null + } + ] +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.java b/samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.java new file mode 100644 index 0000000000..c8e951c275 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/schemas/ProfileChangeEvent.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. +*/ + +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.samza.tools.schemas; + +@SuppressWarnings("all") +public class ProfileChangeEvent extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"ProfileChangeEvent\",\"namespace\":\"com.linkedin.samza.tools.avro\",\"fields\":[{\"name\":\"Name\",\"type\":[\"null\",\"string\"],\"doc\":\"Name of the profile.\",\"default\":null},{\"name\":\"NewCompany\",\"type\":[\"null\",\"string\"],\"doc\":\"Name of the new company the person joined.\",\"default\":null},{\"name\":\"OldCompany\",\"type\":[\"null\",\"string\"],\"doc\":\"Name of the old company the person was working.\",\"default\":null},{\"name\":\"ProfileChangeTimestamp\",\"type\":[\"null\",\"long\"],\"doc\":\"Time at which the profile was changed.\",\"default\":null}]}"); + /** Name of the profile. */ + public java.lang.CharSequence Name; + /** Name of the new company the person joined. */ + public java.lang.CharSequence NewCompany; + /** Name of the old company the person was working. */ + public java.lang.CharSequence OldCompany; + /** Time at which the profile was changed. */ + public java.lang.Long ProfileChangeTimestamp; + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return Name; + case 1: return NewCompany; + case 2: return OldCompany; + case 3: return ProfileChangeTimestamp; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: Name = (java.lang.CharSequence)value$; break; + case 1: NewCompany = (java.lang.CharSequence)value$; break; + case 2: OldCompany = (java.lang.CharSequence)value$; break; + case 3: ProfileChangeTimestamp = (java.lang.Long)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } +} diff --git a/samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.java b/samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.java new file mode 100644 index 0000000000..42e9f04db7 --- /dev/null +++ b/samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.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.samza.tools.udf; + +import java.util.regex.Pattern; +import org.apache.samza.config.Config; +import org.apache.samza.sql.udfs.ScalarUdf; + + +/** + * Simple RegexMatch Udf. + */ +public class RegexMatchUdf implements ScalarUdf { + @Override + public void init(Config config) { + + } + + @Override + public Object execute(Object... args) { + return Pattern.matches((String) args[0], (String) args[1]); + } +} diff --git a/samza-tools/src/main/resources/log4j.xml b/samza-tools/src/main/resources/log4j.xml new file mode 100644 index 0000000000..15aad1cde7 --- /dev/null +++ b/samza-tools/src/main/resources/log4j.xml @@ -0,0 +1,43 @@ + + + + + + + @log4j.appenders.webapp@ + + @log4j.appenders.public_access@ + + + + + + + + @log4j.loggers.spring@ + + @log4j.loggers.public_access@ + + + + + + @log4j.loggers.root@ + + + + + + + + diff --git a/settings.gradle b/settings.gradle index e50e816215..06141ac14d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -25,7 +25,8 @@ include \ 'samza-shell', 'samza-azure', 'samza-sql', - 'samza-aws' + 'samza-aws', + 'samza-tools' def scalaModules = [ 'samza-core', From 608e4e0b7635aa00a9f1e01647de8d978051d2cf Mon Sep 17 00:00:00 2001 From: Xinyu Liu Date: Tue, 12 Dec 2017 12:16:39 -0800 Subject: [PATCH 10/36] SAMZA-1534: Fix the visualization in job graph with the new PartitionBy Op Seems the stream and the partitionBy op has the same id. So in rendering I added the stream as the id for the node. Also resolved the run.id collision issue. Author: xiliu Reviewers: Jagadish V Closes #385 from xinyuiscool/SAMZA-1534 --- .../execution/JobGraphJsonGenerator.java | 6 +- .../runtime/RemoteApplicationRunner.java | 7 +- .../execution/TestJobGraphJsonGenerator.java | 75 +++++++++++++++++++ .../src/main/visualizer/js/planToDagre.js | 6 +- 4 files changed, 87 insertions(+), 7 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 2729fa3598..2be01af5c4 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -28,19 +28,18 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; - import org.apache.samza.config.ApplicationConfig; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputOperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; +import org.apache.samza.operators.spec.PartitionByOperatorSpec; import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; import org.apache.samza.table.TableSpec; import org.codehaus.jackson.annotate.JsonProperty; import org.codehaus.jackson.map.ObjectMapper; - /** * This class generates the JSON representation of the {@link JobGraph}. */ @@ -220,6 +219,9 @@ private Map operatorToMap(OperatorSpec spec) { if (spec instanceof OutputOperatorSpec) { OutputStreamImpl outputStream = ((OutputOperatorSpec) spec).getOutputStream(); map.put("outputStreamId", outputStream.getStreamSpec().getId()); + } else if (spec instanceof PartitionByOperatorSpec) { + OutputStreamImpl outputStream = ((PartitionByOperatorSpec) spec).getOutputStream(); + map.put("outputStreamId", outputStream.getStreamSpec().getId()); } if (spec instanceof StreamTableJoinOperatorSpec) { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 3e046afe11..1ead8416aa 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -33,6 +33,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.UUID; + /** * This class implements the {@link ApplicationRunner} that runs the applications in a remote cluster @@ -57,8 +59,9 @@ public void runTask() { @Override public void run(StreamApplication app) { try { - // TODO: this is a tmp solution and the run.id generation will be addressed in another JIRA - String runId = String.valueOf(System.currentTimeMillis()); + // TODO: run.id needs to be set for standalone: SAMZA-1531 + // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); LOG.info("The run id for this run is {}", runId); // 1. initialize and plan diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index ba5c922c49..b48c82dd89 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -27,10 +27,13 @@ import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.StreamGraphImpl; import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.operators.windows.Windows; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.LongSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.StringSerde; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; import org.codehaus.jackson.map.ObjectMapper; @@ -48,6 +51,12 @@ public class TestJobGraphJsonGenerator { + public class PageViewEvent { + String getCountry() { + return ""; + } + } + @Test public void test() throws Exception { @@ -147,4 +156,70 @@ public void test() throws Exception { assertEquals(2, nodes.sinkStreams.size()); assertEquals(2, nodes.intermediateStreams.size()); } + + @Test + public void test2() throws Exception { + Map configMap = new HashMap<>(); + configMap.put(JobConfig.JOB_NAME(), "test-app"); + configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); + Config config = new MapConfig(configMap); + + StreamSpec input = new StreamSpec("PageView", "hdfs:/user/dummy/PageViewEvent", "hdfs"); + StreamSpec output = new StreamSpec("PageViewCount", "PageViewCount", "kafka"); + + ApplicationRunner runner = mock(ApplicationRunner.class); + when(runner.getStreamSpec("PageView")).thenReturn(input); + when(runner.getStreamSpec("PageViewCount")).thenReturn(output); + + // intermediate streams used in tests + when(runner.getStreamSpec("test-app-1-partition_by-keyed-by-country")) + .thenReturn(new StreamSpec("test-app-1-partition_by-keyed-by-country", "test-app-1-partition_by-keyed-by-country", "kafka")); + + // set up external partition count + Map system1Map = new HashMap<>(); + system1Map.put("hdfs:/user/dummy/PageViewEvent", 512); + Map system2Map = new HashMap<>(); + system2Map.put("PageViewCount", 16); + + Map systemAdmins = new HashMap<>(); + SystemAdmin systemAdmin1 = createSystemAdmin(system1Map); + SystemAdmin systemAdmin2 = createSystemAdmin(system2Map); + systemAdmins.put("hdfs", systemAdmin1); + systemAdmins.put("kafka", systemAdmin2); + StreamManager streamManager = new StreamManager(systemAdmins); + + StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config); + MessageStream> inputStream = streamGraph.getInputStream("PageView"); + inputStream + .partitionBy(kv -> kv.getValue().getCountry(), kv -> kv.getValue(), "keyed-by-country") + .window(Windows.keyedTumblingWindow(kv -> kv.getValue().getCountry(), + Duration.ofSeconds(10L), + () -> 0L, + (m, c) -> c + 1L, + new StringSerde(), + new LongSerde()), "count-by-country") + .map(pane -> new KV<>(pane.getKey().getKey(), pane.getMessage())) + .sendTo(streamGraph.getOutputStream("PageViewCount")); + + ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); + ExecutionPlan plan = planner.plan(streamGraph); + String json = plan.getPlanAsJson(); + System.out.println(json); + + // deserialize + ObjectMapper mapper = new ObjectMapper(); + JobGraphJsonGenerator.JobGraphJson nodes = mapper.readValue(json, JobGraphJsonGenerator.JobGraphJson.class); + JobGraphJsonGenerator.OperatorGraphJson operatorGraphJson = nodes.jobs.get(0).operatorGraph; + assertEquals(2, operatorGraphJson.inputStreams.size()); + assertEquals(4, operatorGraphJson.operators.size()); + assertEquals(1, nodes.sourceStreams.size()); + assertEquals(1, nodes.sinkStreams.size()); + assertEquals(1, nodes.intermediateStreams.size()); + + // verify partitionBy op output to the intermdiate stream of the same id + assertEquals(operatorGraphJson.operators.get("test-app-1-partition_by-keyed-by-country").get("outputStreamId"), + "test-app-1-partition_by-keyed-by-country"); + assertEquals(operatorGraphJson.operators.get("test-app-1-send_to-5").get("outputStreamId"), + "PageViewCount"); + } } diff --git a/samza-shell/src/main/visualizer/js/planToDagre.js b/samza-shell/src/main/visualizer/js/planToDagre.js index 0421c338fe..77adaa2cfa 100644 --- a/samza-shell/src/main/visualizer/js/planToDagre.js +++ b/samza-shell/src/main/visualizer/js/planToDagre.js @@ -36,7 +36,7 @@ function planToDagre(data) { labelVal += "
  • PhysicalName: " + stream.streamSpec.physicalName + "
  • " labelVal += "
  • PartitionCount: " + stream.streamSpec.partitionCount + "
  • " labelVal += "" - g.setNode(streamId, { label: labelVal, labelType: "html", shape: "ellipse", class: streamClasses[i] }); + g.setNode(streamId + "-stream", { label: labelVal, labelType: "html", shape: "ellipse", class: streamClasses[i] }); } } @@ -67,7 +67,7 @@ function planToDagre(data) { for (var k = 0; k < inputs.length; k++) { var input = inputs[k]; for (var m = 0; m < input.nextOperatorIds.length; m++) { - g.setEdge(input.streamId, input.nextOperatorIds[m]); + g.setEdge(input.streamId + "-stream", input.nextOperatorIds[m]); } } @@ -78,7 +78,7 @@ function planToDagre(data) { g.setEdge(opId, operator.nextOperatorIds[j]); } if (typeof(operator.outputStreamId) !== 'undefined') { - g.setEdge(opId, operator.outputStreamId); + g.setEdge(opId, operator.outputStreamId + "-stream"); } } } From cb92cf18b5b1707d98a442a32a9f6c3767e13527 Mon Sep 17 00:00:00 2001 From: Xinyu Liu Date: Tue, 12 Dec 2017 17:07:07 -0800 Subject: [PATCH 11/36] SAMZA-1512: Documentation on the multi-stage batch processing Add overview documentation to explain how partitionBy(), checkpoint and state works in batch. Also organized the existing hdfs consumer/producer docs into the same hadoop folder under documentation. Author: xinyuiscool Reviewers: Jake Maes Closes #381 from xinyuiscool/SAMZA-1512 --- .../hadoop/multi_stage_batch.png | Bin 0 -> 65364 bytes .../hadoop/unified_batch_streaming.png | Bin 0 -> 17371 bytes .../versioned/{hdfs => hadoop}/consumer.md | 2 +- .../versioned/hadoop/overview.md | 46 ++++++++++++++++++ .../versioned/{hdfs => hadoop}/producer.md | 2 - docs/learn/documentation/versioned/index.html | 15 +++--- .../versioned/jobs/configuration-table.html | 4 +- .../versioned/yarn/yarn-security.md | 1 - 8 files changed, 58 insertions(+), 12 deletions(-) create mode 100644 docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png create mode 100644 docs/img/versioned/learn/documentation/hadoop/unified_batch_streaming.png rename docs/learn/documentation/versioned/{hdfs => hadoop}/consumer.md (99%) create mode 100644 docs/learn/documentation/versioned/hadoop/overview.md rename docs/learn/documentation/versioned/{hdfs => hadoop}/producer.md (98%) diff --git a/docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png b/docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png new file mode 100644 index 0000000000000000000000000000000000000000..11425400f31a19ca5a711067defb6b8d99ca123b GIT binary patch literal 65364 zcmcG$WmH^2vo?whF!*4>Ed+OW4-i}vY;bpXmqCIi5Zob165QQ_4=_k@cY?daCFi{7 z+hKv0}c+M84eEK0*L%tbE0q> z@cM#x(~y&bs~98Oe=Y0UXz94?C@Tq>IXSSIm^+zTuz5Q;zqW#d6Y&;$Ejn1Zn^1W> z*gLujd5gaNuNFeD<$u-eZ>j#PiMyTXTODOJDrqNI3o2eV9yX4*VrW!UR3fhCmO>iu zW&hLt^-J`vwY$5s5IehuU>EHy=lL6K_^WH=6$@ z@;`LmTez9I+Bm!0I5|@NqibU73`L}iYkUC!v4S0 zCWfZeL$3=5Cjlq_UQ){&{?Gu$U#o9MJX6Xk9~qY_KNVI~miJx~{`EOSB>^DGsN-Di z<5Aw~u0p2k#~HD7Ly59i&+ zv#2$SKx#k+06qd5JS133f;#X2#wR91NJi7=dlUpo5vZPQe^lj$^cSfvGz40~PE{5Uc2PH1)3ZzLTat4~EB*jqISUN!VwP|`co2cP?y zLB1z}>%2`lUnIB>>e@yme2y;|AN-rEC!X%qzHMk}FkWbd?MJp109&^Bh*%qgFv9sS z!}|FAyB$AhO;Gp?hh47O+C#i91^w%Q>}9)FXFLU~NyxfyXbde&PoGVzle(jL&n2;{ zz2R7_-q2PYbCLA0nkXe@S##25@=@>^4zyWu7qYK~20C%r5o)L~4Q~VOH}5v*yKa!i z(E{zau}|sAP$co3Lx{KAl!++&LtLD=`f-?C$>7&e^#Qq*|9cgndVf)W0AW~}UMMqE6g2`Eg>26Z23`B}Ze*`hj_ z`lliGm(s@4hZ0-7OW$1|cpFL_Fm2Tjt_yFowoK-xpN4nbGg^%F5&Y(fjnG8z{i$og z@5RY%#SlxfNxTLO7^uHd5Xv?4?j`hscn0~Aw#i~PwB4xN-%V`vL>~MZr0Hi zTP?%M?1}+@)GmYGC5V7S=f7H*zWuaWI;gk-x@zIJn^jm!+JUCb^a97^gl|_0F*ceK zaG>7ETfV_9Utyj++`}1oe`UG#|Ds#GinB(HWV0LH@irD361`PRG%C0t783TT2@#}W zMRn`Hz@7obT7$WUlLnNbk&bz(g^0H;f?ozBlRvSV)a|y&Tl)!xOV500|8vjnhxv@2V z3%BCuZQ!|?IdbHV&sjfLCuy%ASphpsvYtI`?46|`KTGk#0TD|7&p|if+FjC35X-y0qfpJmp$pl(Mu8SPhQfGnXy0C!lL1^}*T*DQ45$E%F zej?b^N^2zvXx8qH_2mYYyJP%ZXfjX7GCk56mn&U}OR@aj`8bDk>DTZ!@11QZ7lm1@ z>Ne@e&eyWc z;_lKYcNh=yRM(8-8qScC4aSzf>Y`EMWO$1VSoi_yIps(%)QmR5tteL~)cfsfuohAd z#41KkqT1pG5yaJaxX%f>UpL&Bl@YSuUwzI*B?NaU2+s`-h=06y3ZY?XXBcnG2cuqH z--&rVFBhEDik9V_SsW6o&+XR)<7fU0V#}?79t@J6*vohCV$E~trx+sH1I2gHp&vGp zLprQvN%%2msnrUMwvE$X@thfr-MZ;ODNkJPzw-SB zQP0~w1L~Y!;DM$P#8ZjIHET;X7VhESR3`5(+Puf)(n+D6+zA%_J*)#9pEuLQ`-5Hg z6#dAbUe`P702EK)g9R@_*k!;KIPFYXm`*!nMQjnlv5AeABU6;hJ-{)-)pSCY{AC>P zzDrbWI9AA5&Cc?-%ZW=s6la|4x$uZOUi%Pdasux*QHSE7w1G@KHQZwtPXFJtex75* zZxeve;>Cmh&Z{pu>u&9Xim$NV{*zmPbgTt*d0H4gJ{ z&ef4^)Oc#`QO1AML-&Mpi0#m4niruA+@G7VFZj$zd~jrC$uK^f%7VpW(i+K7o*(-- z2Dl5M@61S^jFt9C&AQsyiKe=GaqBmglI4VrW8DPZur2SPh2--$H)JP+WTPwxhugPW zJ8aOf*gF)#-y)oR$q$jlmvr2di)m(QkwjBO{0teyv)xH)6S8fbaV~*9u@CP?2K|O~ zKl_t~jQjYXcToT4z7lrhseLD6R09RY``I}RchJ!l=nfc;IeuZ9`lN`yDbmNF#=$OfV<%}bxM@7dQRx<5RKzf5y zM4c8|6--5*0GOh)EHX>v_#+{}>ySiDwh_5OZn?%*$&!t?y%Vup@Ou~|c{Ml0G+F8$d zI{<|h^-o;q@A}*TcR2mA)>szJ&Sn&$Vg3GfjPf(z zI+@dicaVF8SQaIP7ScMps5=Jo&Y%p4S+v>f!1K4SbMS-!AEdijZ!GJEFs|3eH?@2Q z=jF9laGks6=`PDQzHV;$$o<9W(<4LJClZ9S=6Rxl&^!YX9+_XV2{si)M4LK^k~QVM z5km`VVl(HZs6zDkK*;hf}}HW!TRl*l8h>$~#`^<6MQJEFCyd z+H^@hmIk64r69Ce`OHuOWya`Q&tFZ?h}@?sv!pI$7sx@q7++^G7_atG0mW|g37n~> z$G_WxiFwk9SJ4&2cWY@cOf7*D)lj{!;}zHG;uWl2XC(bR_g}^(>o_Y5IH!?I>QD*1 zgb^NiMZwd*S!K3;XJgoh{h|d*KT~rA*v`IBPp6_G8-DPIrg_P^VXDv1%8{i8)N`eD zbJJGZ<%&8i>0E~7QSV}yMRa*33%1sOFm&p**cRZROT5d%Y>pVeB6GLIGU<1k5wHYx z_}!{l$XMMK{6^W{@&bl7kV!MJahUi^LRr&X$X7M*NP8!g!uh6R1#1GL#cNK8D$BY$ zL8qzp%LkA8(b`fTtS#Tf_(N|7?A+;kdD;u<#Mw3R1MMgkkkv z^GQNVB#U_S%dD5nqqZe$_8&T}rE=iq`b;F(W{yZ-?5+`b`iG)LOBRuAya)G9B%>+nMEfmQVMO{)jRceg=2NURiNmknB`LC!jMQ#5xa} z;ra_`F9bLB0xYfV&K^@NWS{=X#&1*!*81lN+I}(5J%2lyw)kNuhHHw5V$&JOy;GZh z_tDb7@@v}Y5YFT1qMFUBeQA`I$U|0R zi5j^%vc{>^(*6l9?A=YOacSE^J?g1yS=9t0=rxtr@rB}-w3VC9DF+Gu9fBiohjy*G zjGof(h};0Rw8M7L>T5e~AY%V~t>RLL% zQ1i;V)E;G!hA^ibr;&x@7{+}Bv7V^^>^qxHE>(yiB7Bhkq8svG$1F(B;@Ic!dC^FF zvDGX+Vf_Mzn}ZP4e0#g3YAAu($~!Dc=`?lZ{CDa6Lunxp^aYOO#i7_|81`fh2Ee{( z;#nL@7`v}305F;87>(rufi}}e2YPhV8A3H(<4a2{E{(#7r!#cpPZ$O)$c;l~h%TKh zrw52k)79ysiG4VsdES;qh3WM?nRtV8xkSPnd+Zs+b!o&tXV5g^ijeJqwbHlPRHwo> zArHpU;{4YMlzK~~$=y#Uy|I1~m+F>vAy(Qcl0J{Yj%G6rq?Ph+T0k6cF*QrUT9$;} zz6I=49W$Tlq*X{6nf8_d!a0QuQD)+eeDN2DoW|%2HSC)J5$r`r5Ewyh@kJXVxImP!tu_gYUlWp&Kl{%SXPhQ@m*@tH z8{|mV&R%|m3$xx#JA?YfB8bi;nUTx5)dMh3Hg9EjuAncL{i&4UG#jT*{K3s+-0?^T zzaP;quqp5JAZ}pglqL7(p2(v$T;^)p1Ba10vWY)Z8mA@93p$d|9emPnV=2N85A_aouIw;P8K*v;^4Vd6 zdgpxN*o-6`e|Jp&7t>P3SjB8*Bx_LQn_$=-7k7tTF2{|PEaeM#gg5e;hzutRXCWBF z4Xu_|Z#dl=Zxuc{9V!w!-9H%I78`w@|}<3>WqcOB&G7HP%x(}wngV>RUp1qe@l8K z@i^uTwunB}FL52OVa~$GpHQ6aJd(j`(jVtFoY7{IPqRM zTEcpvxWXbdpZzKSnE=`X?`sy(gYykBXlCE zZ|icOkQ-s&x%XUV{WGJshQGgm$7N_^&&fVlk4F*q)IQ?cUximV#|dW;R+~8t7htRA z+n>!7&_ADHLHh3Y$Q&%-T;Cy#T|65Oe)^W($3Rs`u9`23UJtUQ{fV@A_wYlHgQ}~8 z_i8|AFRl=7*>>`K&$9*^q+$;sjnI>dmjFiIXMfb{%Cg(S%>7}>|m=aU%f7= zmE?PZm|u;J9hIr8L4iT}R6X~lKgZ25w|!)*gk#ixr>{F{e37D}r*V$#+#K+Z6E2=T zxY@sgKj3z6!K_!D9Xspr|8qa~GY5UvD*>(fUO%&!QM8GswdVMDr~^-s^f!sjq*O+u ztjwJ3!UJm(r0-8<%%`V}Mr5X+%oftU+iBS0SDn+Dbia*Q%zmmxd4#`W!e7}PzYz1a z@$&Zdm9Tqb-Fhw>ztj9d{P(j6E3YRc_$R1~v_(L4UmKvd2DSZecjEf7w_`G1ZBuoT9ta#G_AUmpGz68yT3v*l|c{)Nm7YB|6y(#;x<&riGKge`u; zt2&eWzIsi+YMcv_4Q~R&K0^Xe={c~BmNcbP^ACFT^sKisAExI1=nx>Rybp#DK9*${ zqeIg)>feiYYU)^5YaX<|!HjWkK<_pZ=WyZppc)Woy$#1Y5!ic$YSQx-1%DYn7x_D7XeS1a(BC3gmVOHW)r^(ks- zz^Ce>WF3mw$@6({Id`Gu)BA1o0DCi+gg3&jfk6SH-?2BvSzt>{8n2)wI664~lQ}3r zw^l3rPlf%AfkWBVa-jKE?&gn3KVep~Tepo|~J0)ZG5Jx&e2Tv@j4p+;hwwy?nPuDal^ zSZx30SjqB}cro>c$`dv`SKY4i?%r+fAGXr5g^kL8E@%OlF3A+r%7h!D+I)+u27Urm zG{#Bn`5SU(g5T&@k22~NQ&rA((l+N~lx+22^g^QTeVc>3qt&Gy1PIL-A^0mr*D@=a zkf5m8*tShOH|>K~O|RcOSK0iu!m7xN`V0N6W#p@4y=8i*sDQa=UXFm0=X<_u%QbRS z_*5f1W>-+iX{*+VFV8$D>9l47%Sei_|CMfYxM9@8$-Z2a)?8U-*%S>}k0-Vz`pI!7MO zPwJ98k#o!+d37UXztYB8LPL0cSr{BlyVQ7Ya}YqxkD=S^1dQ1fcs9B}HWixSgMQ8XmvT`xVoCZyPboGgy%m zO}L28?#a|wNP2DRV<6%mc;ek|$REF3J|n}v;{T^!N+Jx7dHpg;7nU7|9uIhjueMt(i+fo@hPT;8Xbtn>5ej)iO1;c95CA__g9q%Sc2S ziN-&wLL@`ufrP3U-yDVtL?%?@Poz)M5v{SZmdRx#z4-UPMa#zr6-@)4C_;Ap&98;t zxJ$S5;=PU%=tDZ=)d~Rz49BPV*TAd^G1YksL8t71%N85^mbGi`H5Qib3wC3F&pOA( zXi@*bpo=cJb)>@2;{|!h#jm5Z#wo>?B$TtU;}#`&RgsazG0%WT#X$>pw2urlW+d3- zDE4|_{_)fi>B1B4qGML6=M z4u8r_^icf!5}|4$`!A^>;S&GQ(n5{0RK@e~7_gx?K(sNm zXr~-h6&rkv{_M!Wnq%SY>h3rP!KIn22WOIr%JT|BB#tHX0-9rJ#fLnzi(>XOLIi^fn(ZRy z9jT^D?6;?+lEs&nu5?Q07h;1AZ{FCs1;HRZAF7(;nf4liT}cS){gq=XRD%0Fqpuri zS}7zHiQ$)&;OD@t)=`qHPh6qT7k7~WOt!k8^qY9iW4d1g9V95JKcuo4 z^i0z&$gNr{D(;~?<&y(PqVx+qs!8|OI;1ayXr2|BH&f6YE`M`svJiaqL4#kFl~B#b z%2}LUXhbQKwPHrlwR?9tJunlY^{QpBfgPm<;P>g{OKJ?~;DlzvE@p(+$%LAhGVTr< z%(p_nK8MO#D$T!4(q8tKy zG+f0Fr}OH1aDw5_{0+^sfuwj}Up7l6wc0*t&v}Il%@Gx|AJFR;ALHM*|{q+LfH)h%9#@QSl+B} zpXh-NG_Jgr++vo>NQwUUjf`aMe+bbcE zhTD=Maad@*Gb)rmU+GzYF+56I5raviE(&rKbKh^Khx;d2I@#tp=I)TKr#+%K1pmp&?5+J8p;u$w3=PQ5)C^&4R{PH z)eT5^ZVi&U&?Ipz;T9VYinak{A@5ZXAYK%yy(@hF>j|PO0f)i!#1NJ#^349^jny2u_i2>i18Q^g^ST<` zTy(G@rGph_Cu5GQoGM|Z6|78g-~CWYD7*QqWU7-nOnGd4$o+PMBCxaaril>^3Y5sj zdcfwjpOl#Ut&pj!=pC61G{HPD2#u@Y{Ve{5ov*~EQhZ?p=S#R3vb|ofCFdNX`2B6~ zg6Vw!0sjyBVdErHzS8aysVO;byKl7)?+ISlk_e%#r5&k~NsNz$mrNKm_?Wt^GqFd? zzZCDOrT&_U4&F0quoy3y5?yc1B)BAgZ&hib#)V#6!nVKf$#k)Nh%_I1=dKmJ+!~Pz z#0UQ8dG+`)#iq>#H`822$lpo`4aSBwm zFONnX$YV3th3pX#^!Cw2>n7)sz35g34Qt#R@XVfCZQe%{PBQKnNNV~eOpR$giwL2y z6HA!TB9xkz!J%PPHBAdF#$jH0pkgxNf3C}4LzJ0&(+JA72?}9v0w*D+f(gfs3VLMF z#Et2hQ4mI%9``?$>&gII_$yXHeATm+Mpki$4kmA?Mx<_+`&KZV=Me3!*YBC5M5#X{ zA$!{dGs0KoJ55Ihl=H|)+`aeYjyV7JV~Cp`CGtJ_zcrkp4Zmu9=4E~#oZtCzNP_Qe{=#`vkzyzMk?1YeYfXU8c%+aWRJg1 zRPVEjv?L?qsecfmT2df0KCaq!COL{$EkXVkR_Z%1F{!(XN%hl5bQ%e8x}%@KINdI0 z#0&STZCzXl@bE3i{(Tlw72Aed6~dq@maUo+A}{yi1ufu~H{enz_l>J+mLefS39DyA zMThpD+h(_4-HAB~?@wYS3u%At$v8;vDf<0&MZZLS$?L|xr9=>#yxLF?`Wr;k81P#y z&^WHcW-0PZ_XGDi@&S-Oc6YPt1H4#rzn!I3!+3975id3WQd{1u*}zBOqh1s0RN?cm z(xswBv5=ZnmIdTWQ- zBq**{c~VLrqyJ`Usn*u;_2S5>41kNF5Gi4V9Rjwt88;lwOKR&DxcjuCr@TGBGi^8bQ6U)Y;9 zS~h*3FQQc!d;ZJz9?}2VW)kLN z82^CAXfmUH98MyqBH3ld(=5NFEM(J zIhzM{zx4K*?xUeaj?4_Bl>xNt*n=89UdQ_pk~ee&dsi7l*nemMP1|(4eKb5Q%CXEj zQM9>UlowG+z#jVZv~l|Rmx_(sEZ{r4@vC(iZaB=YWPTW9bzoiL7&lB)-bQ51v;8(% zpODT)7Aw%&p?~nk(ir6{W{!PZ`?Yi}x|5JI`M%h2V1RZ8_$I$n?-S4)37*AqL^tf< zNM>wh$jGdlY}4^VtxB7p`*bx_t>oqVobVl3hl0cyn7)EL1K-SiAF+1wqv5@@r+-t; z$Q4(c#` zN4w<<=Wo+{5K^HjN{Gt;`yz{3Ep-KM;W>MD5xOQcNq!P50<+KmnsqRryo8|x9A@s2W0sftQ*Vq1SScv76X{+R4aKaDCECO`uE9Qh$;sB#7W z{is6#Q*Gpu=fG1?hy8H!xVB|TZ@}ry$f~tCCF6;_q&8~v^L2RX854ko%6UD*@Rq60 zdB~}jYD;SF3W1TDA0771IRi=HuSUFVur^ID$kn7^4ZTEL$hVey?^tA=IIiNnVr|br z;|d=^EWGP_6PX*bB~8k3cPcwIh=$Lq+}CTX*8D?R*=Ov%`NLJtUqv5ue2?NX`vX(F z<%(M#TSIPv#Tqs?HE7>a*$xX8TrG;<6QaJgtOX-LBr5oZ|9g)ve;A1%-*rX1FyT5s{ zq}F?!O@37PAfjwzQ&Equ(Cia4mcvaEfAp@Hh_pY7&y9)p)Msl4|KbXZgyRh1>e00j zB-_Z&lO^jTyv_9;EU0{=;WT*{Zjk)N1O}vfe=oHmc2UgmWXO~)N+DPJlXlCfGwt1i zREH0V&mM|LWtQM)b2=%C`}d;quKt6)^;#<_nf^n}bm*=hnB4IYh7Y@yyM9=@y+iP> zt+!kit#lalKq;M*4jpwnUJ@wbX+Cap6KosZ=T4n~nRLQlV*}ITyYuaQ!j`&X^;Lff z2-+KG-|J<=`>5lhVvWO|TyjVBo}&_@*zXhKFRXy2ygUip)g4Zfho=F2z2flRqkGf7 zaJQMpnTX_a1sR1De$-0RV(k2+hfk>8gi@9zH&zaTegqy0@7If>IriP1$6+kwCd;Z5EYrmQFv4nF>BySB%?)BNCN#WD09lqO-+xYl zQpi+xEJf432<8fjh{G7m*#J}UCqc@JpOzXKJ@C6jC<7FGJ2REqed|gHF8X1Fxz{BG z)%yEo@lV+#g@v0g+M;PpIzsD~y{ppJec|vtyO|YIUFABv@>3v317THy!ki+jm8^I6 z9)t<|%J3}So)6~LASRapV)XpHs~~LJB5I7AM_0Uo@$`iuCIn12z*50?8VFyr%=o(d zm-^rGD6^9te>B5`7tglMoV$2i$;f3r`&_yqLpX(?W+yacIN7eM-G+m3V3XA$xp zFtM4H4RRdhuMR$qXgNF;8}N9oljQH)Pv^byI62p9$e2rrh%Jah+h*{b5YU{TVWrIX zzdN7g7k^e!)OIZJEYGz1JCO3b8c+UmA(-9zs^th!{*xPU5$=Mg%W&|FVV5e7uUsGN z*t5yT47fz?9Q_{Hp6gbOo?Djb-CgtM=J4!D%J~lx0G0Ki&_H~YXD4F3O?fUE`G`4O z0gPR~^dY%yvJ~;i=phoeVyLvF_?_keJs*$oC!bGz13Oomn7J*C1-=%=+W9B%5erWb z3~M%^CV`>e**zAh$(CE(&P-pGs2{FExopbi*VTvYaF55+M%6Yc6riiRt?0X}^MMn1 zWg_)zTIm?H-czsP05;;c$!-!;;Ce|C;;aRpXa8&^(-HOn0Bl36@%LetV+Jx_$4egG zSydOwm%VdkQff&ybt?$B*=)&_}0>mNLC~7XbmpHSc5E?y#{{qYT1#knoIFvmj`>QjBQ}z%mY29C@YZOx){3`)V*18AZ0_f zE4G=LYx}`%iHooa5oL&X0l2Ys-_PuldcL5{f#{Mk;RG)#nwHi$jyyk-)nAeZwN#9t8K42*Y%A2QDhP5oL&L7|m%e{QMGT96h^&vS}GbC-=7a>77Q$-x*x|FlqWO3_k@S{AeIhS_+n|aX;bC#s&+B^- zb}UYS|JfCQ)K-@rw{eQfI@7M=Ohoq250-G7gZiffG$=tO8NH_W=m3Q8 z%6mzqp*;>aWBRLoSZ0+Sakq!g1lJdL_X8G9uc`^8Y}GmCKa~5!vlx60gVIlHZAdbP z-;o{mq1EL*{_rGbP27re?Xe2svqgi*4&ecgIgc`Z0&26k%Sw>`1wqWt@17rXo`4LQ z;0I1kUVMc3w^j7iR#~R$%a7G}hj5EKV&LWa1@No6cZEsou1y*x6p~uaXk>m$z~zZ% zp0)p$3V}t5l+O^5ENTwz_Wg7P(!y}#pDkKGNK0d62<~Eg*1gdAS4xY0NH8UkFu=knZNg=M{Tfv*!Gy08( z^^<0M>9n?&l;=kYwOO&qHz-?S>H9V}yafyE#Npx&@c}NtAF4Oxie7VWJ$3o2FsR89 zqkxg5?eh3(L*CtmG^5e_hH2P*GB!y~oCB>U%3L@ycep5X(Dz(J*36*?1i(^>A-Qs@ zkJ}<;t_TwV3~cn!zs8AGwvGw(m6;aqc)e)#Go_w#RsV*ZBkf)r=S5Z`UG(6fARd6h`)S&&Phf%rWjx{3j!g-US+GX$WA5y8b-*&Ed3zVVwfDhJ~;zW;O z3@DO-)=Lw9G!uM$;;UbvjF+ondZ{Ggt-=n#C=D><6R@$0ACibX`WC=NLxU60*_!0~X} z9&WUrTI2HA@4hS0R^K~Z4}0}o!+r56!^0<`?IFJxSoWf24<(jOF!rKU--sTWYwdBa zIvyV;Oym7>N$L2_!#-K-AsP^f&({Pmn}lRf7BK5V(`7*Z=s1^Sc;|_2Am*V496QaS+Hc0*jly*Cns;mzuyem4VUBV6+5B)0*YH=m0kR^4da1Di}ts!bL(#jPV1RMXBcR=Q-)IBgdQDscHm@OU}kzvh4vi)p3$EvWwUmyldnS zI&Z^9Ypxn?_vMvQqR5Z{W`GL1KrJa1b1VSrG*qp!QLPUQae*zoK*Tfs`*nQhN|Dz&^DJ$wODY%pl~t{UYWB-7RwB zuzPK^V|;|#TlBYR5*_+Z*q34^y*Tc}%<_4G%-KfTa^}fF8-6%pJf}3k_NZ zw}l~Oq+Mn~$E7Cu1(Dx~2nfe}SNF9$cOU2}#d|x#D8cB?tn8ImLfPAknU!k+n8L#f zXh4{H4$HIuua zV&a{L-TuKZ=Vq5DWXQM1N91umQKPW|JHIAXz>(oIdU^JJ^r;ONZGSiB6hnOblKVDh zzxQHF62l$t&Y@5%IMv#9)nfKzN;n`5|I#<0v)Ka~;hIyYeawg)4X5IzX8-FgTQ`55F4o^ZH<f3z@HHp9i1<|c#`|p%X!sg zA~Q!WHN~^D(iijKoDw1Ioa)Mv>FD#K^6t1no+-6V*=vGAi)|@SB(djVJxtDKASFB5 zcNa_MQ}otuIVu6O`m_LE0WPhAnzI^3ORwzY7}^Pk9}%xJM9*(pIEUPqpOA>$X=q98;FDn z<5*dAE7RmFB4H^{=g5@nVxc*KuUB_~6!y+~=FwFdQdIp}>$TvFwC=1!m0?<@A*YEV zd6MBg!@uPR;L3=7<-A|oiJ}*4aoK!vhyg`5dL%bqM@y(l>6s$SxI1=wUrTIkSJ%1& zXg>@npoK*UeZEapZ1ENiQ+VStl)_ol)?zES^PbESK$R7a6apk!E!9KL{Gd20)rw3F z(346d(Rq8gF>AZ7R{RG~*=K5DwdZJ-p7QdIBb2JNXKb3{D*^>yvwS{wPc!@1W@Wbi zs5oC(QSZ{`H-53gK_T++L@buzmknLSb^=Qd`Ty(b^&9v zS5eUkhS~kM6^@$NOn9zXqb*x-CmWsw$fmMqeJ|RK+}jITt~0q4Vfs|V^RZMbZ203ZC5sWc8Y)vTmExX8 zj^m2)XWGMG+piY~yh`isD#jvD(W&QfC9NaWEK~*sUAKS)%pOL!S;BS{^|p(+B(|Ox zH+gUF-Oq_X8`MO+W%^iT`{&2RD8AUQuu-V3?4LZQKTOrZf!e^%lB6*;z$PIP{yRWA zft$AGHq$p5fv>&1l* z3zTv&nPQro*3*KN1gr6qb{?x0g;9&KVEgI|jry**(WCz3L^_8h7v`azfO{W~q}QpR zwDn9~^K^xqzHft_pTaJ`V$kC~VZg=Rjj8{M|7FS@hrNn)R{-~pL6U@{qRiTjhErzSn^9UEODpr8@G! z&rLN-G6+^^%wWS4JsMt}naf3Ax5R5mEgQ-IL{lLRJG>skRcQ5!oCX~B zv4^X=>0&?78@m=O6SC=s6#L&M4vM>(#}H92gc~!f*7%o>W=uR#_S)7lKS2-9g}^6Dns8(D;vk1pzg$>=%vcL@H{?CYB;Hsa}w7Re(k?a0G%r7G&M$h_jN;Bk|`r> zzwLaO=qvTzHfcqbCHzoB-emqf9YDy?ck|MN-O`bR-?v^<>`;a- zO5N694@iw+-W(aVztAX&7Arq1JkBTPaF`QcS~#k_t@L5~-A#Csa{Fm+zQOS-%0`3| zo18UA)a|7?ha{Ee37^4x_4$v+6G94pS_+!&`RQLpiu#{JZzkx-PjW>p16hAwMrX(L zMO)7Nr6<=>w7Ts=A!7-NbG*h4zoR?KDVgt#D{iVEDV^9<)jZS4*E(6Ze$#L~(Y*Cu z!93r~?NF-4{}}`F(Y%|!h6y6~$!*z>-4De<`fg36+gu%W)Z>Dni9I=;zusnw4y{~m zAvx}Hs~I_gs94GD-M1H_%e`x^fS#WCPfa0jFfwZQV_x!nG3_)T285>`auD2TCovAs`=?=}-_Zo2}| z|9o6^Af6mDj~?#fHmq#etyKvG>N2lK>Mz~$Si%C zd~JU+QSN*8+O7y84RpcHab2FB*4%xg_7+W~H`A$5BNdR=?yviSqU^?*=RuP~;0|i% zcn$+VEZOedn#Q-rg!KM3&z+uwXy;&kq1sQ62H)rrS+d`n+sS9}+3`xCeQw?2Up`*9 z8rKYWO&Q5P%3iM#BIa|`>3$vSP{KG&7evhag=%jM%%{Ahza@VCp4<01or1+Xp$N-h z(IO_veuS_$v+Pzj5$61srC+K%rH<4`TbY6Z@W&G|&}=rr!UmOcs$(sZqV3G9u>q}J zj4xzHnVbddp<8y7H1+HIcC3?U8G?eGyr#QV^fJX5E=@OsPjx;s9AZPRT;EtI zWdh_V3+MfpgqJj=2A78yEbYM&RT%5Hx$KLu7Mn0-3je%2h@E^>a zQgQaY$S!JpaWq!l)Ov)#@;I;9nv#I`^2qw9maa&T>F20$w_}$vy=B?+wT$>=KXGo1 zGH&DiN8Mhlm30}1ss`T%mq7uL+JdJa#8Qb!o>sb-p+L_;ot=Y0$J#l+59#|#v$8xr z)>PRoO`YFG23NqtwX&)30)fAcys`0G2B%rSPF)|tH@CHc`?)x+L|%P8c*3dQ2M1QI z_4Gi^f4OmpHV`jV$^08I{|RI3%H%A<2b;z)A{Xwl=zS3KKWg_@r@4{ z1>>@h{8OU)uYQyyLI*-?{OBMrJdf-3O~}q#p%CPBHv+ z%V!8(tlO!L7Zq=(;PRVk#M$+FxXq#Q^~_bhPc?qV|_6buTD~oa<9ewHyV|?=?h8GzL7tmg~dA>BYeHNs=uRieT9<3 zjz7|5iZsHkez#EQrITY;C89fU){TNZzxyEc!!;%kMe4c?#U7N*owU4*MWuJP-TzSu zYzEQQ;1+OLyGbgd{pJwl;1>>tAz>E2bJNy=T`Ke(dtcl$I2odBk6u8hYLtm>AHMi7 zZOvlz#o&g|{Q?^DW_|ZsZ3_~qE_9G=O%Uf6SIwg3S-9Q27^uv}!|EH9*J2Pi9O+eC z_3UA$24Q)p^}RHmkwC!$%FV7BN53}aKk76qk&+sp(5kR)BH_NY)#tpaqv<-}27%L$ zz~@Qc4iTDjrwW)-o|{;Ri1`2-l$|sj7KDc7Qni)5k42&B~0~y z(Jd!#r}xHJd_F@KQ~aMGb#-+vESGf7{-}%F6&+qDD;Yyc^!m?NLD!;6|MOn}iJ15D zx8}Vw>9TsA>Z17`!abf17QgI4p$*)gcO>d3F((l<&ZmO)4%aN-Oha}i1}KYb!;aUt2(xA zc2Kc->ivJ?8)uxmx~hx)?Ded@=bFFyobcFehLlTPli3ofgd$+kONFgsOmFFPe4y`0n?Lx`SEzd_TyiN@Iru_#%G$4NX#g9!*dY{pw)0v&% z>zFJR%=N60ha25@tr7VjJa9>X8m?cGL)TRHFQ5G4Zh8O2m1ch!_pGVVw>$v$$O51n z8N0dr2X>x6naaqsE#QQozaaLqSg|aBDazaOCz|>@SfSrDXtxCz z=Tdw9{p$*}DdFCBrG)DPkEHCjzJCTjL+D36;eNWv-1?WAc zJ4~8oo@<2jr0xF|A(_B0$LtOEZ&riVW&>%Qu33=uE={i$n*c-Mhuf&0jtfL1zUK0{ zRidre0?a`dJA7U@6NUvIKiRr*adBmbCki=9xHTiV>BMTYaBiS7;4f0m0?l~o#nvsFOBjP>1rr_E=wL7T$8+}! zowCqz&sH};p;{xVf4>s99EixR!kojcj~MLGK}`HIN1yhQ-*DfY7%TJP$X}v?WS&0!1a|; zK-iM*Yew|>IGg@9dc`kBIEN1h>GZw)z5oi3n=&ebgb z6tAsQ=;b9byOBy59;VWz6_BigvaKVi6ZaWG-bWrkT^rIaucfVJ-XTk@wPbLfdM0b| zCruV-3Ac!CJ|Zmms3KR_^J2|~*d`4xiP64^l#=hatzNC0N2gxVw}lUUc5|yI-}G!j zKBus$k3BvnMj^MnF76ADHJBMd`n-0bQ!a+H)9uWpipvz~e#O@h00M4D3QSNk=vR!B z&n80OC_Zy2Ko_#+sZY=aEWQAH7#rrgKM&inl${w?Bcp=y5`l*M8J8kQWd zT6}%Z>xl_EnF_p5%12;tLF0-3GbJdlo5aGv)DUP54eHZH=y&5k(@Vsb^ z@@Cec=~xg0)ED+Wg^=rHu1mhSg<7L97e_#ck|V!=L!KAF;p^Mg&K+Vo6Gr^F4bf4r zw3zarpPCz*Os4a~=36Ytfnk71S5Q6!L#&a69)9=^SPQ`ZoR4rAu)FQW;OM^L0p7F_ zA0r{17#HErX%Yy}poSD13;-#C;yj%w8FZ=)W`6!PJiTJa3c1Dayc}lBDe}t!@Au0p zE%;YMOUF7YpPyZHB3%#L?N*$zgaSdOvH8;3>UJgb(iV&zkBt==oLIm!CzrGj(cWXi zYtO+~)1oCH2%V2M=KTUEI1utD3|x<(!jt;>RzQapuhLgDVdZ~r_w^ulpBaQR)ILgS zBSQS9W?)AbQ`r4XqnT-%hN$AS=tqTcZ+$=*HfA;wVM(g2aoC@4y4P=i^b|4jw0@~; zp8&;?AT!E_&20i$)K=Q4i%}hCB}kj(2kV={v>-RUtYCCk z^B+1e9i5PzEX_%0v}MQ@0XF{i(5HohVCml$){m5leelSy&lGZ*vViaHt&kD4Lh?l? zOLq4yb9-adod7odYge&$mQ#-pRqLODSMbu!q z?o3yL^>jU=BnpOV+q{GQ$xhcyDna?0-r2(JmuV&SJkTG znOebNY`J!#JU-A9Bc~sGW>tW&q8IE8j$V{WVkD=^w=>d@1w4+!6)1(*f6T7uBN!3dx9reA+Mntr*T*%Y zE)_K73~`jNIKr+^RI*og$ZJaGr;X^`2JWO!T#)a|_rHm;wrNA<@`Ex$61V-IL&8zs zB|NvEtZq=S^HF3q+jvke_Jycd@3XiE@i~EEuTLiyCtY41vqOGV1xvm}&stRS?R69E zy*Wkm&l3~#Ihoby&bh*hi)wIBaF9rJu(PM#kBS}6u9_@@eZ4kq`KuH7ga`Ip_&6A= ze7m%3Q0xqt(=IvfQ=dpoZ#Va zjM5OltVLRu8>2;EqeHN-EdG_zk`G#0tS&k&yRAU%b8mM0JwoaARYAhluJ~peCM95D zP2daRoZsantNDDmBKgniiLZVJw^dRv#{F0H9MFcX-Ym4aQ+n(v$r7Y{ll%UDP=hbGPA+*J)UKJ$uBzsElP_C>*X8M1(( zEES#3bGb4{f(%xRT?1ltxlAmPkBq0T=S8Z=iu;4G2e*2M6Blnl(3@(vf?B`j!A%%_ z$_O7=aX3C@ z{@+LTo#TK)fV}paW+qr<)?_uX(Fx;0jhlq$&@_}zo2wsgObzxjv({bgIi#&r`q|;h zl&L=Tu9CsBvX)*o)~^ep+iRs~R2NO|TyUb-YD=!iuf zu@u+s@BD7(zeRA#vL*K8YPDH;@0vXMtLN?Gr;)wk^pj!fm!EJe zZh!OE(RW?hZL&cOsU93C94m;&4A{d64!TW|lc^$bE(4=E8NIzKLB1=|j@u6#OV53h z&!hqw!|44b8T@DUwov|FyWGRter=I_9d3YM{;KvXq{;lXwtw?}y@fqCVml4U5La_S<Ab@l^-|W>$qv8fMgAmQeR-UlOeXCg**I6HJZfz68Z6t?7H_i&> zW$#J9eBO%V@9Lu;S?V15P7>iSaZ~6(1?2l*P-byo!wRHUvgLa(_09inT_HYqpAWcXqedRtKEa(XPle`Dg2e zzlZ$gxIZF%<-ipEO~`T`oWZHk99rj`nPzC}=1ke#$ig_^fsuS~3fGVA!)iDb4n^t6 zA|B!KS(q?GXM8!hrG_!Ei{9A9Uy#$v@773H6}nM9qqK|(=^y?DS>E#Y7m+qw%5S_Y;<2e~eN@?&GuZSHp-`xgtK~Oxle2UQ^5O{98xd+HcbXbH0^YG{i4E8PgQE;hQPpB#7ZvjXoZi z@bh0+ZkhIxY&jh^if>Y%gb-|vO?>oq6pdcNw9T$DBINAW<|{l$^0xmE8HL9!GCPTtyPpy%z}+(9Jw2vQDe*A zIWYPHhGG2YLOzIVP0qnr85mfb;|Kca?rpVXUUJ=4Yoffh___3wC2Riy#5odwSRF(< zlS|8LBhhn#czyUOCOG_02ons;wgY#vI@rM8d7q{0uOO%HW_A?Ef zm=C7hUO8E8#*%I)Sj*=eHPtR~MOPR6>IW7tZ-4$sDbw!f(CFDB8an!RYr%vn`F1p! z9r2#~LE^q4*w1@ql3r6FNn}G;N+FL6xjM9}XIdp+TLk18irxG`UGobf$Q>?i(eYGS zeka$hGw9t#oL4fr`U3fMOFT-gO~iA^lMF*}8Zu9JCb$`YD!8a+LB;meoL|5L6E zYz0LSe$?FX_3pUT`dAVGcN*OJDi zZp<>QLn3Irs~|3xLuahrJ89eRKE-E-bh?FI6+>yCiHDVV~M^>{cu0T?->PDvtg+T62* zNu(5B)bUFyp)6v;sx;fKM)lycW?@fiO1y9R#wMXF%yxFI``>Yzgw4R!F#Wrx&W=V^ z__)?47Eznz2`3L|aZ(!TkAH)?iQG>tS^iD(u{f2@jKZkhEajOzQn|`l9qATPpQo08 z(RTEYkI1JnlZ7*==I$*M@g%ZLyUoP2Ql41+OSaRQx@&FGPRtCAQ zi~z;CjjVb(d2?59k6v!P-6ct!LEWHamu#>VBu5&%?~z@>iTp}ajy2mE(?$~5Jps>1 zaNk%SLcReikvPX4$wV4ndTDA4(+c3f_~2!>VOQ%JMxDX367yB$TgpF&`@dX}^aSdS zE~9^4vsq*GrCitU%_RwPM>B)}>1N%pZ{gsbT~0 z_4Qv&OiaMs{5)|cixIHl)r@r*Ie+`i`TXbOj&CJc-W=GU02F|!xF{%Eg~rZC3W3-f zsRlBMI;33@t~B@dRSr`ILz~`4eoM#wrORTK>rvI$0z6;54KybLXCeUa#X=K5|7zPk z)NLQUx}J|KhnHIhzhCVPi^lJ25`_iutd+N8N}q%(E*F@6Fjvm|>R+>6&or7`mz1n9 zZ@XnnWfxIU5GR<|hj&hWP!D(eWi_~(Y^FdkpD|0hY%G~}7Wn{(iRlMLp6-m$st>&Z0C=YDT)vU*e|zS8oceJ0s#chVv_ zcj3G*<;qpgBJBIX?r|@uVbmgxB3aoxkO7URv&ORCp#Hgq?-VmANX3uhSrtk+VO0Q} z&#WYx%v{!8X5uYSFnoNcC)Ef^KoJAuKhIQc+J;k%BDG@I|CzSE$YK>51@#osL)YnS zUDTM#en(-}x81cFab@BNFC9D>buKl@4q>ty&y$#89>MZ^_11aV z*}F6|BAQ9ki}8|KxsP+Sd1iWlyPd|+_W`v?R8QuPZDI4gMi0Y`H#IQhl+{aXn8p9X z*sWxUWUrpN0T%B*O!BHWMD?rTax`kWsF_bHGU?`nV%HTr0q0l|I&4<6zv)|LRVFNrei^bI2`G*CaBsy`Wz1(o0nbw?|g;E7MKGtlmZcrm%^cCk2F8F`^2h z^Cbvc(&d_~${{Ssp7<}mr@_KA7gTx^e^eN*@_O@(($9X_;?bzcNgh6@Y5a_?`(1X`i~MZX7cpK-a_9u;;^Z2iVYU%Db42=Q$we zx#d+7pDRzJrI{Cm>dKav^MRpl@h@w7j{IBNK_)>CO&B-X72aq!C-tRl*yV3tZYl8^ zs9f~SIr;OxfwjEu!yWtTZ#U!2I9}x1uv@eW)qL;xHht;P7F0+(uBXLG-D1UhW1Ae z)Y{e#P`ZV}nh`S>_Fzt?6HOgN5B-J-!64Qz%#*)3!h?9$=CuFNAn~ZS^_wF*{$o&r zEnnbedG(n`=W}eZO0`0nvcvrzkALw59swEIT;73fG9RGH7K>!d<=rN7=s0FT(%ZCI z@$&znbZ;OrSrA*jB-HSfsOBRy_@QCYRDlUL1c-dGK=uwqtr@~${8{tI)5Wt$@T6(` zdNjz}>KB)?=kXDLv(K5pgN06-dY!)froMOFDO)w8gldsJJDPrvqxi}#(V9yrSR{}h zlnsLm!U=Y0M<^&I4{e$M#nBEXqjX6Pi6OaWS#ux%*m3zzGc~ckdOO)!(I^xa-DFJc z>#lbkE~O;c;NqlKfk{blCV=M5dlJlMd6(C1+#zl69`pQG@9m#a|D{Efq1k??Gmn{u z5bARC?$f(Ee?72r_T$BIN0REZX-$B1A^425hW^}5kvbRc!;ACkD**ABDH4jfI1R@f z^?$KT$7to)*ds?~JlgHYd9s{Hyn>CeI&icD=Gbnxo^O+X(4C39aOfn#0ON=mT>bXO z^2&uPA!27p*AKUqGN46ujcS*tlM&UG?%AAm)sAG{@rl_EWg?o8<}>o);M;Wp%DW0PJ&#Yk$5ij@ar*JF3OP3dlV2w@1-dP89) z1;77dx525ZC@|oqa^L3X@muBGjP95J-bEvnd}h^XiCKUga&>3->-~@ddZn&@@G_O( zrTI~ez-Mhr&#cJC za=V0IY`XOR^`Wgc;0u^Np>1zu^nu+U>v56Q!G9QBLu@MTdPEMH`35w`q}H|8@3pCWmhtUxrgVLCY=|7(`v^Qu4!!}hZSeRBCXb$PwtpTQdxZ%VyB_StI9n% zQUn*mYh+sd!KDXeiMLx%tf(yhki6AqT|aE+P&26n-@cTf^bD?0p;|GLN+ZSwE$gj? z!MhDvLSKnw&k#jyoo-8hZn_;2&y5}@oLmJS+57B;0*$Sc6PMw$Mmx5CRk5t-zD_6w zLs`B=F?BaA4asDm@Cr>9*6hPR{fd`=C4BI`MXpmWr~8vFHdFYtX5BYa+CZu2M% z;$wC^{o9|H#1%LTE=`)wYp>G|bZ7ps=F|W7{}n!T;W}GcrpnP=hZ+9KHOHi{@dj7o z`QUQr>`9_d?f zRbyxowUxsLgf^=SOe&p9@cRuOTX(_X_|s(9`JiyB1{_yZAm8kby*5j}sTEfbf1+ZG z++m1+PC>9Q2`cpZ4J8z9vu2YM^pB3TDF zk1EIYe-$h;sJ4-~ov!BnS#6QHGOTxag;;B_!ogz36`9oLmrJdb)TF=Qe4bOoJ&z#m zqt54JqX@%Wl^9$ljsR5S2@?=vXKlBm;i)mb5hz@Hd#V7oR%sCED~ zxUyaeRrmknNqvX zZqq$E25DbWnte+O_G`bI)>~q-YYIxUW{V>dmMM}aCH9-sIZLR?%IE(Aw}`|@0+3Pd z533hib;$}u{`bKELkNhjqq=pjDD!uL{2lEH9pG%9-D|mhGR-_(!79J^RohjH_MjWOonFDCAhvON zZ8mfup8hs{W6YQOpmzB0kLa-0eR-T8mSl`s4KX`|5JZH_Z_hABYH3fy zP@Z{=9vp?k23=eHkM>%GC%3y2Y`L|+J!*3GKT9O+&w;1~ibS83v-zmgOg?n?&^^(S za&~rB)cz;_B0xxdF-N${ejQ#p*}H;P^rGkb`v6#x!Xm(wYayx>=705|F0pb2k;BJ& zg1)lL>$6Sex04_j_fhqP1u$dPj{P|a<9BGkG4mVpXtYuKRdeVH{ilR~oCr&y(J?1(mp@ms?7qy=ZLPpI$+s@LR5I zz}hc#1zj=wa-0xkbL14Sh@XY7A~biRomYO#!{2!R=4shcnN1Mar-yC-NL>mqv9Sof z`@E9r+COu%L`N*e4utdr@`j*0>-n{+`G8|NB!s3PG{89_zXMLnm>T5MW|`#uDRUpHqvj)(E(`g_ZS#Wz3q6k$0`x;7QTc`%wDfyWsjwX((G zSzIgoI$H5KM0gwg)u4mUCx*Ap?+h?=lgF1Q#-ay-8>$~t$_XBwkST(@cvtKfITXh= zTxeYnxYo2ISjNkTeK3kBkZY|0_ooA`ADXy(i;USFR6~1w;cvrnDiaY$;YkpRcisVe z5Y?l%xzyi=Yi!$o4;4*(-k9`+Ns#k9B;fD?UEgQ33knweSGQt#yW758`QaGqWPt`I zFi+LpY2CLehu7AxbM5267R~gGIECJxp`o(e?8ljlnpc`?V=e1<|y&pW-d|gbLDK zDyx+u-+Mi~)+#bcpnq|9)2aQQjQxu!AAIBi$hlvCT3qt<6?SAjz&*t2p;iB;prx13Y~{ZKRB=G`=xV_#3c3S;``-D(k-tPDDF2!2a~S@NSZFyxq!p%rs907G{E0N zwwC`LK@&%!4*{-E8Q)6ylL16(IETcYY_Hab-|Zo;Vf4Jfbm(~uC%fol>Cuh`64T6# zk(UjRp^$j;AOZ`z_5Br2^Z|JD-SXHT&}~dPalZ?X4Ty}Tl6560p)nEbX(x9%ArScl z7Pz1+-dewG;2{)PJzROO*;wR}yHt>xh2jo{R(Vs+RL zZGD`b&o@Aq-x4Za+mdkP{~dx?Xa{(YDa~cq5E>@%m@iW}5rv-^{WhsYFJ~Ia$wuq{bpnQd&?!Lqo%Qr4iyDcZq!c0I4iKw?rln%eC}!vNnFh zm+(Nw6uu`fWUTKp$wGI|7whzt=wg03l@ z`qV{W6`HTU9R6G(X4Ld}c5*gV_=u)*ul~lU-Qvi9h(p1tK;d*{qa82N z&5?s@u>76doh#`rYmu`QAzcj~!o}BlQVF79?5Z(TF`J3e44cE$u+w#yHSEnN;1X;cjB@ubslS?(z?`!02H9>Z!~qqTfv42tF1WCB+$+8!L*AykowQ|3QGP z->`>QWTU0XphNh+0*5Yd3nyRPSNdF09zvDYSNk%LW^v-Zfn?QRqx?5ysE3_QCtyA2 zPybPIImWT&UGcFC-Oyt*YQ1E>E+bYf&tP<7e17ZOpFI%1M=0~9IFH-;H#_}yJ#!<` zH+|G?1foOvmm1s>R2%?3v~j2pu%O)4rd*HKyqFb^`$U}rMUBw`cbT24T|xkrX$p=D z67tGt6EDy75n7`kJ$sIW7-p7n=T|KvNpYg2!rUy>u4-ire#^y1Fx~INW3OJOB%@+E z>31KVy_n`BCuNAkWur|8*_VT?pXg8i`}6fB`762Q#9>PpV%Lc7T{=HR_!TnW|oOAA>25{dH8X7X=2^EeYbQTQ; zJ8BasGFzZ8(5Xu74=C{Yd+@cuJNC@<%prPG4Aj|Dx(F2X_}2zuTCRqTA5TKU!T=EN zQx|AIiUAlFK&r~x+7G}6|1!&dwWU&2RD8KDgHbKkM6=orE5Z5U%PyK4L{cOWZ9;vX{jmv(n_zsy)vlOLD$8-AEG`jd(=uGeH?ESsgOK?qQ|#DPn8S~XQ=-Itjj za_T5K+buNQK)Ae7rA%RT+9BYRgj1MIz+vQq-1T~^z8zOKuV-e?iz?tGUY2JjN^qidFo6vlTHt;4p)6Rk?G7u)R&$8csmgS*2~ zZypVfKmDBLl|c@das#^_4e2Cc5q(&ZyBHgN1DSkh2aW)H05v)ECtTV-nXxU###?`P zM6=JlK-N8oufzG8?w(*F74;P$0kW3B_=0M@&`W?p&slzWXR8pJOXXm|tj{wAUrcau zNR_`qU^wH0Sc?FsvQ!|6->!v@rL3ZIV)|tDvj6HewlCFEUV=Hl}Fc-zKC+&)sp6EI@A zCtB0{dd7J?8?0(*vUV}~TEgt$ZWk6I_N#lyt*LWzxh6D_99)NxNUYbS*J6!B**|=} zu3sg~S+QS5NXUk`)lcpUqQr!$!yIb0ffoqYq@y4fNDi@`5wGLyFAb4Qx=)rZ4@wRv zGRx$6oc6lJMx)+((Cc-XX=x8NzZH3PL9tvT>Wel~oNM*A(e(%@Bu{eIV_`pksFq8S zHo>=;XSahg{G4Nb_Ex?<4^h{B*NBu|hlc2i&}~2T&7N$8x_!G>wcblDjq1cp-Z=kt zbQTjBLK4{A?bJH2@-URQCD~x{U}34>CdN&7sGm*e#XjCEB#|8PXgWV2wCy2{1mK~ktJ_nf*|2w0p;AWGd%s#GXq~2ZwxVKM zX}tzWjA{E*{GqlMhl)G(mhBS$69z@cpSMV{mht+aXHs%!E*7&40k;$d&GcWFC%Ibc zWU{F*f^Gorp{)O_WcJ09b_HcMIvFl2>ht8YVwAK9@rLP`?}2+mB~m;A zri}bugpy|bXG%5LWMW22$Fd(zKemvi>?caNvRWPWh7{||SAxC;&o$ZDNHek8e|K_J zf!5NH8daRfQ33~#SPr;!n<6UaQ;j*GrZ1j-mN+Kj{%e63HNz$EU@89>!%bf<@;tXt zX2BaW=0bXl_GGuD?EU`PQ`G!gV9pM3JdyA|E;4|lz}cgAM^gOU)wA9j;1>Nk_`8JA z4)^+AiB5kg(qLr5Nq=KTnO=D<2U03%eDwcZGRYfk(Yrsj% zNnS-c34LI|J4YsEn!p%tnRqk0Ok_|-sRvSAtTS_ZRzev_-r;tbaz0&Cm~P&r%(=k~ zx)7~>p4)e&d%>d9&axlOOL`d`=tM}e*?I`N-XDHNhz4aFQ+p+lv>_@hD{D{W@xSU) zg;l%GwR^y7v--dgt~mYHcD0qM-&?A^9JJD^3GMB%caB!96(C^n5)}W=R;$;oa6xnm z{sF%)1<{YVA=9p1k!coFqw_R7b^mjq*iM&eug*V1(gmlsM5SQdG+Xck z@VipIf=2qH|L^e0a|7|*5__*SS-15jla~8Hy?$k~ZbI>OU4R041PtrwIF`pArYSn3aLB5D{fkDul-Jo)=6%*nQ3v$HBNmKl=>6(0!RzWH%D)nq$j|)As zZ-6I>0T^3L9Tq?MRJvmcG%~r>ogeSE1yL&ZqXO-5w_G?=T4MR~&rwqY#GJAxE#ph` zt@$59B<-lg+iPt1*(f7*iMMSZ1DguTN-iuBmsqzRVR*3>kHv$|$bv48xOSP(Nt)fS z;Tb&X*=b3>l}9?0ClMy;c#r` zQy!0WlsOIyqCnVygtZ<>SID4)!>F7AjL%Pe#Ik8n{<=LAu&1*vbyMxNnHev*be$A@ zZLK|0r`Icf`=Ouykt0S`_R-BNeO6Az!0(` zI?F8HvG!GI$iV_MN6h--u-FCh%tjxMYkc9|nQT!UD^W)RPMfr^+=U`dd?+xY`) z3m%&q9`Djyg8esPVv-$U2gYO``z$nuW54fWET8oBiJZj6@JZ2grO{rPT2%%KSx547 z={%QFzbHr=@3TeHcENC=lvidlYKwe#G%>JKYsf5hP{`Xy0 ztCgUeOu7q78E(C8eSg2hCO=9~RHTz_H4U_ox-iL{RAv#;udbscZROwQ@Mn5(`(GS> zD}42^goU{jobBF!SbHWnhFTN&aiW)`z7#myYdQx|ir1)+DAJ!pf=G^XcukxW`J$}V zZsC}&0YJK?0j*7C7OI|>4(vx^^%5yX9hz^0KcT#nMdh3wTGmg@AP}ycEJmZ+yNPrT&b^5{x467BQk*4qt6>$JuWQan$EAYJ5CP6@cHXr z4`Pm3x#MaKvV$_1C9iw&KhcG`iUM5^C`jryb?(!kaZF)uM7C$|mM36v7t_= z6iOiOYi299K>XE?{BY^BD)Zdyd6RPc_1(EI&UzE~KhSi{y8D(tOVdBwWaJ^Y6R`Cr zdAJFwquTxN;;9d|(I(z&y%0y9i9#QrrrH}%Nsa8{;*qHn%o0|pfk|B!*C{dPM$h+W zd~tuEK9+O=qzM}%a4AI8Q@Nw&6C7zQ$LScNfHWW@Fdu9{#jc;;XX>kYWvFrG90AMo-_@~Ow#NB2)G-RAmuaY+Nj>{N`HBL zo(|5K4IF}aGZplv)&zv$V7MGeJ<)3ErC&1oi=J_S3Px0pi%V8>{Ld?xHn3c!19po_ z}nfIeI=ALJ1P+oGos&+OdtUlx3xJdJQ`{%CD7#XD@bkuxMKmJ6F0Xm^?>_g4gf zoLRVOQVE}-rTyF{vgZ(B$E(SL-hGsR4F@e0o0(Y6=JJ{d_&puICXM`==rejR*XZnN zwmGvuU8`zRmT$Q{ug-np5jtY=tH?{`u#JqYdAZoAE^A&xA^xe_vr?n=uF;pOVuZnc)%ZoOW#2O0~NzCVshF+Wk)j9^8jK-Q3=&G*K@s<@m< zeT2kcZ)mm`ach)?VMp7}XcXqT2Ha&8&h>??jN_QpDgez}weQ#5+>)I>t%WbS5=Cvo z%A36r8*l0b;X8hvM=~})e8a~*pUm6F`r8ug3={AE2^xp6T^OLv24V&&u=H4t>0wFL z(xvG>9R*tgPho>0dbQdR`)(d*J58)^0U>j9u(B0^KF}92xKPYm_$L}{(GmuZe6{;( z8pUjQ*&kEvX3wXfo4aIMW@!y*-=c5TjZud1gv4!fffE9YjgWr`e1A7o#8MIlO|GMi z;9$!KNAr5B|Al=-Rex4%Bw;A=gb3n=hvov-X%qL0ucihBWDnj$EXya-H=OuuQAXiR zDicl5IcvMzbT5!$0arhT>j9OIf4rQnS#X z^m?lic(4kEyvv0LUpf9I=JHM1HYX46G3AxXf4)i6AHU0pM`4#0FuxPGTyl?)!ZCeC z^*W$<=C~_R&&`hZDhtfu^f2f2E$iy>IQFL_Q~zXxXZotWS%r5`v=VLDAEx@?T80&^ zA*&qXqVc2GcB7RFv7bFPr%;|98uupdLQyf<#@ff)#5jg1?E>Gw(UJUVC;D;1U7pwJ zyqF-%VE?ODI9#u8mykGoNO+Ny`z`NXz?1ETiZ2+Q<4yU&P@VREub_gk{D6E1+q5kabdOFW zvBbo8ma7=0)EsKdMVt+A(2xtsmq$=Ia%j`oo}7UEUt~9#Yn#F3$@;x9$_C2DU(32ecUH&4QPg%@VUpF&42s5|IFZaTT1d zj4fq{zslG+qB;tJ{KZy}IN^KM?8b?pva!ff>k~E4NdAv~C?zler)MkDCLdkqj_R+~ zeX#Dn?DW&@XH6Khn(&Wj!;sak(b&**n1SL-xdmv6I3suk)}PQF;tZdoH@?hj9BFURqhC)-Hl zBSjK+-u`>93vG5diE^a3;|}4h?w((&3=U>h7XDcASLnqD>4|-BsB%63_3~AiS9R4i zl?Oa=%F5>)>SkQ$@u7B@5f^}p3UK<`>LqA)-Le#6<^Gr&Yd#od#%`HX zmtjBC=0jbj#Mt>@>&0L8Oz6$;Wj;E1r2icfx%w%me)n6N(s^#*k>SLChRbdTDq)W} z9=)M%pc^GjcPUnht_bMKsTJ5@z&%LW(tO$9=FEr80dV30cjZ8uaR?< z7GfR3AbArA!mLPn8`+Zli`Qhy`I|-FVnLQS7Ogw-FCOUfN(_exKNh;m2m~=PIvN|k z6{BNo*^;e$Tz#)uZprIu*og}M-pW+?ExHBpN4FnR zYb(Y8W6JZh>avyv3%FLd zQFgeKdtLIw>bS>TLRls$(rErmc^7J|sTv4d;6rz%Mg{4m@HD_~UFl4{=A{2U0{W)$ zmsly+IXUnkm4T`1?sPGFERBv?w29X4;0{ToGIL73^L(a2)M}~KrG@#7zxTnG9iBEv z^f62cJfHm6A5z_0#vJg$rlK>LSEF#$+=ZO4uGvFsHkL}7Afe0aO%vm8|D3K5hbRB@ z<0DqPdGz@*kJj%mYo)|?=GPh^yyFaZD0;|-C%{1Rwa26V4o4Pr_BBrf>u|zc|07S! zU?laq`?1gH?g%!2{F#u4c1f-+r(4GoLAKUVd^adqWZqDQ9xqo zkP@UzT3Wh636UPUJ0+z%q`SMjW9X3X?gr_SZtmgtzxTcG$N4blth4vpE1&gj*>;Nn zbfz#I3=Behb{rsI^;wyu+vY}?BVmS3{=HDwah^W9@z}>4Sv*qDeU2wmln!J@UU{w||3d(6U5jS{|LpNgYo|CP)yo~?t z&pDHiK6{uKxCI+MjuBEEENAXblU%9{x-GcFs)BHzL8dK1pEyzwb%gIY{n*XlulbQT z(1`SQ!5K~Qu1`Oyy~gI@VRD>zT^#9S;*O+H;(3T*7>R0NtA4l*QDdCu<$B5}zWWm- zw*z5k4uO5jnUCDErlhe**)wGzX(1=INnJAUzsi_t^f_8`%jozXW^>Ain}pJa zOEKx5w`i|okMB>1x6sel^XVr|>PwBCwUzxbhEF4>L^0Km3!HkXycp)&4bNTH++7f( z-(L-emvZ$K)auvSNPaEISqN@$yAG(66aE28}X;X|Wn2tSuvZvl+zQ~T_ zHtZ4t`Ep^C;P~wWTukr9-_!IKvkBy9*1i&@m71Tt%H^BitjT+pPQ7lYq$?wptq0G< z?GqJN>cNL=m9>1Y&xs${5ByYF;|C95Y2$<&90yxJAe*i>JJxLv7;81ww!tfCD&U)E zfVR~7D>*rA*8STbciqwfVsob@E<*Ma8hzLCi32&LBgb6h7l*wYq6hb(`K!@7<~%|3 zBeZf53C7MfJ=Y;sQ^A!7C8bP!dwsvfY17TF=;+Ob`7zr>tF`t*4Ij(eU-we829u$m z3sjh#8_8i_6ZlJuK#ZZk zAO=r?AXY1zWF%}i+{M8S!JJsxPPPXi$c?(-sJApp#V1Ne93O4p|W`oH2U>K6l zw*%Z2f|z|e*qu~4o!FP=6vwLLa9L?mr90i+Bz>26KY2WDe^Zi#2aphAeeYaX{a4=; zSew|77UP*+Rbi&?%dsr%wX~PQ^Qj*{QNLKJ;%|dZ6xebh<;%TAvW&0`uB~X-Fo>b^ z*Uhq1z=U%LLlzeeS|Y|K6?_GnR356uW)*_Ud7WGq;5aR=m;9s}DPhIGK*^CRg z-qvy%*}i_;Ue6&|n>D@i$E@LT8qegmSLqW`25}5rl^9tl{jQ4Y56HW&|20@P$5|}I z`P!)pX?9jNr&i!1yvWHdO_mrX7y@BswmypE%8>t&n@AM&66p=aD|~K2pGq^kWYvSY zMYG@fvs7cF1>SKe?_~3X?+!E@PB(G$8an!HYMNMo&Aod&;pf&9;!k+bX(~3l>@dLI zA=;4xcWJ0z{Rb&!wTQP{&-1TP1YcwfJO8Y+#+k03+Eu3reg&S zJxA4xWRy4&g)36Nv6+1YdeCZ{t>WU5qrbLV6Z~akL z>kY?=eZK6djchtcCGU3ijU9hqFn;f^-@SE%$G}ssdzF<_ zxu1iBaXN7cC)_>Er&22mVzn*w0)Ig;AcLeZ{sqTNgmjMfb$dvO{^^@6F%#wylH5$> z)Ivtw(m?ZX-H7i9d01Ou31Z0UNW$0mHOE3HSPX=9#Oztx3}RxpnwUwf2vzpGmgX7; zH{ay9{aflpP||46ybq@|6dburS#!C5centdcL{+MI9sn`dhY$c>fbuB`$l~2nem(+ z)?M{$P2u2=+qdJA+me%b;q$s>&fAXyO%Ebg2Zt1Q<lyG8M4;X_w^~C<&`V z{3ghS5g0Chee-ctRXo?XyPttiHhm}kyp-TDT9d#h(IMQnB#rk0e2v2&EsM(H5M9?Z znP(gQs`<|0K>JHaMd_F`s$sip`9t??4!)H(=QwO=g@-Oo<|(9Z* z6Xd#il&z$z!I8-1FJ7?{OX~cykIa_cl-^H!QYBhe0VyG)&;!BBC&RN-iUclP7RHZ%yWR7|`o8vccbZ9FC-!8t!^mYnd_Jj?>?ySQlkW%6`Ep=zes@`kY%KpCTeOIR=Lb ze>~IPyNCtuSy<(x0N+?hk)#-ElL5U~Ax%w6Z(D3cbnfraJ}0OOjXyR;X3hfE>Kwki z99^2Ag5Ydf7cJ9pfh*QTs6Qzg^7=)(At`RBQ|tNK&57#~xGW!?}%P zEKt-n=W;-#Qt$%1NKd|clcT5#O-jT_9Cld?sK~;;2t6I*Ob%9ChzhP}t_}rS4Ml!h z)dQM=$A({bvznrzZ*jU{k-G|Dqt-p71%|LCRcTEIxHw^&TJ&;W<%AsE(7k?hGNn^6 z^T14ox44H-O--?ilib;eHAhM%(u-^P@=P6e%uk;u!XMQ`9a#rd68L!+3pDjzMK2G- zU+q&e-0t3ws9*}L3^y_)&}K!F@Cv*1et&HgNTOtH;SSKZXFncJ{A@M&;`8-@80ayk z|LIM$AsB39ULGkH0G+-VWe#-E7hGi*+uma(?|F>61roCja$wUk^@A430NifVkAK}R z`!5F9FfhkY7Bwj36)25TZ4onWQ&`DOma~oT40qX9-gTYaOfiV0X~{0ueuX_+=kZ%Y zdk({BydCi|+LMgc`pzQ>)`a=%Z zB7O}VBwpJvg5a?5J}F?>_LIOVe7QCxBQT68>LifEVA-+Z!m`&fjb^D>1EITL)8~wl zp?QqqZ|@Vrx)K#xM79B3SK61oKn64e_%MQ@rYVW<63CRd!eREKw)#^1e+d(`D8qz6 zEOqrBr<5Vw4dP8!bB(OlwFH>)$vY{br1K^Kgi7f8q5_*Zy(|QGzzdCaNu4qrN_dqD z#C(zZr-rcFAmS;I3ZEALmMJ-Aq3 zS)J+qgIGzW4zuOskfp1K*nT+>A}P;@{gSM5%g_kfgTR5gMx$A$!zE^+#|Etoicmb5 zi*;<0Mg?nDu?4rx*7QxmVmY&!S8Xw!ss`{B&AdX=$A$OgQwtK4$X$yzrgdWLok-ck z8@PcZ4gr}|rByd<_BSs8I3hbXady=`P{s`Cz72g=*Nr&k_$lD>;DnK|Aa^hjn8~UO zr-$Zhq1MNZR;7MBZnA2q=>)7(Acs3l-;@4@X5}JuUs+($bO=F;Eg^tC(#3*N9xlFJ z$Ddnt`qS<%A*GYQZ({Qoq@;bXCK7y4uFAuD=)F&_7*TMHV~v=^5RbwK39?s&t3Z07 z=EYde!lRz}4MbxSb*lwVMw6}pi)=xPz-%$;7IgBb)klOMPqV44MrFTc!Sw{7!>4$A zLv~n7rp{_CZs;C7OdO0$70f*IUPtk`Fo1p~^60Lec`IiN9}%;zH)cnvg;{ZYwhz|z ziNhY+<9f0YD>dy(p~a@>r0r986IqYOaKotCVmA)AB zV=Q;4EwuFUV$_uh$XBCOO~>s1tsqTzv`Js9M(5Gx+p-(fby2rRe`(ka0!}(P5@p&C zQNH04Hsgk!l7H+F)Hl#Yr%|@Qts11zcNm7D0eQ`WR`H6woI-)8I|f-eW$opxBK8yk zKqX9W)~`hYWXD>G6m5czOXrmd!f)6)7no!Bw-3sUvam58--+al)`M;IjJ z`cq5IS7M+_wZHWnvn&BU;=k9m=QQ;O3*ml|$Vq9Nu-uz;whqH$7F~_IosP?IY)siNy1&;OupFB~&2I^q1anHx1wz8ZxmRWhfbAQdms^G0zr zI}DAyB==vuOtCKm<)`yTX)BhvB=0j0GH7W~O9=%kM8rdkzz(-oYF1LtxW(_d0-oS5 z6{#YTj2j~vH2D!6^BqIE5OaRbAKzF`3LT*_9suFAVq}&z`GL6W#tX9iBzc1w9065L z3MwI+Qc3(ByN~wA3#@P#m%J60$k6cKF&B*H`E|E=-ETA4k~f1K2(iagSLXV$)78Zu z+qXdz)*&k$uR`2AHu0F9Au1d=s>fNxyO$j6=n1%*9b?<9W-G6&^>%XNKpPn(=KRMd zcYmbobp=Prv|4pL@3VKOxUf~R_GZU=e3KPmd+~%T#z70vFy|Tpx$R9bx9{@6PtIMu z?0eVES9jX2mRP)wrJVZ=&yBl45A@~NY-&Zipa>&^XH>Cto|WZ_Mal}Sxr#%6 zjl?N~O0e}x{-q(a7#H~!<*Y@Xc9`ldU3`(%d;%-S=NRcPDY|Npq3G>7xk25m?q53Y z?iGJ(21kwI;tCH?==EC7jAUZqN@0ST$SRn2U+3aE>pQGAZ*mE98Gf<+hLYdndTC$@h9x;(#4zclUy z8ua3+PG>=jN_6h)eA4M6uQRE3xpbH4A4jVitf&ZbeeIaF_@<#KU;>K|T}nRD&0D$P zV)HBUJlmH!p+*>mL)cd5?jmZ%@U!g@_2HvesbMl}MmuZe{a1P7RFr19B+=YAW!u0;{iPGvJ5-CP;7K7rh)Wc^nt7XjFU)XvLT!| zk8p75?@_Vno>QPhzmbesR+O!3kxp+Nx4dfNd6PIS96lT8(}S;rBm~Dz{RQ>YwwhdX zZdX$EQ>c*dqXaBC12Of&>WwqSgsY z@`rj#YW2Uqv^@4ob+I!CNq-MEC}mu5FZi(jqZZ}R`=f<4*(QL;C12WuR2-!!Xg{_H z*Sq_FTU8oN6coJvNFZZ@D>F{=@grY{LcUaP!T0nOOLMn^xllg481wmC^1JC0Mk3d` z{J)nf(`!};MY4$r=3F+mZdKGYG&JvHRQF>Vh%}_>%k6Bomfyn`KL_)Przsj1bW7CC zue$_z0+kn1nL*f6CfI9I60b|>a?MutX0E##X}No3`W!n3*6tY@avS zre&b?%^BCu!(0C)jOB7>cl~vvu(IUvcJ@@Y{AL2JM5Kj7`)#eU#uWD= z<0pf|rwAryO}T8Xexh;NK@12?D)#3oaYnhY4wK1`nL(JLsewfUM)=SYoe3aa1nb7Y zd*$oZn%!TzZ2>RljH-W!bx4(?G(OCca&kIFlQyYnw1x0EW`ra%ShM(s?!g?nAHQk9YF*tVZs~!KKx?5K4`NGy-{-e$+(Gjp8%n(v1)5eTT^po3GqEO z_N?;v6N~=dK(^Q2>sf0~X}2mY=l4L4_;3p#KGjZ39XDTA5!knn1P>-oe|%UZVVLQ|Dq6_;%b-vIL9RbnSS9X)zmBpSp+m}j+*&Vb&8#x z{j3V8Zj>-9z*Af^KN&ERfpq4I;z@wvvLxL}MwJ|V97gT9p_as2F=tMT>1fgDMHKea zg%Xf%Z)qv`-;hncBkeFMx?x+bjzDO+0PA4=O6_*S2k2CCD1;Yd2?@=rfNopa{U2PJ zbYv5DQ#xeLIWk50zpd}OKo?f~6jSNyXO%xLZ)A(aim~d9{zjzfe!gD=aNhgB_7Tme z`Z6!@1|%!0wo32`Mh$y2lnw37nS3?ZqcMbFvc981^b3EpsGJGSjV>CNEoM|0P&YOI z*tB>=V?=Da(S94n@SG|3?->X)!pl_x>W zFYc*gt2}X-!gG5d1hDC3|8a{jUuQk~XpIrPGf`kKZH_bH+kAtZRVbHHLQF8qK&4zV zVvSA*Jva*f2ra2mJq-(6>TFy3POUNT^I$z?^9m=0E&@o2=|oDHTpju;cZYj4Q4Z3ff8DCCKoAcJ zZ_J5MsG0%T5LqopzA^)r(Qjljec^2v^4{lN_4#s(Y&k7LU~OfpX35*2Se@Fd33={K z-D0;l*oC%e^YlZ0D$u~(1$AdJW;O-_xc|Jq@@F;-hYi51BDo! zRk=G1G8&z``t}86k)wh>+kr*Prb9U-x;;L_gb77}yfr!Tc)@-t04uXm`Ku1evzxMA z{!6#3h7K%8BLb$_EnD@66K9N7JRSGynkJsrkPcb&Q^CB$`jV98IJIkf7p1Hl{-uFn z0KRF624iFThZJIwXbD`(jI4+-nb3wDtLWw>QkF@D*(pV&a!eOr^Ri;S55b@=Q#FY8 zALwR6N_ha=sQMdMx?8fr(#0{$O0mLO(D`_hnO|A`ZYcX5Q2r=l%x(#WJd?(EGf__W z5*FhsBknX9!>OA8;HXrl;SxpQu$U{$?290xXMKQt#9f@^!B>}4Z88Xo7#;{_Y^Evte zOrC7p1=dY4Si@n=HSOQXQl!tLJvFRl^1l)6kCkiR@-OEVKHaF$@w2jO8Ge7>v}&_6 z!`md+leQ}rq=;aJ7q;#Hy-k~|O{m?Zq{j8PrfdCNEB$^nlk!mdsIUu%5sWdnfCD^3 z?O)7dg9g2yTX$bQFsWfSu<+va*V*D63PMVuiwSF)`EO)GPi;6jI#wWo%XU9=y51t!dU7SsE(qXru4{r2C~Vh3@IwY$gSIAi~4yzI`Q#1T>AZk%pWOQ+i1m^gwg z#QA^^-8pXdiz~{*@L?$A2ER`S&)Ih5SN~Xpm_GYlH|%9q)S2!I0_01yzK*JyX&_!I z(EygN*&yt76WC)sXrtcKf~0(NZDZCxyI7{EnP*O-`iLJq4#5X>jX+E`>0ebe;@KiC*_ z9Zg)|&!$$NgY+w8`0rs@5(9l(+sl>Lry=&7V>iPAiRNh^8hP$Z!8ZM{YS z?r$67)Q;a8kMg8^xQ4*2!VEA^<_E;We=Xwf*rQ8am8%RPTSCyJ?Kc$}w;LE0qNb*AdgKz80t2#nIbVUV#qxj(3~e#_U{U^hyjq?v-P z3p<_iz`yEoM1m{q3z_iitEF2|Ez-HnrlYJiTfS+ip5+n0YZz zvKGy<{LQD-01L92tBSBy6%}HQQkTTpAMXYvd`s8fK?m$xYt{RWWqc{jg_Yllj#ZNF z>19-cM2$V(I8{rpM*xT+ejdot=Yt_*W1EK>fj5~5VUhT2mzD~g9bJv2D|o)x(Z8)z z9<~^a&*d7344XKW0xaq`!&w9-@$;B*5K%utI?%!cp0TLdq z%xOk^k;h(xQkFOPmRrXM{Q*Hx8#%&~MSS66h@hHn_dW9I+-lAYA4LfDn*i?NW+1L|mb0_o$dWudQ;?d(kzTsehu`A$>&Xl((|ZoJGR91BGW2@0po;}1d0F3 zg8@eegWELl^BfHuJ%jJ}WuAT_g`#BxImR@AHFeM(?+P={`JNd&f4cq{8P%=t#(2uZ z?|GF3G+96wuCcfyFB%ztL{RQMce$?CXl&jyKnPW=asXtNSh~f9`*9BoyVu{3bc#&; z)8s?B9#`QF>~POnf_3j|!VI)vwAMTvk=i&miL@H;3EmD08O(8oZV-Ba^4n5Aib07v zFWd>&xv!Z^$$ZTj#$y@|hP^2+w=RzwJ(hB?l;Et@;If{6ph_)%4Hl3ne3@PKJ1wgq1e9~Wg#iR1TyraORWc#YN*KIO_o z_jq{juNmKi(ZM)T%~}|)u0h}bH=IUN10i2&@=!gzx=cIE7gu*0cm)*STZ^JIL9@k^ zFCuq^`(MB3+WL{{nU4c<;*^gF3mJh!43_D3O6;aGy0^0;tdS+CqYOk1W zY$Cycts@&KM?P7pS|(4iT84w%uI@roiEz^yIE8)2DK(Z)lcczj5hU>W5G zE>**aw#S5gwD*}~()g+ugr2k~@rOe_;)B|4j6jMe=+Tc2vF+i6L6;tf} zPO0$r!_{7mL-$gh@_SdK3+XzOJ%JRyP5x&4BkKevKbnYyyqacJj%qc=j;E}Q0$qdT zmY|l_86j{;Jy(jAcBAr|+jF{VkraoVtE_wOh_#-)O3!>H)8@8v3$@P(UFt&I<6U6D z{0ypn$_e=JZa@38tH~C&dV_61e)EQtJtoX%Hp2cNuD&_%AZy55>zq?+$ach$*D_{lTqQwq zeFjWVAsWrbVRnrpBkvpE`{3X*A!C4v5S&RQY$fr(w37G?m?M`pnj?NKA+M-N=D%h? zqc!??(&D6LZl)x3EPp`?gHgzk8# z2a(|Ji3=rPwH_73V($?ZzeJ0Z{!vrZF~cPKO!xI}zaml!rFlwypC?zQidkvdVoQHQ?O`k&;)toQ6efVjC=wPwG9xkU z#Mo0%FYdpO!YqqaFlx~_G-|}S%sH+!HYw-sp9(L)5avw)(#>!VY|3D4GJhffYM-Qd zq~3@CE2(p_M`A{kZrvjKKOp78c0M3*nU|(9tn%&dzjn239f?RtFxO_fJ9guIiKt#2 z;VV@w=qrCRoAq^MmG}S0PoSOak?1r9m!-H}F(d8&O2@ncE2LYC%&Z~}!RnN2bSqs` zTS0vCCxkT|+4z?Sq4|_TnqYNaYT!?DO4nqDnmyurbMn259OfM|V1vGJ!MHe}GM{)B zJ&#N^WA6Z&3lYUzx(+1fVe;I^Q}12`O;L0g6M)S4OpX1B=U2mDMBZEb9?$WFXIS#( z28~jga#A;Z1t`BOc(haF$**w9JY&1C4-8KpwD+84qTw2kCW=zGe@E#X5XqA%Jsceq zC%kYx%Y!vE%6?Tdc-BZ+`M(q8(_+p{fyIOAsSl7letx_ahg8w9vid6E?r=Q~upHna zzpDAa4AE<1U?;+hnL+H2BCg8C*NY+>uvKB>Iw(0!vRezNz%4{TU)gKaW?u^u9L({= zz!I>_FW_Vhd(rbN@sh(_U@`!H{H*y0%ioljd#ybZAXh8!9(Ba`aez(QS(;`Rex)ob zxB5=W#m${;?7=H1q1t)T-r)UIp;hxuyMbylfLPkNGV9{>?*KL)(u@wD#Rui<5)oi{ zewLUzlMSW`8=AW5=6d6L{0uPbI&xw4dVpEymDm6>EIQTdvawN(AB53JLe^x`ubO86 zsJs@V=LC@^nKbJ@*PqR)A64iT;!6UBYed@6@yrO$GOWZmw;<6uhM^l{VS?^I`}Vj; zanh%WpC$80mIV%4im?8OLimMRcPbc!D?P z4>&-ylFotfH_Kd0;r$8j?$`ay1g{dtYKC94lryxLb`|cVIA?9>9@AjB?L?QRPm5@O zhLu^bjs-)?!@MSclK`MBpJAE-SxS*{+TX&jFmGI#O)^ZSj-Oh&oN-+wxMG+u)m0Vh z3Q?mdJZJ=~TokFv1kYq4VniKFPI%lR;nhFi+9Mk)fX{Y3-5bs|0KNwYqcgB}8dF}W z<9Fp`xzB#HYO$gGf2Kn)N3OuqSb{!PHb4b>)Tp7l+``FN(U+O+EvnM>dUO08Cfx-8 zIes$RhXu8g9W|ej_)q3B#-z>H1he&^N#&a4E`AZ!9)wuzIeEmBeywJyel+nz$Z+C+ zt5PNSw(sj~dg1?D=FUL24&&>QDWqz1LXh-lO}x{UW(yS!DI>rw&9|rL)q|U?f7>KK zsETPWbWr#M6;JgXWb0T=UEn*EF@c4lOIyKzJ5z)(1BN$Zl(<48VEcSMX~Ay}c;o6! zT-C|b88M?M6|l2yr76>KU5=|#iE)I77|8g^>)`)MWb!7_<+R@Fn?$>ykGvmg%OD44 z2D3ll#TtP!be-iXUF>hB91e!`T7^U9`E^OudKBw-#aJ^!W5oZ zt0vkbk3%s7| zR=|yz8<57(4{AVNW=6b3Yn?Y0>re-N5|v;rw4w5u>-}WC11B4Y1WY3`Wed30l%mTf zDd4|vcM}2l&{`3Uy5KKgnM9%i;2a#&cBxB(v9*veL$d z8oN=nqnLTW2K8sQ)xdSx+wd3)(6wML6B2iqoQg$q%;pSG5Z&|H)p z3jx@XeGisTM! zQRXbuR`F9&Qck@PkrnV`SM>g;>>q&tbG+lL7y^Ec`3YHCT4n+lk4*<#>sqk@1TNX# z0Y$IDYcc!e7rqodx#EX9luchgc&2<0$gJipTOoNn-*$iv+PxrwLeB(rcM20jW#u0e z6H|u=mtH~yc8Iu zG@}8DkKqAd=XCiAj0qTvDmWMl$q0r|?ms1wR0BTD9lqX#^8)R6;AtCY1YaPjN-m3; zr9|XPs&55gh{n&QcH_kQtT#FvAYOYD2T#zQmpWQK!rTJWGMv0>`R#8F@rSm0HK#Wu6mUEahlYYjfbJXjG~@S{P$!Vc5Fi zP+gye+*@#8SGEaI&V^#yk^&3KGx4X!M0s2bA_EK+OS*}BJL3Y;A|^gr4qaKw+kAKZ zwGvxu@=~+UJ~<~N`G*CzN%6yV1)~+y;UOY((T$lCt*~}6`oyvV3Np!b*EGH_N2LFk zFRA(-gnJn9ts=mC)N1q;XbVV0WJ=Y(s;4gpW?1wcF~ny4(%lBg!iOYr|^nPEY(jD>o$9=ObxpAGoA%YUjo3}mwa zEcbf{jMaALtAy0n2`0@C$FfLxj0~co+84GTy3|#QCU}SH9wSV$MKrIb_swKb9y+a_v$EadPOA7k&CuukiCG zce5|WE1jA%a5yJIh1AuThPz|vvC>98N*Zm^;EuUd)zHidJ&?tm@1*_A?prxx_MP@8 z`}R2*)SwX=2^!90G4U|eQE;{2>hYgHLPyO>b1CdEt+E3p>i7)G7-7nWA1{1LB}vH2 zpOS-y;vKT>s4f7Ob;&rr7>m0zUuhs}_x2-;N{fq=j9Q5XO1_>UZ3BoSBOKs)RHzz!!`AC4OHaAsjN}# z#hO)k{j)h)Uz-Z#1Cz zVg0gNY3!@&Mg!9^#_t2nEvGp%ER=i(TnffNPfL9- zfj1-u*Q&cU3Pjy#He((0#^Qezg!O@mf7v@e7dBju$`KpQ+(iAaGS$DA?G&j7-;ejd zM4A)Egd#G^O_`Zar6z7tMbl*1xPbbmxVX5^`AnPcO#8cp_OAy@;nCeu^HQUJW2G!1 z&!u*!M70xebB#W9gM;9T=cy3sg+8kll60MhZ)hP@<_16`=x}HkGE0IKt63^KM)Y<> z^D)2R;ED*mjE0?3fB6Er2D)KJU^+@--N|q${XO4(rj&bhGS4c0VD1#-`>~x2gE%oa zt+Ld2&IclzB-JO?A;x{+5A#OF`1Q#*XuUO4N>z;1wny9_ec=S500F0Q2jhDnQC2gm2L5cXKjM43U?jp` z6Dc4NmzW(L4bXsNFNBwMKMC1yZf;hX+!EiPsb-6dNfX(VOD5@s8c!NDvwq26o?qC^ zv7LgKel^dA$_bq?p2LW!&_#`FLYL?i#^{Whs&^}9Egg;5FuS4@c1Hn;`=!?gAY&*q zozMvyi8o7XeC?j5!SZIatrgg~Si)=9t#Ra>uf~lCse&*; zDo~S18#x>MFfm8}BkxFqhVdi#w5F-_4Ifr#v8_#T{a{N6{?U35mdBvM z6LU)s-aFblBxh{ms$3;Ev|^j@C)kNAZ&GnjT2aP@#UMUje@`tlp{8CFiR>m8%mUE; zDdklC4I_X1j51+X@lTw{^%)+u!cnMr*wz92XpHUv<7OQ>1L1FmUA66$>@hoXS*d0?7w$efoFfmW=h88%sbz2dZ0wGwkI+spcJ~^{KxIpIKf8t;C*OPvxdsagvNLb91<>_P+OowY_Zqbx{K~$ z1&Vd5haGBQ2(o}El)Z`ng5_=*7H+yF3(Pp>yqXsWACrX8J7};RdI)~NO%%@F%QpLm zRz49h^-K&+Bryg<)7o|FzLkPM5+PYx_%>YE2_KMH1-a7OLi=Ib(rYO+fAT(_`&TK8 zH?G6E>}zhj$Bpfe>_;Hb#O=`4hz}h~hT+PTEVD?j#}c4q4OY`qYEoPL7^^9rHDtvK zA|3$HkuOH2=)T_Cjx}$$}R4UGP#jrK*)! z6Td`f1V*RzigFN^QJ4m=0=;OwG+>)6I56OwMS_f-&z_nK;OhG# zvkGLI$Y4|xZo@CrBgVk#e$c!Y-RLR_go;(1&Dse>5rg_SoK9<))UNl3)c#-K=Yx_@ zGXHOX{TLb=qp!T%H`GQF3&J2|V?ce4aCFo!Ok&uZ1`x^n;Yt4%f%4r0a+^{yYJ9Zy9mdK{lorCrI81;2FZ=;g5y+5M`}NSsTBD`m zw7|&|dSXr5%Xfq}z&kCnCR9P-S2>4uTdyKJOD15yd8>snmNe#QAhQVgS0T!bl=&l8yqk z5Z?`JHI_^y9NH8G;1-rK;{-Uec3pZ{6i2sTx3lOY4N{&jU8?MtdyCi;Z4{&07rY+4 zC(_NfOzK#BK=a?u9}0JF4@MFs;ug5}di84hGQ@pBxCgPWfOEQRF27pDzGO4=8~Kj^ zk50J=oMFuHBpZRb>&mw~39#pm@~Y;7NUXZPoKIqhL6g*FuZZ5r9@EH6e<`n2 z|7yn@<{ggd?bcmwy0`F~kRV=0d8qlvcQfzY@r_3xmAV)Q3}$k@rrC+=^|ZyWtNCN+;?eM|Gxa3WpQ_D>p<(32 zvwWtuw|aGkYl2rdGhA#H79=BBJNT!Fq>b&Kj(<-I%2R%vfemS(Rq z{#;X3y;P-DktEY)+~YWUhOW zW$~~zYMG72LYr6$a%a2m2FgWb=}t~5#)b8{z7sMcGBIH@$w$0_-{V_TCV@$lsot)Z zULhX)4HC8XvO;E5cdyv5F3)iZUYK72zB|0c)|uTVxnj?LmJintZEYbdK-1SjyDW@m z)U74E)$`gicn007S@|lbmWpkD zzvGCzhfg@n+rj|dmwfV8&4=1oAo1h$Ujb@LB>&N*w){*N#xNrzrw({3H4zl=QqI=i zo>L1Xbke2K`Yj;+EYU`Nk=?RTZeZtx+lJ>Gr1%f3cL%az0I?e+XvT2#p;Iam(sG51 zinKYAF%ZDAi)*l1p+!U-_zDbjeDFGE!>OD2&DgIu+%6mNUhMu2n4|G5qQvg;I2d^~ z7x%4B_hx-DVcPkfq1|259_8RHgGQw=j~;(%|B#C~*sfIRa)cBjMHrsKkp}~Z6eS>C z^@fBu$hG!gzwX?8&W}9bt=Fgv#dk+5+u&hwADNKpxJQZ~8vbd6W#w0tdRe1e#Y%_g z9v3Cx$lscNTC1rLr7laWdfnjx;@$loBUZoH(@|8UA-;6w_PPAg3_twgG|3o;K#)K1 zh9O0S#)&!Ws5|FzW6ns8`TV+LP!ZzlU-2v)HT9T$xZ@D>Y2aX=L~f!BN9V^*MUztB zpG04P9gMyMEoYl|Pb$XkD>!vK_Yb1shn>63@#e{lUd5s#NOFxC*h2&D&Ra_LCK?tG z?~Gr;X35w&AT5J*-2nC@EHo^NptJHTR)-&oWC>zSsusAqrE%+EZ1h*j?n$cIR}*ry z>E9C`Z;VfEzsXpY>$<*u7ehGoZtW(3^F>A`*n!*3ya$NGF7RI&G$6jz0H9?>szu&P z*psV#RqD9~g$(1P#oL~YwwLSm_Od<_Zm%UUvr_a7M{vXVJ^*+l#Zb;sR*qVWWowtU zqGYdvqa_+IhK%nTA~o<7J^}b0m0P{jIz4qq2fY2#OY0VU-{@_l$pfx%#{az>yf@kL z=#W6Ii4zs;>LmW0@^!s8 ze5jRZTk4KUtrWs6`Bc^tRTpPS_kXCLHn*1(h!HK!9bc_4+R0v(ZqI}RLK3hlu)__1 zfh5G4`V>bumMLai#BP8VqMT+--7ee*`Cr-)P>JtL2I{bm&I_ z5_jD*&`0jf(*g{7YIGUu5Z%R+I5(>%%Y03?Jq@!C;k}Kpd^Y=~;bJCEZU4};-Exl_ zYr0Tuv4#jQHn2Hc3gp)*Hqlc3ZotdVj?? zC;Md9?%my0byZbYUv(iP^5x^_RI@87-}9KK+pV+LIIAOBu)&RjvQx2>9?=^PgFZvYdO=@>ckEgF3qvJR8$A-=Bs!7QMHUtor`H;k^kC)I2FrLXOT;L?T%?z*2H`KECeJD!@EF!WJA`29?o zB-W?;el26N6($$bf{2Gbr+P=IqOafZX3_i_V8d~iB@+*&bT#K4`kt8>^!9?BSVGwK zU+z9G5N7s%Pym?&J{U@*dlEA>*}d^uP^;~9i~d*~ zZ+<2+*5}Td{Y<|-6z&;q@ucGl%WR<^Rkx%Y3ntAADrJF8<&;A_ULc#GTR%i|_r-rTYI z35|~SrSTrez)HMEC`Vx|B*>s#trczg-m-=eN$IXu+qmPN-os(DFz#(0e11zp%d#b8 zbCztGs-LGRRxGmB6E$@dLs8UjHCBzp+7N8h1vmTo(}|tyBTkza%GkT(mq3a6^>*_( zF)5$b1NmZtSnrvH3ceAo1Pg6rmTtP@r~N%lXY$pyX%3#2lywD<3(X(5w-_3jnihCc z5Cgyj1Uz{6o8oQ_yJ7Y<#{RJfALFO{Ybudu*!7V^n{~{UtZj#FX@k!~tCh=2x7P{d z9(F*JoZyWTds9t284iV@hQR!*mrcaw zfDfaQTwWb_J9aOnOHY|nL_24OYIW`RgC#rag}!bRW6svuz)GY@EDkZu-I z)Ea~a$O_nl-<(JYF_?A4>c(*XW9rp?juPo!(t8?dv?VSfkdO#hL45eC-sCCcg$hAl z)nv>^3ksE-B=0Q3KUZ3yzhxJ0mY-!T@OH<#$&b z8a)U~v+Zj3hj=`3|JdV`O6z)?|I-j+zv`*BLaB`bOC5++Mq~dI?BoR@a`M1qi;F6=5w&-Gi7k# zC793vQqQZ4n&uE*fDoUH@F zeIH%*#5fU^`_3=^rwnIeJx=okvr`VeJ14LcA3y8&E)zn18g`KOCPol!-g&WTtmePf zOUQGqu#n@MXuIRvXul0Bgj65@t&+msZSlvLkHRn1GEAa2Skh?_o|kWFrxu(aLILj0Ka29S;1e=GXP*$UzP&PGwH^5c?2EN~vfO+x zIzx=soV(9^m`M(~xJid;I}$8(@VnppOmMZ0FblA{vreJ4+7~=|07O1tY-4?(*`xD!f)xH0w@pO$l9SDaUZvDKoM2U9QwWL~<@X&%JUR zZ>rq#<8*S!Ih&e*?HGm2S8**k)9;9K$WsluVl$x0LP_U#AvP_H@siW>1n>Qy)*sw> zg#^XyVoA|<7Hi$Kpv1*;njQD>L3J!fjUgg0u$X|A4?9n_dN{KlBtj)E3)9T}uI;Ef z3N43V-N{VA1Tt5Pw$3sCx9|qaxxunT{TQY;eLbqSMa~@e`hJ}q&I|%Yab@ktCoDI^ z3fa=KdOCX1tLLhVxDnRsJ}sleze_u^f2H9Wn2HfF^*rNuRu&Xi!Fvt!x5f`|>`ylj z4a=s#tf!(*db6eSpWV%IOc4EAH3@Uu+Uo)6`o)%Sd8XKXcVjEV3O_?oN0JmH4~})JA!Vxsu?sN(zMd__*Z@KCCH%Ix)+}HN|;Ta0k*P z4SaU~44T|yB0NmK{p*uG+g(km?|y?_>u+7%5r_8O*4Qfo^@Fn7JU#D zdFHQ8ndr*Tw=le%c@g&nVEPsIaeW>1^$6C*k8#^?A?5+P@G zK1-T5>uEB_XT;wyqc=MiC$<2n3;@Ps;W5W=Qv`cK(NivE=ybuWkWD^oWEQG)rHvs&RIKC8OrM{FoA0-~arG|uH{`J#p=jc0r-cvb(kfFJOEW@|Vv0l|g1 z_jHv|J|{x}+D@hT^!wW~hkm4ZU?@=F{7ljel}Y0KIl}!B+Rt~Cy%)~_W|Jgg%}k2T z$kw14KKSLbGXx8tnE8>Efb*DkF4IWMPS@*;3KppQ3xD?63nb(62ZPbzC?kU%a_Ofm zuIdspiD;FYU#JyUV=X_F8`0*=4f0a|*^X^{Mt|Y*6Gz+;)?XiDBz$9M$=&%?<*tG= zilmUwJR=@pYl}Gvew;%vgVOyJ1;X$p_aWL&GWxhp*d$fmeXs+u+krSe1ek-n46fk) zTD>vX7a$p58&1-U<`mtiuXn!gkK`=Wd?M>kYLV3O&%D4Wlt(O1UZ?6o+1rrlnD zIaf>-kW(7myJ(0o*nLubazkEKm!P|W2kT%IN-$ko48Xyk(MeOD+e8C5Mlno3WUkNI zmIiV;Bq&JF4+?MO{!1BCQq9)d@_Xgy}vQ4PevKQj!8EcYiJoD|lce(QobK z=Pw`m*s%f8;XQ}gvPw7fW8Z=DWe@Z8k-S%gqvfIs~91C${ssSlO!#_2(1=IZv;X$u@`M`6R#`$nY>o9Fj zeP1HRGYdDa45Sp9)BV-A;^`+Cpp{??iv;?0f1JH{|0eM956_A_B|-0cmwALNT+p<8 z0Tn92qSiatWs;-AW_tsQ!`PtRSAPi0u3@UEh~W2;v^-6FrP4l$hAeEXVUO=J_6Z;H ziqCI`LV+Xm%rM_=SISe#+AODXwe*&ztAjj6jrE>-`KIV%S1wg{G{O!xiosmRjy&akQSK)Ml$?Z7NixU>6;JoFz@y(C>s00R`5b0c2dwd^2<-2+$KCF}8_Q!cnf?S?v~b1u80Qsce!T##I_!!*8@fCUFN z{93pHgsZT*q&BO-&E%bU&YvTDB~GEhfsR>j9v=w!T%>`rT$wkqv0$ zoMHn5JTb%7v@`~&2&2HLo3mSJs2HOXJ{BQ<5qySe-ifi&rGT%28!YuhZ?*|~`<=bw zVJ#KC&!j?u%ft!%rq;hy;D=^Tt9_UmKF>WV`c!y`W$sL{()^g}3A`uv|nRl0Dl za5Q5oKFM zG#P33lPcfIj@F;282QPn+?&c~J*=w`O()ygohXoEqMw4Ah%#T zFBjp+d>8vPYu-}#58 zwCO*b@fS$ZG~oEQP2!t&x!9#}&&`5*V)SFduzie;|+~aupP=eyly(){nByV!^d{Jw`Mwm?* znqD9~oaf(dUNOgFT@Wt}5(bMf#(fbi9gb6>3nRJXSk)@m&Y$imh--DvQv7JPZOVLr zG2Hc9GQa^_uO(5J@_wO}r=@5S*@{y3`{rP>O5S=H_(9KpLUVBmwo8t!%)fwf>Myg~ zBipA)u7(0cz5Klo1!s#kpysVp`!t?^#0>*m?;<}1*aT7$4YKx5q9t~ndht$5_i{!h z7Xw&V_n{Pgl@NA2bv=x)>BxW4E#Ty|Y?W6;6Qt5VA0kqyvoQo(!6E3%Lod7OwV9SG zC}1vN5w`XmFg_|`N1)QI3?2L)G5V@dFX>#cKF970+Q5Ah3ZgsOXp42ux87)id@Zz! zFi2_LexAMXS7-VtxvC9x1Iv~9v*uBCJXL$^z$`13>&i@EBWO07aPpx6w5n}kKmBx8tSV=(c0ui>C8xN9-ZbAb6xzmup8_$>G zy1aWVnNPJ@8eR#jyl1QmIDh}OJa+mY^A5ti=zCnFzo)&Xe3=|o?-0qG7&l1C-)MA+ zG17ykSVEVXRcH<%^pR1i?i_f@?p&O&p8r_ljqhvnvZ#k9ZJiC~10tKtGuaBOSkEE- zBPdp$iG2@?@DV@ym;M&St6so9RMz-vxm~q$U49>buY!#JsM21TXP8@D-Q0-lUfq*V z^Uwr-^WVy>O<6`eoxiEeSV-6kL2?R=Z5f_CjyP@(e%i9NQB01mHl0NHgc5poEqU*_ z9X>`RJHGn7aa%D@izl18R>s{I#D+>K|B2X}UM7*<7QcUEo`#^8!|-9BC{zjal?8%7 z)$BYeh(XzB1gD15$!@1Fx7=iEvzfAwDeke=YNNrCZj3^}n`%vQugdgsT0agKGSWE1WdrM0tbDDk$6ifGTBdjJ=)Nd~ft%XZ#vuL~ zm00rOAt1(hsAL4O|Ahdj_}DgtP_H9KlVYrF<3yu9NS+OT@J!Q+7-O7}^CzeY$Q@@z z+%mmuI@96!rZg`&x;9jgok;xIum#jn5s<4)I-rbOkNeVj$M!lx`a9uD@?tFzE52@3 zLU(vOds4+k)sa)M>2ByMWK@6aGDzT0<9jk`&crF1(2w|`sbZ=c0@W{XLTK?;iI-A6 zd7N#K>H0o2oR#-tENnr)^AAH+O6x}JWzbdQDC!Jv+zm0`1D z%lki}<3BEdAX~<#X9IJ>s&(WDrJb>lxe<-H%$6%wKgOn7tUh=pO-&I%dc%4$xC|N! zh!oXs7d095e)DWhCE-ZbzpSHAPpg<=EpwE1A|egs(Xn`UTJPm6gq6-`;0f3|LB7}L zZtUQgPv>p$X;noineiLyyKr;~-7}Vz)1_5NbyJH9W-Zrm#<7uhi?=U{TVU3P#Clo6 zf722N!oWc_1QB1PCsMegPz7OCosG&$tTab;UPN9?{)?zG8Q!8nXh>*dEz3VV`9Huh zeqIOvuai-qs*faRO!NCQc(S??_-Mqof;PYWR*Ps5qZQ-ce=_POQRnlYTw%X2$g#Ou znDl$SB^)`MtuJZlMGOk0!p@WhYHO{&I|lc;b2a_IReVb@H&h_BwEoJs!4p~mC7j6ehytIF`N^8hMbRyy1wtK$q zf2Wy6eo_4gEqIQ%m@u%&+_l!V;|}Y*IcDBlLyX{b5c4OsVChRXSp)sd<;}9fd%Ui| zn2kkS(&RwEgoQLbCKE)v@{#NmL%omASJXrlyEPN;Ar!FV(=*0TGcDk4hrcNeJ&;xo+g16@P)cRKLahrQ-#?Pi)0XNvq?eL8-UO5{5Q28 zKm3^4$Gz7&t)^Cap`KL6Iz!Vt@VCA@rLLdWgFxqGB73-}?vS=2G7g$r*!_2na%EeS zUlode?!;CHkwuYpY*LH*4eV2`PDBOHhAk5=f0|X?QN<>A8YvfR@d)!=+H5GfwL7GW zA9W|jiG;Pyq|O6l|az2td>kBa>H*-(k|+0rl7)XsW(UY2n+zTv7A zl%LKo*c;rDaeGL;&r_i8+dbzi5#0@!cEyfc;w$aX7Fb>~mB!Yw3ADiLgOMhJZCQaPlcT?Aa<7Isg_gfJ)$ z1pSb@r`MEWs4KDmqswPQV)qw)Yc>CDVFls4WTw$SNAFBjl-=hmOG7nr!c0qw(w;m+ zW~PKG;qN%;xGl?>6`ufteGn@=1Z#TD`yJ@2K3`G%`GiB?p~IK#jKpQxxaF)l>4_S> z7`La!{YRT&&f$2Fb39uQpKC@wU?a7DmF=+J@5KMIH?PV-f7l17gShpt;&Xat z8!A>liHER@c?txj;StPAN+@`PljZT{wmR=yf9kke?Irm*FY-l{gkj(69RYptLZl=gcFspZLOZ$Gzii4R>f+_=1 z=2dRhO*}Q_OwIZSwZ?mN4BL{K!Pc<=_20Ke-Z^WDLgMpo!)96g$6=kkbC8nU ze>RA?Yi>ZD_hV@Q>^52>8aS0jD&5uiE%9q^sBTbMH67C$Rkr2YVY%^#MmZwW4|Zf# znagK1Y-a>xSl5H9y-U(ClgL=~f7O^6RyL}Wct92XQIwliEokRL;&-1E zlVVS`rCWJ-mwU6fTVd^V-!Un#spX05=DW@N7xI(z9S(w!$lt#1@6>p%KE&H_CJE9n z@Wjzp;nt7L7m*GHiRQnddL1tQa24H`a@ZZtZILHtX??GPePb)^M-R?A3&&t8Lj3In?cXthKbee}WhO9o zaYjg)rP7_#{d$?_l*&`SGXzTXuIUL%SMP@*x6@L1PER&_nY16p7x{ScofP)VX|{U= zb(&+K(YJW&+}+n2ECz@+B5y}0n0iaWgADal#jeFF8#@DbSt|%U~4ms|&*M&>FnFz8`vwscl?K%#Db|e_Sk$K(&*}>1tyzO7{wIR9ya`h5h=A>qDc?5>-3Rl}Zl2$g zcDhnNe&^aRI3zNS_L<8|(05Qq?&oE;yW|WGcKYRa;bHG{a8u8+UTUG%>P&?~X@ua^)H)loRs1X({>)aN)Y| zn;#Pj$SGz@TB($a%&SVnpW*~jL~}RzG&?)Oz@9_Pcz$*1qx!Lw4J|d$Y0*Fao;dnT z-}N(*BYljEu{4C}XTaJLQso`=h=3N=ZZX&3Fn86b%4PsE6@&cA4#K3X6diE^o?P|v zOzv8kIhu!ec(M1AbMbb1lyk+UR(`ZNC$sjkIiXjGRuzIz$9fr`68fMeS!b9iez;SNpL0B|fK>5}m7bz1D^^AM+A z|^4qn(aCu4unykU)wZ?9+#Uk-~V}J zzd0{W+1zyUh)(K$W(fq6#cpGVu|CegSTI^``MK~w}ZoRt=jAiJ%;)y7@9sOb2$+D zbLIEei2nNhs@pAceW3&0>a}ZM@A>Mde@#4kcLsHzlhQ4hCXo{{@x8; zCokGfPpq&03$5zs&-bERKR?+x>dlXz>?u1NZn_K+=cirDaEO42faT*Gt(wD1$V2s< zEz|~WkF8|Ip2>?X5FmX0ydTrey_%T$Si9{g=-K>?16>~s|2M4tVuG{ACQ9o!tOH6$3p!~V7T(`eCqi>`t z3j`wk3xtu!fi!|z{w5k}RyF-n@QEolEvW+9+LAu%b9c3}DhYEEF&uK8{$dZZz(qsy zy1^}MZj`i2^LwM$qvthM!1nzhMF60I6IF!&)atsA0Lxvlz5oh0CF_~S`u4iu?L}Lg zcR)t5ID#v~hy=G4F6?(Nf2Sd=jv-<)c=-XrCA^Pl{H1mio%ThS8wgU^ky4_V)-vOz z<*Oo?Xbg`JDaD}zM6T2Ys98kW84djj>_3PEz$JzThLS}Wm*GFQ2^Y*j>``765V{v<@>xc4MU8BL=~M9K|ZYEB!o^6Du~oE;u5+cd7@eu<8YpbR})E(Y0mg z{jRXH*udXt*U!Jqh+;4#+}z90)-3t4vk8&5Stuv51`FeX zrs+x186J;Von~u@1`c_uG-xzSE2HA0U6z*n=(I3BM+gMWMO^O|V-B8wt<+bsuK4+G zD|BP{U`9iy8y=W0Aijc$I_V2^rpsA;zI%bB>)wR0jaGM>8aSxYNhaP_d;Igj8T{n% z9z@1~47U{tW9w4&OuJy5e^pMH#|6^U3Cc~o#q9zBcu+SjaVyBB{$xPDvtXfFO+^o^ zlQnFAk?zlzOp9k-8F_Nc08V8 z{)&^iDfHTGL9cdZoP1R}q@L<*o&q0{-HxK1?Hv~KiC zV8))EAUSxZ4y}J;-9EaR^jkzrZ05n41IWsX_)|c7%&C$ z)k$f8Lp#ol30~btLtVn_F`_3~QooiIRoI)~F9U+_HHA>iAY(~a^5}^kLrGWkboK?= z0t{I`XT8w2b zfAHJ^>tWfemFu>=@d#tm1AApAm+&sL_qu)Gy#6>KAVy= z9zsT>wkBaw>br;y*hBI$p;##e!4)xLQ0syKYY8yU3F~Bh1C_ls%zbafzg^tS2z|&K z6UWuTNb`L{1544Yhg7z?NB%EKlFT3t;4uy!;p=AoF*aVe`4r4)q#a#}0>?fO5~Z%ux9#B)H%}P{IiikOttgAdX4UXQS~_ zDRr$lk%^}HE8$jKm2VV@?RQYx((!>Gg|3e!|4Rhk3zYL&Sad@CYny=kF84%MbI%nj zjdnRGrAj+>Wo;$udT-}dfN6I;&$3+m%CvHyTFpXr>*Uj1&V#GkK^mihY99Ed-9c+M zH^=9wH^{FXkcNQ)U8;9-V{)I$cb`P!YH?v{NDBCPg||MS;#?jLF=$#$n7kdpa$_S~ zD%Q8j>qBoO74sOmR~uiLwe-ZWRR38ZdrX2cPpFJ{2#v&rfV$Nr`NQo|hZR;1^$TZy zJEu@Kb8J^GM_y|6d+k?Fzw0vj%rV-ecE7Qy+ta^^w^!Nd75ZTfO{P~K`tGpnUJhKO zZRf%%KEZ)ufF-*hrY0`4>M%fH5+O`N39<<22gY=V1B)csBfjq7Oy#~uU_{Y6H_Acb;Q;Pv2b)MijR*67XgRgVf(>n4=0C1&4lF= ztNWE3Z=D|x*NhIzt?w$)L_fuzl|-k_KSBGImAfvhvb!7~RPwX`1~oV+d4+KFu^^zH zZ$E9$Iz*tUjb34W#qbaiVEqxnWIQy?hju8S%=gI*khw3= zet-4)3C*rv6E_dm^-7tf&N!&%qUVHOEbmN=xjL6!D6JC#4Aa=46T7-Ett2Yh`0 zmCyXSTm9vp-!BglaaOt5b1M3>j~N%`3coRyh|*l6(OT>Mu!{+iMa0(oT@wfvEU=FvBNk+1MuBF_UPnew~+y1cYQ9@3@66H zbi?>Gafd`+-evorG;eTZ3VI(x~{{zsgLtFm|}f0n%McGfh1(Uj>xw5qPRWB(NX z(AR@tVGzc!^usA4cY;}RS2j6_Vv&c@S>rzwM+=wDNIt+nKzYn}32UBg6N*fQxbzl5 zbqf&`0A%VF9okvOKJW`H^8J#KvNTw^dvLaHb=+E_mxw9m9aG@JKyb!@oB1}?L^~?s zg7#1m)h7B=NCdwdiYMApfw6`3%Soa5u)AUPQ)>N#cPR?=*+2DV>QAEU!|zyoF(y)o zcpasAB9AUM*M4R2O1OUDu2INh7h8tkT&t|;N+*J1Twpw)c$tk}htQWOT>SwCmb8sH zkWvB>iZIt0k)zU^mIL&wuHc&N<`v2fNdw)xd{tX27`T(DL(M8f2`e8yj&AZS9uWb~ zkU*552Q0JWm`yBwLAHM};ng2mxNAC7Z%dtt35PN8{*%uhRs_Zc8a1KihK1sSK+s=T ztx13To95jCSp0BTGJrS=+(>pL`Nrb1EuXL98~XF7!b%4@h!M*|(Lw!e7pu~;8l|G+ zosHAa>=;D%nlhy~k+5j2Nek*K>$yP*(>$~G=rKLbs_t#4OCW(@Lx-qm5IywUnTm@I)J-bGUPws}F_lptT4Xn=ix=VlAIaY__z!^-owc zl9MOxbS_~hEzGi}6%MPExb$s&QBOP~cncS!=UG;0dkyKE^}uo`>^`~is{~NqZR#OrQl^M?% z;U5O>BK#j1ZomGCsmd5F(#7y5PsV^^s&GC=U9v$#J_3Pj^s|eIvjm|zM%S#oy_m3b zmee?=B_ThHPGou+EhYv+hZG21V(PA|*cl~ zhCine$Gixz`kx>|^%=Q=oic8gQecr6~SMG%16rta?z!^Ktcn^LXGV+e3KsYt+$Cru6bg59O zTu?=$>iRmnPkHm-D=`5}hZ9wNyi}j$5|zb!Xh$^Qo2C2+qu^6nkl;w8g(aJQtDr1L z?7a4{+On^`tf;C%wYv{1=m)#cA3YOZcl>(|1pVYSxu>fT1}Cz;Dl$O2NRQ&G-2g>x z8-96Tjr?6wGuKVJW!UOZ^A6~m{1t%X%n~@VrV+UyOw1IK>Y*HICRHq6r&7cf2@q!u zWEK(LrU@Sjxwc53!McN6G-G4L_|Lv00*JUZx6$QPANS#^MZKdKWzOeSl+cf6#{NPh z91+l{AfI|T*-jlXY$&iH(Xa51K`t6P9MBrDblQSpm*SvW9xFe%Lko8Nnx05TIQ1Pn45^BV%LnhsMQIi~8!te1NfUn&xq~h%xd$Md#>pG73=V5&byoUsSCC;R zZ8@n(AuW%Zp^IivLf=wpMoXt)Dy<~3T|gNYbgmtvCjGx9+0q1D_Y#~b_XTrXz?Uw| zjcG@cG;o}mMgkBtvhqlAJh)P6NT!STt@7Is^=^!N#DpWy*dE5OzWyDH%_XMV$H^Ttb0*He z4UI?dS4odZi-8Knb2>Q**T-&>I9O(hA>-7opoFBvT=y~xzsR*AyUblUIQKdoS$A(L z=dFYcBsdpgKgVXaz#?rp4`)^Aayz&pTwMR3b)jQvrw;x8o?8n>KH;`PN7zk zcZn+8$Wwbk46P7eayKAr0hjG)ivx%K081o0=|8KSH#_=W2006{0^&Qo_3aEX?a-Zs z3YQRavM${v-cKYZRByuX)4{1}q)H zqgCK0sKCWC@wG5~Zue9kdoFC~(hIJDekfg1^vtp;VcGNm z+{7cZu@G5mK3OaPE|<)>jD<+@0y2&gn5%gY@eN<+9D=qLl8X>NHo``(S;%>LtvkMR zi*HB;5UlI?U&!*b-0Y?ELMr5YbCzce>ltTkzPNW^e9#5qD-01h--#8MF43i3EsU zqq0>n0JwqD1u^BRL=szULn$!CO|h*8yYKoku&1oEzj%Ep*Kz)ka4ZnST3no^qMU|H z!)xu6145DD8VhMF1q<^-XFcb~TB|R*_&?8`v`?fN#<-nXwo<98590cEMqT~qcbSCt ze0U??tdMsO=0CKHn5x|)Z30{~_d$HJ)&&ahmINF{`SE4wV#`lq%mwmTryBLoY@BR)u zaf2KK&Hmn-zN}glK%G-O8}f^7M82PjpSLb|Z_S!!vX#kx;^{W4Ht!QNdHwou;U_Rz zcFlh`B^QSXdjedVyoY?L8kn~yJVwA(OTAl<{&=zVY+pg8C8Yn7VJAV$`^wYxp)boQ zKq(F}2q*>$8?pO5P^ySp?2@5_(n9>kHzSEZ*5<3y=;rLy=Em~G9^W#=<>O3;y2uUm2_@4Sykgo zu~;3f_eINE=U;Jq!a@**32L-`ct$fAeou8SLK^W3olb$BKKm&=%8jA_LC2qNM zK`++(>j5~XtH=612BsLTZVu*y;ksohj&fxR(05*lyUkI=>^bY!9?2QwzoJ+U)deXz zjMPCggrFkZYqmVr*#i=$yMtrLxs_i&9~!#&q=pvn1dJ)PgnZlm%>8+_iaAb6o_GNc zUax*oX`q)hX2Oi%kq4zVbe{1>ucCRn9M^0TbHwnSPvRTNe)NSYm@UX@SsfNUk(b=M z+A8eifM1t=8Ra(*VA{wmEX+7v!F-a+$*b>d$oQoy35*0GXv3O>&@RKauXm9~6&{3# zXKgLQ)3=j+Dt6A!i8 zO3K~cPBf*|wQsCTh?kh5e}zozs+`2~qu1(b%2kqC3GVJLfx+EvaCiNN zkmuR=J7@pfUCuqjZh50=rmQH9fkuq>AG8paZcral(2U)qvU4gVr8cmMx&&p6m&8*2dGL) z{d*n$mk_lj80-LGV{>(NWp(9bwRf^$d&|$y&&JNd#=*e?Z@~g`w*woyvDkrV9y|F@ zKayr36DKPNu$8?XU$S2s!{C`oh1N|Eoe1L2Z->|)9WoP?u z-|$xjAD#k~oUF{?lpp#Rek=I5<^S*5KRSYJ55)fmGmj(v`xHJ@VKhOu|B6i*O}>?0 z=gAY%C+{W2)ZGwvlQF#T$7(vEwung2Fo>zFW8MY&r3L)_89~gXiOrv1T6~H6z|IL@Bj9g=84lS=DNxLo5;V=2rt5unK+gZgVX5MG<~ih>=O@3 zuV4!xggOb!OjnS|zuRlW(z2Lz8(m`R`N}3=`~)g+Aar<1z_^ipv|_E56{?&$Bs@;U z{N@|hS`^JGdkj$KALD5nNE^0%&*XF5>0=;}<)m+kFI_Mv$t)?8Jqc& zwOL~@BMz0s<1KW!j4=i%O}n_CBsr#vviT4AbJV|-?Fr@3 zf0DH-L6i*;F#H@sxFW-k(+jSYpzrh05n1`1j{6_7c>ZO|2?L>!kb#cPH%V%(%7k6) zX3t|BDa7Y~#bju90z;intd5v=@GD*|Hl4Y|gqTvm4>3Z~)Bq>- zZl{{TvFOO~mG(auTkuy5;DEGN zGTcAQ!lP&vCG3v2OpINYd0xJsV3~yZAB(5;4@uf~it(b<^ELO6z=z(C8-as$ul!6$Ndde24VGavXZj4+ z?ZNsq|4XGW5nz+cW{ppxb5|OvyJ~N`mS|DXa1V!kP92ggGC|A&rKp7$ z4wx&3W`RP~z`$ZLDg2X^Xj$4q(6A2ZM25wm)&KdnK`?tTP?af7&JCBqLXLd_HxmdZ zasa#QlU_H&{%A>rifc)o5|PsV8yF>G0i^?GuY>9Bv8Ya!uA2&_V$OfR>ZfJLz2ti~ z@cTfVA*ZBDx(!_aA3MbM*Tm+-G2`wY-Jws|baV(%m23@}}mgA9s`es~HEqEXl#X8!+X@ z!V%IQ$?Qo$B2(8y#)a;1)}Xy8ks;RiM;Dp-1VKcRVuK6T9u^t#amvBgls@QqNJ@-( zno%m=A;~G<^lwN)AP{oLUo^%+ANz`hAty;lZL^XAtqNq-J(BtCDG69C ze$Ru8#+6Nif&3(&9myg6e(c)`ICx4qT{omEfOPLmZ2l8v^1pHFnBo33&mMv^o<^hH-2x_L{U2aUGV~zH+pZmQkuIm+IFHD)cOvot>e?j z8bk&te?aB~?~^a3!vs07f-9@tvWKK#Uaal>yIB}t?n#M5I?*1O?&0B;*z|K7m;h^c z1uBno;`LTu5@Q|DAx89fxvhs207SD5o?-w-J_tiT_B0EV+$kh<(@i5=R+L z8YIm1B7$I?n}!E>Y1vXRc_>K~kUX9@#NJ0B30+ysIQ~q(+79rtq&j)da4lVYTPX?M3HtLh&u$zZvTP65Bzs@Y%a9Qv)KD@5@Ov zJoLU1DKRS-q$f7YigT<%nqrJlaeiIUK%gT=&+$+Y2Pv<(m8WowbKTX2jvzp;mu~~G zLHeB1m|=z()$DRi8SPqhMvrM{LY1M<5V1dac*;!7gf5QlOS|AN(AN_8Sej)S2A$y& zg}~v}b~fl#PPXYgE{RuNUf zO7cyf_NZ$DwT*yvu$6}FgW4{)CS70Gw+~+VK(M3@VsIQeG~@%B|Bj>5reCYF4}cA; zcj($c8BnV+xSd{t`$yWMk(ijxNO;*D?+Bg2{)tbVr&UJlc5khF z!9M!xs{4D&e6_#9$=@6sg*)1UY?zN4)Jn0)I7r9(Gi;@22z=Ka8R)bsiOhxCOW}l; zjG2WmE8Wb)AWs2=65p30^{k{9HZJ17PS(Z*zgLfJcBBRq#9(p|$}YYA^UWM(w+`>> zMLAOM6?uRmx;JU>*)r;G0+#dTq>mt@#H#aPl)vP9mYrX;QE?Lu_CP>kfNH=lY)o?O zT8}uw(aYVN4=8^ZgSHXa|2MsG8eWJJBS7q4M zK_o+Mc&dwJX7I%qGXdS~noabdAK#UseCizGzb?@>m}W}VOyNY|;_xbJb5L+Y*)o67 zcvWKaLqK$dsBnz*`0f_8g?}4_OF|xjH2vOFXB@SUJ;jBcg(07YqL=66wnPlUI02?* ze#4*Dt#rnkoaBamU8AzOv{LzK@#LKM*V&-I>5bNE0qkBF{!7*d82srCDukz zWa5JS<<*5g!>G$)F<^=}QQ0-h{)W0-1JOc1Mw3jJ)8`@{MedQe=y=g8*FtE%J8cz! zg=0waW+(;bXy&-HH8s%C=un@s&k}hN^_^);ZTE=}h`9BYNg)RqOQHVagN z$&Fuwoh(9jP&*#a<9*uey^v}zK5v2-sv6sa}>WKe##MC8`!j`elsos27w7fPbvNX^ekM3K1tbGR~sh^I+5VuwkS zz$0Af)=S;hcRLoNy4p-FWu7ij7!t4QdKi3_A$m@$46V_$h4RsVS3VdcUU45$`>j;T z=Cj+_!Z^Y0Eb~AzI$2Z*w&gi*;3XWtLzZ!?PcX}Ag51Q8XWw?KzVY+wae?sF4{2E_ zP#$*to^R5sQ__dqcydY(cW_Ti!NaZm6H^1!Hl<`6ZMQ;r(P_d9AlL7i80LvEf6mTvi3i$i-5L=?F=FB)nexh_Uh zuVb01^ga>XTWob&Xf*616%zg+1~eSY-<-Xt-rvH3h?js*Qz^Gr&ypN*++sd}^;V^e zMOX)J%gfnHetTQJdnYwDs-o7X+nbk`M%h5^X`)CQ{euWi!$jg{P}CN=+XbYe^(rBJ zy-bjp>d`-A`yL@hSUF9uOuZZO8Hm~Pg(d|qi>E+75V>|AF1&? zdSg>+<$aALQx3z*M*)8lrN;dVQfSYkPW)&334Td7sz$;@kB0a46Vj?a-eGR{>&N#q zUsJ*oMq`yaAB5tELQ9RnKOtGq^lw>5`5P5Hna4*_fBU|ydB^u)-tpM^{Kzq#LG`aP zC>c?}aUk#8sfvEY!LOj*y|u(=O7V|$aijgRgYk|^yBYp5`atN9mQv+ELbU#vi7yqq z=dtIyK)8Yf-{s+9YKv!wKgQ+qww@s+Ux>FPuJ2ulT3@)qi@oC3$UrzM~pfT;>U%+HQQf;$IU*tf?gYPZOAH&o(5QDeBnfrNeM09N^6`ny$nnmes2N+lcAG zP8vAonhTe1uVZ<+JtEwqlI;A%4e$|gWZoh=6*9VreJc>qUOdq?;qmPJ1_HkLI{d(N zG@NO1!yk60>KYpFGa61C95Oa#;V;3b3@zM3{0QH@esEK8j(OoERvUdv4Iga{0d4}X z7vDv_e+&ve3hI*SCbUAgmbJGaA&-!p7fI8fJqbT4wKAM8(Rk4~mTi)oT+iBOU_Y;_y9V_&!s zzO=^_5NJ@=h3vbt)i^X{SdepSXo!Fhgyx1#>Kix~OyPsy}8Jcc-}BpqWWj*;6G{?72xi?`5|BSAKS9}2X~ z-#uQd0#|O8HNI=-I*uqvSQN-^aLjOV$U_Qf3M=u0dfz!T)>a`Un; ze10Ty`I>pWTRUbU+I6vj^$L6seih8xJsHM~W<3}KwBmw=2R484s7yTZ6QV2Us$X|jD5Sp=OT zIvO|<^u=`=zXsaWgJZLtUIN1`ZZ55aJl?z#aKtv}B>7;6iZ5cYd=v$$#C169wJn*YL*XRgFN5JQ2&l^#}$<4S5lpQsjbi_BZ(U}!ES zwHA2M#V@RrYy)oB^=ovzlBPyvlxa4av=`k}kN9v8vLb}CU7?aY_e;rh2#no>f34@E zafZi25&;+UlRUfEmny6@Sy zS!SXs#T2_~{|{e5@4ySmdtXIIBUtYi*153Agp zyb5Tj#CoOa4pvTJ3HAPG({clza8Wsw4>r*4fmF4mp=b`q#EE0It9!v8hK4UacbEAm zzL&&D3y~i7Q+#W{Dmhak^>9%&%t?vbLLVZ$U2Hd04pxg2bur)Xw)#C;^5g7xEtBf- zFh@^}Z`aw@k+$bHzZM&QEm}OW!7~EA_tc?py*+L-19J)#O-bs$0bf->_7!h6jOut1 z@+0C#8&F428Z#|H_Qe;p1Uk)K_}}x{W<)COPm{KTs4Q65j^Ed3bBOSOqJzWP35Cs; zTP*ImdUh0nAfWXQZJIYn z&uG0AEjIO_5X!~WTt=$Gxs!#N^S$eY?bZS_&F*cIe_ia!-pC5JtyZ=42pt|Xvh?Ci zrFx^9Zt2>Glg;Xo{P1mkgZ0BU=$qEd^Izff?zGUzi*F^%%oIyFgOcucQY2bV5})SB z(w{vf#`b$8n|~AT4K%nrd^_r@u8uW(FhKZJ2&#U%n|QdMGS$#DSz!+)<(r<3%MZiN z|Ew-OZa-IgJmTwnr!ij1BCv6PX4hLSm26X7oXpB&&bU(Tx~Fw3LSbCLRpd_fq-DPP z)!UC)GI&x>$ovLHqYjY^U#l<`8&N!O3v+*M)U@ux2%?5Y)LdSr=3XPv#!S_@ao&0uN+!Fz)j0+aa(U%7|gjs<9r@|HkbQ*Ta9Y;o-s#tZH!j zgOJfRCKnztdF1|E=eiYdePg7_{#&K}oUNwdnZWTr)&e(jszZYD6+(uRHFx5{>~aOev9BZi#%)ga_yP(Ii_PT2*-%9!u5n@X;hxt)+Hf28JmDDe zy#*&n4IR7QPu8C_W0+gc|9*|JIron5DESh5;bUG@KLwCo0+MVB;r~Wb9uKhM=AXXb zKZF@*Ss>kMOu^Roc@1pW7aY?0_p6{h+a6w*o5-)I?18>3>KQ2IQSju61R>YW3H#U0 zXSGo8Z9Yw1pNmX%1*rw09j>>@Hm3t*>jg0K@(e~rUZ(`-i%HI%PmoI^k3}iYjf0Xn zU7vmkrEAY04(Zf0Z#-GKw(_lAmtIhlM5DP>sz+Fao)^7Hi=L5wa*~ht$`tYo>9D9R zrjdI;i9a~SR9-T+0956x=!cEc6_@_)(7pAY>K4ic2o>;EYabKc2#gn?`sT%yG38a(s9U_d4sc< zC5Et8ij|_|+rViH7p3_dq2u*s-oO!VC{?;pG-)}mMHtXh*u-Y8An|T4_ zk52~+B9jRW%{uNm98Y=X?K1p$#_uHX%vt5)#^nx!aI%&^;pd>g(}|=bdWf4M{Rx`S zD!tb|kDdq_QgWwT5dTrupbp=)tt82vL+1Iek&qS8l>0?fq(x$vu_h7`5yydD9dtWwIOZQPDvr;*%;Igbs z7&$JPz-a0xsdozd3IH_t77rK)j4jkbfazHRg|Nd5H&&D9wtw8sk#e6NO|zqa_r7XO z4&$31PZOaYOQS=f7u&n^X*K!?QWqv%_!+>}O7mX7`PcLBD|y&~i^g3# zYxqDSsW79T4a*bg0Ugue0Y)IemmWOZ;!}Om2_dZ+X+%JV6z7E*%**nL6xL=B4n@53 z%$6(VMN0x$r0UTmrMR?`kWK$d{LYdnTnahKW3w?Q(@$AE{PjE5*rQ>ZRdeR^JH2`z ziXzW`h;85{c-{wT-0Msy6}}*}nVEW(36IU9`0@&fq;z{V3m>*^H^43eR9%qf4L`zdW!F z!0Q#)yFC+XuDoo~yLZyrBI9q(YO|5ySQMR6yC98geJ7#6wr#2<4=okJ2b6u8+p3h4 zo+maosQ*m_zL)?y3U_(K^}F*aWx}M{KXkOl_!BUY_Z@WBvBKNM?s$dBg+ks_4ge7~ zd$<42p%`!V!#2kcP(hzWIzHXOC`)XjN{+sA9r3vHH?=k|;jSUvRF=X!7|pug#BIk8 z@3MAca7MQ8ZNWN!7V}6nP2Wg~Q|`BL=YUpNSW@ooogAEpY-iBf@MSyNwqtn2 z$B}8=b&z&%OB}>`Wd21N3oqI_HI1<6`e@y^G#VGje*VV54!QhT=fl?@dr~cCtc(>6 z1l&E}-)MQo4NSOK`)y{*+2JPchUbd1;O+UxDi&0oLkOBmufXU4mT0(bRzW|;WG7lY7yKT7Y-uyZQ+ zA@aVDfBJm9G@He1;LWB3;y}yU=E0BXOoRyiZ>!=xe=2btDAL!(xLq0?y*ePay6XUs_nRP<-o z!4kOCFI&`Fs9+AM8IEH%z08kJ)H`se%i%T&M)weA)HRa^-(Srr{PM)wj@v*|ig(vb zG+GJ+M#5duU9bU$5Mg)>A0o^~fON>`_UC=sRHOh*`c-nO1@Gnu0B`}lPKKMV1ii`) z#J;?3G0s}O+-nRt^PLSFVlc;b;}q{2h+v&P))(GBqtRI33y;Qo{3AHqz9xd&ZZrcB}(wy=YTFhWT{650rXUjYVX~MTVQ7b;^Vf__tD= zN(eL2jeo>Q)e$R)-ZLK;8(p8KpYk69}MNP1b%X)!tEeAN=Q zsO2{I`r;4YE7*?948xIBK!)W+kv0h*$7(+!A?gsT6NovMDz^1;`0PVV$>!T##!s9@y=@A}+wLa<|ghN~0>r^}zI z_3f^d5nFna7PNcc#v3h8*8TcdoVpZmJemey4v_KXPJO>QKkXVj%yCBSFMLPzF6}!~ z?Q~7$uq^?eza}2v{UkyGYhwGeZN5oum~5BFwGk_?=T2i4zw@k_`C?;RKcsipydyVw zeTwkj7L(j7pWk(B$J{v;a)5R2yO0)PzGknOy}i%ZmgIHg`X?%M{LQqDV!Spi*P^gL zJzMZ`lZu6EG5gbsaNydf3Z;E{&}B^%XW*H@C`6Utx}SoxKk2$4HNKS_nmD%4TCodc zxo>sY;$vVf%*`E+vA=G8?X6Nd1lZUmSPR*N8{S0^;P_73W~YW(iGHA747Kw+u!Ykr z#w;;Er-J(LW+eUN*}^OTYRn&);c6|HUGiDJ$ zH@BCy0=6~PxO@Xmf^TQmS1gbFo#p<77YSPLxnB^2`Wm#M2j?Vzyd&>NXWb^i*ZQ$U z@uBX4xL@j4*kXwQlT9M$o^39=%ua;`bU99s16IK4&YK1DOFPG@fI{2P)=ZRTVi{nlPof z$PvkNWhtD=E249^8(&^7DZ{XvzI{nOaUKS{KDr{j+JC1qXEEHISfVe>A+ZYS==6mg>bgEefe}`TeL8Ib^7cY>N z`}RP;oN5W5os6kZbv!n|JUr*D<&(?ISwiw;wRl{mvhd7_zfdJC)nj1H7q(2aJLR{v z#fLcmwQhgAdRnb)oxO3S7b^%kID=*nv0Y)|8xSe{JS5Oy^Xu^nw5|{WT9G2TY222> zn*ES$^9FjxwOoFGYvZX2KdI8t!q`d(5%@h;e)4fbRShz>miW~-%KN#;Op;uGNvP7> zYjYTJqh_0y!~FGuLR}n#O=JR)4XgbpUE!=j?Gfs?F*f9`u7VY?^XaETKEv`=^6h*g zXl?b!i_Cf=hio(RC6Rp&dUAm|z(~>#)%cL`$toLgM*rsIX#3c}`K+69Q)@vx=%y}C zA~Z#rzM#h6V}RfpjHjyj$eAC$b?s7(bYsq(%q(rHGg%pVeM0VW1pZ=a|F-TF`qrrF z;C`nZwyiX0<7&CgDn6U-z$aX`?!vgNC~kkZ;-gl&_0C4cZT0YBWuccipp@+>^1+T0 znw)}}3O3#?XOe~nhp#Vlx-owVwaQCcl1^+>9^WPHxCKubOL!_aoXwnza$2{(sH?*x z(e;9H7q7giqoX_*n}1p-?3mZNJaHFRIkF9`-rxDvtQkN`=7bO3PlgDeMw@>#y?6iC zit{?y6Vblw?oWQ;seLoo+jG9xMNB)C1bsglHN_!jPaRxSjO{H=FLB6fVzTKtjg)viQC%o@lXji;cN5bt;Fzy z-hPn-#qIDjqHTn0ijUclreV9ljjJf$E4}e}R!^XR1i)jI7W8C3!nxIIy`Dyiv%{fR z$8*Ofqh|X2D^*uK{Zh=-%j7rP>85*?v6p)~yzw1!`aB z)c5p;)`t3rpWID=Vs(It>E~c-#5w=w9FqCcxz(H=7fstTHTb4$k%+LC)DqTNn}|@` zNqgj}!;bfMc67}<$|hb}9fuXye30->BH`5iAF$wTNMuH>Ypjr2(|Bvq(IDgZ4^75< z+&8-p>jqv0`%UR+M_e$6Bobxgdgr?Ew2gY$8!1g+hv*>ey z?ve}bAZ{12?T?~?>IsX0!xqpT$&$5^wV?4;m11Kl&*l^%Y!YNABNjm)h)ibuON_{4 zQKg~<2YHRiT90kTjZT3f>8Tj@Ft=jB5C=zQnsA^_~cUo+l1h6djEE2zu|gWbDYq8beBrIzJ_x zc3zt^ttj56icCe3N|%EVZrq-T{80=Q;j;hS)AZqG9*4xy>%>qWj*b-GvrtbKkdiT) zNWZj&EWK~*m+j@3EB(l+UA2rZKD0|=+fYvF=1aS~RiVk_fChdC^q+j_4*O>A{Rux7 z&_X?yD1MBeC37Ow_wW<{a4`4dm2TN?w_jy#m)>$`nd=wT&mfqxMl5j7FR){OkJelB zx+gd~epkXTgMv4?hnQcN45EUWV>LHQy;25}KKhUa?nf)5XdGB3Wv1DbtafVFfQAD^{OZKqNI^=+3f9BzUwX5ls>2 zm{}ZOF|Ujq5|pgIn_o^ru!u|;u#n-r38+jKt)BqGVbj|-lf0hnFGVAO-p_W`RR=1P zMROBYDcRvmzSEan9c4*s;ffn^Ngzi@K@N!84}9T;FZ5ngGE1MBE)X0c)3fEA(nU5# zUak2?jVKv&i94{>i}wJ;#G4EeJ$k~Tem6&Oam*qHV%uh1 zB~X^{bS`?Vw&i@oAMq1#TIGAX?VhjNc&$H0WL>dyqm?XA%JUdz2;}knr zMw};Z1*F8jCbuEUwT}I{`!;KaJf!{pon>T#Mkq=q3Ias=@_kv_fpk*1TU_U+nPTko zV8tMckFFVY5{J9vX@Seji~!*HF_DzvpDru?&Vo)vDq-eT_1DQ%(j9j1Sf#Cen>Ig| zSr5%lZ_a(F!$=-cgk10`kdl&h0)B?{R?()UCYy+CNHW&9qzykGz(2@thiA8azs*_4 zt-~EgdsQ5Z?WS8Sww_|<#~0em-7+yFvQVXwCFqLFx!2M8Q)xT)c8GzWC>r{1k#E1^ z*XXm-hse*B>940BuLMxt{Q~8ljk)_4V}@FhS`CRc;zq25b}lr_nxK&l8(w1-2w9|b zxEh$ud|WkridBi8w;`j;)W#mL!lVaqK}*$WJU(jup*dpe1K=w3aXGqb^|Wa!NbnqW zyVg%+{*rtUzn4*#Gi6^~TRl$7-wQjQU3B)q0E*llgYrT43du|~@B{Uxue-}+Tl z&EIoFm97eJqXs&csR62hUx!+YWje1kR67m40^RRYQa^5;+4KCdC|D`FC~*EkZ{*1N zlHI^^eL!$Xam|Sa9YU?dw@>e4>M?s_(;v-WrZ9A5VH*b}uSxry$m{TZd&Mp1GwL(jhPofF@xs)ntftj`-CX-4o?d}(zN9Aj z&V!QOZ4MXYb)kl5@%tM+vExK4yc)jsC+XU09Ts`u>^*?@;gILqboE^)a}Io~76nmz zLqn@LEXasgDqWJ!Pl0yJph>b~Y-;Q~g&Zu#(gT`OvHuB>7$6O)-I%SW`B zk7=_~P%P+#kni@6Z7>;SBNNU$T*Vf(vSU>p&scPC0MH}(c5e;^ij0GFgQNC70eFV9 zv6L%6_N{xE@n5ahn#_jLfQXy$w?~+UXkJHUPuH z0@d>UWnu#r?nyCuAKKah4uBRMyfA$)IYom=Ci#JDzGBTrXtp^Z!Tt@zemO?=+n4oC@lOv<`auJn^dNx(kLp9Wx|vD2DvXw z3eHk!fs^<%RhXbcP-+;E$)}nMHa*wZmGk|kB*h}-&>IeXXo}tv1EE~JKG6pqIR^?J z4|Q4no>p2ohIO7PjI@?W;_qoA8m<NXx%MCBnwh`s=k%&yGOc*Rj84Pr(aoXceOCK`VdIfjXI&LhBt#@?jXU4q;fA-L zdNC!p)sa*``85q}f4?ik-%?JjLqt{&mh}I7yQUK! zQdDonc8LTZKYuq+A;u-U!GKtwn0nbA!YSWw6vT4{Ru{3lHHJ&aHQ}iQ~A|q%^;*62Nr>ylP9+*87|+h7i#g_ zUjMYO=eSM@Nq!uf?lbi0>ct@4-C%L$@y|MQn&L$OD zl|0Gdr%U0PTUG@AHH0sg#r=%x4eQ;1Q*4zmtX?)?&3h7-$w3EaXy`IIl`*LyDWG7;CaHMEis>h4t><11YHGW+WN8;Sj$$CH4YG|T^p4SJb9D3Hx zhn)Z6ySOpoynAbX->6l?_ZJ{ase`fs$EVe(GFn@VmhhApZ}!5M0`t*74YaD@a3Iw< z_|X!Y(X8aYE*@koo(*|h25t(}U%B9O`P;f`){pdlaDL&4Q!ymy0bcX|0XM_Qs%A|z zCEm3K_;2$Ar1CVzB{r2UqF1QyPm4|O(9T};!ODuiZmvst)ZoYOIc^r=$68Qu|AI;p zj;s7dPdyxJQ5$~8ui8Agy&AVOH`K1@hvRif919JHe))Uv$vM()rNEIypK%7u9X=Z< zJ4-(zgti|X$SwERXOBA`j8brxI#_?z())QZ~iw zV`Z`_i}wJRl6-(mebUMznZnOiBpKFr74rI0TLOLndp&>vH;Q+$BFTm-_gv18fElrp zSwi>bLwpt3>0e&V5 zX6rTmDL0yl%@*@xC2G|F{5Tg-PsRNghDs(1$4Na+%k8bi zJ+R00QEn)x`28272IhgE#JTb~R!s0AKm|2_jGRw*ax!ZCC7tLeRT$>&U7P4!^<(XX zUvH#kZnreAiTbMt{BTO4!6xC05K}99-9*G}(|%izWxego2a#*TmG|IzB@yprS3=1d z!gsE^N&@RYE81!+{1tmB9DBK|Tu-K7vtPvVnyV^M&#G#x1l6}@j(PG}R@`tSxXsCh z5m?o$5%r*dT5Pzwk!oMDaGQ4r(iaT*612 zLzRTvoIh)b{G(CHV8HqM?MOC;{%CVZ)NsoXC6vg1G^)(!{(O1C4K=@qxKG9Z+VFp% z*>K7Jzq_X6{UWcHH$B=@M_MaOzuwccFno)wx2>ywW)|~T32@T5DSn{*;No)1TpLHz z8^sM7OHdHRnX{Y@!a;F4r6|`+S@)lEBizI%$t2t&oqsxpE+(Tk1}*2`@|}hKSZeHW zR~|p6;(Fn1#a987&w0LbjAI`Eks;W*YrO>Z>Ty<5BKAo74MZeRuqO;0(J`z*KX-ym zYu^yVdL1$WM8uFVi>nt5Yfc+3znI&O(bM5Ifh!{Z!bCzA7oZKxf3-qzRCKf-$fLz- z>ElVzA(r^u1lTUDa|m_7Z=M{ZO#k(0mVM{t8-|>3^x`^D&5bc-NC5wse1Gt|M7*N& zDJy1><3FG_w*M;4Vv=4x4HgJ` zBmv7TzCd~WJwFbyUZlbx#a|V}p@1(D!KR}(el7fbsZY&r@`l_1V>W3?#F&c5FQrG? z_&A!-ITHrr9l>#jv=mV#X3HHN%z1qOUfys*On<4Z{JH-8-Eo~oDhRs~NB4R1bFybZ z+1?DIi~8~Rujs+^|0x2K@jz7IpFr3!=F3e&YebSw&&XPxY3kGuUP0VQ8FJWZQP}73 zG2@CFh#Tou7w)>n?)_~X_Se1np=8n`DE9GWdr8SUqDRcPUfwFR>QAGt;lB;eP}9On zhP_~r{*@}ZLL_aLQlSi3g4bepDAQl^g++j@{y|sZehNp%ra8f@Jr)VfJzx~V8f^H~ z8&@_RlvO79gxKE@bq=v0sPQ?AzZ0>W?}pZ3PP&)jHD3%Q7CnwBEmDll&&h)yuUF*= zz;LXx+%cB9QS^_u$Jz0J@(D&mU{hrF+BGw$OeyNh_hZFscV4x6^*!~8bDbD>70dJ= zj&fcGPHYrunp}rB0gX@QUPah!iISH%^Js*Sm?S;#4=nA6r74O1tM8ce6Rjsp*pN4u zzE-4vi`#py8;Hr9FiRE6ZY|VAz-<)9Q7)u8AdPe_y5w0glD?e|_EP8x>u$!vC|6%k zZzfP~%+o_`Aoy>2TUsJ6O2)!01|!2aoJCAoF;p7%=e-8slN?Mwr3ISAC0?sJS!22c ze!>|#Ui2rCRz6ODHkdZZ4?rgRVsSeV@-ZniFl-0}FSmMq=M=R0!ToqK=xj|whd$U! zOCnZmGbXux9dx+YT`$blP50$M(ef+CcKR*^OA+Y7js-P{yVQYM{whl}(~%w&Mglgv z9m;=ulr|W_pM5$Lgo2hl^wSc_cakEg@!8rh$PD4S7$YdHY)7Y`BE&+nt%I*rA0Y-t zI;WydA>COdPADDuM5GXWVG``z|0&EgQ7;{=4%W_3 zbrwW16XHy_P2|N-Os^6p=?%>Uw0gY(VT|%`0f?~Up0b1tyPO$Q$8a#`@?hhc--$E0 zr#?$270T(CH_>@04SuC`?ThiR5;`mn}2JKM%7(i#Qk07we=XW&Ue0 zi3L=3^^oH~qLay$?au*e>?ar-VS`<1u40?{lN0dLwdhck`43#*BTu77iqhy*ZkZvZ zIw>#Ad=qV$uY2wQ>3Hx;Zsg-qyY(ih+2@p4q}G4s!W$nD)6K_4a4$>KvXlX7jLLUk z)sE6{(k*nK#fTi?XA4-^(hy&iAvN&f?mu~gNc-@=01hAO)-s-6YAy*P*V5+7uP`Lg zXaKtP8G^gml@h4R_u=OtmZ)Id6NHTlWuM^C-HvZzpS)0})f<7`ATluRC@K(Mx5gS= z+7D(3{?hHj{S$A6VKLF{uBGB>ff} z77m{U`Keu~%792__?(8NePIjaq2;A9cHxSuWC`+NBVNBW83JXiT#b_Md~$y=z|V5~ zB<9i*`8Z3$Ksad}UM+8cZ1CIwqrea1H~-9RT6P1UAtx+e7!yFIi;7Fmd?SzR>-8H5CtA|pgUhJ@C@BrLqC7*CReQ8o^9Y-b62d1a-SRxKy6Pve*ZyV32%# z$rMj9;EZE_tj*`8SetT+mY9x6Ur`9p53C;NAn#WQ3Zc=@faqu=rD^%GJp9I_BbL(c zT=0t9O0pS|*oi5!e!FcmvNuh`@3!CzjK>0||HIG0mx%XRO5s8-cxMT^5C4Ya{X0d; JLh%m){~rwL(-{B& literal 0 HcmV?d00001 diff --git a/docs/learn/documentation/versioned/hdfs/consumer.md b/docs/learn/documentation/versioned/hadoop/consumer.md similarity index 99% rename from docs/learn/documentation/versioned/hdfs/consumer.md rename to docs/learn/documentation/versioned/hadoop/consumer.md index 41a1a51544..653dcf9dd7 100644 --- a/docs/learn/documentation/versioned/hdfs/consumer.md +++ b/docs/learn/documentation/versioned/hadoop/consumer.md @@ -107,4 +107,4 @@ For the list of all configs, check out the configuration table page [here](../jo ### More Information [HdfsSystemConsumer design doc](https://issues.apache.org/jira/secure/attachment/12827670/HDFSSystemConsumer.pdf) -## [Security »](../operations/security.html) \ No newline at end of file +## [Writing to HDFS »](./producer.html) \ No newline at end of file diff --git a/docs/learn/documentation/versioned/hadoop/overview.md b/docs/learn/documentation/versioned/hadoop/overview.md new file mode 100644 index 0000000000..0820127aa7 --- /dev/null +++ b/docs/learn/documentation/versioned/hadoop/overview.md @@ -0,0 +1,46 @@ +--- +layout: page +title: Batch Processing Overview +--- + + +Samza provides a unified data processing model for both stream and batch processing. The primary difference between batch and streaming is whether the input size is bounded or unbounded. Batch data sources are typically bounded (e.g. static files on HDFS), whereas streams are unbounded (e.g. a topic in Kafka). Under the hood, the same highly-efficient stream-processing engine handles both types. + +Unified Batch and Streaming + +### Unified API for Batch and Streaming + +Samza provides a single set of APIs for both batch and stream processing. This unified programming API makes it convenient for you to focus on the processing logic, without treating bounded and unbounded sources differently. Switching between batch and streaming only requires config change, e.g. [Kafka](../api/overview.html) to [HDFS](./consumer.html), instead of any code change. + +### Multi-stage Batch Pipeline + +Complex data pipelines usually consist multiple stages, with data shuffled (repartitioned) between stages to enable key-based operations such as windowing, aggregation, and join. Samza [high-level API](/startup/preview/index.html) provides an operator named `partitionBy` to create such multi-stage pipelines. Internally, Samza creates a physical stream, called an “intermediate stream”, based on the system configured as in `job.default.system`. Samza repartitions the output of the previous stage by sending it to the intermediate stream with the appropriate partition count and partition key. It then feeds it to the next stage of the pipeline. The lifecycle of intermediate streams is completely managed by Samza so from the user perspective the data shuffling is automatic. + +For a single-stage pipeline, dealing with bounded data sets is straightforward: the system consumer “knows” the end of a particular partition, and it will emit end-of-stream token once a partition is complete. Samza will shut down the container when all its input partitions are complete. + +For a multi-stage pipeline, however, things become tricky since intermediate streams are often physically unbounded data streams, e.g. Kafka, and the downstream stages don't know when to shut down since unbounded streams don't have an end. To solve this problem, Samza uses in-band end-of-stream control messages in the intermediate stream along with user data messages. The upstream stage broadcasts end-of-stream control messages to every partition of the intermediate stream, and the downstream stage will aggregate the end-of-stream messages for each partition. When one end-of-stream message has been received for every upstream task in a partition, the downstream stage will conclude that the partition has no more messages, and the task will shut down. For pipelines with more than 2 stages, the end-of-stream control messages will be propagated from the source to the last stage, and each stage will perform the end-of-stream aggregation and then shuts down. The following diagram shows the flow: + +Multi-stage Batch Processing + +### State and Fault-tolerance + +Samza’s [state management](../container/state-management.html) and [fault-tolerance](../container/checkpointing.html) apply the same to batch. You can use in-memory or RocksDb as your local state store which can be persisted by changelog streams. In case of any container failures, Samza will restart the container by reseeding the local store from changelog streams, and resume processing from the previous checkpoints. + +During a job restart, batch processing behaves completely different from streaming. In batch, it is expected to be a re-run and all the internal streams, including intermediate, checkpoint and changelog streams, need to be fresh. Since some systems only support retention-based stream cleanup, e.g. Kafka without deletion enabled, Samza creates a new set of internal streams for each job run. To achieve this, Samza internally generates a unique **run.id** to each job run. The **run.id** is appended to the physical names of the internal streams, which will be used in the job in each run. Samza also performs due diligence to delete/purge the streams from previous run. The cleanup happens when the job is restarted. + +## [Reading from HDFS »](./consumer.html) \ No newline at end of file diff --git a/docs/learn/documentation/versioned/hdfs/producer.md b/docs/learn/documentation/versioned/hadoop/producer.md similarity index 98% rename from docs/learn/documentation/versioned/hdfs/producer.md rename to docs/learn/documentation/versioned/hadoop/producer.md index a157cd8106..a94d5de25f 100644 --- a/docs/learn/documentation/versioned/hdfs/producer.md +++ b/docs/learn/documentation/versioned/hadoop/producer.md @@ -66,5 +66,3 @@ systems.hdfs-clickstream.producer.hdfs.write.batch.size.bytes=134217728 ``` The above configuration assumes a Metrics and Serde implemnetation has been properly configured against the `some-serde-impl` and `some-metrics-impl` labels somewhere else in the same `job.properties` file. Each of these properties has a reasonable default, so you can leave out the ones you don't need to customize for your job run. - -## [Reading from HDFS »](../hdfs/consumer.html) \ No newline at end of file diff --git a/docs/learn/documentation/versioned/index.html b/docs/learn/documentation/versioned/index.html index a710383ebb..e23b29f9d5 100644 --- a/docs/learn/documentation/versioned/index.html +++ b/docs/learn/documentation/versioned/index.html @@ -83,12 +83,15 @@

    YARN

  • Host Affinity & Yarn
  • Resource Localization
  • Yarn Security
  • -
  • Writing to HDFS
  • -
  • Reading from HDFS
  • - + + + +

    Hadoop

    + +

    Operations

    diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html index b6ae0d9e45..ef81887b5d 100644 --- a/docs/learn/documentation/versioned/jobs/configuration-table.html +++ b/docs/learn/documentation/versioned/jobs/configuration-table.html @@ -2170,7 +2170,7 @@

    Samza Configuration Reference

    - Writing to HDFS + Writing to HDFS @@ -2210,7 +2210,7 @@

    Samza Configuration Reference

    - Reading from HDFS + Reading from HDFS diff --git a/docs/learn/documentation/versioned/yarn/yarn-security.md b/docs/learn/documentation/versioned/yarn/yarn-security.md index 7b66ed8cf8..0aecefe372 100644 --- a/docs/learn/documentation/versioned/yarn/yarn-security.md +++ b/docs/learn/documentation/versioned/yarn/yarn-security.md @@ -91,4 +91,3 @@ yarn.token.renewal.interval.seconds=86400 {% endhighlight %} -## [Writing to HDFS »](../hdfs/producer.html) From 7887b6d868ee048767c11c2db3a5d38093d9abf9 Mon Sep 17 00:00:00 2001 From: Aditya Toomula Date: Tue, 12 Dec 2017 21:41:37 -0800 Subject: [PATCH 12/36] SAMZA-1502; Make AllSspToSingleTaskGrouper work with Yarn and ZK JobCoordinator Sending a fresh review as I lost the earlier diffs. This is the new approach that we discussed by adding the processor list in the config and passing it to grouper. Author: Aditya Toomula Reviewers: Yi Pan , Shanthoosh V Closes #383 from atoomula/samza --- .../AllSspToSingleTaskGrouperFactory.java | 50 ++++-- .../standalone/PassthroughJobCoordinator.java | 6 +- .../org/apache/samza/config/JobConfig.scala | 1 + .../samza/coordinator/JobModelManager.scala | 21 ++- .../stream/TestAllSspToSingleTaskGrouper.java | 125 ++++++++++++++ .../TestZkLocalApplicationRunner.java | 152 +++++++++++++++--- 6 files changed, 315 insertions(+), 40 deletions(-) create mode 100644 samza-core/src/test/java/org/apache/samza/container/grouper/stream/TestAllSspToSingleTaskGrouper.java diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouperFactory.java b/samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouperFactory.java index 2d2297713a..d3c5080011 100644 --- a/samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouperFactory.java +++ b/samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouperFactory.java @@ -19,35 +19,40 @@ package org.apache.samza.container.grouper.stream; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + import org.apache.samza.SamzaException; import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; import org.apache.samza.config.JobConfig; +import org.apache.samza.config.TaskConfigJava; import org.apache.samza.container.TaskName; import org.apache.samza.system.SystemStreamPartition; -import java.util.Collections; -import java.util.Map; -import java.util.Set; /** - * AllSspToSingleTaskGrouper, as the name suggests, assigns all partitions to be consumed by a single TaskInstance - * This is useful, in case of using load-balanced consumers like the new Kafka consumer, Samza doesn't control the - * partitions being consumed by a task. Hence, it is assumed that there is only 1 task that processes all messages, - * irrespective of which partition it belongs to. - * This also implies that container and tasks are synonymous when this grouper is used. Taskname(s) has to be globally - * unique within a given job. + * AllSspToSingleTaskGrouper creates TaskInstances equal to the number of containers and assigns all partitions to be + * consumed by each TaskInstance. This is useful, in case of using load-balanced consumers like the high-level Kafka + * consumer and Kinesis consumer, where Samza doesn't control the partitions being consumed by the task. * - * Note: This grouper does not take in broadcast streams yet. + * Note that this grouper does not take in broadcast streams yet. */ + class AllSspToSingleTaskGrouper implements SystemStreamPartitionGrouper { - private final int containerId; + private final List processorList; - public AllSspToSingleTaskGrouper(int containerId) { - this.containerId = containerId; + public AllSspToSingleTaskGrouper(List processorList) { + this.processorList = processorList; } @Override public Map> group(final Set ssps) { + Map> groupedMap = new HashMap<>(); + if (ssps == null) { throw new SamzaException("ssp set cannot be null!"); } @@ -55,15 +60,28 @@ public Map> group(final Set { + // Create a task name for each processor and assign all partitions to each task name. + final TaskName taskName = new TaskName(String.format("Task-%s", processor)); + groupedMap.put(taskName, ssps); + }); - return Collections.singletonMap(taskName, ssps); + return groupedMap; } } public class AllSspToSingleTaskGrouperFactory implements SystemStreamPartitionGrouperFactory { @Override public SystemStreamPartitionGrouper getSystemStreamPartitionGrouper(Config config) { - return new AllSspToSingleTaskGrouper(config.getInt(JobConfig.PROCESSOR_ID())); + if (!(new TaskConfigJava(config).getBroadcastSystemStreams().isEmpty())) { + throw new ConfigException("The job configured with AllSspToSingleTaskGrouper cannot have broadcast streams."); + } + + String processors = config.get(JobConfig.PROCESSOR_LIST()); + List processorList = Arrays.asList(processors.split(",")); + if (processorList.isEmpty()) { + throw new SamzaException("processor list cannot be empty!"); + } + return new AllSspToSingleTaskGrouper(processorList); } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java index 87a1cfa3fc..5147169dca 100644 --- a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java +++ b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java @@ -23,6 +23,7 @@ import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.JavaSystemConfig; +import org.apache.samza.config.JobConfig; import org.apache.samza.coordinator.JobCoordinator; import org.apache.samza.coordinator.JobModelManager; import org.apache.samza.job.model.JobModel; @@ -125,13 +126,16 @@ public JobModel getJobModel() { StreamMetadataCache streamMetadataCache = new StreamMetadataCache( Util.javaMapAsScalaMap(systemAdmins), 5000, SystemClock.instance()); + String containerId = Integer.toString(config.getInt(JobConfig.PROCESSOR_ID())); + /** TODO: Locality Manager seems to be required in JC for reading locality info and grouping tasks intelligently and also, in SamzaContainer for writing locality info to the coordinator stream. This closely couples together TaskNameGrouper with the LocalityManager! Hence, groupers should be a property of the jobcoordinator (job.coordinator.task.grouper, instead of task.systemstreampartition.grouper) */ - return JobModelManager.readJobModel(this.config, Collections.emptyMap(), null, streamMetadataCache, null); + return JobModelManager.readJobModel(this.config, Collections.emptyMap(), null, streamMetadataCache, + Collections.singletonList(containerId)); } @Override diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index 083dbaf124..de8391936a 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -79,6 +79,7 @@ object JobConfig { // Processor Config Constants val PROCESSOR_ID = "processor.id" + val PROCESSOR_LIST = "processor.list" implicit def Config2Job(config: Config) = new JobConfig(config) diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala index c2e06652d5..99b1abe40e 100644 --- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala +++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala @@ -24,7 +24,9 @@ import java.util import java.util.concurrent.atomic.AtomicReference import org.apache.samza.config.ClusterManagerConfig +import org.apache.samza.config.JobConfig import org.apache.samza.config.JobConfig.Config2Job +import org.apache.samza.config.MapConfig import org.apache.samza.config.SystemConfig.Config2System import org.apache.samza.config.TaskConfig.Config2Task import org.apache.samza.config.Config @@ -49,6 +51,7 @@ import org.apache.samza.util.Util import org.apache.samza.{Partition, PartitionChangeException, SamzaException} import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer /** * Helper companion object that is responsible for wiring up a JobModelManager @@ -104,7 +107,15 @@ object JobModelManager extends Logging { val streamMetadataCache = new StreamMetadataCache(systemAdmins = systemAdmins, cacheTTLms = 0) val previousChangelogPartitionMapping = changelogManager.readChangeLogPartitionMapping() - val jobModelManager = getJobModelManager(config, previousChangelogPartitionMapping, localityManager, streamMetadataCache, null) + + val processorList = new ListBuffer[String]() + val containerCount = new JobConfig(config).getContainerCount + for (i <- 0 until containerCount) { + processorList += i.toString + } + + val jobModelManager = getJobModelManager(config, previousChangelogPartitionMapping, localityManager, + streamMetadataCache, processorList.toList.asJava) val jobModel = jobModelManager.jobModel // Save the changelog mapping back to the ChangelogPartitionmanager // newChangelogPartitionMapping is the merging of all current task:changelog @@ -211,7 +222,13 @@ object JobModelManager extends Logging { containerIds: java.util.List[String]): JobModel = { // Do grouping to fetch TaskName to SSP mapping val allSystemStreamPartitions = getMatchedInputStreamPartitions(config, streamMetadataCache) - val grouper = getSystemStreamPartitionGrouper(config) + + // processor list is required by some of the groupers. So, let's pass them as part of the config. + // Copy the config and add the processor list to the config copy. + val configMap = new util.HashMap[String, String](config) + configMap.put(JobConfig.PROCESSOR_LIST, String.join(",", containerIds)) + val grouper = getSystemStreamPartitionGrouper(new MapConfig(configMap)) + val groups = grouper.group(allSystemStreamPartitions.asJava) info("SystemStreamPartitionGrouper %s has grouped the SystemStreamPartitions into %d tasks with the following taskNames: %s" format(grouper, groups.size(), groups.keySet())) diff --git a/samza-core/src/test/java/org/apache/samza/container/grouper/stream/TestAllSspToSingleTaskGrouper.java b/samza-core/src/test/java/org/apache/samza/container/grouper/stream/TestAllSspToSingleTaskGrouper.java new file mode 100644 index 0000000000..fa3b33f90b --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/container/grouper/stream/TestAllSspToSingleTaskGrouper.java @@ -0,0 +1,125 @@ +/* + * 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.samza.container.grouper.stream; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.samza.Partition; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.config.MapConfig; +import org.apache.samza.container.TaskName; +import org.apache.samza.system.SystemStreamPartition; +import org.junit.Test; + +import static org.junit.Assert.*; + + +public class TestAllSspToSingleTaskGrouper { + private SystemStreamPartition aa0 = new SystemStreamPartition("SystemA", "StreamA", new Partition(0)); + private SystemStreamPartition aa1 = new SystemStreamPartition("SystemA", "StreamA", new Partition(1)); + private SystemStreamPartition aa2 = new SystemStreamPartition("SystemA", "StreamA", new Partition(2)); + private SystemStreamPartition ab0 = new SystemStreamPartition("SystemA", "StreamB", new Partition(0)); + private AllSspToSingleTaskGrouperFactory grouperFactory = new AllSspToSingleTaskGrouperFactory(); + + @Test + public void testLocalStreamGroupedCorrectlyForYarn() { + HashSet allSSPs = new HashSet<>(); + HashMap configMap = new HashMap<>(); + + configMap.put("job.container.count", "2"); + configMap.put("processor.list", "0,1"); + + Config config = new MapConfig(configMap); + + SystemStreamPartitionGrouper grouper = grouperFactory.getSystemStreamPartitionGrouper(config); + + Collections.addAll(allSSPs, aa0, aa1, aa2, ab0); + Map> result = grouper.group(allSSPs); + Map> expectedResult = new HashMap<>(); + + HashSet partitions = new HashSet<>(); + partitions.add(aa0); + partitions.add(aa1); + partitions.add(aa2); + partitions.add(ab0); + expectedResult.put(new TaskName("Task-0"), partitions); + expectedResult.put(new TaskName("Task-1"), partitions); + + assertEquals(expectedResult, result); + } + + @Test + public void testLocalStreamGroupedCorrectlyForPassthru() { + HashSet allSSPs = new HashSet<>(); + HashMap configMap = new HashMap<>(); + + configMap.put("job.coordinator.factory", "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + configMap.put("processor.id", "1"); + configMap.put("processor.list", configMap.get("processor.id")); + + Config config = new MapConfig(configMap); + + SystemStreamPartitionGrouper grouper = grouperFactory.getSystemStreamPartitionGrouper(config); + + Collections.addAll(allSSPs, aa0, aa1, aa2, ab0); + Map> result = grouper.group(allSSPs); + Map> expectedResult = new HashMap<>(); + + HashSet partitions = new HashSet<>(); + partitions.add(aa0); + partitions.add(aa1); + partitions.add(aa2); + partitions.add(ab0); + expectedResult.put(new TaskName("Task-1"), partitions); + + assertEquals(expectedResult, result); + } + + @Test(expected = SamzaException.class) + public void testLocalStreamWithEmptySsps() { + HashSet allSSPs = new HashSet<>(); + HashMap configMap = new HashMap<>(); + + configMap.put("job.coordinator.factory", "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + configMap.put("processor.list", "1"); + Config config = new MapConfig(configMap); + + SystemStreamPartitionGrouper grouper = grouperFactory.getSystemStreamPartitionGrouper(config); + + grouper.group(allSSPs); + } + + @Test(expected = ConfigException.class) + public void testLocalStreamWithBroadcastStream() { + HashMap configMap = new HashMap<>(); + + configMap.put("task.broadcast.inputs", "test.stream#0"); + configMap.put("processor.list", "1"); + Config config = new MapConfig(configMap); + + grouperFactory.getSystemStreamPartitionGrouper(config); + } + +} \ No newline at end of file diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index eb087bb3d1..9c5dad5f54 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -40,8 +40,10 @@ import org.apache.samza.config.TaskConfig; import org.apache.samza.config.TaskConfigJava; import org.apache.samza.config.ZkConfig; +import org.apache.samza.container.TaskName; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.TaskModel; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.StreamGraph; @@ -97,6 +99,8 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne private String inputKafkaTopic; private String outputKafkaTopic; + private String inputSinglePartitionKafkaTopic; + private String outputSinglePartitionKafkaTopic; private ZkUtils zkUtils; private ApplicationConfig applicationConfig1; private ApplicationConfig applicationConfig2; @@ -113,7 +117,7 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne @Rule public final ExpectedException expectedException = ExpectedException.none(); -// @Override + // @Override public void setUp() { super.setUp(); String uniqueTestId = UUID.randomUUID().toString(); @@ -121,6 +125,8 @@ public void setUp() { testStreamAppId = String.format("test-app-id-%s", uniqueTestId); inputKafkaTopic = String.format("test-input-topic-%s", uniqueTestId); outputKafkaTopic = String.format("test-output-topic-%s", uniqueTestId); + inputSinglePartitionKafkaTopic = String.format("test-input-single-partition-topic-%s", uniqueTestId); + outputSinglePartitionKafkaTopic = String.format("test-output-single-partition-topic-%s", uniqueTestId); // Set up stream application config map with the given testStreamAppName, testStreamAppId and test kafka system // TODO: processorId should typically come up from a processorID generator as processor.id will be deprecated in 0.14.0+ @@ -147,15 +153,23 @@ public void setUp() { LOGGER.info("Creating kafka topic: {}.", kafkaTopic); TestUtils.createTopic(zkUtils(), kafkaTopic, 5, 1, servers(), new Properties()); } + for (String kafkaTopic : ImmutableList.of(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic)) { + LOGGER.info("Creating kafka topic: {}.", kafkaTopic); + TestUtils.createTopic(zkUtils(), kafkaTopic, 1, 1, servers(), new Properties()); + } } -// @Override + // @Override public void tearDown() { if (zookeeper().zookeeper().isRunning()) { for (String kafkaTopic : ImmutableList.of(inputKafkaTopic, outputKafkaTopic)) { LOGGER.info("Deleting kafka topic: {}.", kafkaTopic); AdminUtils.deleteTopic(zkUtils(), kafkaTopic); } + for (String kafkaTopic : ImmutableList.of(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic)) { + LOGGER.info("Deleting kafka topic: {}.", kafkaTopic); + AdminUtils.deleteTopic(zkUtils(), kafkaTopic); + } zkUtils.close(); super.tearDown(); } @@ -175,7 +189,7 @@ private void publishKafkaEvents(String topic, int numEvents, String streamProces } private Map buildStreamApplicationConfigMap(String systemName, String inputTopic, - String appName, String appId) { + String appName, String appId) { Map samzaContainerConfig = ImmutableMap.builder() .put(TaskConfig.INPUT_STREAMS(), inputTopic) .put(JobConfig.JOB_DEFAULT_SYSTEM(), systemName) @@ -197,24 +211,25 @@ private Map buildStreamApplicationConfigMap(String systemName, S return applicationConfig; } + /** + * sspGrouper is set to GroupBySystemStreamPartitionFactory. + * Run a stream application(streamApp1) consuming messages from input topic(effectively one container). + * + * In the callback triggered by streamApp1 after processing a message, bring up an another stream application(streamApp2). + * + * Assertions: + * A) JobModel generated before and after the addition of streamApp2 should be equal. + * B) Second stream application(streamApp2) should not join the group and process any message. + */ + //@Test public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNumTasks() throws InterruptedException { - /** - * sspGrouper is set to AllSspToSingleTaskGrouperFactory for this test case(All ssp's from input kafka topic are mapped to a single task). - * Run a stream application(streamApp1) consuming messages from input topic(effectively one container). - * - * In the callback triggered by streamApp1 after processing a message, bring up an another stream application(streamApp2). - * - * Assertions: - * A) JobModel generated before and after the addition of streamApp2 should be equal. - * B) Second stream application(streamApp2) should not join the group and process any message. - */ - // Set up kafka topics. - publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); + publishKafkaEvents(inputSinglePartitionKafkaTopic, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); // Configuration, verification variables - MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), "org.apache.samza.container.grouper.stream.AllSspToSingleTaskGrouperFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); + MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), + "org.apache.samza.container.grouper.stream.GroupBySystemStreamPartitionFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). final JobModel[] previousJobModel = new JobModel[1]; final String[] previousJobModelVersion = new String[1]; @@ -231,7 +246,8 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum // Set up stream app 2. CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS); LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(new MapConfig(applicationConfig2, testConfig)); - StreamApplication streamApp2 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null); + StreamApplication streamApp2 = new TestStreamApplication(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic, + processedMessagesLatch, null, null); // Callback handler for streamApp1. StreamApplicationCallback streamApplicationCallback = message -> { @@ -251,7 +267,8 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum // Set up stream app 1. LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(new MapConfig(applicationConfig1, testConfig)); - StreamApplication streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, null, streamApplicationCallback, kafkaEventsConsumedLatch); + StreamApplication streamApp1 = new TestStreamApplication(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic, + null, streamApplicationCallback, kafkaEventsConsumedLatch); localApplicationRunner1.run(streamApp1); kafkaEventsConsumedLatch.await(); @@ -268,6 +285,99 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum assertEquals(NUM_KAFKA_EVENTS, processedMessagesLatch.getCount()); } + /** + * sspGrouper is set to AllSspToSingleTaskGrouperFactory (All ssps from input kafka topic are mapped to a single task per container). + * AllSspToSingleTaskGrouperFactory should be used only with high-level consumers which do the partition management + * by themselves. Using the factory with the consumers that do not do the partition management will result in + * each processor/task consuming all the messages from all the partitions. + * Run a stream application(streamApp1) consuming messages from input topic(effectively one container). + * + * In the callback triggered by streamApp1 after processing a message, bring up an another stream application(streamApp2). + * + * Assertions: + * A) JobModel generated before and after the addition of streamApp2 should not be equal. + * B) Second stream application(streamApp2) should join the group and process all the messages. + */ + + //@Test + public void shouldUpdateJobModelWhenNewProcessorJoiningGroupUsingAllSspToSingleTaskGrouperFactory() throws InterruptedException { + // Set up kafka topics. + publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); + + // Configuration, verification variables + MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), "org.apache.samza.container.grouper.stream.AllSspToSingleTaskGrouperFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); + // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). + final JobModel[] previousJobModel = new JobModel[1]; + final String[] previousJobModelVersion = new String[1]; + AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); + final CountDownLatch secondProcessorRegistered = new CountDownLatch(1); + + zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { + // When streamApp2 with id: PROCESSOR_IDS[1] is registered, start processing message in streamApp1. + if (currentChilds.contains(PROCESSOR_IDS[1])) { + secondProcessorRegistered.countDown(); + } + }); + + // Set up streamApp2. + CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2); + LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(new MapConfig(applicationConfig2, testConfig)); + StreamApplication streamApp2 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null); + + // Callback handler for streamApp1. + StreamApplicationCallback streamApplicationCallback = message -> { + if (hasSecondProcessorJoined.compareAndSet(false, true)) { + previousJobModelVersion[0] = zkUtils.getJobModelVersion(); + previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); + localApplicationRunner2.run(streamApp2); + try { + // Wait for streamApp2 to register with zookeeper. + secondProcessorRegistered.await(); + } catch (InterruptedException e) { + } + } + }; + + // This is the latch for the messages received by streamApp1. Since streamApp1 is run first, it gets one event + // redelivered due to re-balancing done by Zk after the streamApp2 joins (See the callback above). + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2 + 1); + + // Set up stream app 1. + LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(new MapConfig(applicationConfig1, testConfig)); + StreamApplication streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, null, + streamApplicationCallback, kafkaEventsConsumedLatch); + localApplicationRunner1.run(streamApp1); + + kafkaEventsConsumedLatch.await(); + + String currentJobModelVersion = zkUtils.getJobModelVersion(); + JobModel updatedJobModel = zkUtils.getJobModel(currentJobModelVersion); + + // JobModelVersion check to verify that leader publishes new jobModel. + assertTrue(Integer.parseInt(previousJobModelVersion[0]) < Integer.parseInt(currentJobModelVersion)); + + // Job model before and after the addition of second stream processor should not be the same. + assertTrue(!previousJobModel[0].equals(updatedJobModel)); + + // Task names in the job model should be different but the set of partitions should be the same and each task name + // should be assigned to a different container. + assertEquals(previousJobModel[0].getContainers().get(PROCESSOR_IDS[0]).getTasks().size(), 1); + assertEquals(updatedJobModel.getContainers().get(PROCESSOR_IDS[0]).getTasks().size(), 1); + assertEquals(updatedJobModel.getContainers().get(PROCESSOR_IDS[1]).getTasks().size(), 1); + Map updatedTaskModelMap1 = updatedJobModel.getContainers().get(PROCESSOR_IDS[0]).getTasks(); + Map updatedTaskModelMap2 = updatedJobModel.getContainers().get(PROCESSOR_IDS[1]).getTasks(); + assertEquals(updatedTaskModelMap1.size(), 1); + assertEquals(updatedTaskModelMap2.size(), 1); + + TaskModel taskModel1 = updatedTaskModelMap1.values().stream().findFirst().get(); + TaskModel taskModel2 = updatedTaskModelMap2.values().stream().findFirst().get(); + assertEquals(taskModel1.getSystemStreamPartitions(), taskModel2.getSystemStreamPartitions()); + assertTrue(!taskModel1.getTaskName().getTaskName().equals(taskModel2.getTaskName().getTaskName())); + + // TODO: After SAMZA-1364 add assertion for localApplicationRunner2.status(streamApp) + processedMessagesLatch.await(); + } + //@Test public void shouldReElectLeaderWhenLeaderDies() throws InterruptedException { // Set up kafka topics. @@ -495,7 +605,7 @@ public String toString() { } static TestKafkaEvent fromString(String message) { - String[] messageComponents = message.split("|"); + String[] messageComponents = message.split("\\|"); return new TestKafkaEvent(messageComponents[0], messageComponents[1]); } } @@ -513,8 +623,8 @@ private static class TestStreamApplication implements StreamApplication { private final CountDownLatch kafkaEventsConsumedLatch; TestStreamApplication(String inputTopic, String outputTopic, - CountDownLatch processedMessagesLatch, - StreamApplicationCallback streamApplicationCallback, CountDownLatch kafkaEventsConsumedLatch) { + CountDownLatch processedMessagesLatch, + StreamApplicationCallback streamApplicationCallback, CountDownLatch kafkaEventsConsumedLatch) { this.inputTopic = inputTopic; this.outputTopic = outputTopic; this.processedMessagesLatch = processedMessagesLatch; From 2e5ceec19f52b300b98efb06b292cf9a3db60388 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Tue, 12 Dec 2017 21:47:38 -0800 Subject: [PATCH 13/36] SAMZA-1437; Added Eventhub producer and consumer docs Still need to add tutorials, and configs to configurations table vjagadish1989 for review Author: Daniel Chen <29577458+dxichen@users.noreply.github.com> Reviewers: Jagadish Closes #382 from dxichen/eventhub-docs --- .../azure/eventhub_send_methods.png | Bin 0 -> 52427 bytes .../versioned/azure/eventhubs.md | 197 ++++++++++++++++++ docs/learn/documentation/versioned/index.html | 6 + .../versioned/jobs/configuration-table.html | 87 ++++++++ 4 files changed, 290 insertions(+) create mode 100644 docs/learn/documentation/azure/eventhub_send_methods.png create mode 100644 docs/learn/documentation/versioned/azure/eventhubs.md diff --git a/docs/learn/documentation/azure/eventhub_send_methods.png b/docs/learn/documentation/azure/eventhub_send_methods.png new file mode 100644 index 0000000000000000000000000000000000000000..477553a23fe3bc3216a73b4ff709e0f16ee14b41 GIT binary patch literal 52427 zcmd43cT|(<*DkD%jua6D5e0<-DGCZI0s;a`?>!J&Fe;sZh;$Mhr3ffZ0RbUOCkYTr zKw2__iim)e1V{u#q&F!^AR*yXl^YklR9j}_NS^4xXb``&wB*WNMF+R})R zN0evJo;`df#`?B<_WXtc{`qnJ4t%3a5;oqm=l&iO{pEkh`{P@zVb8#`3c$mK|MY~ep5EXWv5Pg{kii!_eRk`+l~NIbb8=^6tjXM1K}aBw^hh!fE&6_BgALzcq+DW5Vm8dfq}_#uY52 z+kDBz;>K*XFJu2Mh;xK<({qIvl*$G{2Gcil1O_^4@d(h=Wq8i}n99At#(IfACYh){k$H zo$OP?LPEjL$;pssQ8tjTaS0(v1vlGC6${kuBqYC*h!uJgo1^YZ)5Cqi3h zob`X^1CAs7f#QeP7i417ZjQS4ZMbm0Z?<>HD-EuemyCUu&qqZ=PJVRf2fn^U6AvnH;s4@@&#BPEc@=iL~bUC_?@;292JMbE7c?EW99dYGdVpe#kimY z%3QwzJ?sBufSwGJBahHXlvP7NVaDwlyPPYC+JqdjN6uaF;PT9|2mE98X&=g~0z^Q4mZ1J4Ak3C(Ndl;*FxU_kVh z^g2~o9sxBkv;*SE*cgnh^|@&TxsxeK%I_uUXv{Gei%wTZzwVs=p&*_@+gj1!tRjDa z;SKL7#qWdwo$f=dSV`}XEax0~eZIY!{y3x`ryyG7ewB7NwR`vKdiGr1uvS6D3YUUp zv7dO@VEFl%SHMg!9hGCK$q(0Sn@ft6>P@*9_wG_Db3t-&t;ELMiA32k_R)4zuKB&d z0-sDNafG?)3^cA%R;(2OL^l7N>9ja}nY1>{F|3hDOx2gG_`1KJVb9Gd9pm(n|7J9D z@2?Gqx>{CcSYeAe%k1*^b!tCbS;9B{`YzRab950Q!J3~|;~Ag_Y+7!5Zjk1vjiSmM zBue#Lmr&I}!bZTo(Y8K=-3bo@EM#T(g${l9b+FwgMw4zD8}ipUGxL*<#7>uZPU!>G3$4-uIY<2gG^{gJfM}0DLzCZrGLZQ)a zw27zc<=&*+2?TC14rIFXVWZlxQS15L6@LE|lJ-U;_ya`>vt5-MIvzsz{vp|MX`{>P z$viE2YNHB&n$dZRp+CrGdc}D#oMv@Azs20fub@RBLV-mw3R*Ki`lr4POUcC+ALbkX zfO*4Nq2_HI)Q`~n+`Fqx>UuPNc`kEUO`opK?wZR^J!LC>(5RZQt($G{r0~iXn)!^j znS4vgDmF8%;Msehoddw#xt}YigjUvMwxevJV+_Wv2&mR2rIG|s&#qes4XWF=U8|3f zB*v-6m!tZdheCo^;g=xv7}0|B=LY-is$GwFe^Z}u5pZiUl}vAXNqQRd?Hu1-R>wheTUai1+N08SBuKu z`r47*?xopI-iQOy@A)!!U9lKqv!u~UvE@y+8_%UO$<`M7#A(wiB9C6GOf*e2u_z14 z2oy1t*q*hjA90|kkXb0%6m6KWgJvo1L;xD`gN`8wRgN9QEk5wuEyL!=GG7+4cO7QK z|LWv76TYPzuIdLGbBq+Q@i8ctDP; z2ePCdVSdXnXPHpM+bfgZni~3vdq-~X9ux%Z2&-C6N|ujRaRJLX2sUBB9`+zT*zSCg z7j7g0mC;Pt91-diL8WI5x9Mn{DLCGh<+Cme^Q!e)-@=~?OG*(BF>Q@G|Mb4^_GBl# z3d6JbsuR&*J$k4p*tZ&MYAy3swf`x?k!hb!tW|nJU2eZ<|0c5@iCCFltDSmpe$Css z(yIj@{=IrM%CDu$ym}sIQGJJ#$P1l=L78;SJe4EuIxJPxsS>s0(KRGMO8F4$5A8XFsp_ZF>az- z8v~WwJJKqBfo;l>McXBonO{ufaP~VaWXQu^nPaLtw})>J=-aPXOvqGBVxfrzsC0-p z{rHyEd0LrwPIl)_I@VzzzJUu}&ErA^cUxWEa9LXw^fL*&^4&>F;f+V5NC`tB3!Qq4NYhtEOx;aZJqtfb_60`lUE z8BV5PInNIsZ7wSY)8yNu4f<@|rV6WV*&w?ltmP*hMnBN%Hm0pzW3WWg+C0s>y8&b~<(}=~+1-#C zvF0CI9ii+SzOHP5t|c&7{oBK15{^OWpsX7p6Bt)DBKy-!2Gd%hf=rgSnErUns7KJX z0_&-jKrFEF6H@niUe9<(Zd_cENJ{vPf`#h(>g0L-$X;H>@UkB^UZjr~tNDn0pBih( zAgNeh8dm3~3Vwe6s4wZ-$yTl&Zt1QDn4%u|;OpfbZMCLhOA{wVu%SwoD->qQ_`~0^ z0$SqKPL@k&S_JHd1i0yaHgb8OEk0xk9n1RO^fmZML1)pMSr0cucbozE;I#AMWbtgt z^`5Q>#)MVx+|(KMS)+&S1;x}M6Om%|Q@fM0>VQsY7w&eo#Z%O$b7 zX~y&t#%Kt&1y$t)vU=LlxKaG)!Sl1jRXZj)v{NsStM7^;J~J z6OB+@oH%jvk=s$@esj53aMuvOd?B*U*#{S;vbBQ-Cj++Ik4{DpCBFT7O~CM|PXdU( zVtM4SM-8Ryyr#MySg!)bi|THOHn4rwBLlkPqFuKSdOoXLIVGx2b7lWK0P209%T zne^caFA52_%?cYTHscG}efU7h@b~6EyCUYkl0Z`ltL`hb`b%*k$>g$yzVGdY+3V=Z!t}O)vH_{Ce4D&`;J~gFpdP zKR?8r&%?4ML8Vm<1hOOW1Qmoyns+PZ#C*BPvuhoogY|Y`oyjaFJu0$x$+_`dt4M_2L2fjj>-h^m_1n>g z={&Sz*TpD=*caxwTsU$`8DDG;%@;WtXi>Gpoe#w4``w1_QRD1OPz(|!Cb=LgBB*}8 z#O2b)mu{UW;|ThbP;T_wuth(9Bc;|_X~u?vj#bYLfu%uaJJ_WjaX?>KD6nW7-#=uf zT0G4dR#JO2bovfi9usNk7ZnTY|Lm6EBI3*KrPzd2JL-_<+I3TzpVH(_yQSlYt=9L` zbcf3pHz2#4w-2-1OS%^yxh^tI7WX+(GF-7P4Vt4LGWOiE`vdb%)*;um6dU>G_`GyL zDf$s#h*aGbgWmy3{D>w?VM4kP#}2kP`xzoD&q^(_UjWmGoPhh@n#=xPyGnp{7n@CG z$j?s)j4}X0++{^1^m%DrHNc3us;Fl7XNz?oC7poz%k87}(xdw=nuktmVSQ{j7v*%s2&(55kQypV<*7$y-Fy+VxVajgUB~(z zemNsT#NB(o4mu%`_1o2imCMk~zecrkoVZKO1hdUni1l98&YLaLmWv>GrnT>LOiGb_ zsg^rV5Q;-pK>{C)IY5bRCC;UiZN3TO$wf(61b-Z;Tc(I?*qQF1FpCKuyx^Um7jNvR z8KsEUT1ZG6Uw@R>#HU1Ch_I8^2$K_``b-tEv>wVGtZk5{j~88w_H48R8)Zq}asp>M zrcV0Yt9DwtU8-K9LC$aYjir|>ewwShymv#$?i8`BJ<{kP@8YA!eU$iXWI#?xkPd}f zf2r5Qu4+0B5-YsN_d2+7(;Eug2nKEkWQP)1sKc60nHwd1DaaLpY#RXKvpY|9MA7!N z_n7i-*2DyK6g1kI@Qyg%E8x)OuFj24$gnt1e|kP{0M$7*ccMDu8tAf2673L3%sCIo zZ*jAj#{=9%WJ`BMR?0>XEVez4w-L6dY<8~(n02F{bwa(UGe(#MO?w#GaFH|c6zM(2q}ZcMzkdYw$4DQ;J}bYx%5>KsXbNBU@AQ*y{t zu^9e^O;s*GcR62|j3g(dYd3e?$0H=G@6I}r`iFXPh(qRQ_pa%s!$y3)c>P|-P~Y8u zrVUB|@a^SG`T5kAH*I06*Za3#>HwA+pW7rmO0Rm8Ew-E^6Yl*!3X&ZaI1+JdUA@wb zT;BuoU>0w#4Sc_PaKG@kH7JPk8{;jaXlk6-TY6 zARjgOy9nuYsGL_K&|1p3^Bn<@Y`olithLt1HE`D@Ot*@9=nX|M$TX^S?nCj%;iZPm zx8WO&vxxy8+th;5HBwIE<7Mi-+_Me(&G9X)lORKv(w1e`<3`7(=}EPtyicbUzSL?b zhO26-k1F_>k+w(HZTn_j=Nc?--H`Ha=$X8JfTTmOzo-rgS8f1lH8yP^kG~*Xe0N^# z_=|+|NWsmXY0C#lGyI+B83U<4=4aOmXeOwsh9?qL+EPx9V{pqAE`17pZRz&g`(5=A zHKn=g`aWL=N}aaSo;_o_tOb$Z!-fyozk8|(n>ap3e_O_hz_M~Y#8MI)2)L#{FoI)7 z1%xxs8=JAwy~aHc`=U=P;jCm+-z63{j*n-FeA@U1_c%2>Y?YD%37^j90Bh6u0{|oB z-bz27CZ)qcZOUl$Ao@C3=4BwBWm@mkZPEX}XV_yV)n%cmabk4*Ta9Y2bpJJs&1_i*BggwE6aEP7SL zv^qxigF)(3`&%6Tf?mVn4({#SScyU02!mV=IRFzOr5peh>5hZ~@Z{Qob6{MAzY6yv zWwrH4+YVVT=cybEzOQ7WK>)yt0=gNqHB|w*9q!sd<8pDR+t`&R=U9;59Tnzkvlm{x z62aK~boIxl;X<`M_p_}l`C$`DS-X=P{cWid1=)igwsrm;HT~`ow*-I(@pS<^s-B-# zk^siCPX!|3Tql)w=l)x;g0Q;%1R+*XcWP0m`xZnyyJV)T)X%x~caFl}c2HGW&*pGb zsCWh_Z_&>V?4q3j0+D7K-Fv)Td@m1ejafbeamWw5OO2BT1v8v26aS1)p#BbB=Kcea zSV20#91i~+LJ#byqLqY|!o&)A^86$T6Wg0ha$~8P!!bw^l2w)f>0#0EMD$(`J5!&S z?Vw^YTg_SUX1HR&BgQ>?++a&YnLmS)*RLcVrOp^#h4`kw<0l+t%~Uu7LU`KsbbXfs z_~6aCjoqAV!iMt3hv31Y#&Nbww(;Byp-s3lD%52&?@4w2homyLD6nZ7aF4<`_lQ65 z?CT)an7zQ+zU;Ikl-5+WD7_f>M#weK(v^%7KC#^{Z`DpTd`Rn#5qpwDkYsF-)LgET zXP(Stf0HCQMNCy@H)EQcy;)gvLt@BBhMyuZ28O-xbGg(QWMB7|3r8IjJc70Uo*}s7 zqewy+u%QOE@sycLpQO7J&Znk(VfnwY7aRMhlvX%N%@Jo^P zYaF)g0hS|V`;QMmd7V<2Wg*h;IVp@g_=o=H2VTxJWm?I&Ansoi&GIe+EHRe?hcxm4 z2)Mu8uK|7Mw%eOgKAu)0B)Agdg7cl-nYm#x_xT{7tqz5L$AHn}%Zz0vM=B~Fe19*v zi1|FS8H?ZDVHR~%hhNbM?Jd$$8onwD-^@5zq`h0iQ5SnV)in_TV0;16VF8lwVPMkh zU_hXI1ROZ(EZ_jVy#_wQ`S5EGe_(+1#{K~i|9kFZL#-h2jpe!bOf{q3@PK9R#R?t7 zrO4L#kpDEdFT&!tX)XOpWxJyJ&0iDef;Uw4k$LW7>f7;hh3J?K-TLjoKuJKo0JjXB zqi7tcl}*y@A9QIjN^Se$ya~Pq`<_@jT$BxgD*%ip7qI4^lkXv(!HB65#{Je~)cmJ) zbN&%=GH`MJvjtr)Nx&cM*#ju`-+S4^*%enihiU`%LCl3~Gx}`Ux2%j)6SbfIY`dr3 z)c71%WW9i-mJGvOinCEpQ!a(;fEa-H*TCG@Ij>(r0wnYRxE;Xv_pyQxI4{mY1M~fX z*$*}N*$8TIn+p)@&pH45Ml$=Mz>C9b|C*pZ>E}qCj{l`E;MM+a^sn!2VeKeb-Q9tT zsTO@tL=!D>ZU8+if*1~>pV&$tf+bh-ZX^>^JC~n_F|~3iClQ+$4VW*#sGbW5Khx~_ zrm%fgqtmRaRgoP*j4z#QO(vxN+7f5qnLLY*z1nJ7%@;DTRcR+o%Co1cj*(K~J70%r zB2jK+1fE^V%R?};4s@xNXFGj!UQs^{sK`G;u$0CKK0gLJD1+c<2l*vWcS}TRS3M@L zI>&VP#R$oLAwjB#jrUPoD*d*`l>5sJQxk7>cMFqnVvY40YA(lk7HbmS}P;?1|0%Pgng+9Y_%jH z?=s##xIgni^7S+i`mIg%+CKm8R;+1!81t`{n#&6(@md>6NwGxQ>9wInq9rvEWY5Ch z)Kn7_#B9-eo~x*_)t0Q5r|OmEB>yE0h;)h=wF!)Gf6jGSEGYV3yNIQqWKCp#u|4Qb ze%Po?ZuGc89F^hpz*|@kruqG=9xoxUeyrvmY$ol-@?aKo$+0o!T|4R)`hibpmOuH9 zTVf$Co5V)XOptEsWZa|l4fm?+jHj~14|CJ#*m%7MIGA$`O5@4D6jURvlB2ZFDt$1uWz~wC{W6trQv6?XH;vyv2gg z(Z0k+cV;T#(~PaGw!94Hl)CgtwOUjC6zG!lR@F3rqciCkG_%|kT&Y8i`}3{DKeq?C z6_wUt;W77$S8q_wriQS)l4Hpzr%;&ysQ+vKFkZhal$!4YD}+jYFe`|B8bdlvEsNia z*71vIm-(p#d!RDKc5y^~P@!CZ?X>6uI`PqEW+igXfgY8cmUSA#F6g}rkei?5bx|J! z&tMQ&YI`k+7A~apcE+@c-N$1NI5$>>(!P{a+b6{R9{#$ONgo7aaTzAro9Ao@mGTv2 zc2uxP{c%N%WMw2Kd1Y;AsZsDk&+q<%E~a1^SneA~dg}sv_--6Ap0uzN;@Spba>s#Y z3nv@NhEkOq#E9(NdtI^x@^uCab)HPE?~jV6VA8FKt;&LYXAJ+nlm_IFQ8Xe4)us{eDV=q1 z@gyum78F916Lj9aBF0^_!grV{^sj@F8589WJ+Bk>ADeU?j6Ns257W8e{T#r>|2720 zE=65Am!rumVFv%~qGAWi@KTZsDG;anJ4FvNwKNUP&&$N`W@U*N3rb zq9#(6|1q_ks027-&Mp6Gg#mNLF`==UKke^-KBYMyb3kv5CKXxzA72L8c_Nnr zh%yf1<1Fc?SNM;i|K-3q&W>aDIVS$UnM!~&+fQc=82JCGJK#zH>kW8PTqpP9RM?oX z8?7hMunZY~^Myj6j!q%22T@)4Q9?>_FEY8?xGHdxgfu&ds%*2K`Xs}X-7%H?)!V!z=BDFP2a^P z9x;}%dP4+v!ra>~6we(szoOOHXsmT(p zYW$Qdyjw+mb)UUsq?4q87tO%17$Ic(f=l4}7cI-~og_c$2_IcFRdmI_LlejE_pmEB z)@?UFQ=gA76Km?J2PNVIo~tV#Q!s&FIpj=m$qQM+YL$=_QqMt7R@X^(`7FC21}$=y zH&)Xb_LR{m;xQ-=zIfFh?nq#=eB55g_$9bkH7o|d46WerfHwMY3^`8$@%QtCh@q4p ztshyB=e5{N7dz|0r5olMqJ@FrSZc9dr-)&X6C)#tW^V0%ZAR(`2SQHn zMt5Zn8!!*;81HP5pSToOzkmlM--=niVbR?_9+Dc)PV-kC2rn{$*pc4C%kEk3-Mmu06 zQ1`Z>&G@^186_T}=lGzcJ_xDwMUY`@eoEL}n&ev3{1)kVN~*NsP0_==+OW+z@AYi_ z(bWmw#ewY&UOn#+Jb5+kx5H<+N@C4npYIBV z1OjCZ&0XwQa?pW1lD4ZRaKeB=brWkGKdW`=Ras%)N~;pviAkHqy`Fp(?i4j`3bPj9 znIg?)`yn5t#`HWe03=F|iUZwsLftKXl@9SV<@O_|-Kfm=&M_v#A0PWiTCp-7MgtSv?IY~JQOrDMhFM=d~FaOS4LLj;{Jg|#~}^stmdg4MIfSTAxfCO~=Z?2=X2GnB9in{{5GZhT$se;>}NEewwz$iNTn^#Di2WA00N01#xHd zt8abwUXKtFE)IW>Y3e^)X7zZZt%;Y1($m5!xI(6NKP}bIhG^+j-4$0`2wq#KWMSm( z{W`5)oJ3~JzvP61&Ax2dnkvG*EMNU>3dq~yp3Gl@-lyCCuiz^LJf=hzEB-%9Evo_csszaOhSN!Hs z4e7Vz;XcH!3vBqs(M{$h6B^Mvzic-etb+=~RmN(aBrP$Ac+n`V4?WE{2ma=Uds1vi zEz8Rac>sP<5b=I;GpF(l7&iv@b=}QJ6Ru0e6QARRlXZEB7Taxkl zK7XTVBoL6%(GDgK?=%IH_vVA1Y$^z>gi*chRNb#`a@eAIAQl9&ZBu+KF@uJFBtzEG z2rXmhfi<$_j0&amBle)a2_o#zu029;M>u|h9si`OqWP@{{50gd2X(f-i(Xv-f zHRW0kj_~@0TUAwm2ZC8zTtW(ZxcJm?d?8v8vth~HP2}_OG`_Isg7ovW-U=+@-tP4v z1AD=WC_{xxtf{3;$}x-F?+M5Kkf=~!%EQB2QdvgmkG@PH{2k|(SYnpn^E)FAEm=A* zp)Wz?JhzyFT{TH?(l^h zoPSXXejw{}IjF)O91>Gl+{atItZW9hz-~J0yckK&g$Y?Y$h@*4TD6~jb4X?^e(xf< zgybi9nzfYh+WEjpubULQA$Hj=4<-ik9*qC!%Y`Yfsfz`%%K9?*_Xk#Q*nvhTrhIM7 zC%XadiJG8pXY5G;qG~*Cw4NIk?OxuqphQX4Zkasv7unF=c20}9$~Y8{1i{k_5j6M9 ze>>Qk_U0)|&3E+p-15*NEzB{K;`rHx0_((=vZm|R>38C*tAWL@IImE?qRW8<@vI2m zK`%)q_5|nXaadCad*ahSjIVCCXqg&_FkgK#b-=k2)!mjB*#>T(qM*2s*>BB6wlns2 z#lF3Ev#{&VJ#!_K3^`a4%jDB#bFk2HlLr6i{w3}55&c2Ze`vVpC&*_sJvPbqm+-A4 z-&UR|P0Z~{^AiT+%&P06&coUzG*{_)+l}~k6y0KR;HK~ZfNzi|+K|3hTjKM|#Y(!En`cdk%)-9Mi8v7l=7Vdj zB5D%;_z#d(i-gOvJz*}mj8?u(+r~w~h>eWLK4R30m=zT0J`DVN+#Md@9)$&%5P+tb z2SfP`aDYq$2taJ#8P8(+|R8g;2J&rQ#7IqNJs0`*&c*=0gG@l6N zbD=5gC#2%~&HEUIZ|R^du0PIaAEK!G)`2X~NA8ALB6KFFd^blok$P?X`8I9<$@7zD zNP!>rQx|ryxF_9om4mBJ#8RWOu)Z0xCTLmX^>!3|9&g_U(Sx^P7uX;^71FuCM9Jrz zi09g!=*5vxoJo$o59zyA$ib7xtrj?uZW{#LX6)Vm@=hoj!*pj&T_2!%WZEqEp`s?? ze?p8|>cu@TC|L_ZzC4v6qU)_2Z* zz-ufK{6CN#!}K$)scucHI7X*-7}Pxn>r5zy3HAQv>MusMk(rGL5H0|X0B0M&C0~Mi zohuApKWj3y-<&;)np~7AB}0Uywr8>OVdvb$R*fjJeNevCDf%oOnz682hclX(OV@Pj zD+Lf038H|>7Y1DgEnl*!J#iPbc24H~aw3S`Iy8#?Z2b&1FXJL(+ui>0W@RU@ zpUMH~fuH183*;sjpeRz9O8KbfVOzb6DpwtauXUj$#3?7z5BY7O;tLiE?D#H{`g~GH zD&CjFRZosID!rIK_7j?N@PZ5wl_g=b(iDL7HL7go#L`D-1*&x6b#p4o^HpWZ7A=G( zKq?T$Q6>@||J?5{oW{u*{W1#xG6k46gI`47{}@q_!-)QepaVeRKdk{|!kO`ZdCtM7 zoMAZ6ekP=Tb+8R?Cj;aYdBV(gsz2%p)mgTJPjZ$+K|7j5`A z^#?#x&Wpou{lrF`z5dN-a`300aNRC~*_L@LLsZxItk~;+2D=XSJ0N~Qmn#*wmqUX7 z9OD15T@HK6&0$XgRB8VZOFDJuk6_4V##|eRiueaBgHB`YVNs4MNB`CM7xDS4&3)hu ze-8BDiRRzX{nuQ7cPDXxFJsLk+kk`Hv}(hHQd?ff0044sK5BLOGaL`#(9JEkuy_4^ zK9tt_$dF&C^&9gK1t<8RQ^a~fRH2UA<$F;RbGt(M5#qq$95DXt6waqltBoB!6@6pT z%k@<72oxqbJfY}()a?$*=@g=XgU_QaPc>O}J;%`-6snBJJjrhps&o0i~|-`b{I zLt@t62Zt=gDWIsB*7$ zB*H1igSuU;gt)j+*KxRits`FYCoZ*>Y1g<#`ZCACrt6t?eNQ-qX!&GVU;Xw7hv}`~ zshOQZ2JXyP7XoJA1UjZPtlj@@#Ic7^sYhW^1KmDX1sef2B3HlxlwRK_=Gpt{%S>Ti zUrJ+*JX&8@ccC#I?A_GmOSZOMHu_fPk+E0Ib5UkKBQW}OmSI$-zH>+Bp#F%o*=#`O z1%r&q>8O7Gn30<-uT*u}EGxf<%SJ?(eB{amU&3$5D>wC{T>%_ac=5E+F&bLc5+Fr{#EDVT{#TA>dDq{$X!8YqcNsSeHibXEf|zs}Y= zMA*~shuIx60wA60iZUgE#2k(Vd~9ciKLotcM0%(0sJhpm_9$Fxjmh&Y4n6grGIu2-Te4d%3zY*8$zeDz~C}>L`d?q z;B|LNYWkPluPe$wP9K2sxw|j4O5>b5u{<#Fbs{3+$rbHmfj_PMm;s_jG9OB2r>-aCn~+TJX4@xN7g>sy8_CZigw;mN_4o7+%)#wl*sT&!vBy{kg{U1@wm zw02*?z;=)HGZdQaJR5%_@oATP1y#@jcmAXz8ebpmNQmutPOT%mV)&y=>R*^->)H%! zwISREukDTwMJ-PR!fl`+$F{NzW4O~dy5tFdg>4#s@v6lmxc#4vsv};b{K=$Q=S;9r zG+a^uAqkx=3SKgVj@eH+rCaIoc_qFmey^;FJfTv+ouAnk10yF@L&7bF@^hl+cf83l zyO`2oPF444GKuJ4<=Tp$iNmcj!%_}we1GJ<5T=wG;8Kgyiv^|c-_LO*u_`C|a+y}u7t72oMtLMoQF0Czw!&VjRXKBg_JLuSZq!Tf|?BI+UP^QgW# z1RP>6_8!For)aK!2yfEjd58MIa1_3dF3mk7>=FoOmwt~Nb-K|`(RzKkCyyh^`hCtocyyp5;}L^zMm}WG#Ja#OJ(_A z-b#b23A~z5AzBp=z;`WQ~ z!Tng-qPrm2d0Sy70lD5l%E_-<#6c`8mwH76ostua1A%3De1ztnOPl!D#+oo|UKM8Y zp!e54`y6NcE#ZJ)c{#{tFct(eO&)7Rqe>e<)8K*f;kg%>LO1`mSckhZf08bB)!pu$ zK<3JP$HN(;-<2MN)HR*KL`9qtlb<;(zbLx!BGl3WN^8%h<4Y%J~)rf~r3HV({<_H`%A0Rf8? zMp*wr2~RmH3E_=Z{3^5Gc~)rzwEXPys8)LJZKInaJm_rKJnJr0CQ6+AT}##^1Hzpk za#^g_Uue-@9)lt|g*ZwT8Q%;(@l-LV*bGb*S5kuvVIGv@FI*oWgvOL1vhMimKw_8h zpz{j2aZfKGbx1C@RW)~_RKNut{g7kbKFf7UrQ^R*QW;=4OltT|2%5gt)+gY42iDtVukEae~DS1 zleYOB>Rh&vwf|b}VjN%z$YpLzX2$+VK_R|}Jxx)^WRl28l21%V?wz{@fGFcH`ymC* zO>#+#txE;(E}u;oh_ta3QmGtd``4afp(mDAd{c|fyNAS|#^f2i9*WG*CXE(3`#aoZ zq48_AYHkzQB42Dd#l1EJ6b1O0i(J%D=VQ-GxGx4O~WeB+5|>V zJ-Xg5e`Gl7W}hKwvo32?IX7iPiRG}UcKcmGo@^hIh7T`c2 zB^M6{Xs!m}_?j*oBcho=)y?;t!C44qAT3t3`!7>%$1^1x>?k#|-T6U7o$SD*;kAMPa4B znj6td(NJqseR08bu>)EUsJn}%k;#*;wB=4mrZo6fu{VLerr6+)vtec&Pf%tsXyw)I zl5YVp2}Aus*2>Ztt`Fv=$c@(J1hKw5E!Zuab#Cf;Rq`jjW2je3)GJynY>Fo!lk5BP z$j9-+k#6KmAnVLK1X)a^1Sp-TIvK%Ga;t~H3qlZ1NWNd8ZDc9d_km2wkSYiK_8mTV zq-}Fyd#%{m;X0ptwZ}tkRp|nm5uF53|65aXs;tj?AJh%^>TJRGu!K>!c`k@Z?M&NW z2Fk7jREWb&k$NA?r%h^MRL!qEYfVA)Vo+hU=b}1`K{bh`D4C4gppG_cCno0Zn9I#} zXV~nMzY}L}N*6g$H34|B1|h#t=tHiiz=FrpIN_%`ul-)f$vbdZnFgK($%76)WLRdQE=nqRn}f*P*h- zr2`+Pul@u_rpB`%*v}f5UjUJ_sDJ7>I6xQp`%i)sc>fEd{_hH5q%QqW#Vr53M*m&4 z3tUx-R@3@1SP>5fus))Wem47Sx8|6S+=*7p0TPcWdqzaJs-BXQiB+a;&iLX0A+E zmrO%mmcLrd+t7|N?Sr4&e#z0VujgBQ!p##PU{(Mv@lt%k@6^D4T7woKQ&I?2rQZ*| zHIdbGrp8UL)~$kndw*k}MzhG@s2`A#w?bZq`PrJW_S-GOG7VyjH1)KV1OJC}`F|$z zX{A5@+57)3E%-GXNnTvzT3pJW-QEXP?glKSQ#x1>sjKJb&)pBn#D zYyajP{!JqQYIhy1c>ypFAC~por2m&3zVMfE(24&(hYzg!R~r8x+TzzIfC2f1#{R?o zaMbQ6HNxot&;=Y7`?)aS6Zv#36`&rTWVyz`Hz@|m0^I+F3JN7)6&aKKMZYaBIA1Lf z>tnNuex~?6mdQ!37QZxzddIrbc(*#va+y_-bESl1~1esw{9x^tIB|+C_v(XOoULXgz7nG zjTl7qEB0T@jYd!LTkp#I4r}U{MKcCr&3jf4HEFWm3QYwsopWMtry_=YS*MdlI$58^ zVnOb|7f%BKI5HFKX`=BrkUEVaxH9!O0cr!eP0IxzN^k&<9 z_3Y5o98?;j4#X&=7g*Ex9ERcE%=3!|9<=xz<)~O}Xeox1#roa-=mH82N{Wv zD#YT-URZ_kF5;t>j!3wlKJ<%Gs{nq}l&xRJ5t0BHQ=AfJP}cG>I-yU|C9A22Q)AHm z0)YUSpl`g9eA`rtH|v>lueZ&spP;;Y(3;Za&ak~YWV?9y)FtVvFFr1mPBXM)C-d{7 z|D%;53IkdA znY(_m>MFPlWLH0Z^j80NT3yA&4FG97bt$J4>z6qNLa+Bxjue)Bdafb@Sh1)wtm$j% z+wU6EqvyW4i%^@xty*q0CP&BER8Lss;x$73qt}$q_G>F{zgOca^kQDPz6H>B?~GyWU*}5Io#HS=g3f^&Ec+y!%JK*FB&n7 zkcM>!HlHmh{z&!)$O$}AN%fQS0GdVF&UUVz?9wQeHlKY3B`?^WA6%7P4uv?EA}&hsAFqiTc`$fu|@=Tf@;A~o;~d%-Ts{hs+L zmwlBDi!YY8G=?ujaec`t6&?#>CRKqC$9aL$fyq8TzBN3NAn2{h&lgNbUgSiJ-N$yF zNKxlrQU4`c){Wi0{ShKDgv{3sAh3lqH4=?5)gWB;F?D@+b&|KtK207|&!h6ok$cdjpj`XP5XnXOoFtC?pKUYP|aNOptWP82kOY)RDv0_Ml#}luv(O zhAIIF?6{rJU>I%#w<(KWF$h$oX>3Q5ACNowiwS*$y2fG007^_sM#DF6qEY(OewV^x z9&GivNt*QQX-syj$)yg-FB<$$ecK=zD~;)MOqz*h2dtg-45(OhtfaQQReJI4Bg4rS zMR$SKP#5%sj854xxx5M;;)dkcAz-2osyPAVlGqsdP`i_{F7?9vQ;LVXG2jxrONbiIUr+>lQ3BfkQmWo&^Hw2A8~%O^`Lxs0&wG7;-9tX<7y1D z`fO)6hZm~XLuv8~ojCQ}!=%t@A;mGE(s`Z#r=He-7a9*Z63=Vo&XL#`)? zAL*`^FUhc?STa9f@HjPzB;+EF~9rgjuB8=wq9JZ<2} z$;}Lp%zkT(IbtA|7#KKPwk)I3^o%Lcp&Aa&{KQyz89Tl*L4{>RS7cO7z=x2s77k+| z83%aL1iYjcU}oFp|0Jp#7c4`Z7k}e^9{({uRd#*3TSka0ZdTr8P@7^4O{$0b`S13LSo5X04ScMrkzenDY5PE6Fq!Ry*@D+BjuY8&gC-YktuZ2d%)7dOs3rs za^O3WwV#QqcpLuVO~NtSt6St|4rLL}v<9ohI^9H@o&y{SO&&^&Yf@w+Yc(4Z)lERT zOPQxJ4mP}UpV!)bbQ0*Pc2s4Hsw8f-#4(KId;IzgMOyt*|DmZ86G7?g z>vSqZvhy7YEq?||Qa6qTZG$}2mQt}dy)#N_qt>1v#lc>pJUR1AhjR)Rb;*ZrxVST< z7!|mTAFJQsPRLwrvz72LDP;7>SIbU2&H5olxV?d&tbY?jWU)TqV6MkWzS#qi8oJF@ z?HZAvUWdasoYMI| z?EamSW~H+UrmH*SY;C=s{+1J6&Lfia;O%F}S$ScH#D=$BC@*qqJQ~O?w>hw0EtFQr z1L;!9PweetK?Q2&SO-cqfu4c_et=m*THI2_kfgS006wc)knlswPrtICAGglWMhcJM z|KM7ME}wzoxKoAsdKrI*D=D4iqfeW=X5EHvUa1myERyU1{)+=X^}}%>y$GC@ z393?adNXmY4Fv!yF;NV(gA+pf#exK1CWkLO4!lIlWM^1**SOzEnVT916ZOwL$>c(lOIbcT495)r)LUJG4sC{%C>zVD z1`N4s0|)(!+{kE;HL)$HY83omw7my3oa@&<%xFOn1kn>UdPMIMq6X2%L(f^5XSI5Gm@Nhe&zjt@3;QnTC>g?XP&ud-)-O5-q*G7 z=*(oz7A2WGHl!x(x$b+JrjPm804bb+%Dq)TuBNp>()vBS=ql1hz2)>cAVu;?yojb~ z&oQ-8{)=ys^YAQB)jPPO#o>A$>%>gsqaXX56IYy3!8268-(H;@<@GyzY4OL)RbRWR8bl> zP_dTS9U@PAao2I4wKJ;Vo@uU@t#lTGr=?WA^4WgF#8phPtzA`0e9o4IxlWU zajAR_sA!;2+3TQuDd;4Jit96>VeDgqZH%a!mNWv4fMdg7ki{ zO2ZYhCr}_;qa0Q;x1jta!;r@jJ*C9~&D>rG(_0TDvnAWE z{1XPJuh9d?=}wDXF8!jae7jXxPL@W7v^=UXu=Q)xNSdjuMa{rNE8m+`e~VClmks^a zK?D5y?-&b9t^>CGhZyXapaS>|8;1Q-be}4V{|d^m@fof}=@+3^at?9KJ~MRe`GzTa z#_3rl8RL(SRE@nPiLa8hIREI#Q%&-*eL5`+`9Bx#0LK0!O8ODG2*VZ56I!3>!t`f^A$`)c^Dy zAc^$vcKS$Qw(~v4N1j3Z6q=%KMNCaTL%Y+2&4!J-Yr9>{8FkQ`>2%wzb!_?hPzE!! z(f|Mtb6#PuEXqNo?lce$B!-^oe+D2T*bv#cohtAiU}fks8{yD3@+RF)|H>j};G&&O zKrSfoExA0<0J5LWCw2#BsAN1@OWS`0#_}~}`W9od4WlE~-@BK4CtA|m)~rnScY-RE zmW=CN>=k`$pig%gzZz^H+n2>Y7cr;x6DBy`m6Azk?DVxyNfAueCqNdDK-~xLT3;{g zxtggpD%6vEsgu-Y`uvCbORM3IX8ZPvK=S4x(zbeWq}Ih&@pbq458n%zAPEzmq)3Lo zL(gz>Msu^t8@iP+Im}e|9kU0zzy(ZJ)4&}jm5_bc1TzaLa<@F;dA^CTAk6R%MplG; zr0jcr;imm1-@7YZY5}PX*^ak)y-{V~ZK~)ItK?iKHsJv!+1PuD%Gwfa4jU#$du!}k z%PToNOt=H*BD$X{JsXnsHuu`P(m;#Q@kHhj2lf>DPamqTMTCBQZeLkaCv)yeX|RUT zPOVJm$$tHp$TO-BRq0BSW6cD8-?B1=_*E7&FE<+Hdmz`Ih}C%-n}e_D$FFINq2ef=L-8kqOxvKV(dW9qjp^##{*vO58L4#A&I3K2zNS8u+bpH*@vRU2SDx0rQ{s29 zw)o)8n$U(NZ?KEG&g4wO5^eT{F5Y8);$e~Uxezm>{{-=Bz$890)aWts)9o5gNc>29MgPszNl)UxwM`{bc!y@- zOt$RE#X)1XKPY(x=boS98Vx|uyxiF91rAVD(cW3XiPJJ)! zWDwG4Y)L}BJxyC-_vaoSaUJ>2Vb;X`2oEIKC>Th|-0$g>e%ToOBN};abE7#+Z;$`I z2%38mL6jaGMFY0zWl8c&lSMWpy>_31N!t3`9#%V}7jX^~m@lUfS6a;BdV05;q^ZZp zKe|tM%Lt`&hbH!PRyLZH3BL|v?7YzwRe5DR;I2XCS;p9P?MZu$<-DuUjXN6ovwa{8 zDLdb?bgE9aY{*O-Cij~yQ@`Uk$c-=E=T%4Sz3Y^e;_YaVOi=WzwqW#Mn{>hqzp7AJ z2Skb9Ptg8uKZu?Ox*qv1%5Yc<%M)u>(iYd{#O)M`|ow^_{&`yuTorX zdZBzhx4OU9)w;AlBJiZ8-z@!`hjsd5Tc@a}&cR7MYm=UP8lt${Lt)|gAvRx%bwLVD z=83M488(2sClJsZci)GZ1F0Sd-Dg*HUX*B!2i&=MTb;YRWdj_02}bL_s-ml2t+kyK z;f#pAL}ln9^s=--fzyzSyJ(`0Yt&u3w9?<}t-q}34b^0^Dur=p21?RC$3SyoUyUEJ zW_u02ycH^tm$_$bxg{+AAo82~gVPayVhit8NsTXf*e(Durj^%*$7MnsnE|!5SCswLb;~&V0 zs3;5M!}r`chEj~Ac?oIU9)~JUNT|H1;vEz(4uMY`ckp1w*Hzn}+2qrXcWTyhL1|$o7^34)hoSw*mtx3N+p4oGdky+6$4C*_pXu|oVe<|g> zRb@c6Oy@HWIs`njO;PN2%bJqdW3G<6t_EIa^Yot!`6WEun}sJJTfu~>-5ru6F4JFV z{KYR)uWT@4Duz45loC)i@vTvIwb1u8jFC?o#WZK=1)C7ga-W@j-8rhury0&V#0c9^ zl<3kTl2errMTFXUI`}1Em4E<4QuZU;E58FbcQ`xkx95BQ|59-F^6sp zW`;c6XpT3B#nn#@>Mi@1;oZwWrhNDHU8wj~{XK;2-d1+JgGkD_kw1rCf0%MBr(PrS zY*p%vt`+JH!%UzofA*jWnJ+!kn9q2Wk^9rk;TZXDHHZJ9*~LAi8MNyB1|8UG@`YqT zd>E;d#~`AD@kz86;Sb8ufI|BhEw$2%s8nuP%yij9=|@T*T<&&l5yx%G;Bx%fHH+`d zO_yHrF0}TH#e#70%jL37^v?JfT`P+0iiUE+QQ+@v7NZyPyP5?~oHXM-*%~G6Id`eJ54`<})*P}8K&h~H5Yr(vYvT8LOo^+mK{^)8Hj#DpsVzRnuO-u7IVr@7 znPmCIqrB3WL9b!j`PGXChc;S+cLOOM{2k*o>Q3C8;QjcT?3B=}Kq-#-W&!eR0?XG@ zpoWI~NgekA&BX`mcb-+cl8_;Pa-{1wVOah$q2rvQ(}>cb>ip`a6Km{eEe~Og8<%!W zI7;979N1#jY`$tzLi%OZ8D!qskE;t@`7)zE)$k6zSg{%nA}@!ong_j=EQf0^FSDny zNa?ujcQx#LF*EZhh^ZtUF_7~yL7uJPduM=9vM{nPy+JD+N-u2xtbJ@=-cQ zBKT~-_Nu^N_Q<`iP)Kr;IcvzUfOHqyv_kDu9bT%E;*CQ<`54yY&NvB=x)UQLG;Jj6 z)hiO=-xsUO)kDwfQTIVLnd{H=3l8XUdK9s14!&~2(S=6nK?XC<^TT=b7$<4SdFE{E=KP$-dZ6=viW@Ae-=4#e`tM3hOT`=z6A+5_}h9yCvz}?8N%b9YVM8dHYq9-TUr~;SQ-v{#$M08PDka{v-Ouwfmb#{CAZ7AHVY-ko}66r}qguz4wHc+CV^4ap-p# zkU<=B*Z;@0>yA93)!3?5ERBU@y|ChE9H#$WJ^NS1>u)#@^z5`0_Scti0oq7vgVJGt zPnfQc(cbaOMXcr6un8cb9l}lm8;|uS&#a0L*wFwn5@0(v0tLQ}yENfE2jV$_dua!OCBWVSo8oT&E1_rUhG=V>U8x48$#`v!lkXUakNl zRJBvR_NfM(2yaEvO6Si0xOG%0rQ(m~mRolz*aPPFCf~~~_wX6rwJ>dvHFd|J2BJmk z1byyr&4bZj4snWKfFhzkS$~cix9Oh890CRW4owqHQM6z`sBU<`&sMJ!pCSw1gQ^|F zbJu|+^6y|XkJ^e=%>6!a<4qD)iEIWP*?aQ|t5@Er>tJ)k7RGv|R*B}v+6O#J_n$|- zq-3NysRIPWbkB?@9h;65zm{ReoBM?rB#Hzvazk&%L$6+Uz8>vTcSD6z`r=H}3(>-> z>x=4xsT#L(XOgvXQS@o7ji(G;d{hvRRa}Q>c4l)KxK|${?pRwHEgEcl&xcSGT{A{O z6H*^X+3KV+odit1R&O7uIy5&nU@bJWYOq@Bfn4Wq_qGXumdeG{kLU64opC;db*-;Z939}>-vXiW~lf> zQvfwJ_e&aWE+8dotXo92>0}J8FtZxDVj0taWkNmP-ie(@Nmftw9CoGty3wNm0ue}^ zHfeQ5$x;ka+%Qmi4sFUq%$iut-fJkx0aVe)#%+>*QH6U&@0xx0Hq#H!-i~p`sXuO z4L#c{58J6i-*G-M&Gogw>IQ&23l5)SD}U{5fznEI9&vzcE%;F$g$Ax9F!&%)C494W zGc9zo6sSt=aqc_l;|4DDcq2Il5FRc4vK`EUhlMSh0EtV8%SH^x!;|FK6bHK-4|;zB zSNgdY%x*M-L*FZO8zY_Pc_yQc9k_F3sd`rm3-U<|3$hQ*(IZHp&`po_`goSEsnUMk zY~;R8)1cy)Cmdoa?zm zi@-qa{Q&6t7&toWwf%&Tf_wY7L4>v=mgSDuDH@ek(top;i&;ZWM6=mgBaOV{1d)}J zeq-zw8wTie*Kij4JV(6|C_#eXk}kpvpBp>4GFL~Ah&-3m|+U{KUj z@D4^J+L>;l2wZ#denAn*gXhLJx6gk_G)G9 zbvobK1YWq~Pv*HkqV?`<_As+Rgl%kM$zQ&(*nAiEhZ@ojW(2atJY zV(VYe3V`Fl&hW2Yl3Wn|N+>|(h3<>)FSX@k4mRGJe8O%JtUJbe)-z5-Kn8qlvpsazwZs}gjh9xe*^$m-wd3-Hdj$s{+6D`3Ozk?xfVlpBNvcPHpI}i;vnHX=%6T~| z(p17*aRXOc)mSXfficDYz6a$tfNB=zAyO|M--fOBQ%qQlRSjN1FUu#SNP~|eZe5*fG=QbwpmD$L#bANY z+UCdDSZv*Hbr}D(q5c4aOF9`?zOD50xF}cQZ58Oy3ql6D5}+dTjRTUqfCf@%s|79x zT%sfQN>w^gYhBV-+Is0DP-&?0#5kA|ZMe+khD{UxH6n$yq@+fs5<>;A-sqF&ap^QU zNw1SyNDCc%13LKuECX14;c)D#+zUYb=^|=nOlNTQf?7wh5k!JzoE;4)Y6V#O3tAmE zK8!ziRzd_c!2nhUAgX8G-YYVgSk=75-edusQrMO81YTt|$W76vu+gKrb>*(Hk;G0B zdtYq6%Iu+98?gBH_11=u2)lUGw-)+o7C4M24T02h!*hQl}YPR;;O_3v$gFT#Ml zZ^pyff&>u?hIsgkNsDqz{cB6L1`r8Vne*Tnr*Iw}1LGPw;3|@jrh$7e)Mv0*0w=fv z3I>v&8Pn2aQ#7IWvnM_&zcEy@diCOiV>SHKNO#emlX0v2Gvk;`#{94VddoT`W9;=J zDnT2^w{#w4?Nl-ba3}fbn4;+#fO}Pq7bO7==;MH;`xsbP!3#xnqsl)9*a?t{J|E1x z1gnX?R8`=(mS*f+g1*N6EFcr;u;Y_L`<5K7wAKP&r42JEWo@n`aJ5F5S|;XX%y%cF zuIs9WuLuRjk9dSV%SP23XxzFkG}%yqHnr7`!ed_K2o zIjE}K{_qSjK5a0|;4>aiqltHmmh3Edgerf??eaD8Bhg3J{x$LF8AWvGM629lYO%#g z$xOqj4?+yq;8|AeKiglv!u%>xIBnQvAbZ7Hs=3R{TMr`x2X#CsvGMS>cFW&=i63li zX{^N;Y)lRV6(T@Rpu0sWPtWUr1V3zRgupKQ3p-Z4E(uC@hH{ zk&Avy8WJ&jWb-wzP_X%(7QV*1iD#PGsG2|o4mRe?CIlB@-;l$6Ac=Gd2N4?Tx||iD!sdcl z15hD7gb4ewLKaV4B2j)j8H^03#m5bI6oQk%Ovz!`CgXs|5ZLD{aWCw*e*by;1D^75 zUF50*dR9@hoH z{$_r-qr$zQk-UwuuotJs0G42TMvwcDKRdR$0Q- z*litW%6LXQ+3$^Btp&zyB({Bm$iv~)Tm~iYB}S|fz-XP$ku^D_Uj{Wi&uW-+K3r+% z?F)ufUl&rZ7+xLzfCdM{bTr;Y0}DF1aNV~zWz!!);0e|bL%Q5vB{e9-e7AN&Jl`X=5ST?wfhs{@rElYZT%^sz?zb7pmm&Q8-03%BKE`gi-Z)9O1<00mxV4PoPBQNR*9CSGYKr{ zej~w;ew@2DyF*A3k>gOBF`ldZ+-&-`FPPPP3FrD57rAB=JtZEuj zqLjSk5DT(fb}3%>4%p6c%W8|ql-1i=Fj=)d$PK7^{XIz?g5Svak`)~PR^07gDS+O? zW}RoIR>(W3{j=D2mw&7dg)eV5d?1Asa#SU6HZ}1H{Ukvm;)1G?!gaA7DN=U2?^zFT z$i?OcOvpT%s3@D3PP>}IaM+nBO~MC1UPctPRW7GneCKr3QgAR8_ve~8t1(29`7xH0 zBSpP%L7oijF02_LZnYb7O8)N?kE;B`wM$O?42K`}zDUs8ku6zfz7secq&inhraC^G zZxOBVq4H57rXsy{+I{;M03yyjbfo^`=!jGFU{gq;&c)Ny^nK zY2FH=bnVM7Z(r17~%QVtepLw$AmgBczYS@Mw+4vEYV zR9AFwt#d|xuv@Dr3*cx}w$Iv|!K@s*Gryixp;Rh(fQ*e!G5{8~SXWVawg7*EdpGTCP?AF^|0(PwzbFSJQtp!K(m@a7la_;ew7Z;UFjEYg2~ zDzm~{X~2V<5(m?Omk4zac55uzeK9prw2yc zW!<96n=i7I3`#6y^Db~ZyJ&rvznN4Slp)e^IJvFv;Ba4vKcUUf_vWHbG;Q?S z3c?eBme1XqF$s!UAUGdfC;nV|HhwP^Gpxxf`ZCVmLD7>1bJpt!@jlvxB$ir;yTB|? zOUHMBcTT)0{qqh!dUIA<{ecroMLDsNwpRkU7TFm2n#GF3a@u=2HT?aFEqLo^yX!un z@_Y#L`nu2J>qnb0->Y~=!ipRjWy^pTnoJzU;p51rh#M-RzQ-dMe-*(@?CO?-=6 zHaR-k-?&-J8|TLPf-9*!AgxVf)uqog@M4TCl_|V>gC#7^=an>*^%x5!o+qb*g20&> zut#PR^h+-cgqLz@ISI#Aa1o9ibF)2;_mfZZ^qO05OZ`>s~%T&qmiDArA_jNVFcGdH-_V#43gRKT4#26d(#G|uR3YhEM zLB#N@-;W>@gt#*aIBXG$Ac5u^_pj=Z>Ka;*FhTIkS^7APrHDR)o4YtP>vb8_$;$Jn z(LoOl67jd>naD#7^mr{J#lP(`=4WxKE4_a}nZEmu4zBj>txWZ575JP8&Pj7wI#<5g zRK^!*)-ZRFgB{Af0$S69vGs61yvIHO?xOJrd~dz2uCtHetzJ$S!qbBvYfE&Ml`zkj z3*mIsZ`X+-Z%Y`P6F>q`8qw@NzX5J#F(GoZLAHzPN$yg7AA4Gf%0h{Uh$l5Yc!;Ez zqG>{H%$hHu2B|ncvvgdL!x!kldlT{c^W}~mSp|_S_My$h<*q5n8r4r22;4`ZKcR7h zI73rRcexAa!HQ|>8wgmu`}l&qMs&Khs~?cSR4qfh+#is^dStSTMPPpLh>#o)h<5;j3%K|#M z!3*o4A{Iz+sh28cP}2TW>tM;bsH^MbmPlgAszZwMA3gs% zlVm-Kit$ip<`aDE&N5Uyi(8lw5DwyCk;|IC%Rd0ZA83Qcc-RO2XS@VJ!vFrrZ`AQO zNQFaEIMjv1X1^iTU$}Nt-eI$IYx5=B;=jW2UEEG1-qELYAriphBbDBF8 zQ~aYZD#S2TD&t-7-_artll?aAU%@9goH8UAwJU95Xp#hg%)cYLLP94KQUEmKURZ$q z3tj$84*)cYz<2K-cF-Na@c%0ig~k#f0cUw~GGa!36L|?GOv`g*hVAGdqrW!;#my*q zQ|KwwJq6*|k8yY#i;{i~Ar^?sItHPzNJ|oCTA3Qy;4l=lALJN+W@=I)wXQ)?q~-+Y)q@2D#PI5qb#h>SC#gAJbMv6dgV6VS0yVTi@I zs6r}%;+h=7whFz%;-*GFij)A$rJkIC9{KqYeCCrS^rpz(v|akWQ9v0SrTFdD{1=Cz zuDxib!8@DB=oARdc&^zwMwj2JA%pa1kJLHZk=-593Xe>QR-WBrtaDc{Yzl zG-VsWsW1{0rh>^uGMn_IHo^OyElPJ0 z^LrysU8CAAcW&Pt-`rSqQr-pt!u0T-oze#{Wh>~1>HW*^?f@;|0;J$dhUTyn4WnG1tn=_lf9#QZgg!*IZMTJoPYTz;{FI7vbDs&{HGE+m0CzEW zsM1T4E}bGa0=0Xu32Ztb+rx$V?|N@{wtHqsO)QJp*gsXwUC0$VsJ`qwlOm%tX`ftS zHG!XEEr-brSj!HG!fX#_>ujisCoC{ZWxOR2z+{UUW9~-;$>w*b+-cYJZf07^DZzOP zdwc|V;L{AAfbNHB0RGUJViB{RKj;?zh_{l zUlgsaS=N{RS*WrGCH*h9F>2@q(|yhjIMh0!91A?zhN=cIE`J@$O~~(#NH{*ex6n=# zK>)|a2m>IP>)Q$ww^da7ETtKm0QU1$YqX=PQgD}^h1YF205)k^crCt}$KvKGbuJXS z0J~tHv|WJPpo86c1`B_c((d9qdK|3WOzTJ&QY^L9zrE+r+s}u-;oH0h?FNf;lu}13 z09ZuZ0y?yv^O?_i28buz**D~OcgZ6$6{Ugw_}unbHwNrMdPp4}*z&A&tEv*dR>oQB z@Q};5i2bq$z!Mj16F~xa%;%DG33ELlJF#=;A)#Q9_bNyyHdm?H3{POu;Nkr{K^F(^ z9=1wGEl5DG#7h%P-)GNnDUCe7A%_ohAQQdVDS4>IoRV;XmiULECDARzS2=+X!W+ew zE;#voA9TZI72Xp;WayQrP`!0=w;1Op*e$1jZbx^^;_J__%UIjK?IJ%1N>sh2iMC{TfMn%4LQON{@YurnvMq`I)WoYs zSP(A3ac8(*_g4rjM=yky5yNv<>q>=mSfkofH7bdkL+^nSsT;Tov|^>rSr1n0zVjmw zx+0|{gCtTrm?wW~GtNmq?^)r64tQqq-JtwEMEDsM+Xr zLstUV1gSyX*cMxr8D3voWOE%am;U;c zo$q}NetZKycdV_o)-v?LQ_Clt539>x$@hGd!5z@r3=^aB>~%X8J>oXyjiIKBDpN&pC+_HW1j7b z+c?pcj1bGn-T6dja*eA^6|8VJuP?h*%}n_Xz$)BTp>p;4L=i4b-5aTR1_nYV;(=Yd zsu>+W)Mg`0EvKoQyIQnLX@p;Oyay*fC0#Fls4rld?FZXUCU6R36+m7c726=muY*;E z2b)}s*?TKB>DcJl7HCPM7`aBvWWY{nPJAj>Foyt0^B(scv|Bb|w0S!ZG-d#-Eo z!9jHo?E2P|C`2Uikr+LB3zz8H+n+;60)g!U9VQTv(wj}83 zoTUr6NpRQNipTQmMM3}L%d^WG@Hni z!y~)x$EQb|!!FoDc;4C|YDlKUJJonnnuLY?hnmu0U^%0h@=bPWb_ezg9?F4n__M*s zGdSPENe9vFXiH5)k;#LilzBH7#tX8zmOE&#Wy{YcD!!DM2>01uh8u{W?*-N>6me!& zQs-~j)&a|hQrj}p;_!ew&0;;&9uDG>GEhF-^toe!vlc2JV3pyj0TC@XSItm@xBs+8 z^-N8jjUuvKl$|54>{{iN0YVpOUWRJ)(dJVuJW-&*fkxyxkO0ucb=HGMFBkS=(*_T9 z_3I?Ghzx z^1)sWed}X}4!F?ZUdY6@{B?s!O@$n`!HvOKG0WHD_cLpIdNci!*3`}UK7FA&RM#?N zX{{39pjhD%64@iL5uCUV{2N*L--8^|=x-V&ENB#V_wzZI7hG=cFICMmg!5jdI3s2| z(cdxTpybb6%e)-ZAGQ+{9lcd9!fJ8IIKX?T#=a`xugl=HN&(;YcKaTZsos<)qG;Ft z$u3qgmqzSybZPsPeV+jJbRx3DE>`$a41qxS_^lUZqAwDu67N2Z5C|~)-2I7VOFaI4S((%LK1(iR`o{hPQKEu1IdVg(q-J z3cv+9rJ^qME>?jPo>>ZF;ER!m4!jv6;8j8vhXgkQdu2|%afd)b7L`=x3(j9`m7R8< zdc;OWH(qVr70ZnDisG^f+V*>{-_?<_IHi2I(NQ3sy@TMHpRJ!R9=t>PElpqWu^9WqZdaBmY=XI-TTN=oSjJU`5>FM z_+xeZimS!4BX<7(mhF8?2y=^}T8AbJPXH;q<~R2I$}z(9AVMGjya1wfh^6p)R}?}I z#uxbUx?mRnaW>6*0;ZJA`wV(WR;e2trvo)V*Oc+W*|9~|9@$_Ih(64X(%5BfOP%)=_9()^nO`nOsP@y;bvz;Z)=rA%Y(A__c zAPx*(`}WnWHnj;m*IW>9=AWGZKpm$c>lKa>rYo4xaGl+sR?gghl z(og5Nr1)}=(q2|cRERc1<9=k70UX~_WBF|Lst_E+u`3bz9bf$Qn~4H8JYITbMYE^7U*hq{`)QKO~o%J-T1%2M|_{KoK>f^Ht3UwALS%$us+{d0`o=I719nwEom zIs+%c#NfkrZ`&w0%Fxn8BeEV*D#-vsjbI7O^U8YtL`qM7sHmI1K#FY;Y zjp7U=yN^d2WE^y!T6is!3XgaWKZV*n@{9v87g3=d8CN9RNnceFAcD%yKk~Rm;FBk? z^tC;#0)IcH{DNK&d$!<$t$gYpiN=5lM<{H)+o=oh$=2I*)5S~{BiA*L;LT2oPajcd z0jFa3EI-LElpMB7#dvUCqm)A&C;p^UggBP`OH5~}pp9D(`nuWT96!0jdb{!sx zM0bHG1*ZsZwwd*a-&3og{;csXB#&be??P1PHER1vyO2SuHC@h{W_Qyxj11RS+Qz|x z?*(QakY5u51VkfbBcCFWA*Q4Ey(0Zk6F(>5 z*nI|H;P7=0+mPOxm0v_$Aa#m7~PUl`I&(%!HR zxVCpM_xwDV8e&)b`RPp z0)QJto&g~~ieF5~HEWmu64Ffh0$(PBi-a%J(III^LqeH6XlX|>R1tLFdR0w2Na6(F zT6!uYTEde1-iAm`ZI>DI-#C~a3mluNv$?-Y0z_| zWaTauPEvx;)7*;kPR+`Q;z{K6kx%LvO=iELvD0Um`Zo`JN_M*V(<&J4&64~k8gY$I zocRY8YB@5IY>!LQVAnV>5Mv<3$!7-FV za7rO~n%~Bywhw9Ed0PRfrzUhJGr&FUWg^biC0?60=!8Juq4w_nVh2a->YC~5_#(xo zsSihKH)dD>_v+N-^rw5QM6jT5nxpmQ4wFyrDBkjeZU_+&!&IG4mbTO4p_F!(6p)6K zugC_`Vc;tPqc~<52>n?}VTEkfnS4oFj#nOPCjC&xM@CyfEZD>bgNz~Ap8WObB|?OJ z(qT``V>-LpA5!K|r}w24!H8>FdxqV*f>w|FX4&_i=@%Z1QiOH5jiFg zWqggOI;23L$@RJv z)5A)ZbZ2b*@Z*cx>EKBOk1kw{-1@@fJVesodwhb@(~Coovvqvou^HTQaLqi;B>hU_ zIg!(Hr}!?o=}hAFd#*MX0f(c`*RwwfwF(WM1b$e2K3^?Bd70sH7w4-7Bv~hm%v7-* zc+CzCs`m&WgDnA{hTh6+z>4UrnTY1?gvXBWaT93*Fq^{q~MA72h63s7B-?M@1B7gY5Cf zlC}S}pA09R%L7IRmP2`SmiPjJ-u+YHIWZ}RR+jFmk$V3s68Lsf$OB{6M5E;ykP{Q1 zttfv6ZBUdm^$7QqJ0m=B=2gUQqz6-pCimXb;GgYvnh&1%s`L|>T%r$wd5iVE!Gs8D zo0e4UgpQN>@r-@0S@uxh0qJ-muWjH)?7a-vg1N06#&Y#2mH-*0v}v%$tEmZB3_cW_DB zmljC9|MH&=C|-I&{+#vwy}^yH>61@8k?t0d1Ix)WY;x}Q_pG75A1r{2brLAIAM6}u zGbrZj>8i^*S3XDM0U0{8_E6&W{mhaq%H?oT?7l+0UZoo~GfV-p*V&VC=+R0i=Ebx- z4YkhWHi4*&4=6NPHL*Nx9vk^in9cKRj{5#R`x8UJ#cTM%NbHxpzkt*8vP)I1`?oM z?tn6+#X}IAJRyfQ>;bvN*e_v|ad$qfE~?e|vISOI568PR+^zX^`V4k3u!)7g{L~2W zy<>C+rkE6`yulqXGpc|F2lwt2RJdwX;)eT=L&j#~fKlCetn*tk0{Fd+KOXQ8+WNtxxg_Hz$tg4X#eXhIX&|JxfM3V1C~=yHV*^mOp%wiEifY%7~Oxc7k`vF07(J7 z{+;&0^$XjN=Uk`4jNr<^G{n%tkn$ zI29AlUQ`G^n6uhi;3}5Y2Sh@DI|BYD!@^GH=Y=>3J5fMHi7#L+n1U-0w3?kx52 z@#v&LuHU5-hawH_dijxsAjD!gL&63nJ=|cnI@Ob(J_QOc1ynfQdr<&!<372wKO?t7 z-9u3bMjd!5(yq<9jwtrM$F5TaL@)~2nU|wdM9aS}=p9;at&)ofNcwLs`@5il1fq!K z$q$o7B=?6RL98mrde*w%+1b$-P^}y)fDiov8u-Jc;^X6%du;^B-xM_%Z?{OD{qK4q ziAh6CR(pae?@3iZs&vH$!qu;ou)6PM+PC{BprLzgO&;Ha=+Wj%# z{4|GQ_F$FS-_iFdnddxK$8>NAa6Dni_@c`evp^c3M)>^{*6je2C2!?+tz>WWinhex zhDhRxTOIu*-o;o>*Um@K>Iq(=B?xeGX{gkX_qV9oYR7V0VXKC@@wci09ao&qc!C;4 zbTU8P8BlD!1@stM-`~({*ElFBv}Jnq3k7`e_Na+|G?^D3aLcyc>ZL?BdeGDOQ~Xsv z!p7}yF*an6vM4-mR;V=DzYGyX87{4;P&$NtG2qD@j!5=+MSv{R7UtW(p)}-Q(Gd+& z4RlENU`i*&Ogv8aPs+pvu9eM)43@p`qE`4jX0h(m8E*yfXCmJL#nwHfQ;AfZ|LVvr zJyxT@cpKA>V2|rili?_e#Sahxls5BN+N4>~W~debGWJr>ZPr43#?sXstYHdHZe)i*KJP&m^t7GvA@!JbzvU>)y zshTCw>42vuG*FhYjF-i;Ro=xh0t7Es#cVD|%fw@L`G~-MB+g&-GSUOwWeOX`k?4x)?l~OYqOHQ*EVZ1x=FZ_InsMiIkuD14s zK(p`G%o)-E5n^*cUfd$eh8c4Jh|LmgE>jtOuR^m$UKW6YGU|_8x=6bHH|4x_y#+T^ znTN5wQrCh^ygmQqU@4#{u}C2S{Hn25K5NnJusP$fUNG?6I1ItVmW8*~?t4}l;*9WC^&}Pb)XPhWu>9UyJdw{x8JTpY}gjTtJD>J+c1E`9A-nV)mj0fgOQgtXa zh?P+cqO%ECgWr6791axfqO4k_3(t0@Ul>} z-aPkMmM>eF@x2@rml!)!NDZjPw4cTKgp@Tpv{)nrG3a*!1m}SM@%&?UiTBdzdLoLs zv$R6oWVOIQGNtWx_8g($?JT}cL)Rd>p_dhuRrsa)>dj7BytxbHmc97c^A4*A*tikK zmqC~TdfcPVeN}TR(w`m5l4O&yqB3xemNYb$L02zIkI;DIv>ddhDyoZznzlS4xN)Xt0F zG<7dZBc39KEJ%20?Ucgv^zj6!N99=)@~(zo)IXIDi5n-4y4}rukW|aE+WmSSFgz#! zV~QB|?DrAJbD}ID0Hl?UStKT`V&?$akXDws`h=*hjnz^qAh&YMB7V%24D2H8nVEDU zh_hFsnc|@ho2^u{mMZ;N50JBPxPAY!#Us*0>lr?B)*HTff({i~Jda~&dZvumcq`vZ zg0AFq(i?+n9bRhiq3&IzmnN|X$RCXDRMz@7lh}!*f{5VYrzf}EDD7i3+=Bc*06fpf zVH^&u(|TxU8QJyc0PFQJB0A7#`yzEe94qPa5Id-2w zV@Y}<(|jVjEwUDvfYfGd!Rlw;6xf|dT|@X_l{}{*B)}7dfMk)L$tht;ClHt;uNahE z4sR0Fr6X5B5kojnrksCM6mha;$AAP&cI@B=?1jv~$9^~w4>6N?`BG54 zaQkqlGPu^=H7$VrXLjN46563;WNZ3@G*Q&;1}Pl}T^+Qlm0MxsBlp|@ceWMvLdA6r z913}l|Cc~o{oP-QP2Af0r82<^rLp7okHqXBA^0hr!G=Ho!y~^WDA)!2r%{|c1QbEQ zXTM?SpGJZFLZdI>OcrKo6uNrtPJjOtoB=xT-$?Nk3PuV z4fNM9|F^cVUgE!e`4?hcD_seV*$dsQMYOh<#hf~Z(+B|IWUMSxS$KgJK>R*kWfid#(o`9WM{eEMUNf*V!c!hw0S zb508UW=%OjZsYU=K49$>RN@)~-U1e$+VhXMSQXdlFW{!{_gicY0}#yslLB~W)RNAr z?FQHyieI+k`c-r~7ud1*{TAD$-#`DixBq4dw*KIMZOLCo0i*MuZ2G@u9(LmX*X99B z>i1|fnQm0}Dum9mH2W+cp4^@)+nb~)nfe*iuG6bA{&=DF#LVoN%EI|)6mQ^_f<2+* z!O<{1b4=R{rIQsY^^rz$*T#}Wv6D@vV>DV4xZ5nfe!FUZEJkVKO1kU%f?33_KRc$W zSb#EMUzEZ__}**>gAnT$T32DyqwTKiWRq3jjrE0cwotU9+iKY;#?|eoJ8-k879**K~Gv9YU^0LFgML#HEkb;_(mrw`~|Uz?P~ z^aJ5~E2#Bsy>^1c2HI5{99ivOuWR-7%_N({x;1RIG2F%b;irwdM+eqAaFVHZ#pcV? z2~P$!#!2dP;At)E_G2AX8mrMl_3S!mu>8)Zj#B(qK-OlR2d1#;g3!aqzI*A@I^%)Q z5~x}l3(Jz<2hTh<-eccuS-17b+DuNLe$dBigVyc0{2B{%$&=WZ?`$v73CeCu4A93C zUUX@;UF=43A_gMq7Q4Y5bCnz7RHQ=+{{JxP|9wpofk(2^YWKH|E(6=>be}=D%W`VtTlGQ8NSvz0n|R4n`I-1LD?nd@av`KJ^x1!!W+OX$FN zWe~DfHqq!{*XmKKS;FpZ%W2VOABem8o>yV(+kA3=c5%GX4SM0N zNwn&B6aD==NT~@|o#uxw)9A_-y{x%D?YUBB%Z+^F{)Xw2@D18G3$teIck~o$wd!VJ zP3dDGS<0Jzt)K3KoTSc;#-w%_(Z*mF$~N5C^a`R&jT>Bxo0O{V9;@8;o@h{5-6`!{ z9yPIaMgi9VtveCHRu8^=7DEGrQ06En$q9M*4OSAwet%&?pb10sM9eT65w>|fC7iT# zUwB%w`1qiGS-zO!s6F~3t-zW-);$9b2w{~k7sHHFFFUjFj9#jl#C7MfQ}1eFU;)Ov z_-l1ovd6J-QG#Z-NY9#0z&GZ~jiJ|p<3#XAi2hQMrowEGrSPGYireV%1sGepNQ^BZ=;ZKvg&b-Aq$!GgPSG%0tJk1}F& zcTVmQZ!N6NzN_d#LNg?%A-^2E>^iD=Jx6A4>Gh$`MzTPrqLQ`?oLw`65ARBo%!%PN za_!jA{pb^yEUF{R?`{<>Tv#hAQZZ1R;M-W*42<0jABG<3NZb=M7q>SgW{e(2C$-%D z(0n*8yKZza7Uq_cb|ds8Hi#uDHP*lAZogBfjF$!FGPU zs%6vlY7kpm2mTvCWe(O~+Rq4&uJrSlJ|0`%O8fQx?x;GwWP`7oeT8IcK+W(Cr{ehc zb*je7u>%OaAN{q}bX$S;QOYo;HKR9#l&cAo1q`lAAV=7)BI|KWj8?8PYXfFGHi^DN z;aB2(V z!LzYy{YIKAzHKV(XojRdTisnxBqC&m-6J{y_bu{ z0*iF>0-Q&&=>mNl~>b6bIpiJor!x3J-)+{x&31~tF zHl()*|E(|0c1P{?)Bl`xPW&e$7o+hEszbAMjJD)%`jX!fbV6l*!P+y&G|N3=IjC%Qeoi}-p5$66wN!$QN8mlbm zkT0Hjn8~%Mc3oml-of|S8RM?hEh?aSYlgTG7NYt~r zm7IOu^I*#)^NUV+t&!@}O!l?~_0U87U`>Sb1s=-+wWc3+-Y>Y*lfznf& zX&kP~EXaoqS(u=o_uUe+&4*sMq57gX`NWkF4HE z(=UcmL54Rso0&g5@Ql5^$!6cGc*Dkr>C20NzQXTe&k5eb_B26nAD9}!O4(~8t3OTd1v2J@Fr{e zJMGI;!}TXN-TX_xZ?c{8UN2AEwr?oCiP>QpI+x?&fbsAM^DZ{-QY@tQAJyLQ@dRgk zR#|?jcy=k=dwpu%`dw@0%q;WZ_aXgY!sY&&<>Fg66?ZPv+hryxnpeE_@!m1^j;F19 zQGL_fa=p~U!Hu;x;zcZe!BpMXzD*%|Fnpofpw>-$y76I(_24bL*3}8Y8XnK?+9Rty z9P@KI(*FAY+38=#zQq^BshXeaHp-0T|IhyZSXBpokVF0x{-yS%?pmTYc`to%%TiO{ zF@NJ>Kg(t_m?`j&ugnS)x^|fLwSZl<7wZ39jbU=-nK=YZ!gOMEgp!h_fs~I-VyVk|Mfdm7m4)lL1J@|UGCE_ zdqfK{r}&@WVGYM^^tTDxPY>SA@4u~740IVx`BtkPcttbAyy2Q5jPF{mGcydRH(vyL ziT)MlWy_6_{v9=&%KLBS8sI_v1)-(Qm$42_ChQL( z%Yg=s`SOedRkp4U#Bb-d@xHmqeEM=+6~NV0tVMiNS$FI=M%pmw2odEf=QOf$m-X!Kdp46E)d*ZxraOK0wPU8! zcoXlp9M#10_(#7g-vngCZIslPK1C;sZS~&^UE1}RdxnyG8?(RQ_(-? zfD^!B*qSBS^V%Cu87Nekz7;7ip+Y|ID^gq)Dc1feUXYPXAcDDtN+*;1MO}?1)=0Bb z6CWkn+9lGwJApINYG@Pja`9nfM7hb7cJuhHYoz_QzCNWN{P_ed~$tpJ}oY&<+ zv3!ac+bWbu<5u;2r+yeX6wnmh>3H6`r2f%h4*=c+Q&tej{3D_Sh$biHK=4;78 zv+nCWlD@Ttr#nK(depk@=o#xfY2>S8WPQKAB;jB3jV*6l=Zs^Xf-8&w3bfzONu;4+ zw0l%q8^y2Nla+d^RqyT4ca@KU`U<$(Ki^MWFA|pG6e`iRcLJ>ed2)&aq7W1nSjht4 zi1hWD<`z=R3-}O^5rA#M=Mumh^54;=J-3}wg_R6)V`!UZ%5kTbjADZh_hqylp&?94W6|t&(ODY+H^+xx(gf$wfP*HnzwA2y)MC?x?pmX zy;50wNv4B7pwrt)fUm6UbNpB6+as+6IR*cV?(c99-?}KVfyL3_v@WqxSA9#*%Uw%$ zy4XP4;f2uCTMX&!^6n)ca{df$m+EhK-HO0@BJr#3DK7{UZK3^R3mC_^@1 zvSwrU+$Q#N<^ve=piBF(>3%=<`Gf!V*Y|*QJ&j;Zly@USVZ&T%bCz<8sD0auOI>|r z1`ad+yTKKEAnblG-~|NZkpIm~VvPV5R&e_{c~BtV=DT;_wmqqNLonMrce*)r2$i$0 zm|GWEJvN>$62Q_zR&Qo827629SLA2U!nOgmjT;F`HM%`lMl;zw9`$D&^bnj<6w5c9 z2MEbQ=9UT9hJaY?nO|)}Loh!pkf|gMPNECNXAp?69gk=rlGjq?hG*$W1C=HE zC4pSD9vBo<4Am@5u$@{}T=3$k(x^puy~%%fi-g-E=Wt4%S{fk_oU$uWVK`!&xsaZP^%A0GM+iK`(u>2gUUA{n_)6~n{ zrhtl&fu&HedhBp#jQ42YMrx6Evc!fCX8Fp0eqP?vk6lEHg%tU*Pbda%2U{ zD@HEz^L#E_>TCk3ubniNEFgYIjEY<*tOFy8!Tj2xA2FL6KjNYapOd7?63u1gJ;<}?Dd`B2Skj8Ys;o`6_Aw7Nw2J(^uNtSTcqkI4`NlMa6^Yt6T2iRBI9colUXF>3m`l9BTBFEm(JziHvHi+M|h z)r%sJx#P!~ETXsrmnr-eF+1&oG#BAoK}4(r1-h2(X(?^Cgz%md$?=!?av3@mS*=8) zfMy$pR^ep}$~o~_dnPk~s!@pK2fT)fNIdutyz}TSBmUflNP(B!^Z z%|4p{?~ee$8op?I5Ihl%-}^f8iw6|ML98G`<@=4&>VQ(?EweYOmFrj{p8;sPjMUJ_Y(P{OdS7m0#Z)~wixMRsqTkgJu9^G7vqB1}mt1~7yz6UV>g#Q_ zR|Ev=TYFslL|x;SWE%rO@8IYF>t%mcHD9Ptdrlh!q#bjJVy@HWnqzg_QH+tSB_}8Z$ zSe4uXUoLo(xkjR9&)1osMm|3uGH8`~^2SouPq?OLRS;+tvIUEm>m8><_qZka-UM%=>tB4J=D+<$pN_D&zSK-J4oj~`70W#c zZtW5s&;4$o=3oyPUBi&uOmnG2v=2uNGonWxwO^SKscyTjmAwmVu$$$*drh@Av-q`W z)7flOsXJn&^UlAK9B#AG1bQMtpj|$R^jnr=IelMo^GA3&X%DR6Ug2T6UX|ohwGtxR zv|W-An%U`dG+DLkf_c^%wj?cunk~_dRRW%g@b%ut$SG8?OXIzFUJKu6XM);zDSaDW z+28vG7qN)sImmC^S4E{Cor*QDaUKuWG7qqs@5M+&o>(v#f5B#DWNJ>F>G1sA(?bX& z3{WqeXD2O5&O{*2KceY^B&}jWRZye_?~xWw{50PvDKp|3#5SfnFbh{BWw2e>DLIBy zW}N(ba)JEpI+JjMs=ajL#!8=EdC?nCWnz>ZD1&gBoI1(JH7xz~^-oII39GttQ#V7W z<#YZKo1J9{r2&kbm0rI?(=|y@XL~=iW__dC3MwFEX+U&m1QWvyx;p^HPa;tz*Tw zp~}LSb;U?5$-(}t5V^rd!h5d#euj4Ejet(Ii%GP)U(hNene7i#QoMQ!1<4I&w0y;$ z2b`ax`l*@!mycrrn-uSdJ|XG3>Zsmxy2h=&D8YHo{a~=?46%6_%g5;!H`;d!#dSsM zEWfxC1C9zJ@pos0{{DpG2Zi`CQIy&#z7X#ODb)GNlD>e%*~-XKiOUzrkV#zb2R-P6 z*qf9&LzlucGu#H)aFHY;jcHR!hXiIPvOihHN>REqZnPp(O`De7$#^rg>m*t)4~(9E zXrg+8F=UA~`Lv?I4L_-uhrLvnrN|9BIuLDgb}Uc^%f`vxvqyZ=Eg;vFx`oP{rv5nb zq$gk%!e$ivVG?MF#My5umm5VQDA11s^j3%}q7P+2pY&TlJ-H$90}w-HiY_T^M5|uu zya*&bXlAQAWrI``qf^&z*c97JSbb_YV}EW-+K?rgAUkNKh*lLl`I2&x_f&RQLPK=4 z)x=thNBGJ3az?9h1lB|p;+_&x5(%y-!|N=cduMs4d&VHhvmMqt4fU7QcJ<~AZ^glT z(fv&td{gj0jpP{Uwx?!;A~SSBy~vT?5>g<^zkZfe=~b`(HWZQi4oGg>Wtyg5Nzd4l zV_%4NJdRCe6i#|fW0h9AVPe=2FPk8$K8T~!eIH|vxe(uFt@xEPi*ymqW|0-`ETT|0 z)!r)Cq)B&C0du~wl2Hv6{0a0jrAOKHdO&KZ{`T`M;IX3o(|iY1LY$PRMtw=0K9Qh> z$68CK)DqV?i^;VBlCh-fXV2cNe0xci+{xsPf6>WehV>FZe(cbDi)@{YBdD#w7m zf^?C)k?8c8a(pMRug`@nj=ZE8Sq!OyB1%~dNxXwTIYOn{8q+V)$;~`L+|lmBSjYZQ zB`j*oqklgrR`;b?qHq`f&q&F~?nD}lk$i=!p)+SGgs*ZuR%W&#tYgqr=;PBl2&{-h zSk@U7)e9(9;F+>}N{o`epy7k(SzO*w99lU95psR(mtcX)gXo=ho*BeR8v^|cZ0 zgCVLeshbhsydb4`(S}ATcN&xWoZ1)uc22+1U$r+$RjmBm4iy{=u#{V0lahQP#9B)` zg*KP&-I_n1#BuG@R2<3qiRX{6>qL^qgR-NwLcg7iBMRv*4(21mQ;*x8D(C|ZTPg1J zG}~0H3w^F)V|*`U;A8UbOoryxY1-|S-)iqBtJ3UzG@skR8@$6nphY-wt4E!x5N0h4q%C5xOPzR!KF#AvIbT_>R z`u1cp6v?B2JVf1#T$X|kOF>VutJK!9A+&Qr?Fntzernu zK2)-c$9s8IXPC$Uc`Ru`uv=EpPwO+o6U<#zF~$Q-U{ZV59M{{LKe6;sL@7b#{gQjz z!V@Krybk>*q{vE%@%VZg`gKtx2SH|r5&IKPEKeehNo~p}z*trzCSbG!QNCX&YK&B1N-k=j_Z z>p*JVIi>DY_mr;79uN6xCAY{q49Ma(I;~CN<1SR@_U4I}N2`x@8^qFJ=%3iLD3D=a-NbDFf%3MP!~%Q+KPiW_&kS;=q>tT^hyr+ z@&UmWX%DbCF=iIU|NZ^9YOiF=R841vQ$3KQ6SD2{+A1t7xk-EwpG(Y??92u~eW|ur z!fe1p{78aF#eCw`eMe`+K~EdQqIo*UA~XqcKL9ron#TB?V&t?L)wW1Y4jYL|@mp`@ z&X6*2n$M`?sv6V014D(3s#q#Q=usAaX&}M+nI!7Cl4!!J2weG^R$*-GdSzm^Z{mP5 zV~W0$CRM7fIa@{|jD8JK{1~e72*%00N7GLY!ddA$hjMi&n`|FL`}L!acYJYPEr&DJDi~(4_ZF0?tr7T`Rx)xlZ`3w70^RGKEP+Z)a;%`(f;h z9?8!dOv{&@mKgS|GBsHwVU=;ZJguDC?XI1wbmA2mqedQZN9+nAHQ4zwu!?>XG`u?W z_OJ;hm{^Ps6U&ow=*V%SFAufwntqX(CAX!!E+}%=zLomT9*H+f%mhm2mnok&VJ}YI zkN~E$agXh`chTQpxJ*hK2v43KqYt1*s!9}`44zUAP#c|ap+HrZO}3gLe-+{LjvcaK zQhZjhE8oa!20IllZpguvVZGG7_FK63g$o?Vdf8?hXe742);UF-BSrTr&yAl+Q!M#Q zEQxNMlqT-hifBsi!jnK^njzRC_t8H}kbf#CgP($?Onvu7eWFadNT_n=c+i!ghPZXz zG|Y{ab0nZPhSh1rD$}Xnc*V!qsd>O*ZeIU!E(od((gkiVn&57cSkd0LdGz=-3B4zX zfE*?t)MNMO(_aL1d!c9Zo7#z@$x~hUQs3VUNJQ&e;S>DdT}F_IUY*YRr+%#yZaxqK zsFEtVJ8jMZ+cQ6L9q>=&N(WvJiYi%JYv)YK*;4DE(u9Ky$TXO4>uUr951Nt9n$$JC zF3YfW*IqQkn;V66}gIfv(|NcKNa>IoIo!*U+sxV za;g3vYib)%)Yv36*m-Z$`HjkQ(wuf81>2}Nal#N2J?oF8_}1~!8}V!HZDLBU0Pj?sZ43_zBz2Eqrm%nAY{!sYaYg5^L`wgSYP?1AMaEVv&6%m>cJ9kdxokvZGNG}NV z(jl{h*c7v#*l>b8GGej=wk}3Kak1sBLEGU+18f{`TDuC>aTrn!_kwG&`tdv zUtc_?J~Ju-$x4dVbE^|%;~g!QC-2gyG0}Z*`*y`_z|m}Bqg!ew2Mgp6!i+}($4&>0NOhrdVXpE4auwpb2pg7*K5eks7*~D7)yTRRN z{j5Rrhp?%X$n_KK-kMsGy>=dzWa@dUO^fRd<3SY$i{t%mp==AhaY$V$_RLol@rpgq zdP|;k(miO(aQ(Q-qjgTZyz!wX2q3Ct1wKG}7*xBxwHa52a2)lReBHDdD(~iU*vCql zVTtuVxt=-G*&|=eY}g>~na`sv$FPXpC3^+%^jYF2Yxpmm5#G6elttQ~ehk|^l|mAU z=36o}*EeG~nqCs`0IUkVJC* zDAf?O^i89uE#;iq=q@dfcy)8ea+7Aa@n*Ax`!U`kMf8dJ`s&)%*EUVgOi2Sf>v7XN zr)qt4QWNcgic36{D)X$CcbD((hh9m9ps-n_!WNJ<1s>BG;=}Hp+A_5TyH(pRN9BmM z=FOavhGoXAX_1(2Z>P64j+#`JEu@^^$+JlEet}Beu1Xyfh z>pKIF*zbOIJ9}7(c0q1B?qQU$^lHO+)3)UH_+xI=kF{clHO%)J`Y8!qxbrY%!-M4d zzWItvHQFL8>5$kgqTv^%M)r|;?a|>iwgW6)C@%Ixav6i@ zX17-wULSIFB5}_U*G3eVewYrX6DIlUApXXe1}&bGUkng7iO!najiydY;&YYUB3V#fK2z6Vu_`?h2advv+FVGG{6o~L}{C5)pV$Dw{~Ybo8{uEoBKtV z#6$(QteM0B%^ei%o;Ay!$(MX0m`g)r`2>>m%=247rOx~^7af^pZA5|2IEvCR63JKR z(QbapI&`eyVoh_+BLl0mQlVi*hxE~QHB#3HE}R&>o-d2FvVCE8t&=#pU{nc4id-SI zcxc8+F0P`!K_(aW5az37VL$K50%1rV6;c}fRqqa7Tpzbe(F8s{H=ehUWI*>=)wzbH zQX2I`maaUI2__T$E%u#QNDFv+aQ$GzhFOjR6)|yB&47~LiB$n>Pv#{t!@skv+4#zo zOqB0a(juRi7`(YPgwcQeJgP|0ds$oA(JFk!In(hfr&QwYW)bBCa`6NF=}*cjz#O4^ z-S0CJnZ;6YQVVN@4LHPMRVEVc3M`G>te!mhQF1hhgTR2~D2<*ioDuTQ++iM4XUPZ_ zdwSaO-UWVF$6J4k{ku_02)T6syo*?D(7?2N7R2_-5Ebp7Sk0sNIR|H}(|JdzYW%}Q zie|*?5kII0GT&&b)(=QWUiNpm z@31Zwt2AT`Ts={^@W$zDk`zYfB77~6I?2=DmwY*$%id~)B19R%1dpHVqg(!f7-ZhP ziPyDP0Io}9B>eNy@44D9v5IzLn4HfFPP&!xN0*5n=T5<_(&O&7XZqRt`-n=kYUN&3 zex**&@U?`v^#%R{&TuugYYZt~O`_d|y%6zUiEFxQ`n&oQJ2ays_eG3?sKGx#N1s)) z9+9Kiv@=U`H#GxAzN$6Zzg7FGB&+u{I1fw$V(-B18hi{)v$qfKPL34`>s2|=g=nuA*909oOeibn>$r!*`{wWRLSK{6Bhra|nsKMtsc}%Mq zF6IWa55B@g5H$bt>_Zsm|F0j}!&`|s>EDiJ$MQac;{e>hKWJ<>{@Ug+ z6BBMDYWU%TmSP{iCjA9jmKUK2Yfu-n^F0Ha5PEfFp^4q8%0l7 zZS_xvU9iA|Wumu;wzkVGY6}+YiiQFyd)!0z_$>fC9l_r6 z_g=EP*J>yCI@z@HEt}~mZ|`CIOdydh8hRya>GAK6lH?wAW&ixi1XfRbDhk!qOhf`n zt)2HQL8UVK*0$YX5s|xKw)zN5VE^z-dnoY#+;{6*PdL&5lB=g~9vE>;GFZ2NX!Eq+ z^n&R^A}NM~!Ifb`DpAH6slL6&ez?_}vT6j2Q zEe8{IsxfJ~j%bEmQ%F$tw%!EFM@gk+n4>|&^mdhl)Gye%SI}4&WLa9Zja2tVM}XU| z6Qu7oAHIY3O^I35TBC?S7AChY3i|)VzP9Q}TPbtH3TF^*uWfhrq zAOWqc8hT81JlvT|13gb7h@RC=NsScZQVRjB)%h*i>JgHJ^j zpy#AGJeuM=K_!Y=1{tmRu}55sjE&G!k$ZI4IExrhTq{E)zB(BX9wFUbV#ZgX4K$?< zHeSGLQYf_=fn{Fpb1`PhuCEULWC!sRI_K2>(AMDc<4>^OCSOEr90g8z?gN63hgwL& zA`TxXqwBnX&kP+|f-0jtYp(aaU*!iqitCFMo0&3-6zT2_syUdlfnUtt?1Kq^D&RPs zwD_v{>LIeb90Un+qCcOFtcJY|0DK(cG%!iV775el zu99FO3y8f<_e&@xXKZiN3Xng=jA#2-W&@VZujRgfWs%$@x5063{GqRwviU+lBRsKW zBAw9*IPpKB$s^`p3k%$_U^eVvVZkZnUeN_{o$Z?TfFmF#RRq4d9E6NN4#IGZIFKP7)NlQxW!!mAu)`)U<*wb) zk?jow*IER#@bP0pYqxB$HNbOR3_&Ql!`V)$7e zZtnBPRhM0;|2|AcqN|ITo^ql2PNiH=ChITJ99Afjtk@db%kMt5N%L!P%DrMWaU2!N zj17B258>+0OCu*xuHfm+)CQm77YGP`N1xh&k{=rT*T=ZP5%d#%=YpoX85H^6$DF|z zrI1{dxy)E<1Gu$guPYsDHXYBZnPzW0EOPGDzKpgSBqM`l@(qoC%iiU*x~rcpj6k4U zo13Ohn=zchxz(DXT1rszCZ$cJz3~Y60HsX;KlfSat9$tIXaR>Xu`(TG=0i;-xOMfJ zF&=CuhWKv3LG>N?@U&OmTeFg&!QX7ybEZYr>KzqE`7`dc$DyA)bw(|da07V;K^rd) zAA*)I-O4CukWxSMh2L%@79gvk)IAjz&%@K&nX!Z1Y|vLGLwYvD@0?bgppWYidgT6S z<30FoulE~$KS&{u2L%mRuEui)Pq1K}>fr;a-bQzeKmIcO!D&SX`kxmE8qL3Z&fW+d znE&#deoiOW104BrZn)@^de~?_3G_gFrCPxzHazXN71j^sg`Btz5y z0+N?Ub{0L=4x=y%z;LrbUybql@($VIX|HQqxrZ-KL+D%0%_Vk);c4GWSAKnwzD^DO z+^q8$y~Cq^*;B@G=;Alf?ZHC?i{=E*W4~P`nz@-F;X{UM?T>lXNpl8Fg@+;ZZDy># z*WJ4Cw7TV{yQC1s_?T6lXOx`!jBof$`xsjYmwfk}e92FBy*ba@DG2C;EHbFyr1)ox zfM;0Oc-eP)+GEh^$#$a#HbJkjuFgwCVxJ-OH&$$YtCh~tsI*%(4U?f*tWext(2&+8 zlcFtkz0o*J?$eO)qt(FG%*BiEO28q9f#$@9CHN?nJL4%qbXZq-KB<4j9cKiT8m6CJ za7aUM3R9}%E?Ekpb^8y!9W^H7%Er?e$Bl>-f+N;!-*2CbNNZim@veGnMGn2_u+~ko zVn{g^bxD0@$i{ZWqWkw_b#Q!OpjdLM8lxXpMQH6-7k_7Y2?2@4Jt1`LT%8sS414fh zf4b~588j~mPd4znLrawQFf_(d^pX|Hz%`v#L&aQ9ypAdm#MXX6!!Q8DpsFsj(%W0z zn%H|!iycDmI?n$XyUC!YF4Mq65>2>5dk7lyMdr_}TRx97s?it5Q;lMn`+KO{rCX8Uk9QGAw;NCiy1pK}D_Mm*$H5?;(h} zp&d-*#g~alF|O&{K|`TV@^Er!?D%Z$BDZ{rtwzTwgY@rHcUFyQ!O;LP*A|7j)i-|S zBl1gk=kEoA1`30(nYCUDjJl@2vHT&DyL*{3oD>>6ZJ{&1Zh6Af>Re=+uU$_Kd&@@% zl0xKO@#YrFtVL>YO|vCoYfQju3MYZa{c$x7O7?XTs%Lxx#a-R{N64B9lIZJ~42J zFd>jenRr1Un}KWrG!#WMb?D#5>^V}~?uHIoFI zSGQ>!iO2l?fP+8tVexqdYlf!LIG?uWX`3TuX+js$V*Hmm^E-5A`73&wAJmzdKoIqI z@-UdBzH9QfYUKkT%C^s#E!D=RIraJ&dCW`RuH4?9-cj`FK6MjBqS{30&mSDLb$DT5 VT7~Y*Gi&&u`W0>Eyi2IZ{|^|I@)ZC8 literal 0 HcmV?d00001 diff --git a/docs/learn/documentation/versioned/azure/eventhubs.md b/docs/learn/documentation/versioned/azure/eventhubs.md new file mode 100644 index 0000000000..2defddb9d2 --- /dev/null +++ b/docs/learn/documentation/versioned/azure/eventhubs.md @@ -0,0 +1,197 @@ +--- +layout: page +title: Connecting to Eventhubs +--- + + +You can configure your Samza jobs to process data from [Azure Eventhubs](https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-features), Microsoft's data streaming service. An `event hub` is similar to a Kafka topic and can have multiple partitions with producers and consumers. Each message produced or consumed from an event hub is an instance of [EventData](https://docs.microsoft.com/en-us/java/api/com.microsoft.azure.eventhubs._event_data). + +### Consuming from EventHubs: + +Samza's [EventHubSystemConsumer](https://github.com/apache/samza/blob/master/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java) wraps the EventData into an [EventHubIncomingMessageEnvelope](https://github.com/apache/samza/blob/master/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIncomingMessageEnvelope.java). The key of the message is set to the partition key of the EventData. The message is obtained from the EventData body. + +To configure Samza to configure from EventHub streams: + +``` +# define an event hub system factory with your identifier. eg: eh-system +systems.eh-system.samza.factory=org.apache.samza.system.eventhub.EventHubSystemFactory + +# define your streams +systems.eh-system.stream.list=input0, output0 + +# define required properties for your streams +systems.eh-system.streams.input0.eventhubs.namespace=YOUR-STREAM-NAMESPACE +systems.eh-system.streams.input0.eventhubs.entitypath=YOUR-ENTITY-NAME +systems.eh-system.streams.input0.eventhubs.sas.keyname=YOUR-SAS-KEY-NAME +systems.eh-system.streams.input0.eventhubs.sas.token=YOUR-SAS-KEY-TOKEN + +systems.eh-system.streams.output0.eventhubs.namespace=YOUR-STREAM-NAMESPACE +systems.eh-system.streams.output0.eventhubs.entitypath=YOUR-ENTITY-NAME +systems.eh-system.streams.output0.eventhubs.sas.keyname=YOUR-SAS-KEY-NAME +systems.eh-system.streams.output0.eventhubs.sas.token=YOUR-SAS-KEY-TOKEN +``` + +The tuple required to access the Eventhubs entity per stream must be provided, namely the fields `YOUR-STREAM-NAMESPACE`, `YOUR-ENTITY-NAME`, `YOUR-SAS-KEY-NAME`, `YOUR-SAS-KEY-TOKEN`. + +### Producing to EventHubs: + +Similarly, you can also configure your Samza job to write to EventHubs. +``` +OutgoingMessageEnvelope envelope = new OutgoingMessageEnvelope(new SystemStream("eh-system", "output0"), key, message); +collector.send(envelope); +``` + +Each [OutgoingMessageEnvelope](https://samza.apache.org/learn/documentation/latest/api/javadocs/org/apache/samza/system/OutgoingMessageEnvelope.html) is converted into an [EventData](https://docs.microsoft.com/en-us/java/api/com.microsoft.azure.eventhubs._event_data) instance whose body is set to the `message` in the envelope. Additionally, the `key` and the `produce timestamp` are set as properties in the EventData before sending it to EventHubs. + +### Advanced configuration: + +##### Producer partitioning: + +The `partition.method` property determines how outgoing messages are partitioned. Valid values for this config are `EVENT_HUB_HASHING`, `PARTITION_KEY_AS_PARTITION` or `ROUND_ROBIN`. + +`EVENT_HUB_HASHING`: By default, Samza computes the partition for an outgoing message based on the hash of its partition-key. This ensures that events with the same key are sent to the same partition. If this option is chosen, the partition key should be a string. If the partition key is not set, the key in the message is used for partitioning. + +`PARTITION_KEY_AS_PARTITION`: In this method, each message is sent to the partition specified by its partition key. This requires the partition key to be an integer. If the key is greater than the number of partitions, a modulo operation will be performed on the key. Similar to EVENT_HUB_HASHING, the key in the message is used if the partition key is not specified. + +`ROUND_ROBIN`: In this method, outgoing messages are distributed in a round-robin across all partitions. The key and the partition key in the message are ignored. + +![diagram-medium](/img/{{site.version}}/learn/documentation/azure/eventhub_send_methods.png) + +``` +systems.eh-system.partition.method = EVENT_HUB_HASHING +``` + +##### Consumer groups: + +Eventhub supports a notion of [consumer groups](https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-features#consumer-groups) which enable multiple applications have their own view of the event stream. Each event hub stream has a pre-defined consumer group named `$Default`. You can define your own consumer group for your job and configure a `eventhubs.consumer.group` + +``` +systems.eh-system.streams.eh-input0.eventhubs.consumer.group = my-group +``` + +##### Serde: + +By default, the messages from EventHubs are sent and received as byte arrays. You can configure a serializer and deserializer for your message by setting a value for `msg.serde` for your stream. + +``` +streams.input0.samza.msg.serde = json +streams.output0.samza.msg.serde = json +``` + +##### Consumer buffer size: + +When the consumer reads a message from event hubs, it appends them to a shared producer-consumer buffer corresponding to its partition. This config determines the per-partition queue size. Setting a higher value for this config typically achieves a higher throughput at the expense of increased on-heap memory. + +``` +systems.eh-system.eventhubs.receive.queue.size = 10 +``` + +For the list of all configs, check out the configuration table page [here](../jobs/configuration-table.html) + +### Azure Eventhubs Hello-Samza Example + +The [hello-samza](https://github.com/apache/samza-hello-samza) project contains an example of a high level job that consumes and produces to Eventhub using the Zookeeper deployment model. + +#### Get the Code + +Let's get started by cloning the hello-samza project + +``` +git clone https://git.apache.org/samza-hello-samza.git hello-samza +cd hello-samza +git checkout latest +``` + +The project comes up with numerous examples and for this tutorial, we will pick the Azure application. + +#### Setting up the Deployment Environment + +For our Azure application, we require [ZooKeeper](http://zookeeper.apache.org/). The hello-samza project comes with a script called "grid" to help with the environment setup + +``` +./bin/grid standalone +``` + +This command will download, install, and start ZooKeeper and Kafka. It will also check out the latest version of Samza and build it. All package files will be put in a sub-directory called "deploy" inside hello-samza's root folder. + +If you get a complaint that JAVA_HOME is not set, then you'll need to set it to the path where Java is installed on your system. + + +#### Configuring the Azure application + +Here are the configs you must set before building the project. Configure these in the `src/main/config/azure-application-local-runner.properties` file. + +``` +# Add your EventHubs input stream credentials here +systems.eventhubs.streams.input-stream.eventhubs.namespace=YOUR-STREAM-NAMESPACE +systems.eventhubs.streams.input-stream.eventhubs.entitypath=YOUR-ENTITY-NAME +systems.eventhubs.streams.input-stream.eventhubs.sas.keyname=YOUR-SAS-KEY-NAME +systems.eventhubs.streams.input-stream.eventhubs.sas.token=YOUR-SAS-KEY-TOKEN + +# Add your EventHubs output stream credentials here +systems.eventhubs.streams.output-stream.eventhubs.namespace=YOUR-STREAM-NAMESPACE +systems.eventhubs.streams.output-stream.eventhubs.entitypath=YOUR-ENTITY-NAME +systems.eventhubs.streams.output-stream.eventhubs.sas.keyname=YOUR-SAS-KEY-NAME +systems.eventhubs.streams.output-stream.eventhubs.sas.token=YOUR-SAS-KEY-TOKEN +``` + +Optionally, you may also use the Azure Checkpoint Manager. Otherwise, comment out both these lines. + +``` +# Azure Table Checkpoint Manager +task.checkpoint.factory=org.apache.samza.checkpoint.azure.AzureCheckpointManagerFactory +azure.storage.connect=YOUR-STORAGE-ACCOUNT-CONNECTION-STRING +``` + +#### Building the Hello Samza Project + +With the environment setup complete, let us move on to building the hello-samza project. Execute the following commands: + +``` +mvn clean package +mkdir -p deploy/samza +tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza +``` + +We are now all set to deploy the application locally. + +#### Running the Azure application + +In order to run the application, we will use the *run-azure-application* script. + +``` +./deploy/samza/bin/run-azure-application.sh +``` + +The above command executes the helper script which invokes the *AzureZKLocalApplication* main class, which starts the *AzureApplication*. This application filters out the messages consumed without keys, prints them out and send them the configured output stream. + +The messages consumed should be printed in the following format: +``` +Sending: +Received Key: +Received Message: +``` + +#### Shutdown + +This application can be shutdown by terminating the *run-azure-application* script. +We can use the *grid* script to tear down the local environment ([Kafka](http://kafka.apache.org/) and [Zookeeper](http://zookeeper.apache.org/)). + +``` +bin/grid stop all +``` \ No newline at end of file diff --git a/docs/learn/documentation/versioned/index.html b/docs/learn/documentation/versioned/index.html index e23b29f9d5..fa8e1e3e4a 100644 --- a/docs/learn/documentation/versioned/index.html +++ b/docs/learn/documentation/versioned/index.html @@ -94,6 +94,12 @@

    Hadoop

    +

    Azure

    + + +

    Operations

      diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html index ef81887b5d..6666bb1943 100644 --- a/docs/learn/documentation/versioned/jobs/configuration-table.html +++ b/docs/learn/documentation/versioned/jobs/configuration-table.html @@ -2248,6 +2248,93 @@

      Samza Configuration Reference

      Staging directory for storing partition description. By default (if not set by users) the value is inherited from "yarn.job.staging.directory" internally. The default value is typically good enough unless you want explicitly use a separate location. + + + + Using EventHubs for input and output streams
      + + (This section applies if you have set + systems.*.samza.factory + = org.apache.samza.system.eventhub.EventHubSystemFactory) + + + + + + systems.system-name.
      stream.list + + List of Samza stream-ids used for the Eventhub system + + + + systems.system-name.
      streams.stream-id.
      eventhubs.namespace + + Namespace of the associated stream-ids. Required to access the Eventhubs entity per stream. + + + + systems.system-name.
      streams.stream-id.
      eventhubs.entitypath + + Entity of the associated stream-ids. Required to access the Eventhubs entity per stream. + + + + systems.system-name.
      streams.stream-id.
      eventhubs.sas.keyname + + SAS Keyname of the associated stream-ids. Required to access the Eventhubs entity per stream. + + + + systems.system-name.
      streams.stream-id.
      eventhubs.sas.token + + SAS Token the associated stream-ids. Required to access the Eventhubs entity per stream. + + + + systems.system-name.
      eventhubs.runtime.info.timeout + 60000 + Timeout for fetching the runtime metadata from an Eventhub entity on startup in millis. + + + + systems.system-name.
      eventhubs.partition.method + EVENT_HUB_HASHING + + Producer only config. Configure the method that the message is partitioned for the downstream Eventhub in one of the following ways: +
      +
      ROUND_ROBIN
      +
      The message key and partition key are ignored and the message will be distributed in a round-robin fashion amongst all the partitions in the downstream EventHub.
      +
      EVENT_HUB_HASHING
      +
      Employs the hashing mechanism in EventHubs to determine, based on the key of the message, which partition the message should go. Using this method still ensures that all the events with the same key are sent to the same partition in the event hub. If this option is chosen, the partition key used for the hash should be a string. If the partition key is not set, the message key is used instead.
      +
      PARTITION_KEY_AS_PARTITION
      +
      Use the integer key specified by the partition key or key of the message to a specific partition on Eventhub. If the integer key is greater than the number of partitions in the destination Eventhub, a modulo operation will be performed to determine the resulting paritition. ie. if there are 6 partitions and the key is 9, the message will end up in partition 3. Similarly to EVENT_HUB_HASHING, if the partition key is not set the message key is used instead.
      +
      + + + + + systems.system-name.
      eventhubs.send.key + true + + Producer only config. Sending each message key to the eventhub in the properties of the AMQP message. If the Samza Eventhub consumer is used, this field is used as the message key if the partition key is not present. + + + + + systems.system-name.
      streams.stream-id.
      eventhubs.consumer.group + $Default + + Consumer only config. Set the consumer group from the upstream Eventhub that the consumer is part of. Defaults to the $Default group that is initially present in all Eventhub entities (unless removed) + + + + + systems.system-name.
      eventhubs.receive.queue.size + 100 + + Consumer only config. Per partition capacity of the eventhubs consumer buffer - the blocking queue used for storing messages. Larger buffer capacity typically leads to better throughput but consumes more memory. + + From 54dc4a03e0e1c506e7c73e58575e7c51898309a2 Mon Sep 17 00:00:00 2001 From: Wei Song Date: Thu, 14 Dec 2017 12:04:34 -0800 Subject: [PATCH 14/36] Added document for table API to feature preview Added document for table API to feature preview - Brief description of table - sendTo() operator for table - join() operator for stream-table-join Author: Wei Song Reviewers: Prateek Maheshwari Closes #387 from weisong44/table-api-14 --- docs/startup/preview/index.md | 68 ++++++++++++++++++- .../execution/JobGraphJsonGenerator.java | 6 ++ 2 files changed, 71 insertions(+), 3 deletions(-) diff --git a/docs/startup/preview/index.md b/docs/startup/preview/index.md index ceccc39dbc..20e4aafc56 100644 --- a/docs/startup/preview/index.md +++ b/docs/startup/preview/index.md @@ -273,7 +273,7 @@ The merge transform preserves the order of each MessageStream, so if message `m1 As an alternative to the `merge` instance method, you also can use the [MessageStream#mergeAll](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#mergeAll-java.util.Collection-) static method to merge MessageStreams without operating on an initial stream. -### SendTo +### SendTo (stream) Sends all messages from this MessageStream to the provided OutputStream. You can specify the key and the value to be used for the outgoing message. {% highlight java %} @@ -286,6 +286,16 @@ OutputStream> userRegions keyedPageViews.sendTo(userRegions); {% endhighlight %} +### SendTo (table) + +Sends all messages from this MessageStream to the provided table, the expected message type is KV. + +{% highlight java %} + // Write a new message with memberId as the key and profile as the value to a table. + streamGraph.getInputStream("Profile", new NoOpSerde()) + .map(m -> KV.of(m.getMemberId(), m)) + .sendTo(table); +{% endhighlight %} ### Sink Allows sending messages from this MessageStream to an output system using the provided [SinkFunction](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/functions/SinkFunction.html). @@ -302,9 +312,9 @@ pageViews.sink( (msg, collector, coordinator) -> { } ) {% endhighlight %} -### Join +### Join (stream-stream) -The Join operator joins messages from two MessageStreams using the provided pairwise [JoinFunction](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/functions/JoinFunction.html). Messages are joined when the keys extracted from messages from the first stream match keys extracted from messages in the second stream. Messages in each stream are retained for the provided ttl duration and join results are emitted as matches are found. +The stream-stream Join operator joins messages from two MessageStreams using the provided pairwise [JoinFunction](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/functions/JoinFunction.html). Messages are joined when the keys extracted from messages from the first stream match keys extracted from messages in the second stream. Messages in each stream are retained for the provided ttl duration and join results are emitted as matches are found. {% highlight java %} // Joins a stream of OrderRecord with a stream of ShipmentRecord by orderId with a TTL of 20 minutes. @@ -338,6 +348,38 @@ MessageStream shippedOrders = orders.join(shipments, new O {% endhighlight %} +### Join (stream-table) + +The stream-table Join operator joins messages from a MessageStream using the provided [StreamTableJoinFunction](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/functions/StreamTableJoinFunction.html). Messages from the input stream are joined with record in table using key extracted from input messages. The join function is invoked with both the message and the record. If a record is not found in the table, a null value is provided; the join function can choose to return null (inner join) or an output message (left outer join). For join to function properly, it is important to ensure the input stream and table are partitioned using the same key as this impacts the physical placement of data. + +{% highlight java %} + streamGraph.getInputStream("PageView", new NoOpSerde()) + .partitionBy(PageView::getMemberId, v -> v, "p1") + .join(table, new PageViewToProfileJoinFunction()) + ... +{% endhighlight %} + +{% highlight java linenos %} +public class PageViewToProfileJoinFunction implements StreamTableJoinFunction + , KV, EnrichedPageView> { + @Override + public EnrichedPageView apply(KV m, KV r) { + return r != null ? + new EnrichedPageView(...) + : null; + } + + @Override + public Integer getMessageKey(KV message) { + return message.getKey(); + } + + @Override + public Integer getRecordKey(KV record) { + return record.getKey(); + } +} +{% endhighlight %} ### Window @@ -407,6 +449,26 @@ MessageStream> windowedStream = {% endhighlight %} +### Table + +A Table represents a dataset that can be accessed by keys, and is one of the building blocks of the Samza high level API; the main motivation behind it is to support stream-table joins. The current K/V store is leveraged to provide backing store for local tables. More variations such as direct access and composite tables will be supported in the future. The usage of a table typically follows three steps: + +1. Create a table +2. Populate the table using the sendTo() operator +3. Join a stream with the table using the join() operator + +{% highlight java linenos %} +final StreamApplication app = (streamGraph, cfg) -> { + Table> table = streamGraph.getTable(new InMemoryTableDescriptor("t1") + .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))); + ... +}; +{% endhighlight %} + +Example above creates a TableDescriptor object, which contains all information about a table. The currently supported table types are [InMemoryTableDescriptor](https://github.com/apache/samza/blob/master/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java) and [RocksDbTableDescriptor](https://github.com/apache/samza/blob/master/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/rocksdb/RocksDbTableDescriptor.java). Notice the type of records in a table is KV, and [Serdes](https://samza.apache.org/learn/documentation/latest/container/serialization.html) for both key and value of records needs to be defined (line 4). Additional parameters can be added based on individual table types. + +More details about step 2 and 3 can be found at operator section. + --- ## Flexible Deployment Model diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 2be01af5c4..48d22190f6 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; + import org.apache.samza.config.ApplicationConfig; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; @@ -229,6 +230,11 @@ private Map operatorToMap(OperatorSpec spec) { map.put("tableId", tableSpec.getId()); } + if (spec instanceof StreamTableJoinOperatorSpec) { + TableSpec tableSpec = ((StreamTableJoinOperatorSpec) spec).getTableSpec(); + map.put("tableId", tableSpec.getId()); + } + if (spec instanceof JoinOperatorSpec) { map.put("ttlMs", ((JoinOperatorSpec) spec).getTtlMs()); } From ed8dad54bf04c3f81c2d686786722333325ca432 Mon Sep 17 00:00:00 2001 From: Aditya Toomula Date: Thu, 14 Dec 2017 15:33:29 -0800 Subject: [PATCH 15/36] SAMZA-1536; Adding docs for Kinesis consumer Author: Aditya Toomula Reviewers: Jagadish Closes #384 from atoomula/kinesis-docs --- .../documentation/versioned/aws/kinesis.md | 104 ++++++++++++++++++ docs/learn/documentation/versioned/index.html | 6 + .../versioned/jobs/configuration-table.html | 45 +++++++- 3 files changed, 153 insertions(+), 2 deletions(-) create mode 100644 docs/learn/documentation/versioned/aws/kinesis.md diff --git a/docs/learn/documentation/versioned/aws/kinesis.md b/docs/learn/documentation/versioned/aws/kinesis.md new file mode 100644 index 0000000000..a4be3ddedb --- /dev/null +++ b/docs/learn/documentation/versioned/aws/kinesis.md @@ -0,0 +1,104 @@ +--- +layout: page +title: Connecting to Kinesis +--- + + +You can configure your Samza jobs to process data from [AWS Kinesis](https://aws.amazon.com/kinesis/data-streams), Amazon's data streaming service. A `Kinesis data stream` is similar to a Kafka topic and can have multiple partitions. Each message consumed from a Kinesis data stream is an instance of [Record](http://docs.aws.amazon.com/goto/WebAPI/kinesis-2013-12-02/Record). + +### Consuming from Kinesis: + +Samza's [KinesisSystemConsumer](https://github.com/apache/samza/blob/master/samza-aws/src/main/java/org/apache/samza/system/kinesis/consumer/KinesisSystemConsumer.java) wraps the Record into a [KinesisIncomingMessageEnvelope](https://github.com/apache/samza/blob/master/samza-aws/src/main/java/org/apache/samza/system/kinesis/consumer/KinesisIncomingMessageEnvelope.java). The key of the message is set to partition key of the Record. The message is obtained from the Record body. + +To configure Samza to consume from Kinesis streams: + +{% highlight jproperties %} +# define a kinesis system factory with your identifier. eg: kinesis-system +systems.kinesis-system.samza.factory=org.apache.samza.system.eventhub.KinesisSystemFactory + +# kinesis system consumer works with only AllSspToSingleTaskGrouperFactory +job.systemstreampartition.grouper.factory=org.apache.samza.container.grouper.stream.AllSspToSingleTaskGrouperFactory + +# define your streams +task.inputs=kinesis-system.input0 + +# define required properties for your streams +systems.kinesis-system.streams.input0.aws.region=YOUR-STREAM-REGION +systems.kinesis-system.streams.input0.aws.accessKey=YOUR-ACCESS_KEY +sensitive.systems.kinesis-system.streams.input0.aws.secretKey=YOUR-SECRET-KEY +{% endhighlight %} + +The tuple required to access the Kinesis data stream must be provided, namely the fields `YOUR-STREAM-REGION`, `YOUR-ACCESS-KEY`, `YOUR-SECRET-KEY`. + +#### Advanced Configuration: + +##### AWS Client Configs: + +You can configure any [AWS client config](http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/ClientConfiguration.html) with the prefix `system.system-name.aws.clientConfig.*` +{% highlight jproperties %} +system.system-name.aws.clientConfig.CONFIG-NAME=CONFIG-VALUE +{% endhighlight %} + +As an example, to set a proxy host and proxy port for the AWS Client: +{% highlight jproperties %} +systems.system-name.aws.clientConfig.ProxyHost=my-proxy-host.com +systems.system-name.aws.clientConfig.ProxyPort=my-proxy-port +{% endhighlight %} + +##### KCL Configs: + +Similarly, you can set any [Kinesis Client Library config](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java) for a stream by configuring it under `systems.system-name.streams.stream-name.aws.kcl.*` +{% highlight jproperties %} +systems.system-name.streams.stream-name.aws.kcl.CONFIG-NAME=CONFIG-VALUE +{% endhighlight %} + +As an example, to reset the checkpoint and set the starting position for a stream: +{% highlight jproperties %} +systems.kinesis-system.streams.input0.aws.kcl.TableName=my-app-table-name +# set the starting position to either TRIM_HORIZON (oldest) or LATEST (latest) +systems.kinesis-system.streams.input0.aws.kcl.InitialPositionInStream=my-start-position +{% endhighlight %} + +#### Limitations + +The following limitations apply for Samza jobs consuming from Kinesis streams using the Samza consumer: +* Stateful processing (eg: windows or joins) is not supported on Kinesis streams. However, you can accomplish this by chaining two Samza jobs where the first job reads from Kinesis and sends to Kafka while the second job processes the data from Kafka. +* Kinesis streams cannot be configured as [bootstrap](https://samza.apache.org/learn/documentation/latest/container/streams.html) or [broadcast](https://samza.apache.org/learn/documentation/latest/container/samza-container.html) streams. +* Kinesis streams must be used with the [AllSspToSingleTaskGrouperFactory](https://github.com/apache/samza/blob/master/samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouperFactory.java). No other grouper is supported. +* A Samza job that consumes from Kinesis cannot consume from any other input source. However, you can send your results to any destination (eg: Kafka, EventHubs), and have another Samza job consume them. + +### Producing to Kinesis: + +The KinesisSystemProducer for Samza is not yet implemented. + +### How to configure Samza job to consume from Kinesis data stream ? + +This tutorial uses [hello samza](../../../startup/hello-samza/{{site.version}}/) to illustrate running a Samza job on Yarn that consumes from Kinesis. We will use the [KinesisHelloSamza](https://github.com/apache/samza-hello-samza/blob/master/src/main/java/samza/examples/kinesis/KinesisHelloSamza.java) example. + +#### Update properties file + +Update the following properties in the kinesis-hello-samza.properties file: + +{% highlight jproperties %} +task.inputs=kinesis. +systems.kinesis.streams..aws.region= +systems.kinesis.streams..aws.accessKey= +sensitive.systems.kinesis.streams..aws.region= +{% endhighlight %} + +Now, you are ready to run your Samza application on Yarn as described [here](../../../startup/hello-samza/{{site.version}}/). Check the log file for messages read from your Kinesis stream. diff --git a/docs/learn/documentation/versioned/index.html b/docs/learn/documentation/versioned/index.html index fa8e1e3e4a..9b24c9cce3 100644 --- a/docs/learn/documentation/versioned/index.html +++ b/docs/learn/documentation/versioned/index.html @@ -100,6 +100,12 @@

      Azure

    • Eventhubs Consumer/Producer
    +

    AWS

    + + +

    Operations

      diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html index 6666bb1943..49886ce4d9 100644 --- a/docs/learn/documentation/versioned/jobs/configuration-table.html +++ b/docs/learn/documentation/versioned/jobs/configuration-table.html @@ -2170,7 +2170,7 @@

      Samza Configuration Reference

      - Writing to HDFS + Writing to HDFS @@ -2210,7 +2210,7 @@

      Samza Configuration Reference

      - Reading from HDFS + Reading from HDFS @@ -2335,6 +2335,47 @@

      Samza Configuration Reference

      Consumer only config. Per partition capacity of the eventhubs consumer buffer - the blocking queue used for storing messages. Larger buffer capacity typically leads to better throughput but consumes more memory. + + + + Using Kinesis for input streams
      + + (This section applies if you have set + systems.*.samza.factory + = org.apache.samza.system.kinesis.KinesisSystemFactory) + + + + + + systems.system-name.
      streams.stream-name.
      aws.region + + Region of the associated stream-name. Required to access the Kinesis data stream. + + + + systems.system-name.
      streams.stream-name.
      aws.accessKey + + AccessKey of the associated stream-name. Required to access Kinesis data stream. + + + + systems.system-name.
      streams.stream-name.
      aws.secretKey + + SecretKey of the associated stream-name. Required to access the Kinesis data stream. + + + + systems.system-name.
      streams.stream-name.
      aws.kcl.* + + AWS Kinesis Client Library configuration associated with the stream-name. + + + + systems.system-name.
      aws.clientConfig.* + + AWS ClientConfiguration associated with the system-name. + From ea2b6fa915473e2f120a4860c32cd09b4516c283 Mon Sep 17 00:00:00 2001 From: Prateek Maheshwari Date: Mon, 18 Dec 2017 16:45:06 -0800 Subject: [PATCH 16/36] SAMZA-1538: Disabled Flaky Tests in TestStreamProcessor Author: Prateek Maheshwari Reviewers: Shanthoosh Venkataraman Closes #389 from prateekm/disable-flaky-test --- .../apache/samza/test/processor/TestStreamProcessor.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java index c37132f9ed..a44265ab1f 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java @@ -67,7 +67,8 @@ public class TestStreamProcessor extends StandaloneIntegrationTestHarness { * The standalone version in this test uses KafkaSystemFactory and it uses a SingleContainerGrouperFactory. Hence, * no matter how many tasks are present, it will always be run in a single processor instance. This simplifies testing */ - @Test +// TODO Fix in SAMZA-1538 +// @Test public void testStreamProcessor() { final String testSystem = "test-system"; final String inputTopic = "numbers"; @@ -88,7 +89,8 @@ public void testStreamProcessor() { /** * Should be able to create task instances from the provided task factory. */ - @Test +// TODO Fix in SAMZA-1538 +// @Test public void testStreamProcessorWithStreamTaskFactory() { final String testSystem = "test-system"; final String inputTopic = "numbers2"; @@ -107,7 +109,8 @@ public void testStreamProcessorWithStreamTaskFactory() { /** * Should be able to create task instances from the provided task factory. */ - @Test +// TODO Fix in SAMZA-1538 +// @Test public void testStreamProcessorWithAsyncStreamTaskFactory() { final String testSystem = "test-system"; final String inputTopic = "numbers3"; From 7dc0290ad8babe3abb315bac2aa4155350a7053d Mon Sep 17 00:00:00 2001 From: Jacob Maes Date: Tue, 19 Dec 2017 08:00:56 -0800 Subject: [PATCH 17/36] SAMZA-1356: Improve monitoring for state restore Author: Jacob Maes Author: Jacob Maes Reviewers: Jagadish Closes #241 from jmakes/samza-1356 --- .../samza/storage/kv/KeyValueStorageEngine.scala | 10 +++++++--- .../kv/KeyValueStorageEngineMetrics.scala | 4 ++-- .../storage/kv/TestKeyValueStorageEngine.scala | 16 ++++++++++++++++ 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala index 646b60608c..c42e043563 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala @@ -99,6 +99,8 @@ class KeyValueStorageEngine[K, V]( * batching updates to underlying raw store to notAValidEvent wrapping functions for efficiency. */ def restore(envelopes: java.util.Iterator[IncomingMessageEnvelope]) { + info("Restoring entries for store " + metrics.storeName) + val batch = new java.util.ArrayList[Entry[Array[Byte], Array[Byte]]](batchSize) for (envelope <- envelopes.asScala) { @@ -113,11 +115,11 @@ class KeyValueStorageEngine[K, V]( } if (valBytes != null) { - metrics.restoredBytes.inc(valBytes.size) + metrics.restoredBytes.set(metrics.restoredBytes.getValue + valBytes.size) } - metrics.restoredBytes.inc(keyBytes.size) - metrics.restoredMessages.inc + metrics.restoredBytes.set(metrics.restoredBytes.getValue + keyBytes.size) + metrics.restoredMessages.set(metrics.restoredMessages.getValue + 1) count += 1 if (count % 1000000 == 0) { @@ -125,6 +127,8 @@ class KeyValueStorageEngine[K, V]( } } + info(count + " total entries restored.") + if (batch.size > 0) { rawStore.putAll(batch) } diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineMetrics.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineMetrics.scala index 28cc891ecf..f43b52e1ff 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineMetrics.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineMetrics.scala @@ -34,8 +34,8 @@ class KeyValueStorageEngineMetrics( val deletes = newCounter("deletes") val flushes = newCounter("flushes") - val restoredMessages = newCounter("messages-restored") - val restoredBytes = newCounter("messages-bytes") + val restoredMessages = newGauge("messages-restored", 0) + val restoredBytes = newGauge("messages-bytes", 0) val getNs = newTimer("get-ns") val putNs = newTimer("put-ns") diff --git a/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala b/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala index f6746855c8..ec714b2a85 100644 --- a/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala +++ b/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala @@ -21,7 +21,9 @@ package org.apache.samza.storage.kv import java.util.Arrays +import org.apache.samza.Partition import org.apache.samza.storage.StoreProperties +import org.apache.samza.system.{IncomingMessageEnvelope, SystemStreamPartition} import org.junit.Assert._ import org.junit.{After, Before, Test} import org.mockito.Mockito._ @@ -129,6 +131,20 @@ class TestKeyValueStorageEngine { assertFalse("no next after iterating 2 keys in the range", iter.hasNext) } + @Test + def testRestoreMetrics(): Unit = { + val changelogSSP = new SystemStreamPartition("TestSystem", "TestStream", new Partition(0)) + val changelogEntries = java.util.Arrays asList( + new IncomingMessageEnvelope(changelogSSP, "0", Array[Byte](1, 2), Array[Byte](3, 4, 5)), + new IncomingMessageEnvelope(changelogSSP, "1", Array[Byte](2, 3), Array[Byte](4, 5, 6)), + new IncomingMessageEnvelope(changelogSSP, "2", Array[Byte](3, 4), Array[Byte](5, 6, 7))) + + engine.restore(changelogEntries.iterator()) + + assertEquals(3, metrics.restoredMessages.getValue) + assertEquals(15, metrics.restoredBytes.getValue) // 3 keys * 2 bytes/key + 3 msgs * 3 bytes/msg + } + def getNextTimestamp(): Long = { now += 1 now From e87165d403ac4ae66e5b0ee6b837420f0de635f0 Mon Sep 17 00:00:00 2001 From: Jacob Maes Date: Thu, 21 Dec 2017 10:43:47 -0800 Subject: [PATCH 18/36] Merge script improvement - use colons instead of semicolons Author: Jacob Maes Reviewers: Xinyu Liu ,Jagadish ,Boris Shkolnik Closes #391 from jmakes/merge-script-improvements --- bin/merge-pull-request.py | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/bin/merge-pull-request.py b/bin/merge-pull-request.py index 5f1dae7176..a470045e1a 100755 --- a/bin/merge-pull-request.py +++ b/bin/merge-pull-request.py @@ -350,25 +350,25 @@ def get_version_json(version_str): def standardize_jira_ref(text): """ - Standardize the jira reference commit message prefix to "PROJECT_NAME-XXX; Issue" - >>> standardize_jira_ref("%s-5954; Top by key" % CAPITALIZED_PROJECT_NAME) - 'SAMZA-5954; Top by key' - >>> standardize_jira_ref("%s-5821; ParquetRelation2 CTAS should check if delete is successful" % PROJECT_NAME) - 'SAMZA-5821; ParquetRelation2 CTAS should check if delete is successful' + Standardize the jira reference commit message prefix to "PROJECT_NAME-XXX: Issue" + >>> standardize_jira_ref("%s-5954: Top by key" % CAPITALIZED_PROJECT_NAME) + 'SAMZA-5954: Top by key' + >>> standardize_jira_ref("%s-5821: ParquetRelation2 CTAS should check if delete is successful" % PROJECT_NAME) + 'SAMZA-5821: ParquetRelation2 CTAS should check if delete is successful' >>> standardize_jira_ref("%s-4123 [WIP] Show new dependencies added in pull requests" % PROJECT_NAME) - 'SAMZA-4123; [WIP] Show new dependencies added in pull requests' + 'SAMZA-4123: [WIP] Show new dependencies added in pull requests' >>> standardize_jira_ref("%s 5954: Top by key" % PROJECT_NAME) - 'SAMZA-5954; Top by key' + 'SAMZA-5954: Top by key' >>> standardize_jira_ref("%s-979 a LRU scheduler for load balancing in TaskSchedulerImpl" % PROJECT_NAME) - 'SAMZA-979; a LRU scheduler for load balancing in TaskSchedulerImpl' + 'SAMZA-979: a LRU scheduler for load balancing in TaskSchedulerImpl' >>> standardize_jira_ref("%s-1094 Support MiMa for reporting binary compatibility across versions." % CAPITALIZED_PROJECT_NAME) - 'SAMZA-1094; Support MiMa for reporting binary compatibility across versions.' - >>> standardize_jira_ref("[WIP] %s-1146; Vagrant support" % CAPITALIZED_PROJECT_NAME) - 'SAMZA-1146; [WIP] Vagrant support' + 'SAMZA-1094: Support MiMa for reporting binary compatibility across versions.' + >>> standardize_jira_ref("[WIP] %s-1146: Vagrant support" % CAPITALIZED_PROJECT_NAME) + 'SAMZA-1146: [WIP] Vagrant support' >>> standardize_jira_ref("%s-1032. If Yarn app fails before registering, app master stays aroun..." % PROJECT_NAME) - 'SAMZA-1032; If Yarn app fails before registering, app master stays aroun...' + 'SAMZA-1032: If Yarn app fails before registering, app master stays aroun...' >>> standardize_jira_ref("%s-6250 %s-6146 %s-5911: Types are now reserved words in DDL parser." % (PROJECT_NAME, PROJECT_NAME, CAPITALIZED_PROJECT_NAME)) - 'SAMZA-6250 SAMZA-6146 SAMZA-5911; Types are now reserved words in DDL parser.' + 'SAMZA-6250 SAMZA-6146 SAMZA-5911: Types are now reserved words in DDL parser.' >>> standardize_jira_ref("Additional information for users building from source code") 'Additional information for users building from source code' @@ -400,7 +400,7 @@ def standardize_jira_ref(text): # Assemble full text (JIRA ref(s), module(s), remaining text) jira_prefix = ' '.join(jira_refs).strip() if jira_prefix: - jira_prefix = jira_prefix + "; " + jira_prefix = jira_prefix + ": " clean_text = jira_prefix + ' '.join(components).strip() + " " + text.strip() # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were included From 29ecae891a2c9158b59e81d060234504d6844f70 Mon Sep 17 00:00:00 2001 From: Jacob Maes Date: Thu, 21 Dec 2017 11:43:09 -0800 Subject: [PATCH 19/36] =?UTF-8?q?SAMZA-1537:=20StreamAppender=20can=20dead?= =?UTF-8?q?lock=20due=20to=20locks=20held=20by=20Kafka=20an=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …d Log4j Author: Jacob Maes Reviewers: Jagadish ,Yi Pan (Data Infrastructure) Closes #388 from jmakes/async-stream-appender --- .../samza/logging/log4j/StreamAppender.java | 109 +++++++++++-- .../logging/log4j/StreamAppenderMetrics.java | 43 ++++++ .../logging/log4j/MockSystemProducer.java | 12 +- .../logging/log4j/TestStreamAppender.java | 144 ++++++++++++++---- 4 files changed, 270 insertions(+), 38 deletions(-) create mode 100644 samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppenderMetrics.java diff --git a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java index 84368353f1..0ea8b681c9 100644 --- a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java +++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java @@ -20,9 +20,13 @@ package org.apache.samza.logging.log4j; import java.io.IOException; +import java.io.UnsupportedEncodingException; import java.net.URL; +import java.util.ArrayList; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.Logger; import org.apache.log4j.spi.LoggingEvent; @@ -34,6 +38,7 @@ import org.apache.samza.coordinator.JobModelManager; import org.apache.samza.job.model.JobModel; import org.apache.samza.logging.log4j.serializers.LoggingEventJsonSerdeFactory; +import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerdeFactory; @@ -54,6 +59,12 @@ public class StreamAppender extends AppenderSkeleton { private static final String JAVA_OPTS_CONTAINER_NAME = "samza.container.name"; private static final String JOB_COORDINATOR_TAG = "samza-job-coordinator"; private static final String SOURCE = "log4j-log"; + + protected static final int DEFAULT_QUEUE_SIZE = 100; + private static final long DEFAULT_QUEUE_TIMEOUT_S = 2; // Abitrary choice + + protected static volatile boolean systemInitialized = false; + private Config config = null; private SystemStream systemStream = null; private SystemProducer systemProducer = null; @@ -62,7 +73,12 @@ public class StreamAppender extends AppenderSkeleton { private boolean isApplicationMaster = false; private Serde serde = null; private Logger log = Logger.getLogger(StreamAppender.class); - protected static volatile boolean systemInitialized = false; + protected StreamAppenderMetrics metrics; + + private final BlockingQueue logQueue = new LinkedBlockingQueue<>(DEFAULT_QUEUE_SIZE); + protected long queueTimeoutS = DEFAULT_QUEUE_TIMEOUT_S; + + private Thread transferThread; /** * used to detect if this thread is called recursively @@ -87,6 +103,7 @@ public void activateOptions() { ". This is used as the key for the log appender, so can't proceed."); } key = containerName; // use the container name as the key for the logs + // StreamAppender has to wait until the JobCoordinator is up when the log is in the AM if (isApplicationMaster) { systemInitialized = false; @@ -110,9 +127,32 @@ public void append(LoggingEvent event) { log.trace("Waiting for the JobCoordinator to be instantiated..."); } } else { - OutgoingMessageEnvelope outgoingMessageEnvelope = - new OutgoingMessageEnvelope(systemStream, key.getBytes("UTF-8"), serde.toBytes(subLog(event))); - systemProducer.send(SOURCE, outgoingMessageEnvelope); + // Serialize the event before adding to the queue to leverage the caller thread + // and ensure that the transferThread can keep up. + if (!logQueue.offer(serde.toBytes(subLog(event)), queueTimeoutS, TimeUnit.SECONDS)) { + // Do NOT retry adding to the queue. Dropping the event allows us to alleviate the unlikely + // possibility of a deadlock, which can arise due to a circular dependency between the SystemProducer + // which is used for StreamAppender and the log, which uses StreamAppender. Any locks held in the callstack + // of those two code paths can cause a deadlock. Dropping the event allows us to proceed. + + // Scenario: + // T1: holds L1 and is waiting for L2 + // T2: holds L2 and is waiting to produce to BQ1 which is drained by T3 (SystemProducer) which is waiting for L1 + + // This has happened due to locks in Kafka and log4j (see SAMZA-1537), which are both out of our control, + // so dropping events in the StreamAppender is our best recourse. + + // Drain the queue instead of dropping one message just to reduce the frequency of warn logs above. + int messagesDropped = logQueue.drainTo(new ArrayList<>()) + 1; // +1 because of the current log event + log.warn(String.format("Exceeded timeout %ss while trying to log to %s. Dropping %d log messages.", + queueTimeoutS, + systemStream.toString(), + messagesDropped)); + + // Emit a metric which can be monitored to ensure it doesn't happen often. + metrics.logMessagesDropped.inc(messagesDropped); + } + metrics.bufferFillPct.set(Math.round(100 * logQueue.size() / DEFAULT_QUEUE_SIZE)); } } catch (Exception e) { System.err.println("[StreamAppender] Error sending log message:"); @@ -120,6 +160,8 @@ public void append(LoggingEvent event) { } finally { recursiveCall.set(false); } + } else if (metrics != null) { // setupSystem() may not have been invoked yet so metrics can be null here. + metrics.recursiveCalls.inc(); } } @@ -142,6 +184,13 @@ public void close() { log.info("Shutting down the StreamAppender..."); if (!this.closed) { this.closed = true; + transferThread.interrupt(); + try { + transferThread.join(); + } catch (InterruptedException e) { + log.error("Interrupted while waiting for sink thread to finish.", e); + } + flushSystemProducer(); if (systemProducer != null) { systemProducer.stop(); @@ -169,7 +218,7 @@ public void flushSystemProducer() { * @return Config the config of this container */ protected Config getConfig() { - Config config = null; + Config config; try { if (isApplicationMaster) { @@ -196,6 +245,10 @@ protected void setupSystem() { streamName = getStreamName(log4jSystemConfig.getJobName(), log4jSystemConfig.getJobId()); } + // TODO we need the ACTUAL metrics registry, or the metrics won't get reported by the metric reporters! + MetricsRegistry metricsRegistry = new MetricsRegistryMap(); + metrics = new StreamAppenderMetrics("stream-appender", metricsRegistry); + String systemName = log4jSystemConfig.getSystemName(); String systemFactoryName = log4jSystemConfig.getSystemFactory(systemName); if (systemFactoryName != null) { @@ -206,13 +259,51 @@ protected void setupSystem() { setSerde(log4jSystemConfig, systemName, streamName); - systemProducer = systemFactory.getProducer(systemName, config, new MetricsRegistryMap()); + systemProducer = systemFactory.getProducer(systemName, config, metricsRegistry); systemStream = new SystemStream(systemName, streamName); systemProducer.register(SOURCE); systemProducer.start(); log.info(SOURCE + " has been registered in " + systemName + ". So all the logs will be sent to " + streamName + " in " + systemName + ". Logs are partitioned by " + key); + + startTransferThread(); + } + + private void startTransferThread() { + + try { + // Serialize the key once, since we will use it for every event. + final byte[] keyBytes = key.getBytes("UTF-8"); + + Runnable transferFromQueueToSystem = () -> { + while (!Thread.currentThread().isInterrupted()) { + try { + byte[] serializedLogEvent = logQueue.take(); + + OutgoingMessageEnvelope outgoingMessageEnvelope = + new OutgoingMessageEnvelope(systemStream, keyBytes, serializedLogEvent); + systemProducer.send(SOURCE, outgoingMessageEnvelope); + + } catch (InterruptedException e) { + // Preserve the interrupted status for the loop condition. + Thread.currentThread().interrupt(); + } catch (Throwable t) { + log.error("Error sending StreamAppender event to SystemProducer", t); + } + } + }; + + transferThread = new Thread(transferFromQueueToSystem); + transferThread.setDaemon(true); + transferThread.setName("Samza StreamAppender Producer " + transferThread.getName()); + transferThread.start(); + + } catch (UnsupportedEncodingException e) { + throw new SamzaException(String.format( + "Container name: %s could not be encoded to bytes. StreamAppender cannot proceed.", key), + e); + } } protected static String getStreamName(String jobName, String jobId) { @@ -228,7 +319,7 @@ protected static String getStreamName(String jobName, String jobId) { /** * set the serde for this appender. It looks for the stream serde first, then system serde. - * If still can not get the serde, throws exceptions. + * If still can not get the serde, throws exceptions. * * @param log4jSystemConfig log4jSystemConfig for this appender * @param systemName name of the system @@ -254,7 +345,7 @@ private void setSerde(Log4jSystemConfig log4jSystemConfig, String systemName, St /** * Returns the serde that is being used for the stream appender. - * + * * @return The Serde<LoggingEvent> that the appender is using. */ public Serde getSerde() { diff --git a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppenderMetrics.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppenderMetrics.java new file mode 100644 index 0000000000..0273cb5e66 --- /dev/null +++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppenderMetrics.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.logging.log4j; + +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsBase; +import org.apache.samza.metrics.MetricsRegistry; + + +public class StreamAppenderMetrics extends MetricsBase { + /** The percentage of the log queue capacity that is currently filled with messages from 0 to 100. */ + public final Gauge bufferFillPct; + + /** The number of recursive calls to the StreamAppender. These events will not be logged. */ + public final Counter recursiveCalls; + + /** The number of log messages dropped e.g. because of buffer overflow. Does not include recursive calls. */ + public final Counter logMessagesDropped; + + public StreamAppenderMetrics(String prefix, MetricsRegistry registry) { + super(prefix, registry); + bufferFillPct = newGauge("buffer-fill-percent", 0); + recursiveCalls = newCounter("recursive-calls"); + logMessagesDropped = newCounter("log-messages-dropped"); + } +} diff --git a/samza-log4j/src/test/java/org/apache/samza/logging/log4j/MockSystemProducer.java b/samza-log4j/src/test/java/org/apache/samza/logging/log4j/MockSystemProducer.java index 8d9909416c..fc8de0ba6e 100644 --- a/samza-log4j/src/test/java/org/apache/samza/logging/log4j/MockSystemProducer.java +++ b/samza-log4j/src/test/java/org/apache/samza/logging/log4j/MockSystemProducer.java @@ -21,13 +21,15 @@ import java.util.ArrayList; +import java.util.List; import org.apache.log4j.Logger; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; public class MockSystemProducer implements SystemProducer { - static public ArrayList messagesReceived = new ArrayList(); - static private Logger log = Logger.getLogger(MockSystemProducer.class); + public static ArrayList messagesReceived = new ArrayList<>(); + private static Logger log = Logger.getLogger(MockSystemProducer.class); + public static List listeners = new ArrayList<>(); @Override public void start() { @@ -45,9 +47,15 @@ public void register(String source) { @Override public void send(String source, OutgoingMessageEnvelope envelope) { messagesReceived.add(envelope.getMessage()); + + listeners.forEach((listener) -> listener.onSend(source, envelope)); } @Override public void flush(String source) { } + + public interface MockSystemProducerListener { + void onSend(String source, OutgoingMessageEnvelope envelope); + } } diff --git a/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java b/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java index 59669b7dee..d93c5d1092 100644 --- a/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java +++ b/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java @@ -21,9 +21,14 @@ import static org.junit.Assert.*; +import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.log4j.Logger; import org.apache.log4j.PatternLayout; import org.apache.samza.config.Config; @@ -31,12 +36,20 @@ import org.apache.samza.logging.log4j.serializers.LoggingEventJsonSerde; import org.apache.samza.logging.log4j.serializers.LoggingEventStringSerde; import org.apache.samza.logging.log4j.serializers.LoggingEventStringSerdeFactory; +import org.junit.After; import org.junit.Test; public class TestStreamAppender { static Logger log = Logger.getLogger(TestStreamAppender.class); + @After + public void tearDown() { + log.removeAllAppenders(); + MockSystemProducer.listeners.clear(); + MockSystemProducer.messagesReceived.clear(); + } + @Test public void testDefaultSerde() { System.setProperty("samza.container.name", "samza-container-1"); @@ -70,7 +83,7 @@ public void testNonDefaultSerde() { } @Test - public void testSystemProducerAppenderInContainer() { + public void testSystemProducerAppenderInContainer() throws InterruptedException { System.setProperty("samza.container.name", "samza-container-1"); MockSystemProducerAppender systemProducerAppender = new MockSystemProducerAppender(); @@ -79,22 +92,13 @@ public void testSystemProducerAppenderInContainer() { systemProducerAppender.setLayout(layout); systemProducerAppender.activateOptions(); log.addAppender(systemProducerAppender); - log.info("testing"); - log.info("testing2"); - - systemProducerAppender.flushSystemProducer(); - - assertEquals(2, MockSystemProducer.messagesReceived.size()); - assertTrue(new String((byte[]) MockSystemProducer.messagesReceived.get(0)).contains("\"message\":\"testing\"")); - assertTrue(new String((byte[]) MockSystemProducer.messagesReceived.get(1)).contains("\"message\":\"testing2\"")); - // reset - log.removeAllAppenders(); - MockSystemProducer.messagesReceived.clear(); + List messages = Lists.newArrayList("testing1", "testing2"); + logAndVerifyMessages(messages); } @Test - public void testSystemProducerAppenderInAM() { + public void testSystemProducerAppenderInAM() throws InterruptedException { System.setProperty("samza.container.name", "samza-job-coordinator"); MockSystemProducerAppender systemProducerAppender = new MockSystemProducerAppender(); @@ -104,26 +108,112 @@ public void testSystemProducerAppenderInAM() { systemProducerAppender.activateOptions(); log.addAppender(systemProducerAppender); - log.info("no-received"); - systemProducerAppender.flushSystemProducer(); - // it should not receive anything because the system is not setup - assertEquals(0, MockSystemProducer.messagesReceived.size()); + log.info("no-received"); // System isn't initialized yet, so this message should be dropped systemProducerAppender.setupSystem(); MockSystemProducerAppender.systemInitialized = true; - log.info("testing3"); - log.info("testing4"); - systemProducerAppender.flushSystemProducer(); + List messages = Lists.newArrayList("testing3", "testing4"); + logAndVerifyMessages(messages); + } - // be able to received msgs now - assertEquals(2, MockSystemProducer.messagesReceived.size()); - assertTrue(new String((byte[]) MockSystemProducer.messagesReceived.get(0)).contains("\"message\":\"testing3\"")); - assertTrue(new String((byte[]) MockSystemProducer.messagesReceived.get(1)).contains("\"message\":\"testing4\"")); + @Test + public void testExceptionsDoNotKillTransferThread() throws InterruptedException { + System.setProperty("samza.container.name", "samza-container-1"); - // reset - log.removeAllAppenders(); - MockSystemProducer.messagesReceived.clear(); + MockSystemProducerAppender systemProducerAppender = new MockSystemProducerAppender(); + PatternLayout layout = new PatternLayout(); + layout.setConversionPattern("%m"); + systemProducerAppender.setLayout(layout); + systemProducerAppender.activateOptions(); + log.addAppender(systemProducerAppender); + + List messages = Lists.newArrayList("testing5", "testing6", "testing7"); + + // Set up latch + final CountDownLatch allMessagesSent = new CountDownLatch(messages.size()); + MockSystemProducer.listeners.add((source, envelope) -> { + allMessagesSent.countDown(); + if (allMessagesSent.getCount() == messages.size() - 1) { + throw new RuntimeException(); // Throw on the first message + } + }); + + // Log the messages + messages.forEach((message) -> log.info(message)); + + // Wait for messages + assertTrue("Thread did not send all messages. Count: " + allMessagesSent.getCount(), + allMessagesSent.await(60, TimeUnit.SECONDS)); + } + + @Test + public void testQueueTimeout() throws InterruptedException { + System.setProperty("samza.container.name", "samza-container-1"); + + MockSystemProducerAppender systemProducerAppender = new MockSystemProducerAppender(); + systemProducerAppender.queueTimeoutS = 1; + PatternLayout layout = new PatternLayout(); + layout.setConversionPattern("%m"); + systemProducerAppender.setLayout(layout); + systemProducerAppender.activateOptions(); + log.addAppender(systemProducerAppender); + + int extraMessageCount = 5; + int expectedMessagesSent = extraMessageCount - 1; // -1 because when the queue is drained there is one additional message that couldn't be added + List messages = new ArrayList<>(StreamAppender.DEFAULT_QUEUE_SIZE + extraMessageCount); + for (int i = 0; i < StreamAppender.DEFAULT_QUEUE_SIZE + extraMessageCount; i++) { + messages.add(String.valueOf(i)); + } + + // Set up latch + final CountDownLatch allMessagesSent = new CountDownLatch(expectedMessagesSent); // We expect to drop all but the extra messages + final CountDownLatch waitForTimeout = new CountDownLatch(1); + MockSystemProducer.listeners.add((source, envelope) -> { + allMessagesSent.countDown(); + try { + waitForTimeout.await(); + } catch (InterruptedException e) { + fail("Test could not run properly because of a thread interrupt."); + } + }); + + // Log the messages. This is where the timeout will happen! + messages.forEach((message) -> log.info(message)); + + assertEquals(messages.size() - expectedMessagesSent, systemProducerAppender.metrics.logMessagesDropped.getCount()); + + // Allow all the rest of the messages to send. + waitForTimeout.countDown(); + + // Wait for messages + assertTrue("Thread did not send all messages. Count: " + allMessagesSent.getCount(), + allMessagesSent.await(60, TimeUnit.SECONDS)); + assertEquals(expectedMessagesSent, MockSystemProducer.messagesReceived.size()); + } + + private void logAndVerifyMessages(List messages) throws InterruptedException { + // Set up latch + final CountDownLatch allMessagesSent = new CountDownLatch(messages.size()); + MockSystemProducer.listeners.add((source, envelope) -> allMessagesSent.countDown()); + + // Log the messages + messages.forEach((message) -> log.info(message)); + + // Wait for messages + assertTrue("Timeout while waiting for StreamAppender to send all messages. Count: " + allMessagesSent.getCount(), + allMessagesSent.await(60, TimeUnit.SECONDS)); + + // Verify + assertEquals(messages.size(), MockSystemProducer.messagesReceived.size()); + for (int i = 0; i < messages.size(); i++) { + assertTrue("Message mismatch at index " + i, + new String((byte[]) MockSystemProducer.messagesReceived.get(i)).contains(asJsonMessageSegment(messages.get(i)))); + } + } + + private String asJsonMessageSegment(String message) { + return String.format("\"message\":\"%s\"", message); } /** From ef1c9625c63b6c169585c5d1928298f36f3037fb Mon Sep 17 00:00:00 2001 From: Jacob Maes Date: Fri, 22 Dec 2017 10:36:37 -0800 Subject: [PATCH 20/36] =?UTF-8?q?SAMZA-1539:=20KafkaProducer=20potential?= =?UTF-8?q?=20hang=20on=20close()=20when=20task.drop.pr=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …oducer.errors==true Author: Jacob Maes Reviewers: Boris Shkolnik Closes #390 from jmakes/samza-1539 --- .../system/kafka/KafkaSystemProducer.scala | 135 ++++++++++-------- .../kafka/TestKafkaSystemProducer.scala | 39 +++-- 2 files changed, 103 insertions(+), 71 deletions(-) diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala index 5e8366612b..9eaf895052 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala @@ -28,7 +28,6 @@ import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.producer.RecordMetadata import org.apache.kafka.common.PartitionInfo -import org.apache.kafka.common.errors.SerializationException import org.apache.samza.system.OutgoingMessageEnvelope import org.apache.samza.system.SystemProducer import org.apache.samza.system.SystemProducerException @@ -46,32 +45,30 @@ class KafkaSystemProducer(systemName: String, // Represents a fatal error that caused the producer to close. val fatalException: AtomicReference[SystemProducerException] = new AtomicReference[SystemProducerException]() - @volatile var producer: Producer[Array[Byte], Array[Byte]] = null - val producerLock: Object = new Object + val producerRef: AtomicReference[Producer[Array[Byte], Array[Byte]]] = new AtomicReference[Producer[Array[Byte], Array[Byte]]]() + val producerCreationLock: Object = new Object + @volatile var stopped = false def start(): Unit = { - producer = getProducer() + producerRef.set(getProducer()) } def stop() { info("Stopping producer for system: " + this.systemName) - // stop() should not happen often so no need to optimize locking - producerLock.synchronized { - try { - if (producer != null) { - producer.close // Also performs the equivalent of a flush() - } + stopped = true + val currentProducer = producerRef.getAndSet(null) + try { + if (currentProducer != null) { + currentProducer.close // Also performs the equivalent of a flush() + } - val exception = fatalException.get() - if (exception != null) { - error("Observed an earlier send() error while closing producer", exception) - } - } catch { - case e: Exception => error("Error while closing producer for system: " + systemName, e) - } finally { - producer = null + val exception = fatalException.get() + if (exception != null) { + error("Observed an earlier send() error while closing producer", exception) } + } catch { + case e: Exception => error("Error while closing producer for system: " + systemName, e) } } @@ -82,7 +79,7 @@ class KafkaSystemProducer(systemName: String, trace("Enqueuing message: %s, %s." format (source, envelope)) val topicName = envelope.getSystemStream.getStream - if (topicName == null || topicName == "") { + if (topicName == null || topicName.isEmpty) { throw new IllegalArgumentException("Invalid system stream: " + envelope.getSystemStream) } @@ -92,10 +89,7 @@ class KafkaSystemProducer(systemName: String, throw new SystemProducerException("Producer was unable to recover from previous exception.", globalProducerException) } - val currentProducer = producer - if (currentProducer == null) { - throw new SystemProducerException("Kafka producer is null.") - } + val currentProducer = getOrCreateCurrentProducer // Java-based Kafka producer API requires an "Integer" type partitionKey and does not allow custom overriding of Partitioners // Any kind of custom partitioning has to be done on the client-side @@ -115,7 +109,7 @@ class KafkaSystemProducer(systemName: String, val producerException = new SystemProducerException("Failed to send message for Source: %s on System:%s Topic:%s Partition:%s" .format(source, systemName, topicName, partitionKey), exception) - handleSendException(currentProducer, producerException, true) + handleFatalSendException(currentProducer, producerException) } } }) @@ -125,18 +119,25 @@ class KafkaSystemProducer(systemName: String, val producerException = new SystemProducerException("Failed to send message for Source: %s on System:%s Topic:%s Partition:%s" .format(source, systemName, topicName, partitionKey), originalException) - handleSendException(currentProducer, producerException, isFatalException(originalException)) + metrics.sendFailed.inc + error("Got a synchronous error from Kafka producer.", producerException) + // Synchronous exceptions are always recoverable so propagate it up and let the user decide throw producerException } } - def flush(source: String) { updateTimer(metrics.flushNs) { metrics.flushes.inc - val currentProducer = producer + val currentProducer = producerRef.get() if (currentProducer == null) { + if (dropProducerExceptions) { + // No producer to flush, but we're ignoring exceptions so just return. + warn("Skipping flush because the Kafka producer is null.") + metrics.flushFailed.inc + return + } throw new SystemProducerException("Kafka producer is null.") } @@ -162,7 +163,14 @@ class KafkaSystemProducer(systemName: String, } - private def handleSendException(currentProducer: Producer[Array[Byte], Array[Byte]], producerException: SystemProducerException, isFatalException: Boolean) = { + /** + * Handles a fatal exception by closing the producer and either recreating it or storing the exception + * to rethrow later, depending on the value of dropProducerExceptions. + * + * @param currentProducer the current producer for which the exception occurred. Must not be null. + * @param producerException the exception to handle. + */ + private def handleFatalSendException(currentProducer: Producer[Array[Byte], Array[Byte]], producerException: SystemProducerException): Unit = { metrics.sendFailed.inc error(producerException) // The SystemProducer API is synchronous, so there's no way for us to guarantee that an exception will @@ -172,49 +180,56 @@ class KafkaSystemProducer(systemName: String, if (dropProducerExceptions) { warn("Ignoring producer exception. All messages in the failed producer request will be dropped!") - if (isFatalException) { - producerLock.synchronized { - // Prevent each callback from recreating producer for the same failure. - if (currentProducer == producer) { - info("Creating a new producer for system %s." format systemName) - try { - currentProducer.close(0, TimeUnit.MILLISECONDS) - } catch { - case exception: Exception => error("Exception while closing producer.", exception) - } - producer = getProducer() - } - } - } - } else { - // If there is an exception in the callback, it means that the Kafka producer has exhausted the max-retries - // Close producer to ensure messages queued in-flight are not sent and hence, avoid re-ordering - // This works because there is only 1 callback thread and no sends can complete until the callback returns. - if (isFatalException) { - fatalException.compareAndSet(null, producerException) + // Prevent each callback from closing and nulling producer for the same failure. + if (currentProducer == producerRef.get()) { + info("Closing producer for system %s." format systemName) try { + // send()s can get ProducerClosedException if the producer is stopped after they get the currentProducer + // reference but before producer.send() returns. That's ONLY ok when dropProducerExceptions is true. + // Also, when producer.close(0) is invoked on the Kafka IO thread, when it returns there will be no more + // messages sent over the wire. This is key to ensuring no out-of-order messages as a result of recreating + // the producer. currentProducer.close(0, TimeUnit.MILLISECONDS) } catch { case exception: Exception => error("Exception while closing producer.", exception) } + producerRef.compareAndSet(currentProducer, null) + } + } else { + // If there is an exception in the callback, it means that the Kafka producer has exhausted the max-retries + // Close producer to ensure messages queued in-flight are not sent and hence, avoid re-ordering + // This works because there is only 1 IO thread and no IO can be done until the callback returns. + // Do not create a new producer here! It cannot be done without data loss for all concurrency modes. + fatalException.compareAndSet(null, producerException) + try { + currentProducer.close(0, TimeUnit.MILLISECONDS) + } catch { + case exception: Exception => error("Exception while closing producer.", exception) } } } /** - * A fatal exception is one that corrupts the producer or otherwise makes it unusable. - * We want to handle non-fatal exceptions differently because they can often be handled by the user - * and that's preferable because it gives users that drop exceptions a way to do that with less - * data loss (no collateral damage from batches of messages getting dropped) - * - * @param exception the exception to check - * @return true if the exception is unrecoverable. + * @return the current producer. Never returns null. */ - private def isFatalException(exception: Exception): Boolean = { - exception match { - case _: SerializationException => false - case _: ClassCastException => false - case _ => true + private def getOrCreateCurrentProducer = { + var currentProducer = producerRef.get + + if (currentProducer == null) { + if (dropProducerExceptions && !stopped) { + // Note: While this lock prevents others from creating a new producer, they could still set it to null. + producerCreationLock.synchronized { + currentProducer = producerRef.get + if (currentProducer == null) { + currentProducer = getProducer() + producerRef.set(currentProducer) + } + } + // Invariant: currentProducer must not be null at this point. + } else { + throw new SystemProducerException("Kafka producer is null.") + } } + currentProducer } } \ No newline at end of file diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala index 9117be5380..d874a07d3a 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala @@ -41,7 +41,7 @@ class TestKafkaSystemProducer { systemProducer.register("test") systemProducer.start systemProducer.send("test", someMessage) - assertEquals(1, systemProducer.producer.asInstanceOf[MockProducer[Array[Byte], Array[Byte]]].history().size()) + assertEquals(1, systemProducer.producerRef.get().asInstanceOf[MockProducer[Array[Byte], Array[Byte]]].history().size()) systemProducer.stop } @@ -207,7 +207,7 @@ class TestKafkaSystemProducer { val mockProducer = new MockKafkaProducer(1, "test", 1) val producer = new KafkaSystemProducer(systemName = "test", getProducer = () => { - mockProducer.open() // A new producer is never closed + mockProducer.open() // A new producer would not already be closed, so reset it. mockProducer }, metrics = producerMetrics) @@ -219,6 +219,7 @@ class TestKafkaSystemProducer { mockProducer.setErrorNext(true, true, new RecordTooLargeException()) producer.send("test", msg3) // Callback exception assertTrue(mockProducer.isClosed) + assertNotNull(producer.producerRef.get()) assertEquals("Should NOT have created a new producer", 1, mockProducer.getOpenCount) val senderException = intercept[SystemProducerException] { @@ -269,7 +270,7 @@ class TestKafkaSystemProducer { val mockProducer = new MockKafkaProducer(1, "test", 1) val producer = new KafkaSystemProducer(systemName = "test", getProducer = () => { - mockProducer.open() // A new producer is never closed + mockProducer.open() // A new producer would not already be closed, so reset it. mockProducer }, metrics = producerMetrics) @@ -286,6 +287,7 @@ class TestKafkaSystemProducer { mockProducer.setErrorNext(true, true, new RecordTooLargeException()) producer.send("test1", msg3) // Callback exception assertTrue(mockProducer.isClosed) + assertNotNull(producer.producerRef.get()) assertEquals("Should NOT have created a new producer", 1, mockProducer.getOpenCount) // Subsequent sends @@ -343,7 +345,7 @@ class TestKafkaSystemProducer { val mockProducer = new MockKafkaProducer(1, "test", 1) val producer = new KafkaSystemProducer(systemName = "test", getProducer = () => { - mockProducer.open() // A new producer is never closed + mockProducer.open() // A new producer would not already be closed, so reset it. mockProducer }, metrics = producerMetrics) @@ -359,6 +361,7 @@ class TestKafkaSystemProducer { } assertTrue(sendException.getCause.isInstanceOf[SerializationException]) assertFalse(mockProducer.isClosed) + assertNotNull(producer.producerRef.get()) assertEquals("Should NOT have created a new producer", 1, mockProducer.getOpenCount) producer.send("test1", msg3) // Should be able to resend msg3 @@ -406,7 +409,7 @@ class TestKafkaSystemProducer { val mockProducer = new MockKafkaProducer(1, "test", 1) val producer = new KafkaSystemProducer(systemName = "test", getProducer = () => { - mockProducer.open() // A new producer is never closed + mockProducer.open() // A new producer would not already be closed, so reset it. mockProducer }, metrics = producerMetrics, @@ -418,13 +421,18 @@ class TestKafkaSystemProducer { producer.send("test", msg2) mockProducer.setErrorNext(true, true, new RecordTooLargeException()) producer.send("test", msg3) // Callback exception - assertFalse(mockProducer.isClosed) - assertEquals("Should have created a new producer", 2, mockProducer.getOpenCount) + assertTrue(mockProducer.isClosed) + assertNull(producer.producerRef.get()) + assertEquals("Should not have created a new producer", 1, mockProducer.getOpenCount) producer.send("test", msg4) // Should succeed because the producer recovered. + assertFalse(mockProducer.isClosed) + assertNotNull(producer.producerRef.get()) + assertEquals("Should have created a new producer", 2, mockProducer.getOpenCount) producer.flush("test") // Should not throw producer.send("test", msg5) // Should be able to send again after flush + assertEquals("Should not have created a new producer", 2, mockProducer.getOpenCount) producer.flush("test") assertEquals(4, mockProducer.getMsgsSent) // every message except the one with the error should get sent @@ -456,7 +464,7 @@ class TestKafkaSystemProducer { val mockProducer = new MockKafkaProducer(1, "test", 1) val producer = new KafkaSystemProducer(systemName = "test", getProducer = () => { - mockProducer.open() // A new producer is never closed + mockProducer.open() // A new producer would not already be closed, so reset it. mockProducer }, metrics = producerMetrics, @@ -473,12 +481,17 @@ class TestKafkaSystemProducer { // Inject error for next send mockProducer.setErrorNext(true, true, new RecordTooLargeException()) producer.send("test1", msg3) // Callback exception - assertFalse(mockProducer.isClosed) - assertEquals("Should have created a new producer", 2, mockProducer.getOpenCount) + assertTrue(mockProducer.isClosed) + assertNull(producer.producerRef.get()) + assertEquals("Should not have created a new producer", 1, mockProducer.getOpenCount) // Subsequent sends producer.send("test1", msg4) // Should succeed because the producer recovered. + assertFalse(mockProducer.isClosed) + assertEquals("Should have created a new producer", 2, mockProducer.getOpenCount) + assertNotNull(producer.producerRef.get()) producer.send("test2", msg5) // Second source should also not have any error. + assertEquals("Should not have created a new producer", 2, mockProducer.getOpenCount) // Flushes producer.flush("test2") // Should not throw for test2 @@ -503,7 +516,7 @@ class TestKafkaSystemProducer { val mockProducer = new MockKafkaProducer(1, "test", 1) val producer = new KafkaSystemProducer(systemName = "test", getProducer = () => { - mockProducer.open() // A new producer is never closed + mockProducer.open() // A new producer would not already be closed, so reset it. mockProducer }, metrics = producerMetrics, @@ -520,9 +533,13 @@ class TestKafkaSystemProducer { } assertTrue(sendException.getCause.isInstanceOf[SerializationException]) assertFalse(mockProducer.isClosed) + assertNotNull(producer.producerRef.get()) // Synchronous error; producer should not be recreated assertEquals("Should NOT have created a new producer", 1, mockProducer.getOpenCount) producer.send("test1", msg3) // Should be able to resend msg3 + assertFalse(mockProducer.isClosed) + assertEquals("Should NOT have created a new producer", 1, mockProducer.getOpenCount) + assertNotNull(producer.producerRef.get()) producer.send("test2", msg4) // Second source should not be affected producer.flush("test1") // Flush should be unaffected From 9a8099cdaa2a8ee6fa64d263ebf6704ee5a108da Mon Sep 17 00:00:00 2001 From: Jacob Maes Date: Fri, 22 Dec 2017 11:41:27 -0800 Subject: [PATCH 21/36] Fix a couple sonarcloud issues with samza-1537 Author: Jacob Maes Reviewers: Jagadish Closes #393 from jmakes/streamappender-sonarcloud --- .../java/org/apache/samza/logging/log4j/StreamAppender.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java index 0ea8b681c9..5f41959df7 100644 --- a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java +++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java @@ -152,7 +152,7 @@ public void append(LoggingEvent event) { // Emit a metric which can be monitored to ensure it doesn't happen often. metrics.logMessagesDropped.inc(messagesDropped); } - metrics.bufferFillPct.set(Math.round(100 * logQueue.size() / DEFAULT_QUEUE_SIZE)); + metrics.bufferFillPct.set(Math.round(100f * logQueue.size() / DEFAULT_QUEUE_SIZE)); } } catch (Exception e) { System.err.println("[StreamAppender] Error sending log message:"); @@ -188,7 +188,8 @@ public void close() { try { transferThread.join(); } catch (InterruptedException e) { - log.error("Interrupted while waiting for sink thread to finish.", e); + log.error("Interrupted while waiting for transfer thread to finish.", e); + Thread.currentThread().interrupt(); } flushSystemProducer(); From 93219c78d6f473d2c7ad4d8702a25f704ee61a7a Mon Sep 17 00:00:00 2001 From: Jagadish Date: Fri, 22 Dec 2017 13:58:33 -0800 Subject: [PATCH 22/36] SAMZA-1547; Parse default value grouper-factory config in KafkaCheckpointMgr - Additionally, updated all unit-tests. Author: Jagadish Reviewers: Prateek M Closes #394 from vjagadish1989/kcm-fix --- .../apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala | 2 +- .../samza/checkpoint/kafka/TestKafkaCheckpointManager.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala index 217b2b6b10..e1187c5ce5 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala @@ -59,7 +59,7 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, val checkpointTopic: String = checkpointSpec.getPhysicalName val checkpointSsp = new SystemStreamPartition(checkpointSystem, checkpointTopic, new Partition(0)) val checkpointKeySerde = new KafkaCheckpointLogKeySerde - val expectedGrouperFactory = config.get(JobConfig.SSP_GROUPER_FACTORY) + val expectedGrouperFactory = new JobConfig(config).getSystemStreamPartitionGrouperFactory val systemProducer = systemFactory.getProducer(checkpointSystem, config, metricsRegistry) val systemConsumer = systemFactory.getConsumer(checkpointSystem, config, metricsRegistry) diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala index dcf4068517..ec9f3a0952 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala @@ -150,7 +150,6 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { new MapConfig(new ImmutableMap.Builder[String, String]() .put(JobConfig.JOB_NAME, "some-job-name") .put(JobConfig.JOB_ID, "i001") - .put(JobConfig.SSP_GROUPER_FACTORY, sspGrouperFactoryName) .put(s"systems.$checkpointSystemName.samza.factory", classOf[KafkaSystemFactory].getCanonicalName) .put(s"systems.$checkpointSystemName.producer.bootstrap.servers", brokers) .put(s"systems.$checkpointSystemName.consumer.zookeeper.connect", zkConnect) From 882f61a6925d3950e408ec8e202c0e8b6cdff450 Mon Sep 17 00:00:00 2001 From: Jagadish Date: Tue, 2 Jan 2018 12:16:51 -0800 Subject: [PATCH 23/36] SAMZA-1528: Change ClusterResourceManager to use the async NMClient - Rewrite container handling to be asynchronous - Verified various failure scenarios using Unit tests, and deployments of a local Samza job. Author: Jagadish Author: Fred Ji Author: Srinivasulu Punuru Reviewers: Jacob Maes, Xinyu Liu Closes #380 from vjagadish1989/cluster-mgr-refactor1 --- .../AbstractContainerAllocator.java | 17 +- .../ClusterResourceManager.java | 26 +- .../ContainerProcessManager.java | 74 ++++- .../clustermanager/SamzaApplicationState.java | 8 +- .../samza/clustermanager/SamzaResource.java | 8 + .../samza/operators/impl/OperatorImpl.java | 16 +- .../MockClusterResourceManager.java | 19 +- .../MockClusterResourceManagerCallback.java | 10 + .../TestContainerAllocator.java | 55 ---- .../TestContainerProcessManager.java | 101 ++++-- .../TestHostAwareContainerAllocator.java | 56 ---- .../operators/impl/TestOperatorImpl.java | 1 + .../apache/samza/job/yarn/YarnAppState.java | 4 +- .../job/yarn/YarnClusterResourceManager.java | 310 ++++++++++++++++-- .../samza/job/yarn/YarnContainerRunner.java | 272 --------------- 15 files changed, 498 insertions(+), 479 deletions(-) delete mode 100644 samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnContainerRunner.java diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/AbstractContainerAllocator.java b/samza-core/src/main/java/org/apache/samza/clustermanager/AbstractContainerAllocator.java index b83d83c017..1b7085752d 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/AbstractContainerAllocator.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/AbstractContainerAllocator.java @@ -151,20 +151,11 @@ protected void runStreamProcessor(SamzaResourceRequest request, String preferred log.info("Found available resources on {}. Assigning request for container_id {} with " + "timestamp {} to resource {}", new Object[]{preferredHost, String.valueOf(containerID), request.getRequestTimestampMs(), resource.getResourceID()}); - try { - //launches a StreamProcessor on the resource - clusterResourceManager.launchStreamProcessor(resource, builder); - if (state.neededContainers.decrementAndGet() == 0) { - state.jobHealthy.set(true); - } - state.runningContainers.put(request.getContainerID(), resource); - - } catch (SamzaContainerLaunchException e) { - log.warn(String.format("Got exception while starting resource %s. Requesting a new resource on any host", resource), e); - resourceRequestState.releaseUnstartableContainer(resource); - requestResource(containerID, ResourceRequestState.ANY_HOST); - } + //Submit a request to launch a StreamProcessor on the provided resource. To match with the response returned later + //in the callback, we should also store state about the container whose launch is pending. + clusterResourceManager.launchStreamProcessor(resource, builder); + state.pendingContainers.put(containerID, resource); } /** diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterResourceManager.java b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterResourceManager.java index 715cf6675d..f8a8c8b377 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterResourceManager.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterResourceManager.java @@ -105,15 +105,21 @@ public ClusterResourceManager(Callback callback) { public abstract void releaseResources(SamzaResource resource); /*** - * Requests the launch of a StreamProcessor with the specified context on the resource. + * Requests the launch of a StreamProcessor with the specified context on the resource asynchronously. + * + *

      + * Either {@link Callback#onStreamProcessorLaunchSuccess(SamzaResource)} or + * {@link Callback#onStreamProcessorLaunchFailure(SamzaResource, Throwable)} will be invoked + * to indicate the result of this operation. + *

      + * * @param resource the specified resource * @param builder A builder implementation that encapsulates the context for the * StreamProcessor. A builder encapsulates the ID for the processor, the * build environment, the command to execute etc. - * @throws SamzaContainerLaunchException when there's an error during the requesting launch. * */ - public abstract void launchStreamProcessor(SamzaResource resource, CommandBuilder builder) throws SamzaContainerLaunchException; + public abstract void launchStreamProcessor(SamzaResource resource, CommandBuilder builder); public abstract void stop(SamzaApplicationState.SamzaAppStatus status); @@ -143,6 +149,20 @@ public interface Callback { */ void onResourcesCompleted(List resources); + + /** + * Callback invoked when the launch request for a StreamProcessor on the {@link SamzaResource} is successful. + * @param resource the resource on which the StreamProcessor is launched + */ + void onStreamProcessorLaunchSuccess(SamzaResource resource); + + /** + * Callback invoked when there is a failure in launching a StreamProcessor on the provided {@link SamzaResource}. + * @param resource the resource on which the StreamProcessor was submitted for launching + * @param t the error in launching the StreamProcessor + */ + void onStreamProcessorLaunchFailure(SamzaResource resource, Throwable t); + /*** * This callback is invoked when there is an error in the ClusterResourceManager. This is * guaranteed to be invoked when there is an uncaught exception in any other diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/ContainerProcessManager.java b/samza-core/src/main/java/org/apache/samza/clustermanager/ContainerProcessManager.java index 6a18b8437a..474ac8c433 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/ContainerProcessManager.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/ContainerProcessManager.java @@ -105,6 +105,18 @@ public class ContainerProcessManager implements ClusterResourceManager.Callback */ private final ContainerProcessManagerMetrics metrics; + //for testing + ContainerProcessManager(Config config, SamzaApplicationState state, MetricsRegistryMap registry, AbstractContainerAllocator allocator, ClusterResourceManager manager) { + this.state = state; + this.clusterManagerConfig = new ClusterManagerConfig(config); + this.jobConfig = new JobConfig(config); + this.hostAffinityEnabled = clusterManagerConfig.getHostAffinityEnabled(); + this.clusterResourceManager = manager; + this.metrics = new ContainerProcessManagerMetrics(config, state, registry); + this.containerAllocator = allocator; + this.allocatorThread = new Thread(this.containerAllocator, "Container Allocator Thread"); + } + public ContainerProcessManager(Config config, SamzaApplicationState state, MetricsRegistryMap registry) { @@ -153,7 +165,6 @@ public ContainerProcessManager(Config config, this.allocatorThread = new Thread(this.containerAllocator, "Container Allocator Thread"); log.info("finished initialization of samza task manager"); - } public boolean shouldShutdown() { @@ -383,6 +394,50 @@ public void onResourcesCompleted(List resourceStatuses) { } } + @Override + public void onStreamProcessorLaunchSuccess(SamzaResource resource) { + + // 1. Obtain the Samza container Id for the pending container on this resource. + String containerId = getPendingContainerId(resource.getResourceID()); + log.info("Successfully started container ID: {} on resource: {}", containerId, resource); + + // 2. Remove the container from the pending buffer and add it to the running buffer. Additionally, update the + // job-health metric. + if (containerId != null) { + log.info("Moving containerID: {} on resource: {} from pending to running state", containerId, resource); + state.pendingContainers.remove(containerId); + state.runningContainers.put(containerId, resource); + + if (state.neededContainers.decrementAndGet() == 0) { + state.jobHealthy.set(true); + } + } else { + log.warn("SamzaResource {} was not in pending state. Got an invalid callback for a launch request that " + + "was not issued", resource); + } + } + + @Override + public void onStreamProcessorLaunchFailure(SamzaResource resource, Throwable t) { + log.error("Got a launch failure for SamzaResource {} with exception {}", resource, t); + // 1. Release resources for containers that failed back to YARN + log.info("Releasing unstartable container {}", resource.getResourceID()); + clusterResourceManager.releaseResources(resource); + + // 2. Obtain the Samza container Id for the pending container on this resource. + String containerId = getPendingContainerId(resource.getResourceID()); + log.info("Failed container ID: {} for resourceId: {}", containerId, resource.getResourceID()); + + // 3. Re-request resources on ANY_HOST in case of launch failures on the preferred host. + if (containerId != null) { + log.info("Launch of container ID: {} failed on host: {}. Falling back to ANY_HOST", containerId, resource.getHost()); + containerAllocator.requestResource(containerId, ResourceRequestState.ANY_HOST); + } else { + log.warn("SamzaResource {} was not in pending state. Got an invalid callback for a launch request that was " + + "not issued", resource); + } + } + /** * An error in the callback terminates the JobCoordinator * @param e the underlying exception/error @@ -419,6 +474,21 @@ private ResourceManagerFactory getContainerProcessManagerFactory(final ClusterMa return factory; } + /** + * Obtains the ID of the Samza container pending launch on the provided resource. + * + * @param resourceId the Id of the resource + * @return the Id of the Samza container on this resource + */ + private String getPendingContainerId(String resourceId) { + for (Map.Entry entry: state.pendingContainers.entrySet()) { + if (entry.getValue().getResourceID().equals(resourceId)) { + log.info("Matching container ID found " + entry.getKey() + " " + entry.getValue()); + return entry.getKey(); + } + } + return null; + } -} +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaApplicationState.java b/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaApplicationState.java index adc6e5159e..0dcaace9c0 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaApplicationState.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaApplicationState.java @@ -99,7 +99,13 @@ public enum SamzaAppStatus { UNDEFINED, SUCCEEDED, FAILED } public final AtomicInteger neededContainers = new AtomicInteger(0); /** - * Map of the samzaContainerId to the {@link SamzaResource} on which it is running + * Map of the samzaContainerId to the {@link SamzaResource} on which it is submitted for launch. + * Modified by both the NMCallback and the ContainerAllocator thread. + */ + public final ConcurrentMap pendingContainers = new ConcurrentHashMap(0); + + /** + * Map of the samzaContainerId to the {@link SamzaResource} on which it is running. * Modified by both the AMRMCallbackThread and the ContainerAllocator thread */ public final ConcurrentMap runningContainers = new ConcurrentHashMap(0); diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaResource.java b/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaResource.java index ba6ca2cb6c..2927491d57 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaResource.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/SamzaResource.java @@ -60,6 +60,14 @@ public int hashCode() { return result; } + @Override + public String toString() { + return "SamzaResource{" + + "host='" + host + '\'' + + ", resourceID='" + resourceID + '\'' + + '}'; + } + public int getNumCores() { return numCores; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java index 862e5f9991..9b2b4cf1f2 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java @@ -63,7 +63,7 @@ public abstract class OperatorImpl { private Counter numMessage; private Timer handleMessageNs; private Timer handleTimerNs; - private long inputWatermark = WatermarkStates.WATERMARK_NOT_EXIST; + private long currentWatermark = WatermarkStates.WATERMARK_NOT_EXIST; private long outputWatermark = WatermarkStates.WATERMARK_NOT_EXIST; private TaskName taskName; // Although the operator node is in the operator graph, the current task may not consume any message in it. @@ -340,22 +340,22 @@ private final void onWatermark(long watermark, MessageCollector collector, TaskC inputWatermarkMin = prevOperators.stream().map(op -> op.getOutputWatermark()).min(Long::compare).get(); } - if (inputWatermark < inputWatermarkMin) { + if (currentWatermark < inputWatermarkMin) { // advance the watermark time of this operator - inputWatermark = inputWatermarkMin; - LOG.trace("Advance input watermark to {} in operator {}", inputWatermark, getOpImplId()); + currentWatermark = inputWatermarkMin; + LOG.trace("Advance input watermark to {} in operator {}", currentWatermark, getOpImplId()); final Long outputWm; final Collection output; final WatermarkFunction watermarkFn = getOperatorSpec().getWatermarkFn(); if (watermarkFn != null) { // user-overrided watermark handling here - output = (Collection) watermarkFn.processWatermark(inputWatermark); + output = (Collection) watermarkFn.processWatermark(currentWatermark); outputWm = watermarkFn.getOutputWatermark(); } else { // use samza-provided watermark handling // default is to propagate the input watermark - output = handleWatermark(inputWatermark, collector, coordinator); + output = handleWatermark(currentWatermark, collector, coordinator); outputWm = getOutputWatermark(); } @@ -398,7 +398,7 @@ protected Collection handleWatermark(long inputWatermark, MessageCollector c /* package private for testing */ final long getInputWatermark() { - return this.inputWatermark; + return this.currentWatermark; } /** @@ -409,7 +409,7 @@ final long getInputWatermark() { protected long getOutputWatermark() { if (usedInCurrentTask) { // default as input - return getInputWatermark(); + return this.currentWatermark; } else { // always emit the max to indicate no input will be emitted afterwards return Long.MAX_VALUE; diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManager.java b/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManager.java index 0d13fb1d4e..452cadcefb 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManager.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManager.java @@ -19,10 +19,10 @@ package org.apache.samza.clustermanager; +import com.google.common.collect.ImmutableList; import org.apache.samza.job.CommandBuilder; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -47,9 +47,11 @@ public void start() { @Override public void requestResources(SamzaResourceRequest resourceRequest) { - SamzaResource resource = new SamzaResource(resourceRequest.getNumCores(), resourceRequest.getMemoryMB(), resourceRequest.getPreferredHost(), UUID.randomUUID().toString()); - List resources = Collections.singletonList(resource); - resourceRequests.addAll(resources); + SamzaResource resource = new SamzaResource(resourceRequest.getNumCores(), resourceRequest.getMemoryMB(), + resourceRequest.getPreferredHost(), UUID.randomUUID().toString()); + resourceRequests.add(resource); + + clusterManagerCallback.onResourcesAvailable(ImmutableList.of(resource)); } @Override @@ -63,15 +65,16 @@ public void releaseResources(SamzaResource resource) { } @Override - public void launchStreamProcessor(SamzaResource resource, CommandBuilder builder) throws SamzaContainerLaunchException { + public void launchStreamProcessor(SamzaResource resource, CommandBuilder builder) { if (nextException != null) { - throw new SamzaContainerLaunchException(nextException); + clusterManagerCallback.onStreamProcessorLaunchFailure(resource, new SamzaContainerLaunchException(nextException)); + } else { + launchedResources.add(resource); + clusterManagerCallback.onStreamProcessorLaunchSuccess(resource); } - launchedResources.add(resource); for (MockContainerListener listener : mockContainerListeners) { listener.postRunContainer(launchedResources.size()); } - } @Override diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManagerCallback.java b/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManagerCallback.java index 507962506a..4e6e2c94a1 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManagerCallback.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/MockClusterResourceManagerCallback.java @@ -37,6 +37,16 @@ public void onResourcesCompleted(List resourceStatusList) { resourceStatuses.addAll(resourceStatusList); } + @Override + public void onStreamProcessorLaunchSuccess(SamzaResource resource) { + // no op + } + + @Override + public void onStreamProcessorLaunchFailure(SamzaResource resource, Throwable t) { + // no op + } + @Override public void onError(Throwable e) { error = e; diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerAllocator.java b/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerAllocator.java index 734043a39b..459667309f 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerAllocator.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerAllocator.java @@ -30,7 +30,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; import java.lang.reflect.Field; import java.util.HashMap; import java.util.Map; @@ -202,58 +201,4 @@ public void run() { listener.verify(); } - - /** - * If the container fails to start e.g because it fails to connect to a NM on a host that - * is down, the allocator should request a new container on a different host. - */ - @Test - public void testRerequestOnAnyHostIfContainerStartFails() throws Exception { - final SamzaResource container = new SamzaResource(1, 1024, "2", "id0"); - final SamzaResource container1 = new SamzaResource(1, 1024, "2", "id0"); - manager.nextException = new IOException("Cant connect to RM"); - - // Set up our final asserts before starting the allocator thread - MockContainerListener listener = new MockContainerListener(2, 1, 2, 0, null, new Runnable() { - @Override - public void run() { - // The failed container should be released. The successful one should not. - assertNotNull(manager.releasedResources); - assertEquals(1, manager.releasedResources.size()); - assertTrue(manager.releasedResources.contains(container)); - } - }, - new Runnable() { - @Override - public void run() { - // Test that the first request assignment had a preferred host and the retry didn't - assertEquals(2, requestState.assignedRequests.size()); - - SamzaResourceRequest request = requestState.assignedRequests.remove(); - assertEquals("0", request.getContainerID()); - assertEquals("2", request.getPreferredHost()); - - request = requestState.assignedRequests.remove(); - assertEquals("0", request.getContainerID()); - assertEquals("ANY_HOST", request.getPreferredHost()); - - // This routine should be called after the retry is assigned, but before it's started. - // So there should still be 1 container needed. - assertEquals(1, state.neededContainers.get()); - } - }, null - ); - state.neededContainers.set(1); - requestState.registerContainerListener(listener); - - containerAllocator.requestResource("0", "2"); - containerAllocator.addResource(container); - containerAllocator.addResource(container1); - allocatorThread.start(); - - listener.verify(); - - } - - } diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerProcessManager.java b/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerProcessManager.java index e252b7d7cd..5c2fe4a3d4 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerProcessManager.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/TestContainerProcessManager.java @@ -48,7 +48,7 @@ public class TestContainerProcessManager { private final MockClusterResourceManagerCallback callback = new MockClusterResourceManagerCallback(); - private final MockClusterResourceManager manager = new MockClusterResourceManager(callback); + private final MockClusterResourceManager clusterResourceManager = new MockClusterResourceManager(callback); private static volatile boolean isRunning = false; @@ -125,7 +125,7 @@ public void testContainerProcessManager() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); AbstractContainerAllocator allocator = @@ -145,7 +145,7 @@ public void testContainerProcessManager() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); allocator = @@ -165,11 +165,11 @@ public void testOnInit() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); MockContainerAllocator allocator = new MockContainerAllocator( - manager, + clusterResourceManager, conf, state); @@ -207,7 +207,7 @@ public void testOnShutdown() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); taskManager.start(); @@ -231,11 +231,11 @@ public void testTaskManagerShouldStopWhenContainersFinish() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); MockContainerAllocator allocator = new MockContainerAllocator( - manager, + clusterResourceManager, conf, state); @@ -258,6 +258,7 @@ public void testTaskManagerShouldStopWhenContainersFinish() throws Exception { if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } + taskManager.onStreamProcessorLaunchSuccess(container); assertFalse(taskManager.shouldShutdown()); taskManager.onResourceCompleted(new SamzaResourceStatus("id0", "diagnostics", SamzaResourceStatus.SUCCESS)); @@ -278,11 +279,11 @@ public void testNewContainerRequestedOnFailureWithUnknownCode() throws Exception new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); MockContainerAllocator allocator = new MockContainerAllocator( - manager, + clusterResourceManager, conf, state); @@ -305,7 +306,7 @@ public void testNewContainerRequestedOnFailureWithUnknownCode() throws Exception if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } - + taskManager.onStreamProcessorLaunchSuccess(container); // Create first container failure taskManager.onResourceCompleted(new SamzaResourceStatus(container.getResourceID(), "diagnostics", 1)); @@ -316,8 +317,8 @@ public void testNewContainerRequestedOnFailureWithUnknownCode() throws Exception assertFalse(taskManager.shouldShutdown()); assertFalse(state.jobHealthy.get()); - assertEquals(2, manager.resourceRequests.size()); - assertEquals(0, manager.releasedResources.size()); + assertEquals(2, clusterResourceManager.resourceRequests.size()); + assertEquals(0, clusterResourceManager.releasedResources.size()); taskManager.onResourceAllocated(container); @@ -325,7 +326,7 @@ public void testNewContainerRequestedOnFailureWithUnknownCode() throws Exception if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } - + taskManager.onStreamProcessorLaunchSuccess(container); assertTrue(state.jobHealthy.get()); // Create a second failure @@ -334,8 +335,8 @@ public void testNewContainerRequestedOnFailureWithUnknownCode() throws Exception // The above failure should trigger a job shutdown because our retry count is set to 1 assertEquals(0, allocator.getContainerRequestState().numPendingRequests()); - assertEquals(2, manager.resourceRequests.size()); - assertEquals(0, manager.releasedResources.size()); + assertEquals(2, clusterResourceManager.resourceRequests.size()); + assertEquals(0, clusterResourceManager.releasedResources.size()); assertFalse(state.jobHealthy.get()); assertTrue(taskManager.shouldShutdown()); assertEquals(SamzaApplicationState.SamzaAppStatus.FAILED, state.status); @@ -355,11 +356,11 @@ public void testInvalidNotificationsAreIgnored() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); MockContainerAllocator allocator = new MockContainerAllocator( - manager, + clusterResourceManager, conf, state); getPrivateFieldFromTaskManager("containerAllocator", taskManager).set(taskManager, allocator); @@ -367,7 +368,7 @@ public void testInvalidNotificationsAreIgnored() throws Exception { Thread thread = new Thread(allocator); getPrivateFieldFromTaskManager("allocatorThread", taskManager).set(taskManager, thread); - // Start the task manager + // Start the task clusterResourceManager taskManager.start(); SamzaResource container = new SamzaResource(1, 1000, "abc", "id1"); @@ -388,6 +389,29 @@ public void testInvalidNotificationsAreIgnored() throws Exception { assertEquals(state.redundantNotifications.get(), 1); } + @Test + public void testRerequestOnAnyHostIfContainerStartFails() throws Exception { + state = new SamzaApplicationState(getJobModelManagerWithHostAffinity(1)); + Map configMap = new HashMap<>(); + configMap.putAll(getConfig()); + + MockContainerAllocator allocator = new MockContainerAllocator( + clusterResourceManager, + new MapConfig(config), + state); + + ContainerProcessManager manager = new ContainerProcessManager(config, state, new MetricsRegistryMap(), allocator, + clusterResourceManager); + + manager.start(); + SamzaResource resource = new SamzaResource(1, 1024, "abc", "resource-1"); + state.pendingContainers.put("1", resource); + Assert.assertEquals(clusterResourceManager.resourceRequests.size(), 1); + manager.onStreamProcessorLaunchFailure(resource, new Exception("cannot launch container!")); + Assert.assertEquals(clusterResourceManager.resourceRequests.size(), 2); + Assert.assertEquals(clusterResourceManager.resourceRequests.get(1).getHost(), ResourceRequestState.ANY_HOST); + } + @Test public void testDuplicateNotificationsDoNotAffectJobHealth() throws Exception { Config conf = getConfig(); @@ -400,11 +424,11 @@ public void testDuplicateNotificationsDoNotAffectJobHealth() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); MockContainerAllocator allocator = new MockContainerAllocator( - manager, + clusterResourceManager, conf, state); getPrivateFieldFromTaskManager("containerAllocator", taskManager).set(taskManager, allocator); @@ -424,6 +448,7 @@ public void testDuplicateNotificationsDoNotAffectJobHealth() throws Exception { if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } + taskManager.onStreamProcessorLaunchSuccess(container1); assertEquals(0, allocator.getContainerRequestState().numPendingRequests()); // Create container failure - with ContainerExitStatus.DISKS_FAILED @@ -433,8 +458,8 @@ public void testDuplicateNotificationsDoNotAffectJobHealth() throws Exception { assertEquals(1, allocator.getContainerRequestState().numPendingRequests()); assertFalse(taskManager.shouldShutdown()); assertFalse(state.jobHealthy.get()); - assertEquals(2, manager.resourceRequests.size()); - assertEquals(0, manager.releasedResources.size()); + assertEquals(2, clusterResourceManager.resourceRequests.size()); + assertEquals(0, clusterResourceManager.releasedResources.size()); assertEquals(ResourceRequestState.ANY_HOST, allocator.getContainerRequestState().peekPendingRequest().getPreferredHost()); SamzaResource container2 = new SamzaResource(1, 1000, "abc", "id2"); @@ -444,14 +469,16 @@ public void testDuplicateNotificationsDoNotAffectJobHealth() throws Exception { if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } + taskManager.onStreamProcessorLaunchSuccess(container2); + assertTrue(state.jobHealthy.get()); // Simulate a duplicate notification for container 1 with a different exit code taskManager.onResourceCompleted(new SamzaResourceStatus(container1.getResourceID(), "Disk failure", SamzaResourceStatus.PREEMPTED)); // assert that a duplicate notification does not change metrics (including job health) assertEquals(state.redundantNotifications.get(), 1); - assertEquals(2, manager.resourceRequests.size()); - assertEquals(0, manager.releasedResources.size()); + assertEquals(2, clusterResourceManager.resourceRequests.size()); + assertEquals(0, clusterResourceManager.releasedResources.size()); assertTrue(state.jobHealthy.get()); } @@ -471,11 +498,11 @@ public void testNewContainerRequestedOnFailureWithKnownCode() throws Exception { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); MockContainerAllocator allocator = new MockContainerAllocator( - manager, + clusterResourceManager, conf, state); getPrivateFieldFromTaskManager("containerAllocator", taskManager).set(taskManager, allocator); @@ -483,7 +510,7 @@ public void testNewContainerRequestedOnFailureWithKnownCode() throws Exception { Thread thread = new Thread(allocator); getPrivateFieldFromTaskManager("allocatorThread", taskManager).set(taskManager, thread); - // Start the task manager + // Start the task clusterResourceManager taskManager.start(); assertFalse(taskManager.shouldShutdown()); assertEquals(1, allocator.getContainerRequestState().numPendingRequests()); @@ -496,7 +523,7 @@ public void testNewContainerRequestedOnFailureWithKnownCode() throws Exception { fail("timed out waiting for the containers to start"); } assertEquals(0, allocator.getContainerRequestState().numPendingRequests()); - + taskManager.onStreamProcessorLaunchSuccess(container1); // Create container failure - with ContainerExitStatus.DISKS_FAILED taskManager.onResourceCompleted(new SamzaResourceStatus(container1.getResourceID(), "Disk failure", SamzaResourceStatus.DISK_FAIL)); @@ -504,8 +531,8 @@ public void testNewContainerRequestedOnFailureWithKnownCode() throws Exception { assertEquals(1, allocator.getContainerRequestState().numPendingRequests()); assertFalse(taskManager.shouldShutdown()); assertFalse(state.jobHealthy.get()); - assertEquals(2, manager.resourceRequests.size()); - assertEquals(0, manager.releasedResources.size()); + assertEquals(2, clusterResourceManager.resourceRequests.size()); + assertEquals(0, clusterResourceManager.releasedResources.size()); assertEquals(ResourceRequestState.ANY_HOST, allocator.getContainerRequestState().peekPendingRequest().getPreferredHost()); SamzaResource container2 = new SamzaResource(1, 1000, "abc", "id2"); @@ -515,10 +542,11 @@ public void testNewContainerRequestedOnFailureWithKnownCode() throws Exception { if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } + taskManager.onStreamProcessorLaunchSuccess(container2); // Create container failure - with ContainerExitStatus.PREEMPTED taskManager.onResourceCompleted(new SamzaResourceStatus(container2.getResourceID(), "Preemption", SamzaResourceStatus.PREEMPTED)); - assertEquals(3, manager.resourceRequests.size()); + assertEquals(3, clusterResourceManager.resourceRequests.size()); // The above failure should trigger a container request assertEquals(1, allocator.getContainerRequestState().numPendingRequests()); @@ -532,14 +560,15 @@ public void testNewContainerRequestedOnFailureWithKnownCode() throws Exception { if (!allocator.awaitContainersStart(1, 2, TimeUnit.SECONDS)) { fail("timed out waiting for the containers to start"); } + taskManager.onStreamProcessorLaunchSuccess(container3); // Create container failure - with ContainerExitStatus.ABORTED taskManager.onResourceCompleted(new SamzaResourceStatus(container3.getResourceID(), "Aborted", SamzaResourceStatus.ABORTED)); // The above failure should trigger a container request assertEquals(1, allocator.getContainerRequestState().numPendingRequests()); - assertEquals(4, manager.resourceRequests.size()); - assertEquals(0, manager.releasedResources.size()); + assertEquals(4, clusterResourceManager.resourceRequests.size()); + assertEquals(0, clusterResourceManager.releasedResources.size()); assertFalse(taskManager.shouldShutdown()); assertFalse(state.jobHealthy.get()); assertEquals(ResourceRequestState.ANY_HOST, allocator.getContainerRequestState().peekPendingRequest().getPreferredHost()); @@ -556,7 +585,7 @@ public void testAppMasterWithFwk() { new MapConfig(conf), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); taskManager.start(); SamzaResource container2 = new SamzaResource(1, 1024, "", "id0"); @@ -570,7 +599,7 @@ public void testAppMasterWithFwk() { new MapConfig(config), state, new MetricsRegistryMap(), - manager + clusterResourceManager ); taskManager1.start(); taskManager1.onResourceAllocated(container2); diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/TestHostAwareContainerAllocator.java b/samza-core/src/test/java/org/apache/samza/clustermanager/TestHostAwareContainerAllocator.java index 00198e9deb..6260b71bbf 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/TestHostAwareContainerAllocator.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/TestHostAwareContainerAllocator.java @@ -18,7 +18,6 @@ */ package org.apache.samza.clustermanager; -import java.io.IOException; import java.lang.reflect.Field; import java.util.HashMap; import java.util.Map; @@ -203,61 +202,6 @@ public void testRequestContainers() throws Exception { assertEquals(1, requestsMap.get(ResourceRequestState.ANY_HOST).get()); } - /** - * If the container fails to start e.g because it fails to connect to a NM on a host that - * is down, the allocator should request a new container on a different host. - */ - @Test - public void testRerequestOnAnyHostIfContainerStartFails() throws Exception { - - final SamzaResource container = new SamzaResource(1, 1024, "2", "id0"); - final SamzaResource container1 = new SamzaResource(1, 1024, "1", "id1"); - manager.nextException = new IOException("Cant connect to RM"); - - // Set up our final asserts before starting the allocator thread - MockContainerListener listener = new MockContainerListener(2, 1, 2, 0, null, new Runnable() { - @Override - public void run() { - // The failed container should be released. The successful one should not. - assertNotNull(manager.releasedResources); - assertEquals(1, manager.releasedResources.size()); - assertTrue(manager.releasedResources.contains(container)); - } - }, - new Runnable() { - @Override - public void run() { - // Test that the first request assignment had a preferred host and the retry didn't - assertEquals(2, requestState.assignedRequests.size()); - - SamzaResourceRequest request = requestState.assignedRequests.remove(); - assertEquals("0", request.getContainerID()); - assertEquals("2", request.getPreferredHost()); - - request = requestState.assignedRequests.remove(); - assertEquals("0", request.getContainerID()); - assertEquals("ANY_HOST", request.getPreferredHost()); - - // This routine should be called after the retry is assigned, but before it's started. - // So there should still be 1 container needed. - assertEquals(1, state.neededContainers.get()); - } - }, null - ); - state.neededContainers.set(1); - requestState.registerContainerListener(listener); - - // Only request 1 container and we should see 2 assignments in the assertions above (because of the retry) - containerAllocator.requestResource("0", "2"); - containerAllocator.addResource(container1); - containerAllocator.addResource(container); - - allocatorThread.start(); - - listener.verify(); - } - - /** * Handles expired requests correctly and assigns ANY_HOST */ diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java index 904367b438..a4f0354bb7 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Set; + import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.metrics.Counter; diff --git a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnAppState.java b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnAppState.java index db67de6ddf..fb9a9c24e6 100644 --- a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnAppState.java +++ b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnAppState.java @@ -49,7 +49,9 @@ public class YarnAppState { * Modified by both the AMRMCallbackThread and the ContainerAllocator thread */ - public Map runningYarnContainers = new ConcurrentHashMap() ; + public Map runningYarnContainers = new ConcurrentHashMap(); + + public Map pendingYarnContainers = new ConcurrentHashMap(); public ConcurrentMap failedContainersStatus = new ConcurrentHashMap(); diff --git a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java index 96a4488fef..9be847539a 100644 --- a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java +++ b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java @@ -19,30 +19,46 @@ package org.apache.samza.job.yarn; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.*; import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.records.*; 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; +import org.apache.hadoop.yarn.client.api.async.NMClientAsync; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Records; import org.apache.samza.SamzaException; import org.apache.samza.clustermanager.*; import org.apache.samza.clustermanager.SamzaApplicationState; import org.apache.samza.clustermanager.SamzaContainerLaunchException; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; import org.apache.samza.config.YarnConfig; import org.apache.samza.coordinator.JobModelManager; import org.apache.samza.job.CommandBuilder; import org.apache.samza.metrics.MetricsRegistryMap; +import org.apache.samza.util.Util; import org.apache.samza.util.hadoop.HttpFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -61,7 +77,7 @@ * */ -public class YarnClusterResourceManager extends ClusterResourceManager implements AMRMClientAsync.CallbackHandler { +public class YarnClusterResourceManager extends ClusterResourceManager implements AMRMClientAsync.CallbackHandler, NMClientAsync.CallbackHandler { private final String INVALID_YARN_CONTAINER_ID = "-1"; @@ -70,15 +86,10 @@ public class YarnClusterResourceManager extends ClusterResourceManager implement */ private final AMRMClientAsync amClient; - /** - * A helper class to launch Yarn containers. - */ - private final YarnContainerRunner yarnContainerRunner; - /** * Configuration and state specific to Yarn. */ - private final YarnConfiguration hConfig; + private final YarnConfiguration yarnConfiguration; private final YarnAppState state; /** @@ -100,12 +111,16 @@ public class YarnClusterResourceManager extends ClusterResourceManager implement private final ConcurrentHashMap allocatedResources = new ConcurrentHashMap<>(); private final ConcurrentHashMap requestsMap = new ConcurrentHashMap<>(); + private final ConcurrentHashMap containersPendingStartup = new ConcurrentHashMap<>(); + private final SamzaAppMasterMetrics metrics; final AtomicBoolean started = new AtomicBoolean(false); private final Object lock = new Object(); + private final NMClientAsync nmClientAsync; private static final Logger log = LoggerFactory.getLogger(YarnClusterResourceManager.class); + private final Config config; /** * Creates an YarnClusterResourceManager from config, a jobModelReader and a callback. @@ -116,15 +131,15 @@ public class YarnClusterResourceManager extends ClusterResourceManager implement */ public YarnClusterResourceManager(Config config, JobModelManager jobModelManager, ClusterResourceManager.Callback callback, SamzaApplicationState samzaAppState ) { super(callback); - hConfig = new YarnConfiguration(); - hConfig.set("fs.http.impl", HttpFileSystem.class.getName()); + yarnConfiguration = new YarnConfiguration(); + yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName()); // Use the Samza job config "fs..impl" and "fs..impl.*" for YarnConfiguration FileSystemImplConfig fsImplConfig = new FileSystemImplConfig(config); fsImplConfig.getSchemes().forEach( scheme -> { fsImplConfig.getSchemeConfig(scheme).forEach( - (confKey, confValue) -> hConfig.set(confKey, confValue) + (confKey, confValue) -> yarnConfiguration.set(confKey, confValue) ); } ); @@ -143,6 +158,7 @@ public YarnClusterResourceManager(Config config, JobModelManager jobModelManager int nodeHttpPort = Integer.parseInt(nodeHttpPortString); YarnConfig yarnConfig = new YarnConfig(config); this.yarnConfig = yarnConfig; + this.config = config; int interval = yarnConfig.getAMPollIntervalMs(); //Instantiate the AM Client. @@ -151,7 +167,7 @@ public YarnClusterResourceManager(Config config, JobModelManager jobModelManager this.state = new YarnAppState(-1, containerId, nodeHostString, nodePort, nodeHttpPort); log.info("Initialized YarnAppState: {}", state.toString()); - this.service = new SamzaYarnAppMasterService(config, samzaAppState, this.state, registry, hConfig); + this.service = new SamzaYarnAppMasterService(config, samzaAppState, this.state, registry, yarnConfiguration); log.info("ContainerID str {}, Nodehost {} , Nodeport {} , NodeHttpport {}", new Object [] {containerIdStr, nodeHostString, nodePort, nodeHttpPort}); ClusterManagerConfig clusterManagerConfig = new ClusterManagerConfig(config); @@ -162,8 +178,8 @@ public YarnClusterResourceManager(Config config, JobModelManager jobModelManager state, amClient ); + this.nmClientAsync = NMClientAsync.createNMClientAsync(this); - yarnContainerRunner = new YarnContainerRunner(config, hConfig); } /** @@ -179,8 +195,10 @@ public void start() { metrics.start(); service.onInit(); log.info("Starting YarnContainerManager."); - amClient.init(hConfig); + amClient.init(yarnConfiguration); amClient.start(); + nmClientAsync.init(yarnConfiguration); + nmClientAsync.start(); lifecycle.onInit(); if(lifecycle.shouldShutdown()) { @@ -262,19 +280,22 @@ public void releaseResources(SamzaResource resource) { */ @Override - public void launchStreamProcessor(SamzaResource resource, CommandBuilder builder) throws SamzaContainerLaunchException { + public void launchStreamProcessor(SamzaResource resource, CommandBuilder builder) { String containerIDStr = builder.buildEnvironment().get(ShellCommandConfig.ENV_CONTAINER_ID()); - log.info("Received launch request for {} on hostname {}", containerIDStr , resource.getHost()); - + log.info("Received launch request for {} on hostname {}", containerIDStr, resource.getHost()); synchronized (lock) { - Container container = allocatedResources.get(resource); - if (container == null) { - log.info("Resource {} already released. ", resource); - return; - } + try { + Container container = allocatedResources.get(resource); + if (container == null) { + log.info("Resource {} already released. ", resource); + return; + } - state.runningYarnContainers.put(containerIDStr, new YarnContainer(container)); - yarnContainerRunner.runContainer(containerIDStr, container, builder); + runContainer(containerIDStr, container, builder); + } catch (Throwable t) { + log.error("Error in launching stream processor:", t); + clusterManagerCallback.onStreamProcessorLaunchFailure(resource, t); + } } } @@ -338,6 +359,8 @@ public void stop(SamzaApplicationState.SamzaAppStatus status) { lifecycle.onShutdown(status); amClient.stop(); log.info("Stopping the AM service " ); + nmClientAsync.stop(); + log.info("Stopping the NM service " ); service.onShutdown(); metrics.stop(); @@ -358,7 +381,7 @@ private void cleanupStagingDir() { FileSystem fs = null; try { - fs = FileSystem.get(hConfig); + fs = FileSystem.get(yarnConfiguration); } catch (IOException e) { log.error("Unable to clean up file system: {}", e); return; @@ -454,4 +477,243 @@ public void onError(Throwable e) { clusterManagerCallback.onError(e); } + @Override + public void onContainerStarted(ContainerId containerId, Map allServiceResponse) { + log.info("Received a containerStart notification from the NodeManager for container: {} ", containerId); + String samzaContainerId = getPendingSamzaContainerId(containerId); + + if (samzaContainerId != null) { + // 1. Move the container from pending to running state + final YarnContainer container = state.pendingYarnContainers.remove(samzaContainerId); + log.info("Samza containerId:{} has started", samzaContainerId); + + state.runningYarnContainers.put(samzaContainerId, container); + + // 2. Invoke the success callback. + SamzaResource resource = new SamzaResource(container.resource().getVirtualCores(), + container.resource().getMemory(), container.nodeId().getHost(), containerId.toString()); + clusterManagerCallback.onStreamProcessorLaunchSuccess(resource); + } else { + log.info("Got an invalid notification from YARN for container: {}", containerId); + } + } + + @Override + public void onContainerStatusReceived(ContainerId containerId, ContainerStatus containerStatus) { + log.info("Got a status from the NodeManager. Container: {} Status: {}", containerId, containerStatus.getState()); + } + + @Override + public void onContainerStopped(ContainerId containerId) { + log.info("Got a notification from the NodeManager for a stopped container. ContainerId: {}", containerId); + } + + @Override + public void onStartContainerError(ContainerId containerId, Throwable t) { + log.error(String.format("Container: %s could not start.", containerId), t); + + Container container = containersPendingStartup.remove(containerId); + + if (container != null) { + SamzaResource resource = new SamzaResource(container.getResource().getVirtualCores(), + container.getResource().getMemory(), container.getNodeId().getHost(), containerId.toString()); + log.info("Invoking failure callback for container: {}", containerId); + clusterManagerCallback.onStreamProcessorLaunchFailure(resource, new SamzaContainerLaunchException(t)); + } else { + log.info("Got an invalid notification for container: {}", containerId); + } + } + + @Override + public void onGetContainerStatusError(ContainerId containerId, Throwable t) { + log.info("Got an error on getContainerStatus from the NodeManager. ContainerId: {}. Error: {}", containerId, t); + } + + @Override + public void onStopContainerError(ContainerId containerId, Throwable t) { + log.info("Got an error when stopping container from the NodeManager. ContainerId: {}. Error: {}", containerId, t); + } + + /** + * Runs a process as specified by the command builder on the container. + * @param samzaContainerId id of the samza Container to run (passed as a command line parameter to the process) + * @param container the samza container to run. + * @param cmdBuilder the command builder that encapsulates the command, and the context + * + */ + public void runContainer(String samzaContainerId, Container container, CommandBuilder cmdBuilder) throws IOException { + String containerIdStr = ConverterUtils.toString(container.getId()); + log.info("Got available container ID ({}) for container: {}", samzaContainerId, container); + + // check if we have framework path specified. If yes - use it, if not use default ./__package/ + String jobLib = ""; // in case of separate framework, this directory will point at the job's libraries + String cmdPath = "./__package/"; + + String fwkPath = JobConfig.getFwkPath(this.config); + if(fwkPath != null && (! fwkPath.isEmpty())) { + cmdPath = fwkPath; + jobLib = "export JOB_LIB_DIR=./__package/lib"; + } + log.info("In runContainer in util: fwkPath= " + fwkPath + ";cmdPath=" + cmdPath + ";jobLib=" + jobLib); + cmdBuilder.setCommandPath(cmdPath); + + + String command = cmdBuilder.buildCommand(); + log.info("Container ID {} using command {}", samzaContainerId, command); + + Map env = getEscapedEnvironmentVariablesMap(cmdBuilder); + env.put(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID(), Util.envVarEscape(container.getId().toString())); + printContainerEnvironmentVariables(samzaContainerId, env); + + log.info("Samza FWK path: " + command + "; env=" + env); + + Path packagePath = new Path(yarnConfig.getPackagePath()); + log.info("Starting container ID {} using package path {}", samzaContainerId, packagePath); + state.pendingYarnContainers.put(samzaContainerId, new YarnContainer(container)); + + startContainer( + packagePath, + container, + env, + getFormattedCommand( + ApplicationConstants.LOG_DIR_EXPANSION_VAR, + jobLib, + command, + ApplicationConstants.STDOUT, + ApplicationConstants.STDERR) + ); + + + log.info("Claimed container ID {} for container {} on node {} (http://{}/node/containerlogs/{}).", + new Object[]{ + samzaContainerId, + containerIdStr, + container.getNodeId().getHost(), + container.getNodeHttpAddress(), + containerIdStr} + ); + + log.info("Started container ID {}", samzaContainerId); + } + + /** + * Runs a command as a process on the container. All binaries needed by the physical process are packaged in the URL + * specified by packagePath. + */ + private void startContainer(Path packagePath, + Container container, + Map env, + final String cmd) throws IOException { + log.info("Starting container {} {} {} {}", + new Object[]{packagePath, container, env, cmd}); + + LocalResource packageResource = Records.newRecord(LocalResource.class); + URL packageUrl = ConverterUtils.getYarnUrlFromPath(packagePath); + FileStatus fileStatus; + fileStatus = packagePath.getFileSystem(yarnConfiguration).getFileStatus(packagePath); + packageResource.setResource(packageUrl); + log.info("Set package resource in YarnContainerRunner for {}", packageUrl); + packageResource.setSize(fileStatus.getLen()); + packageResource.setTimestamp(fileStatus.getModificationTime()); + packageResource.setType(LocalResourceType.ARCHIVE); + packageResource.setVisibility(LocalResourceVisibility.APPLICATION); + + ByteBuffer allTokens; + // copy tokens to start the container + Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials(); + DataOutputBuffer dob = new DataOutputBuffer(); + credentials.writeTokenStorageToStream(dob); + + // now remove the AM->RM token so that containers cannot access it + Iterator iter = credentials.getAllTokens().iterator(); + while (iter.hasNext()) { + TokenIdentifier token = ((org.apache.hadoop.security.token.Token) iter.next()).decodeIdentifier(); + if (token != null && token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) { + iter.remove(); + } + } + allTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + + Map localResourceMap = new HashMap<>(); + localResourceMap.put("__package", packageResource); + + // include the resources from the universal resource configurations + LocalizerResourceMapper resourceMapper = new LocalizerResourceMapper(new LocalizerResourceConfig(config), yarnConfiguration); + localResourceMap.putAll(resourceMapper.getResourceMap()); + + ContainerLaunchContext context = Records.newRecord(ContainerLaunchContext.class); + context.setEnvironment(env); + context.setTokens(allTokens.duplicate()); + context.setCommands(new ArrayList() {{add(cmd);}}); + context.setLocalResources(localResourceMap); + + log.debug("Setting localResourceMap to {}", localResourceMap); + log.debug("Setting context to {}", context); + + StartContainerRequest startContainerRequest = Records.newRecord(StartContainerRequest.class); + startContainerRequest.setContainerLaunchContext(context); + + log.info("Making an async start request for container {}", container); + nmClientAsync.startContainerAsync(container, context); + } + + /** + * @param samzaContainerId the Samza container Id for logging purposes. + * @param env the Map of environment variables to their respective values. + */ + private void printContainerEnvironmentVariables(String samzaContainerId, Map env) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : env.entrySet()) { + sb.append(String.format("\n%s=%s", entry.getKey(), entry.getValue())); + } + log.info("Container ID {} using environment variables: {}", samzaContainerId, sb.toString()); + } + + + /** + * Gets the environment variables from the specified {@link CommandBuilder} and escapes certain characters. + * + * @param cmdBuilder the command builder containing the environment variables. + * @return the map containing the escaped environment variables. + */ + private Map getEscapedEnvironmentVariablesMap(CommandBuilder cmdBuilder) { + Map env = new HashMap(); + for (Map.Entry entry : cmdBuilder.buildEnvironment().entrySet()) { + String escapedValue = Util.envVarEscape(entry.getValue()); + env.put(entry.getKey(), escapedValue); + } + return env; + } + + + private String getFormattedCommand(String logDirExpansionVar, + String jobLib, + String command, + String stdOut, + String stdErr) { + if (!jobLib.isEmpty()) { + jobLib = "&& " + jobLib; // add job's libraries exported to an env variable + } + + return String + .format("export SAMZA_LOG_DIR=%s %s && ln -sfn %s logs && exec %s 1>logs/%s 2>logs/%s", logDirExpansionVar, + jobLib, logDirExpansionVar, command, stdOut, stdErr); + } + + /** + * Returns the Id of the Samza container that corresponds to the provided Yarn {@link ContainerId} + * @param containerId the Yarn ContainerId + * @return the id of the Samza container corresponding to the {@link ContainerId} that is pending launch + */ + private String getPendingSamzaContainerId(ContainerId containerId) { + for (String samzaContainerId: state.pendingYarnContainers.keySet()) { + YarnContainer yarnContainer = state.pendingYarnContainers.get(samzaContainerId); + if (yarnContainer.id().equals(containerId)) { + return samzaContainerId; + } + } + return null; + } + + } diff --git a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnContainerRunner.java b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnContainerRunner.java deleted file mode 100644 index cdcf2d11c8..0000000000 --- a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnContainerRunner.java +++ /dev/null @@ -1,272 +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.samza.job.yarn; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; -import org.apache.hadoop.yarn.api.records.*; -import org.apache.hadoop.yarn.api.records.URL; -import org.apache.hadoop.yarn.client.api.NMClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.util.Records; -import org.apache.samza.clustermanager.SamzaContainerLaunchException; -import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; -import org.apache.samza.config.ShellCommandConfig; -import org.apache.samza.config.YarnConfig; -import org.apache.samza.job.CommandBuilder; -import org.apache.samza.util.Util; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -/** - * A Helper class to run container processes on Yarn. This encapsulates quite a bit of YarnContainer - * boiler plate. - */ -public class YarnContainerRunner { - private static final Logger log = LoggerFactory.getLogger(YarnContainerRunner.class); - - private final Config config; - private final YarnConfiguration yarnConfiguration; - - private final NMClient nmClient; - private final YarnConfig yarnConfig; - - /** - * Create a new Runner from a Config. - * @param config to instantiate the runner with - * @param yarnConfiguration the yarn config for the cluster to connect to. - */ - - public YarnContainerRunner(Config config, - YarnConfiguration yarnConfiguration) { - this.config = config; - this.yarnConfiguration = yarnConfiguration; - - this.nmClient = NMClient.createNMClient(); - nmClient.init(this.yarnConfiguration); - - this.yarnConfig = new YarnConfig(config); - } - - /** - * Runs a process as specified by the command builder on the container. - * @param samzaContainerId id of the samza Container to run (passed as a command line parameter to the process) - * @param container the samza container to run. - * @param cmdBuilder the command builder that encapsulates the command, and the context - * - * @throws SamzaContainerLaunchException when there's an exception in submitting the request to the RM. - * - */ - public void runContainer(String samzaContainerId, Container container, CommandBuilder cmdBuilder) throws SamzaContainerLaunchException { - String containerIdStr = ConverterUtils.toString(container.getId()); - log.info("Got available container ID ({}) for container: {}", samzaContainerId, container); - - // check if we have framework path specified. If yes - use it, if not use default ./__package/ - String jobLib = ""; // in case of separate framework, this directory will point at the job's libraries - String cmdPath = "./__package/"; - - String fwkPath = JobConfig.getFwkPath(config); - if(fwkPath != null && (! fwkPath.isEmpty())) { - cmdPath = fwkPath; - jobLib = "export JOB_LIB_DIR=./__package/lib"; - } - log.info("In runContainer in util: fwkPath= " + fwkPath + ";cmdPath=" + cmdPath + ";jobLib=" + jobLib); - cmdBuilder.setCommandPath(cmdPath); - - - String command = cmdBuilder.buildCommand(); - log.info("Container ID {} using command {}", samzaContainerId, command); - - Map env = getEscapedEnvironmentVariablesMap(cmdBuilder); - env.put(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID(), Util.envVarEscape(container.getId().toString())); - printContainerEnvironmentVariables(samzaContainerId, env); - - log.info("Samza FWK path: " + command + "; env=" + env); - - Path packagePath = new Path(yarnConfig.getPackagePath()); - log.info("Starting container ID {} using package path {}", samzaContainerId, packagePath); - - startContainer( - packagePath, - container, - env, - getFormattedCommand( - ApplicationConstants.LOG_DIR_EXPANSION_VAR, - jobLib, - command, - ApplicationConstants.STDOUT, - ApplicationConstants.STDERR) - ); - - - log.info("Claimed container ID {} for container {} on node {} (http://{}/node/containerlogs/{}).", - new Object[]{ - samzaContainerId, - containerIdStr, - container.getNodeId().getHost(), - container.getNodeHttpAddress(), - containerIdStr} - ); - - log.info("Started container ID {}", samzaContainerId); - } - - /** - * Runs a command as a process on the container. All binaries needed by the physical process are packaged in the URL - * specified by packagePath. - */ - private void startContainer(Path packagePath, - Container container, - Map env, - final String cmd) throws SamzaContainerLaunchException { - log.info("starting container {} {} {} {}", - new Object[]{packagePath, container, env, cmd}); - - // TODO: SAMZA-1144 remove the customized approach for package resource and use the common one. - // But keep it now for backward compatibility. - // set the local package so that the containers and app master are provisioned with it - LocalResource packageResource = Records.newRecord(LocalResource.class); - URL packageUrl = ConverterUtils.getYarnUrlFromPath(packagePath); - FileStatus fileStatus; - try { - fileStatus = packagePath.getFileSystem(yarnConfiguration).getFileStatus(packagePath); - } catch (IOException ioe) { - log.error("IO Exception when accessing the package status from the filesystem", ioe); - throw new SamzaContainerLaunchException("IO Exception when accessing the package status from the filesystem"); - } - - packageResource.setResource(packageUrl); - log.info("set package Resource in YarnContainerRunner for {}", packageUrl); - packageResource.setSize(fileStatus.getLen()); - packageResource.setTimestamp(fileStatus.getModificationTime()); - packageResource.setType(LocalResourceType.ARCHIVE); - packageResource.setVisibility(LocalResourceVisibility.APPLICATION); - - ByteBuffer allTokens; - // copy tokens (copied from dist shell example) - try { - Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials(); - DataOutputBuffer dob = new DataOutputBuffer(); - credentials.writeTokenStorageToStream(dob); - - // now remove the AM->RM token so that containers cannot access it - Iterator iter = credentials.getAllTokens().iterator(); - while (iter.hasNext()) { - TokenIdentifier token = ((Token) iter.next()).decodeIdentifier(); - if (token != null && token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) { - iter.remove(); - } - } - allTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); - - } catch (IOException ioe) { - log.error("IOException when writing credentials.", ioe); - throw new SamzaContainerLaunchException("IO Exception when writing credentials to output buffer"); - } - - Map localResourceMap = new HashMap<>(); - localResourceMap.put("__package", packageResource); - - // include the resources from the universal resource configurations - LocalizerResourceMapper resourceMapper = new LocalizerResourceMapper(new LocalizerResourceConfig(config), yarnConfiguration); - localResourceMap.putAll(resourceMapper.getResourceMap()); - - ContainerLaunchContext context = Records.newRecord(ContainerLaunchContext.class); - context.setEnvironment(env); - context.setTokens(allTokens.duplicate()); - context.setCommands(new ArrayList() {{add(cmd);}}); - context.setLocalResources(localResourceMap); - - log.debug("setting localResourceMap to {}", localResourceMap); - log.debug("setting context to {}", context); - - StartContainerRequest startContainerRequest = Records.newRecord(StartContainerRequest.class); - startContainerRequest.setContainerLaunchContext(context); - try { - nmClient.startContainer(container, context); - } catch (YarnException ye) { - log.error("Received YarnException when starting container: " + container.getId(), ye); - throw new SamzaContainerLaunchException("Received YarnException when starting container: " + container.getId(), ye); - } catch (IOException ioe) { - log.error("Received IOException when starting container: " + container.getId(), ioe); - throw new SamzaContainerLaunchException("Received IOException when starting container: " + container.getId(), ioe); - } - } - - - /** - * @param samzaContainerId the Samza container Id for logging purposes. - * @param env the Map of environment variables to their respective values. - */ - private void printContainerEnvironmentVariables(String samzaContainerId, Map env) { - StringBuilder sb = new StringBuilder(); - for (Map.Entry entry : env.entrySet()) { - sb.append(String.format("\n%s=%s", entry.getKey(), entry.getValue())); - } - log.info("Container ID {} using environment variables: {}", samzaContainerId, sb.toString()); - } - - - /** - * Gets the environment variables from the specified {@link CommandBuilder} and escapes certain characters. - * - * @param cmdBuilder the command builder containing the environment variables. - * @return the map containing the escaped environment variables. - */ - private Map getEscapedEnvironmentVariablesMap(CommandBuilder cmdBuilder) { - Map env = new HashMap(); - for (Map.Entry entry : cmdBuilder.buildEnvironment().entrySet()) { - String escapedValue = Util.envVarEscape(entry.getValue()); - env.put(entry.getKey(), escapedValue); - } - return env; - } - - - private String getFormattedCommand(String logDirExpansionVar, - String jobLib, - String command, - String stdOut, - String stdErr) { - if (!jobLib.isEmpty()) { - jobLib = "&& " + jobLib; // add job's libraries exported to an env variable - } - - return String - .format("export SAMZA_LOG_DIR=%s %s && ln -sfn %s logs && exec %s 1>logs/%s 2>logs/%s", logDirExpansionVar, - jobLib, logDirExpansionVar, command, stdOut, stdErr); - } -} From f8d91425e04ae46d5d5bfc72f18663f3d1afdc1f Mon Sep 17 00:00:00 2001 From: Xinyu Liu Date: Wed, 3 Jan 2018 09:56:23 -0800 Subject: [PATCH 24/36] SAMZA-1550: Doc for 0.14.0 release Docs update for both master and 0.14.0 branch. Author: xiliu Reviewers: Yi Pan Closes #396 from xinyuiscool/SAMZA-1550 --- docs/_layouts/default.html | 1 + docs/archive/index.html | 8 ++++++++ .../versioned/deploy-samza-job-from-hdfs.md | 2 +- .../versioned/deploy-samza-to-CDH.md | 4 ++-- .../versioned/remote-debugging-samza.md | 2 +- .../versioned/run-in-multi-node-yarn.md | 4 ++-- docs/startup/download/index.md | 19 ++++++++++--------- .../releases/versioned/release-notes.md | 13 ++++++++++--- 8 files changed, 35 insertions(+), 18 deletions(-) diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html index c48073cee6..75976570bc 100644 --- a/docs/_layouts/default.html +++ b/docs/_layouts/default.html @@ -103,6 +103,7 @@

      Contribute

      Archive

      • latest
      • +
      • 0.14
      • 0.13
      • 0.12
      • 0.11
      • diff --git a/docs/archive/index.html b/docs/archive/index.html index 6b6e74abac..0df2c205e8 100644 --- a/docs/archive/index.html +++ b/docs/archive/index.html @@ -27,6 +27,14 @@

        Latest Release

      • Hello Samza
      +

      0.14 Release

      + + +

      0.13 Release

        diff --git a/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md b/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md index 13eba67852..acbc70b1bf 100644 --- a/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md +++ b/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md @@ -24,7 +24,7 @@ This tutorial uses [hello-samza](../../../startup/hello-samza/{{site.version}}/) ### Upload the package {% highlight bash %} -hadoop fs -put ./target/hello-samza-0.13.1-dist.tar.gz /path/for/tgz +hadoop fs -put ./target/hello-samza-0.14.0-dist.tar.gz /path/for/tgz {% endhighlight %} ### Add HDFS configuration diff --git a/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md b/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md index e5ab8cf400..296203bcdb 100644 --- a/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md +++ b/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md @@ -34,7 +34,7 @@ mvn clean package -Dhadoop.version=cdh5.4.0 There are a few ways of uploading the package to the cluster's HDFS. If you do not have the job package in your cluster, **scp** from you local machine to the cluster. Then run {% highlight bash %} -hadoop fs -put path/to/hello-samza-0.13.1-dist.tar.gz /path/for/tgz +hadoop fs -put path/to/hello-samza-0.14.0-dist.tar.gz /path/for/tgz {% endhighlight %} ### Get Deploying Scripts @@ -42,7 +42,7 @@ hadoop fs -put path/to/hello-samza-0.13.1-dist.tar.gz /path/for/tgz Untar the job package (assume you will run from the current directory) {% highlight bash %} -tar -xvf path/to/samza-job-package-0.13.1-dist.tar.gz -C ./ +tar -xvf path/to/samza-job-package-0.14.0-dist.tar.gz -C ./ {% endhighlight %} ### Add Package Path to Properties File diff --git a/docs/learn/tutorials/versioned/remote-debugging-samza.md b/docs/learn/tutorials/versioned/remote-debugging-samza.md index 4c7f10eaaa..dfa57cb530 100644 --- a/docs/learn/tutorials/versioned/remote-debugging-samza.md +++ b/docs/learn/tutorials/versioned/remote-debugging-samza.md @@ -83,7 +83,7 @@ Once the grid starts, you can start the wikipedia-feed Samza job. {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-0.13.1-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-0.14.0-dist.tar.gz -C deploy/samza deploy/samza/bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/wikipedia-feed.properties {% endhighlight %} diff --git a/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md b/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md index bb908ea553..f6146ad382 100644 --- a/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md +++ b/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md @@ -145,7 +145,7 @@ vi src/main/config/wikipedia-feed.properties Change the yarn.package.path property to be: {% highlight jproperties %} -yarn.package.path=http://yourHostname:8000/target/hello-samza-0.13.1-dist.tar.gz +yarn.package.path=http://yourHostname:8000/target/hello-samza-0.14.0-dist.tar.gz {% endhighlight %} 3\. Compile hello-samza. @@ -153,7 +153,7 @@ yarn.package.path=http://yourHostname:8000/target/hello-samza-0.13.1-dist.tar.gz {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-0.13.1-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-0.14.0-dist.tar.gz -C deploy/samza {% endhighlight %} 4\. Deploy Samza job package to Http server.. diff --git a/docs/startup/download/index.md b/docs/startup/download/index.md index 5dcf38022b..39492c040c 100644 --- a/docs/startup/download/index.md +++ b/docs/startup/download/index.md @@ -31,10 +31,11 @@ Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this Samza tools package contains command line tools that user can run to use Samza and it's input/output systems. - * [samza-tools-0.14.0.tgz](tbd) + * [samza-tools-0.14.0.tgz](http://www-us.apache.org/dist/samza/0.14.0/samza-tools-0.14.0.tgz) ### Source Releases + * [samza-sources-0.14.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.14.0) * [samza-sources-0.13.1.tgz](http://www.apache.org/dyn/closer.lua/samza/0.13.1) * [samza-sources-0.13.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.13.0) * [samza-sources-0.12.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.12.0) @@ -59,12 +60,12 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen org.apache.samza samza-api - 0.13.1 + 0.14.0 org.apache.samza samza-core_2.11 - 0.13.1 + 0.14.0 runtime @@ -72,37 +73,37 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen samza-shell dist tgz - 0.13.1 + 0.14.0 runtime org.apache.samza samza-yarn_2.11 - 0.13.1 + 0.14.0 runtime org.apache.samza samza-kv_2.11 - 0.13.1 + 0.14.0 runtime org.apache.samza samza-kv-rocksdb_2.11 - 0.13.1 + 0.14.0 runtime org.apache.samza samza-kv-inmemory_2.11 - 0.13.1 + 0.14.0 runtime org.apache.samza samza-kafka_2.11 - 0.13.1 + 0.14.0 runtime {% endhighlight %} diff --git a/docs/startup/releases/versioned/release-notes.md b/docs/startup/releases/versioned/release-notes.md index 460cbda7b3..f83978d291 100644 --- a/docs/startup/releases/versioned/release-notes.md +++ b/docs/startup/releases/versioned/release-notes.md @@ -31,6 +31,7 @@ title: Release Notes {% if site.version != "latest" %} ## Download +All Samza JARs are published through [Apache's Maven repository](https://repository.apache.org/content/groups/public/org/apache/samza/). See [here](../download/index.html) for more details. ## Source Release [samza-sources-{{site.version}}.tgz](http://www.apache.org/dyn/closer.lua/samza/{{site.version}}.*) @@ -42,8 +43,14 @@ title: Release Notes ## Release Notes - - +* [SAMZA-1510](https://issues.apache.org/jira/browse/SAMZA-1510) - Samza SQL +* [SAMZA-1438](https://issues.apache.org/jira/browse/SAMZA-1438) - Producer and consumer for Azure EventHubs +* [SAMZA-1515](https://issues.apache.org/jira/browse/SAMZA-1515) - Kinesis consumer +* [SAMZA-1486](https://issues.apache.org/jira/browse/SAMZA-1486) - Checkpoint provider for Azure tables +* [SAMZA-1421](https://issues.apache.org/jira/browse/SAMZA-1421) - Support for durable state in high-level API +* [SAMZA-1392](https://issues.apache.org/jira/browse/SAMZA-1392) - KafkaSystemProducer performance and correctness with concurrent sends and flushes +* [SAMZA-1406](https://issues.apache.org/jira/browse/SAMZA-1406) - Enhancements to the ZooKeeper-based deployment model +* [SAMZA-1321](https://issues.apache.org/jira/browse/SAMZA-1321) - Support for multi-stage batch processing ## Upgrade Notes @@ -71,5 +78,5 @@ If you have a custom **System** implementation, then you have to update to the n boolean clearStream(StreamSpec streamSpec); ```
        -Read more about it in the [API docs](). +Read more about it in the [API docs](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/system/SystemAdmin.html). From d8fdb1a586a4379fcbecbc618a2090a508af63b0 Mon Sep 17 00:00:00 2001 From: Xinyu Liu Date: Wed, 3 Jan 2018 16:10:45 -0800 Subject: [PATCH 25/36] SAMZA-1550: Update master to use 0.14.1-SNAPSHOT version Update master to use 0.14.1-SNAPSHOT version. Author: xiliu Reviewers: Yi Pan Closes #400 from xinyuiscool/SAMZA-1550-2 --- docs/_config.yml | 2 +- docs/learn/documentation/versioned/azure/eventhubs.md | 2 +- docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md | 2 +- docs/learn/tutorials/versioned/hello-samza-high-level-zk.md | 2 +- docs/learn/tutorials/versioned/samza-rest-getting-started.md | 2 +- docs/startup/hello-samza/versioned/index.md | 2 +- gradle.properties | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index 11850aa28f..cfd3b0eec9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -25,4 +25,4 @@ exclude: [_docs] baseurl: http://samza.apache.org version: latest # this is the version you will go if you click 'switch version' in "latest" pages. -latest-release: '0.13' +latest-release: '0.14' diff --git a/docs/learn/documentation/versioned/azure/eventhubs.md b/docs/learn/documentation/versioned/azure/eventhubs.md index 2defddb9d2..7d76be35d1 100644 --- a/docs/learn/documentation/versioned/azure/eventhubs.md +++ b/docs/learn/documentation/versioned/azure/eventhubs.md @@ -165,7 +165,7 @@ With the environment setup complete, let us move on to building the hello-samza ``` mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-0.14.1-SNAPSHOT-dist.tar.gz -C deploy/samza ``` We are now all set to deploy the application locally. diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md index fbd1e05ef8..15e71272e4 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md @@ -63,7 +63,7 @@ Then, you can continue w/ the following command in hello-samza project: {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-0.14.1-SNAPSHOT-dist.tar.gz -C deploy/samza {% endhighlight %} ### Run a Samza Application diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md b/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md index ebe5595e32..b2b1b4ea3f 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md @@ -59,7 +59,7 @@ With the environment setup complete, let us move on to building the hello-samza {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-0.14.1-SNAPSHOT-dist.tar.gz -C deploy/samza {% endhighlight %} We are now all set to deploy the application locally. diff --git a/docs/learn/tutorials/versioned/samza-rest-getting-started.md b/docs/learn/tutorials/versioned/samza-rest-getting-started.md index b889dd3754..54e3bdf32d 100644 --- a/docs/learn/tutorials/versioned/samza-rest-getting-started.md +++ b/docs/learn/tutorials/versioned/samza-rest-getting-started.md @@ -48,7 +48,7 @@ Run the following commands: {% highlight bash %} cd samza-rest/build/distributions/ mkdir -p deploy/samza-rest -tar -xvf ./samza-rest-0.14.0-SNAPSHOT.tgz -C deploy/samza-rest +tar -xvf ./samza-rest-0.14.1-SNAPSHOT.tgz -C deploy/samza-rest {% endhighlight %} #### Configure the Installations Path diff --git a/docs/startup/hello-samza/versioned/index.md b/docs/startup/hello-samza/versioned/index.md index e94e2a5f40..f2461109cf 100644 --- a/docs/startup/hello-samza/versioned/index.md +++ b/docs/startup/hello-samza/versioned/index.md @@ -61,7 +61,7 @@ Then, you can continue w/ the following command in hello-samza project: {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-0.14.1-SNAPSHOT-dist.tar.gz -C deploy/samza {% endhighlight %} ### Run a Samza Job diff --git a/gradle.properties b/gradle.properties index f3cfad720e..bf37e1dc33 100644 --- a/gradle.properties +++ b/gradle.properties @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. group=org.apache.samza -version=0.13.1-SNAPSHOT +version=0.14.1-SNAPSHOT scalaVersion=2.11 gradleVersion=2.8 From 7954815ca0887f12fa11d4b4239eb5cd867b0711 Mon Sep 17 00:00:00 2001 From: xiliu Date: Thu, 4 Jan 2018 09:51:20 -0800 Subject: [PATCH 26/36] Fix a link in release-notes.md --- docs/startup/releases/versioned/release-notes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/startup/releases/versioned/release-notes.md b/docs/startup/releases/versioned/release-notes.md index f83978d291..18983f13b0 100644 --- a/docs/startup/releases/versioned/release-notes.md +++ b/docs/startup/releases/versioned/release-notes.md @@ -60,7 +60,7 @@ All Samza JARs are published through [Apache's Maven repository](https://reposit * Introduced a new **mandatory** configuration - `job.coordination.utils.factory`. Read more about it -[here](../../learn/{{site.version}}/configuration.html).
        This config is applicable to all Samza +[here](/learn/documentation/{{site.version}}/jobs/configuration-table.html).
        This config is applicable to all Samza applications deployed using the `LocalApplicationRunner` (that is, non-yarn deployments). ### API Changes From 5238aaa6cee81a87079c9d432204422ececea793 Mon Sep 17 00:00:00 2001 From: Xinyu Liu Date: Tue, 9 Jan 2018 10:48:10 -0800 Subject: [PATCH 27/36] SAMZA-1553: Add log4j for latest Kafka build Add it so Samza compiles with the latest kafka. Author: xiliu Reviewers: Boris Shkolnik Closes #402 from xinyuiscool/SAMZA-1553 --- build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle b/build.gradle index 04b42f261d..c12b1777db 100644 --- a/build.gradle +++ b/build.gradle @@ -373,6 +373,7 @@ project(":samza-kafka_$scalaVersion") { dependencies { compile project(':samza-api') compile project(":samza-core_$scalaVersion") + compile "log4j:log4j:$log4jVersion" compile "org.scala-lang:scala-library:$scalaLibVersion" compile "com.101tec:zkclient:$zkClientVersion" compile "org.apache.zookeeper:zookeeper:$zookeeperVersion" From b00ebd21a8a3cc064009509e5b953008c89a08f6 Mon Sep 17 00:00:00 2001 From: xiliu Date: Tue, 9 Jan 2018 15:53:40 -0800 Subject: [PATCH 28/36] SAMZA-1550: Update samza 0.14 version in tests --- samza-test/src/main/config/join/README | 8 ++++---- samza-test/src/main/python/configs/tests.json | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/samza-test/src/main/config/join/README b/samza-test/src/main/config/join/README index 01a354b1b9..9392d3b5fe 100644 --- a/samza-test/src/main/config/join/README +++ b/samza-test/src/main/config/join/README @@ -44,17 +44,17 @@ Deploy Zookeeper, YARN and Kafka: > cd $HELLO_SAMZA_SRC > for i in zookeeper kafka yarn; do ./bin/grid install $i; ./bin/grid start $i; done -Update the "yarn.package.path" to $DEPLOY_DIR/samza-test_2.11-0.13.1-SNAPSHOT.tgz +Update the "yarn.package.path" to $DEPLOY_DIR/samza-test_2.11-0.14.1-SNAPSHOT.tgz > cd $SAMZA_SRC > vi samza-test/src/main/config/join/common.properties -yarn.package.path=file:///path/to/samza-hello-samza/deploy/samza-test_2.11-0.13.1-SNAPSHOT.tgz +yarn.package.path=file:///path/to/samza-hello-samza/deploy/samza-test_2.11-0.14.1-SNAPSHOT.tgz Then release and extract the test tarball: > cd $SAMZA_SRC > ./gradlew releaseTestJobs -> cp samza-test/build/distributions/samza-test_2.11-0.13.1-SNAPSHOT.tgz $DEPLOY_DIR +> cp samza-test/build/distributions/samza-test_2.11-0.14.1-SNAPSHOT.tgz $DEPLOY_DIR > mkdir $DEPLOY_DIR/samza -> tar -xvf $DEPLOY_DIR/samza-test_2.11-0.13.1-SNAPSHOT.tgz -C $DEPLOY_DIR/samza +> tar -xvf $DEPLOY_DIR/samza-test_2.11-0.14.1-SNAPSHOT.tgz -C $DEPLOY_DIR/samza Finally, create the kafka topics and start the samza jobs: > ./bin/setup-int-test.sh $DEPLOY_DIR diff --git a/samza-test/src/main/python/configs/tests.json b/samza-test/src/main/python/configs/tests.json index 0d37c301d8..3c72875ea6 100644 --- a/samza-test/src/main/python/configs/tests.json +++ b/samza-test/src/main/python/configs/tests.json @@ -1,5 +1,5 @@ { - "samza_executable": "samza-test_2.11-0.13.1-SNAPSHOT.tgz", + "samza_executable": "samza-test_2.11-0.14.1-SNAPSHOT.tgz", "samza_install_path": "deploy/smoke_tests", "samza_config_factory": "org.apache.samza.config.factories.PropertiesConfigFactory" } From 07f28e9483ba9c69f44fd7634f6fef6dc47ecd0c Mon Sep 17 00:00:00 2001 From: xiliu Date: Wed, 10 Jan 2018 10:33:23 -0800 Subject: [PATCH 29/36] Revert "SAMZA-1553: Add log4j for latest Kafka build" This reverts commit 5238aaa6cee81a87079c9d432204422ececea793. --- build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/build.gradle b/build.gradle index c12b1777db..04b42f261d 100644 --- a/build.gradle +++ b/build.gradle @@ -373,7 +373,6 @@ project(":samza-kafka_$scalaVersion") { dependencies { compile project(':samza-api') compile project(":samza-core_$scalaVersion") - compile "log4j:log4j:$log4jVersion" compile "org.scala-lang:scala-library:$scalaLibVersion" compile "com.101tec:zkclient:$zkClientVersion" compile "org.apache.zookeeper:zookeeper:$zookeeperVersion" From a6540b4e3d1d3916210c50be9b4b2b4920f885bb Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 10 Jan 2018 10:52:38 -0800 Subject: [PATCH 30/36] SAMZA-1530; Bump up Kafka dependency to 0.11 Author: Dong Lin Reviewers: Xinyu Liu Closes #395 from lindong28/SAMZA-1530 --- bin/check-all.sh | 2 +- gradle/dependency-versions.gradle | 2 +- .../samza/system/kafka/BrokerProxy.scala | 13 ++++---- .../apache/samza/system/kafka/GetOffset.scala | 4 +-- .../samza/system/kafka/KafkaSystemAdmin.scala | 8 ++--- .../system/kafka/TopicMetadataCache.scala | 2 +- .../org/apache/samza/util/KafkaUtil.scala | 8 +++-- .../samza/system/kafka/MockKafkaProducer.java | 25 +++++++++++++-- .../kafka/TestKafkaCheckpointManager.scala | 7 ++-- .../samza/system/kafka/TestBrokerProxy.scala | 9 +++--- .../system/kafka/TestKafkaSystemAdmin.scala | 26 ++++++--------- .../kafka/TestKafkaSystemConsumer.scala | 4 +-- .../system/kafka/TestTopicMetadataCache.scala | 32 ++++++++++--------- .../apache/samza/utils/TestKafkaUtil.scala | 7 ++-- .../TestZkLocalApplicationRunner.java | 2 +- .../test/integration/StreamTaskTestUtil.scala | 7 ++-- 16 files changed, 86 insertions(+), 72 deletions(-) diff --git a/bin/check-all.sh b/bin/check-all.sh index 2f9f03c62a..f168bc8d15 100755 --- a/bin/check-all.sh +++ b/bin/check-all.sh @@ -21,7 +21,7 @@ set -e -SCALAs=( "2.10" "2.11" "2.12" ) +SCALAs=( "2.11" "2.12" ) JDKs=( "JAVA8_HOME" ) YARNs=( "2.6.1" "2.7.1" ) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 20a1d56efd..2e45914f59 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -33,7 +33,7 @@ jodaTimeVersion = "2.2" joptSimpleVersion = "3.2" junitVersion = "4.8.1" - kafkaVersion = "0.10.1.1" + kafkaVersion = "0.11.0.2" log4jVersion = "1.2.17" metricsVersion = "2.2.0" mockitoVersion = "1.10.19" diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala index 5338886254..8a6618d238 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala @@ -182,7 +182,7 @@ class BrokerProxy( firstCallBarrier.countDown() // Split response into errors and non errors, processing the errors first - val (nonErrorResponses, errorResponses) = response.data.toSet.partition(_._2.error == ErrorMapping.NoError) + val (nonErrorResponses, errorResponses) = response.data.toSet.partition(_._2.error.code() == ErrorMapping.NoError) handleErrors(errorResponses, response) @@ -219,18 +219,17 @@ class BrokerProxy( immutableNextOffsetsCopy.keySet.foreach(abdicate(_)) } - def handleErrors(errorResponses: Set[(TopicAndPartition, FetchResponsePartitionData)], response:FetchResponse) = { + def handleErrors(errorResponses: Set[(TopicAndPartition, FetchResponsePartitionData)], response: FetchResponse) = { // FetchResponse should really return Option and a list of the errors so we don't have to find them ourselves - case class Error(tp: TopicAndPartition, code: Short, exception: Throwable) + case class Error(tp: TopicAndPartition, code: Short, exception: Exception) // Now subdivide the errors into three types: non-recoverable, not leader (== abdicate) and offset out of range (== get new offset) // Convert FetchResponse into easier-to-work-with Errors val errors = for ( (topicAndPartition, responseData) <- errorResponses; - errorCode <- Option(response.errorCode(topicAndPartition.topic, topicAndPartition.partition)); // Scala's being cranky about referring to error.getKey values... - exception <- Option(ErrorMapping.exceptionFor(errorCode)) - ) yield new Error(topicAndPartition, errorCode, exception) + error <- Option(response.error(topicAndPartition.topic, topicAndPartition.partition)) // Scala's being cranky about referring to error.getKey values... + ) yield new Error(topicAndPartition, error.code(), error.exception()) val (notLeaderOrUnknownTopic, otherErrors) = errors.partition { case (e) => e.code == ErrorMapping.NotLeaderForPartitionCode || e.code == ErrorMapping.UnknownTopicOrPartitionCode } val (offsetOutOfRangeErrors, remainingErrors) = otherErrors.partition(_.code == ErrorMapping.OffsetOutOfRangeCode) @@ -241,7 +240,7 @@ class BrokerProxy( // handle the recoverable errors. remainingErrors.foreach(e => { warn("Got non-recoverable error codes during multifetch. Throwing an exception to trigger reconnect. Errors: %s" format remainingErrors.mkString(",")) - KafkaUtil.maybeThrowException(e.code) }) + KafkaUtil.maybeThrowException(e.exception) }) notLeaderOrUnknownTopic.foreach(e => abdicate(e.tp)) diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala index 5528702070..040e2460be 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala @@ -60,7 +60,7 @@ class GetOffset( val messages = consumer.defaultFetch((topicAndPartition, offset.toLong)) if (messages.hasError) { - KafkaUtil.maybeThrowException(messages.errorCode(topicAndPartition.topic, topicAndPartition.partition)) + KafkaUtil.maybeThrowException(messages.error(topicAndPartition.topic, topicAndPartition.partition).exception()) } info("Able to successfully read from offset %s for topic and partition %s. Using it to instantiate consumer." format (offset, topicAndPartition)) @@ -86,7 +86,7 @@ class GetOffset( .get(topicAndPartition) .getOrElse(toss("Unable to find offset information for %s" format topicAndPartition)) - KafkaUtil.maybeThrowException(partitionOffsetResponse.error) + KafkaUtil.maybeThrowException(partitionOffsetResponse.error.exception()) partitionOffsetResponse .offsets diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala index 013b292d0a..4715141267 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala @@ -164,7 +164,7 @@ class KafkaSystemAdmin( metadataTTL) val result = metadata.map { case (topic, topicMetadata) => { - KafkaUtil.maybeThrowException(topicMetadata.errorCode) + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) val partitionsMap = topicMetadata.partitionsMetadata.map { pm => new Partition(pm.partitionId) -> new SystemStreamPartitionMetadata("", "", "") @@ -350,7 +350,7 @@ class KafkaSystemAdmin( .values // Convert the topic metadata to a Seq[(Broker, TopicAndPartition)] .flatMap(topicMetadata => { - KafkaUtil.maybeThrowException(topicMetadata.errorCode) + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) topicMetadata .partitionsMetadata // Convert Seq[PartitionMetadata] to Seq[(Broker, TopicAndPartition)] @@ -390,7 +390,7 @@ class KafkaSystemAdmin( .getOffsetsBefore(new OffsetRequest(partitionOffsetInfo)) .partitionErrorAndOffsets .mapValues(partitionErrorAndOffset => { - KafkaUtil.maybeThrowException(partitionErrorAndOffset.error) + KafkaUtil.maybeThrowException(partitionErrorAndOffset.error.exception()) partitionErrorAndOffset.offsets.head }) @@ -480,7 +480,7 @@ class KafkaSystemAdmin( val metadataStore = new ClientUtilTopicMetadataStore(brokerListString, clientId, timeout) val topicMetadataMap = TopicMetadataCache.getTopicMetadata(Set(topicName), systemName, metadataStore.getTopicInfo, metadataTTL) val topicMetadata = topicMetadataMap(topicName) - KafkaUtil.maybeThrowException(topicMetadata.errorCode) + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) val partitionCount = topicMetadata.partitionsMetadata.length if (partitionCount != spec.getPartitionCount) { diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala index 82ecf1aee9..8a3ab2bc62 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala @@ -73,6 +73,6 @@ object TopicMetadataCache extends Logging { * partition's metadata has a bad errorCode. */ def hasBadErrorCode(streamMetadata: TopicMetadata) = { - KafkaUtil.isBadErrorCode(streamMetadata.errorCode) || streamMetadata.partitionsMetadata.exists(partitionMetadata => KafkaUtil.isBadErrorCode(partitionMetadata.errorCode)) + KafkaUtil.isBadErrorCode(streamMetadata.error.code()) || streamMetadata.partitionsMetadata.exists(partitionMetadata => KafkaUtil.isBadErrorCode(partitionMetadata.error.code())) } } diff --git a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala index 1410cbbce0..5b0137a1ee 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala @@ -29,7 +29,8 @@ import org.apache.samza.config.{ApplicationConfig, Config, ConfigException} import org.apache.samza.config.JobConfig.Config2Job import org.apache.samza.execution.StreamManager import org.apache.samza.system.OutgoingMessageEnvelope -import kafka.common.{ErrorMapping, ReplicaNotAvailableException} +import org.apache.kafka.common.errors.ReplicaNotAvailableException +import kafka.common.ErrorMapping import org.apache.kafka.common.errors.TopicExistsException import org.apache.samza.system.kafka.TopicMetadataCache @@ -71,9 +72,10 @@ object KafkaUtil extends Logging { * protocol * docs, ReplicaNotAvailableException can be safely ignored. */ - def maybeThrowException(code: Short) { + def maybeThrowException(e: Exception) { try { - ErrorMapping.maybeThrowException(code) + if (e != null) + throw e } catch { case e: ReplicaNotAvailableException => debug("Got ReplicaNotAvailableException, but ignoring since it's safe to do so.") diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/MockKafkaProducer.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/MockKafkaProducer.java index 024c6e6330..e66b7c3198 100644 --- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/MockKafkaProducer.java +++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/MockKafkaProducer.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; import kafka.producer.ProducerClosedException; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -42,6 +43,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.test.TestUtils; public class MockKafkaProducer implements Producer { @@ -98,7 +100,7 @@ public Future send(ProducerRecord record) { } private RecordMetadata getRecordMetadata(ProducerRecord record) { - return new RecordMetadata(new TopicPartition(record.topic(), record.partition() == null ? 0 : record.partition()), 0, this.msgsSent.get(), Record.NO_TIMESTAMP, -1, -1, -1); + return new RecordMetadata(new TopicPartition(record.topic(), record.partition() == null ? 0 : record.partition()), 0, this.msgsSent.get(), -1L, -1, -1, -1); } @Override @@ -190,6 +192,25 @@ public synchronized void flush () { new FlushRunnable(0).run(); } + public void initTransactions() { + + } + + public void abortTransaction() { + + } + + public void beginTransaction() { + + } + + public void commitTransaction() { + + } + + public void sendOffsetsToTransaction(Map offsets, String consumerGroupId) { + + } private static class FutureFailure implements Future { @@ -232,7 +253,7 @@ private static class FutureSuccess implements Future { public FutureSuccess(ProducerRecord record, int offset) { this.record = record; - this._metadata = new RecordMetadata(new TopicPartition(record.topic(), record.partition() == null ? 0 : record.partition()), 0, offset, Record.NO_TIMESTAMP, -1, -1, -1); + this._metadata = new RecordMetadata(new TopicPartition(record.topic(), record.partition() == null ? 0 : record.partition()), 0, offset, RecordBatch.NO_TIMESTAMP, -1, -1, -1); } @Override diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala index ec9f3a0952..86cb4185f0 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala @@ -52,15 +52,12 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { val checkpoint1 = new Checkpoint(ImmutableMap.of(ssp, "offset-1")) val checkpoint2 = new Checkpoint(ImmutableMap.of(ssp, "offset-2")) val taskName = new TaskName("Partition 0") - - var brokers: String = null var config: Config = null @Before override def setUp { super.setUp TestUtils.waitUntilTrue(() => servers.head.metadataCache.getAliveBrokers.size == numBrokers, "Wait for cache to update") - brokers = brokerList.split(",").map(p => "localhost" + p).mkString(",") config = getConfig() } @@ -140,7 +137,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { val defaultSerializer = classOf[ByteArraySerializer].getCanonicalName val props = new Properties() props.putAll(ImmutableMap.of( - ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers, + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList, ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, defaultSerializer, ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, defaultSerializer)) props @@ -151,7 +148,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { .put(JobConfig.JOB_NAME, "some-job-name") .put(JobConfig.JOB_ID, "i001") .put(s"systems.$checkpointSystemName.samza.factory", classOf[KafkaSystemFactory].getCanonicalName) - .put(s"systems.$checkpointSystemName.producer.bootstrap.servers", brokers) + .put(s"systems.$checkpointSystemName.producer.bootstrap.servers", brokerList) .put(s"systems.$checkpointSystemName.consumer.zookeeper.connect", zkConnect) .put("task.checkpoint.system", checkpointSystemName) .build()) diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala index f0bdafdc21..d510076792 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala @@ -24,9 +24,10 @@ import java.nio.ByteBuffer import java.util.concurrent.CountDownLatch import kafka.api.{PartitionOffsetsResponse, _} -import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.common.TopicAndPartition import kafka.consumer.SimpleConsumer import kafka.message.{ByteBufferMessageSet, Message, MessageAndOffset, MessageSet} +import org.apache.kafka.common.protocol.Errors import org.apache.samza.SamzaException import org.apache.samza.util.Logging import org.junit.Assert._ @@ -165,7 +166,7 @@ class TestBrokerProxy extends Logging { messageSet } - val fetchResponsePartitionData = FetchResponsePartitionData(0, 500, messageSet) + val fetchResponsePartitionData = FetchResponsePartitionData(Errors.NONE, 500, messageSet) val map = scala.Predef.Map[TopicAndPartition, FetchResponsePartitionData](tp -> fetchResponsePartitionData) when(fetchResponse.data).thenReturn(map.toSeq) @@ -257,12 +258,12 @@ class TestBrokerProxy extends Logging { } val mfr = mock(classOf[FetchResponse]) when(mfr.hasError).thenReturn(true) - when(mfr.errorCode("topic", 42)).thenReturn(ErrorMapping.OffsetOutOfRangeCode) + when(mfr.error("topic", 42)).thenReturn(Errors.OFFSET_OUT_OF_RANGE) val messageSet = mock(classOf[MessageSet]) when(messageSet.iterator).thenReturn(Iterator.empty) val response = mock(classOf[FetchResponsePartitionData]) - when(response.error).thenReturn(ErrorMapping.OffsetOutOfRangeCode) + when(response.error).thenReturn(Errors.OFFSET_OUT_OF_RANGE) val responseMap = Map(tp -> response) when(mfr.data).thenReturn(responseMap.toSeq) invocationCount += 1 diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala index 65c43f5cf9..2039447947 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala @@ -24,7 +24,8 @@ package org.apache.samza.system.kafka import java.util.{Properties, UUID} import kafka.admin.AdminUtils -import kafka.common.{ErrorMapping, LeaderNotAvailableException} +import org.apache.kafka.common.errors.LeaderNotAvailableException +import org.apache.kafka.common.protocol.Errors import kafka.consumer.{Consumer, ConsumerConfig} import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig @@ -68,19 +69,13 @@ object TestKafkaSystemAdmin extends KafkaServerTestHarness { @BeforeClass override def setUp { super.setUp - val config = new java.util.HashMap[String, String]() - - brokers = brokerList.split(",").map(p => "localhost" + p).mkString(",") - - config.put("bootstrap.servers", brokers) + config.put("bootstrap.servers", brokerList) config.put("acks", "all") config.put("serializer.class", "kafka.serializer.StringEncoder") - producerConfig = new KafkaProducerConfig("kafka", "i001", config) - producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfig.getProducerProperties) - metadataStore = new ClientUtilTopicMetadataStore(brokers, "some-job-name") + metadataStore = new ClientUtilTopicMetadataStore(brokerList, "some-job-name") } @@ -107,9 +102,8 @@ object TestKafkaSystemAdmin extends KafkaServerTestHarness { try { val topicMetadataMap = TopicMetadataCache.getTopicMetadata(Set(topic), SYSTEM, metadataStore.getTopicInfo) val topicMetadata = topicMetadataMap(topic) - val errorCode = topicMetadata.errorCode - KafkaUtil.maybeThrowException(errorCode) + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) done = expectedPartitionCount == topicMetadata.partitionsMetadata.size } catch { @@ -137,11 +131,11 @@ object TestKafkaSystemAdmin extends KafkaServerTestHarness { } def createSystemAdmin: KafkaSystemAdmin = { - new KafkaSystemAdmin(SYSTEM, brokers, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure)) + new KafkaSystemAdmin(SYSTEM, brokerList, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure)) } def createSystemAdmin(coordinatorStreamProperties: Properties, coordinatorStreamReplicationFactor: Int, topicMetaInformation: Map[String, ChangelogInfo]): KafkaSystemAdmin = { - new KafkaSystemAdmin(SYSTEM, brokers, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure), coordinatorStreamProperties, coordinatorStreamReplicationFactor, 10000, ConsumerConfig.SocketBufferSize, UUID.randomUUID.toString, topicMetaInformation, Map()) + new KafkaSystemAdmin(SYSTEM, brokerList, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure), coordinatorStreamProperties, coordinatorStreamReplicationFactor, 10000, ConsumerConfig.SocketBufferSize, UUID.randomUUID.toString, topicMetaInformation, Map()) } } @@ -281,7 +275,7 @@ class TestKafkaSystemAdmin { @Test def testShouldCreateCoordinatorStream { val topic = "test-coordinator-stream" - val systemAdmin = new KafkaSystemAdmin(SYSTEM, brokers, () => ZkUtils(zkConnect, 6000, 6000, zkSecure), coordinatorStreamReplicationFactor = 3) + val systemAdmin = new KafkaSystemAdmin(SYSTEM, brokerList, () => ZkUtils(zkConnect, 6000, 6000, zkSecure), coordinatorStreamReplicationFactor = 3) val spec = StreamSpec.createCoordinatorStreamSpec(topic, "kafka") systemAdmin.createStream(spec) @@ -294,14 +288,14 @@ class TestKafkaSystemAdmin { assertEquals(3, partitionMetadata.replicas.size) } - class KafkaSystemAdminWithTopicMetadataError extends KafkaSystemAdmin(SYSTEM, brokers, () => ZkUtils(zkConnect, 6000, 6000, zkSecure)) { + class KafkaSystemAdminWithTopicMetadataError extends KafkaSystemAdmin(SYSTEM, brokerList, () => ZkUtils(zkConnect, 6000, 6000, zkSecure)) { import kafka.api.TopicMetadata var metadataCallCount = 0 // Simulate Kafka telling us that the leader for the topic is not available override def getTopicMetadata(topics: Set[String]) = { metadataCallCount += 1 - val topicMetadata = TopicMetadata(topic = "quux", partitionsMetadata = Seq(), errorCode = ErrorMapping.LeaderNotAvailableCode) + val topicMetadata = TopicMetadata(topic = "quux", partitionsMetadata = Seq(), error = Errors.LEADER_NOT_AVAILABLE) Map("quux" -> topicMetadata) } } diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.scala index 8a5cbc2581..4dd170fc4d 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.scala @@ -25,7 +25,7 @@ import kafka.cluster.Broker import kafka.common.TopicAndPartition import kafka.message.Message import kafka.message.MessageAndOffset - +import org.apache.kafka.common.protocol.Errors import org.apache.samza.system.IncomingMessageEnvelope import org.apache.samza.system.SystemStreamPartition import org.apache.samza.Partition @@ -68,7 +68,7 @@ class TestKafkaSystemConsumer { // Lie and tell the store that the partition metadata is empty. We can't // use partition metadata because it has Broker in its constructor, which // is package private to Kafka. - val metadataStore = new MockMetadataStore(Map(streamName -> TopicMetadata(streamName, Seq.empty, 0))) + val metadataStore = new MockMetadataStore(Map(streamName -> TopicMetadata(streamName, Seq.empty, Errors.NONE))) var hosts = List[String]() var getHostPortCount = 0 val consumer = new KafkaSystemConsumer(systemName, systemAdmin, metrics, metadataStore, clientId) { diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala index 50c13ababb..9cc2f63bbe 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala @@ -20,7 +20,8 @@ package org.apache.samza.system.kafka import java.util.concurrent.CountDownLatch -import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + import kafka.api.TopicMetadata import org.I0Itec.zkclient.ZkClient import org.apache.samza.util.Clock @@ -30,6 +31,7 @@ import org.junit.Before import org.junit.Test import kafka.common.ErrorMapping import kafka.api.PartitionMetadata +import org.apache.kafka.common.protocol.Errors class TestTopicMetadataCache { @@ -41,8 +43,8 @@ class TestTopicMetadataCache { class MockTopicMetadataStore extends TopicMetadataStore { var mockCache = Map( - "topic1" -> new TopicMetadata("topic1", List.empty, 0), - "topic2" -> new TopicMetadata("topic2", List.empty, 0)) + "topic1" -> new TopicMetadata("topic1", List.empty, Errors.NONE), + "topic2" -> new TopicMetadata("topic2", List.empty, Errors.NONE)) var numberOfCalls: AtomicInteger = new AtomicInteger(0) def getTopicInfo(topics: Set[String]) = { @@ -53,7 +55,7 @@ class TestTopicMetadataCache { } def setErrorCode(topic: String, errorCode: Short) { - mockCache += topic -> new TopicMetadata(topic, List.empty, errorCode) + mockCache += topic -> new TopicMetadata(topic, List.empty, Errors.forCode(errorCode)) } } @@ -70,7 +72,7 @@ class TestTopicMetadataCache { mockStore.setErrorCode("topic1", 3) var metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, mockTime.currentTimeMillis) assertEquals("topic1", metadata("topic1").topic) - assertEquals(3, metadata("topic1").errorCode) + assertEquals(3, metadata("topic1").error.code) assertEquals(1, mockStore.numberOfCalls.get()) // Retrieve the same topic from the cache which has an error code. Ensure the store is called to refresh the cache @@ -78,21 +80,21 @@ class TestTopicMetadataCache { mockStore.setErrorCode("topic1", 0) metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, mockTime.currentTimeMillis) assertEquals("topic1", metadata("topic1").topic) - assertEquals(0, metadata("topic1").errorCode) + assertEquals(0, metadata("topic1").error.code) assertEquals(2, mockStore.numberOfCalls.get()) // Retrieve the same topic from the cache with refresh rate greater than the last update. Ensure the store is not // called metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, mockTime.currentTimeMillis) assertEquals("topic1", metadata("topic1").topic) - assertEquals(0, metadata("topic1").errorCode) + assertEquals(0, metadata("topic1").error.code) assertEquals(2, mockStore.numberOfCalls.get()) // Ensure that refresh happens when refresh rate is less than the last update. Ensure the store is called mockTime.currentValue = 11 metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, mockTime.currentTimeMillis) assertEquals("topic1", metadata("topic1").topic) - assertEquals(0, metadata("topic1").errorCode) + assertEquals(0, metadata("topic1").error.code) assertEquals(3, mockStore.numberOfCalls.get()) } @@ -113,7 +115,7 @@ class TestTopicMetadataCache { waitForThreadStart.await() val metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, mockTime.currentTimeMillis) numAssertionSuccess.compareAndSet(true, metadata("topic1").topic.equals("topic1")) - numAssertionSuccess.compareAndSet(true, metadata("topic1").errorCode == 0) + numAssertionSuccess.compareAndSet(true, metadata("topic1").error.code == 0) } }) threads(i).start() @@ -127,11 +129,11 @@ class TestTopicMetadataCache { @Test def testBadErrorCodes { - val partitionMetadataBad = new PartitionMetadata(0, None, Seq(), errorCode = ErrorMapping.LeaderNotAvailableCode) - val partitionMetadataGood = new PartitionMetadata(0, None, Seq(), errorCode = ErrorMapping.NoError) - assertTrue(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List.empty, ErrorMapping.RequestTimedOutCode))) - assertTrue(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List(partitionMetadataBad), ErrorMapping.NoError))) - assertFalse(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List.empty, ErrorMapping.NoError))) - assertFalse(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List(partitionMetadataGood), ErrorMapping.NoError))) + val partitionMetadataBad = new PartitionMetadata(0, None, Seq(), error = Errors.LEADER_NOT_AVAILABLE) + val partitionMetadataGood = new PartitionMetadata(0, None, Seq(), error = Errors.NONE) + assertTrue(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List.empty, Errors.REQUEST_TIMED_OUT))) + assertTrue(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List(partitionMetadataBad), Errors.NONE))) + assertFalse(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List.empty, Errors.NONE))) + assertFalse(TopicMetadataCache.hasBadErrorCode(new TopicMetadata("test", List(partitionMetadataGood), Errors.NONE))) } } diff --git a/samza-kafka/src/test/scala/org/apache/samza/utils/TestKafkaUtil.scala b/samza-kafka/src/test/scala/org/apache/samza/utils/TestKafkaUtil.scala index 848cfc8fcc..3548aea1a9 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/utils/TestKafkaUtil.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/utils/TestKafkaUtil.scala @@ -20,14 +20,15 @@ package org.apache.samza.utils import org.junit.Test -import org.scalatest.{ Matchers => ScalaTestMatchers } +import org.scalatest.{Matchers => ScalaTestMatchers} import org.apache.samza.util.KafkaUtil import kafka.common.ErrorMapping +import org.apache.kafka.common.protocol.Errors class TestKafkaUtil extends ScalaTestMatchers { @Test def testMaybeThrowException { - intercept[Exception] { KafkaUtil.maybeThrowException(ErrorMapping.UnknownTopicOrPartitionCode) } - KafkaUtil.maybeThrowException(ErrorMapping.ReplicaNotAvailableCode) + intercept[Exception] { KafkaUtil.maybeThrowException(Errors.UNKNOWN_TOPIC_OR_PARTITION.exception()) } + KafkaUtil.maybeThrowException(Errors.REPLICA_NOT_AVAILABLE.exception()) } } \ No newline at end of file diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 9c5dad5f54..97fe1f8532 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -541,7 +541,7 @@ public void shouldKillStreamAppWhenZooKeeperDiesBeforeLeaderReElection() throws // Set up kafka topics. publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); - MapConfig kafkaProducerConfig = new MapConfig(ImmutableMap.of(String.format("systems.%s.producer.%s", TEST_SYSTEM, ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG), "1000")); + MapConfig kafkaProducerConfig = new MapConfig(ImmutableMap.of(String.format("systems.%s.producer.%s", TEST_SYSTEM, ProducerConfig.MAX_BLOCK_MS_CONFIG), "1000")); MapConfig applicationRunnerConfig1 = new MapConfig(ImmutableList.of(applicationConfig1, kafkaProducerConfig)); MapConfig applicationRunnerConfig2 = new MapConfig(ImmutableList.of(applicationConfig2, kafkaProducerConfig)); LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationRunnerConfig1); diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala index 4ba51f34a3..d9261ad902 100644 --- a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala +++ b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala @@ -117,9 +117,7 @@ object StreamTaskTestUtil { }) servers = configs.map(TestUtils.createServer(_)).toBuffer - - val brokerList = TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXT) - brokers = brokerList.split(",").map(p => "127.0.0.1" + p).mkString(",") + brokers = TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXT) // setup the zookeeper and bootstrap servers for local kafka cluster jobConfig ++= Map("systems.kafka.consumer.zookeeper.connect" -> zkConnect, @@ -161,9 +159,8 @@ object StreamTaskTestUtil { topics.foreach(topic => { val topicMetadata = topicMetadataMap(topic) - val errorCode = topicMetadata.errorCode - KafkaUtil.maybeThrowException(errorCode) + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) }) done = true From 9893069cb5186d58e3cf5518ce8e5a7251d8b131 Mon Sep 17 00:00:00 2001 From: Srinivasulu Punuru Date: Wed, 10 Jan 2018 11:17:39 -0800 Subject: [PATCH 31/36] SAMZA-1535: Support for UDFs in where clauses The existing version of the udf implementation doesn't seem to support udfs in the where clauses because the Type of the object returned is "ANY" and when you do a `select * from kafka.topic where regexMatch('.*foo', Name)` it fails in the query validation, because calcite doesn't know the type of regexMatch. To solve the problem, We made the scalarUdf generic with a strongly typed return type. https://issues.apache.org/jira/browse/SAMZA-1535 This PR can be merged into trunk not the 0.14. Author: Srinivasulu Punuru Reviewers: Xinyu Liu Closes #386 from srinipunuru/udf-where.1 --- .../org/apache/samza/sql/udfs/ScalarUdf.java | 4 ++-- .../apache/samza/sql/avro/AvroRelConverter.java | 2 +- .../samza/sql/data/SamzaSqlRelMessage.java | 2 ++ .../java/org/apache/samza/sql/fn/FlattenUdf.java | 2 +- .../org/apache/samza/sql/fn}/RegexMatchUdf.java | 7 +++---- .../samza/sql/impl/ConfigBasedUdfResolver.java | 12 ++++++------ .../sql/planner/SamzaSqlScalarFunctionImpl.java | 7 +++++-- .../samza/sql/avro/TestAvroRelConversion.java | 14 ++++++++++++++ .../samza/sql/e2e/TestSamzaSqlEndToEnd.java | 16 ++++++++++++++++ .../samza/sql/testutil/MyTestArrayUdf.java | 5 +++-- .../org/apache/samza/sql/testutil/MyTestUdf.java | 2 +- .../samza/sql/testutil/SamzaSqlTestConfig.java | 6 ++++-- .../org/apache/samza/tools/SamzaSqlConsole.java | 12 ++++++------ 13 files changed, 64 insertions(+), 27 deletions(-) rename {samza-tools/src/main/java/org/apache/samza/tools/udf => samza-sql/src/main/java/org/apache/samza/sql/fn}/RegexMatchUdf.java (88%) diff --git a/samza-api/src/main/java/org/apache/samza/sql/udfs/ScalarUdf.java b/samza-api/src/main/java/org/apache/samza/sql/udfs/ScalarUdf.java index 719cace87c..f5c111a42b 100644 --- a/samza-api/src/main/java/org/apache/samza/sql/udfs/ScalarUdf.java +++ b/samza-api/src/main/java/org/apache/samza/sql/udfs/ScalarUdf.java @@ -30,7 +30,7 @@ * select myudf(id, name) from profile * In the above query, Profile should contain fields named 'id' of INTEGER/NUMBER type and 'name' of type VARCHAR/CHARACTER */ -public interface ScalarUdf { +public interface ScalarUdf { /** * Udfs can implement this method to perform any initialization that they may need. * @param udfConfig Config specific to the udf. @@ -44,5 +44,5 @@ public interface ScalarUdf { * @return * Return value from the scalar udf. */ - Object execute(Object... args); + T execute(Object... args); } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java b/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java index ab46a9870c..086bb9362b 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java @@ -155,7 +155,7 @@ public SamzaSqlRelMessage convertToRelMessage(KV samzaMessage) { .collect(Collectors.toList())); } else if (value == null) { fieldNames.addAll(relationalSchema.getFieldNames()); - IntStream.range(0, fieldNames.size() - 1).forEach(x -> values.add(null)); + IntStream.range(0, fieldNames.size()).forEach(x -> values.add(null)); } else { String msg = "Avro message converter doesn't support messages of type " + value.getClass(); LOG.error(msg); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlRelMessage.java b/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlRelMessage.java index b5df545eaa..bf945a05d4 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlRelMessage.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlRelMessage.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import org.apache.commons.lang.Validate; /** @@ -51,6 +52,7 @@ public class SamzaSqlRelMessage { * all the fields in the row can be null. */ public SamzaSqlRelMessage(Object key, List names, List values) { + Validate.isTrue(names.size() == values.size(), "Field Names and values are not of same length."); this.key = key; this.value.addAll(values); this.names.addAll(names); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/fn/FlattenUdf.java b/samza-sql/src/main/java/org/apache/samza/sql/fn/FlattenUdf.java index 39a26ce751..e8d643dcf2 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/fn/FlattenUdf.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/fn/FlattenUdf.java @@ -24,7 +24,7 @@ import org.apache.samza.sql.udfs.ScalarUdf; -public class FlattenUdf implements ScalarUdf { +public class FlattenUdf implements ScalarUdf { @Override public void init(Config udfConfig) { } diff --git a/samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.java b/samza-sql/src/main/java/org/apache/samza/sql/fn/RegexMatchUdf.java similarity index 88% rename from samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.java rename to samza-sql/src/main/java/org/apache/samza/sql/fn/RegexMatchUdf.java index 42e9f04db7..1f5307d958 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/udf/RegexMatchUdf.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/fn/RegexMatchUdf.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.samza.tools.udf; +package org.apache.samza.sql.fn; import java.util.regex.Pattern; import org.apache.samza.config.Config; @@ -27,14 +27,13 @@ /** * Simple RegexMatch Udf. */ -public class RegexMatchUdf implements ScalarUdf { +public class RegexMatchUdf implements ScalarUdf { @Override public void init(Config config) { } - @Override - public Object execute(Object... args) { + public Boolean execute(Object... args) { return Pattern.matches((String) args[0], (String) args[1]); } } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedUdfResolver.java b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedUdfResolver.java index c98a4a1b16..412ff3b8ec 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedUdfResolver.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedUdfResolver.java @@ -27,7 +27,6 @@ import java.util.Optional; import java.util.Properties; import java.util.stream.Collectors; - import org.apache.commons.lang.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; @@ -74,19 +73,20 @@ public ConfigBasedUdfResolver(Properties config, Config udfConfig) { throw new SamzaException(msg); } - Optional udfMethod = - Arrays.stream(udfClass.getMethods()).filter(x -> x.getName().equals(UDF_METHOD_NAME)).findFirst(); + Method udfMethod; - if (!udfMethod.isPresent()) { + try { + udfMethod = udfClass.getMethod(UDF_METHOD_NAME, Object[].class); + } catch (NoSuchMethodException e) { String msg = String.format("Udf Class %s doesn't implement method named %s", udfClassName, UDF_METHOD_NAME); LOG.error(msg); - throw new SamzaException(msg); + throw new SamzaException(msg, e); } int udfIndex = udfClass.getSimpleName().toLowerCase().lastIndexOf("udf"); String udfName = udfClass.getSimpleName().substring(0, udfIndex); - udfs.add(new UdfMetadata(udfName, udfMethod.get(), udfConfig.subset(udfName + "."))); + udfs.add(new UdfMetadata(udfName, udfMethod, udfConfig.subset(udfName + "."))); } } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlScalarFunctionImpl.java b/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlScalarFunctionImpl.java index 1200d57ff0..6894c86ec4 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlScalarFunctionImpl.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlScalarFunctionImpl.java @@ -67,8 +67,11 @@ public CallImplementor getImplementor() { final Expression context = Expressions.parameter(SamzaSqlExecutionContext.class, "context"); final Expression getUdfInstance = Expressions.call(ScalarUdf.class, context, getUdfMethod, Expressions.constant(udfMethod.getDeclaringClass().getName()), Expressions.constant(udfName)); - return Expressions.call(Expressions.convert_(getUdfInstance, udfMethod.getDeclaringClass()), udfMethod, - translatedOperands); + final Expression callExpression = Expressions.convert_(Expressions.call(Expressions.convert_(getUdfInstance, udfMethod.getDeclaringClass()), udfMethod, + translatedOperands), Object.class); + // The Janino compiler which is used to compile the expressions doesn't seem to understand the Type of the ScalarUdf.execute + // because it is a generic. To work around that we are explicitly casting it to the return type. + return Expressions.convert_(callExpression, udfMethod.getReturnType()); }, NullPolicy.NONE, false); } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/avro/TestAvroRelConversion.java b/samza-sql/src/test/java/org/apache/samza/sql/avro/TestAvroRelConversion.java index a54db5495c..44b421353d 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/avro/TestAvroRelConversion.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/avro/TestAvroRelConversion.java @@ -137,6 +137,20 @@ public void testSimpleRecordConversion() { LOG.info(Joiner.on(",").join(message.getFieldNames())); } + @Test + public void testEmptyRecordConversion() { + GenericData.Record record = new GenericData.Record(SimpleRecord.SCHEMA$); + SamzaSqlRelMessage message = simpleRecordAvroRelConverter.convertToRelMessage(new KV<>("key", record)); + Assert.assertEquals(message.getFieldNames().size(), message.getFieldValues().size()); + } + + + @Test + public void testNullRecordConversion() { + SamzaSqlRelMessage message = simpleRecordAvroRelConverter.convertToRelMessage(new KV<>("key", null)); + Assert.assertEquals(message.getFieldNames().size(), message.getFieldValues().size()); + } + public static byte[] encodeAvroSpecificRecord(Class clazz, T record) throws IOException { DatumWriter msgDatumWriter = new SpecificDatumWriter<>(clazz); ByteArrayOutputStream os = new ByteArrayOutputStream(); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlEndToEnd.java b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlEndToEnd.java index 769c653f16..8baa9e7ea6 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlEndToEnd.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlEndToEnd.java @@ -134,4 +134,20 @@ public void testEndToEndUdf() throws Exception { Assert.assertTrue( IntStream.range(0, numMessages).map(udf::execute).boxed().collect(Collectors.toList()).equals(outMessages)); } + + @Test + public void testRegexMatchUdfInWhereClause() throws Exception { + int numMessages = 20; + TestAvroSystemFactory.messages.clear(); + Map staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(numMessages); + String sql1 = "Insert into testavro.outputTopic select id from testavro.SIMPLE1 where RegexMatch('.*4', Name)"; + List sqlStmts = Collections.singletonList(sql1); + staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + runner.runAndWaitForFinish(); + + LOG.info("output Messages " + TestAvroSystemFactory.messages); + // There should be two messages that contain "4" + Assert.assertEquals(TestAvroSystemFactory.messages.size(), 2); + } } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestArrayUdf.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestArrayUdf.java index c9bc9bcfe8..b2a2c2533f 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestArrayUdf.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestArrayUdf.java @@ -19,18 +19,19 @@ package org.apache.samza.sql.testutil; +import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.samza.config.Config; import org.apache.samza.sql.udfs.ScalarUdf; -public class MyTestArrayUdf implements ScalarUdf { +public class MyTestArrayUdf implements ScalarUdf> { @Override public void init(Config udfConfig) { } - public Object execute(Object... args) { + public List execute(Object... args) { Integer value = (Integer) args[0]; return IntStream.range(0, value).mapToObj(String::valueOf).collect(Collectors.toList()); } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestUdf.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestUdf.java index baca367260..c718368ba6 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestUdf.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/MyTestUdf.java @@ -28,7 +28,7 @@ /** * Test UDF used by unit and integration tests. */ -public class MyTestUdf implements ScalarUdf { +public class MyTestUdf implements ScalarUdf { private static final Logger LOG = LoggerFactory.getLogger(MyTestUdf.class); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java index 760d5d0356..008eb82211 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java @@ -31,6 +31,7 @@ import org.apache.samza.sql.avro.schemas.ComplexRecord; import org.apache.samza.sql.avro.schemas.SimpleRecord; import org.apache.samza.sql.fn.FlattenUdf; +import org.apache.samza.sql.fn.RegexMatchUdf; import org.apache.samza.sql.impl.ConfigBasedSourceResolverFactory; import org.apache.samza.sql.impl.ConfigBasedUdfResolver; import org.apache.samza.sql.interfaces.SqlSystemStreamConfig; @@ -64,8 +65,9 @@ public static Map fetchStaticConfigsWithFactories(int numberOfMe String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config"); staticConfigs.put(configUdfResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY, ConfigBasedUdfResolver.class.getName()); - staticConfigs.put(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES, - Joiner.on(",").join(MyTestUdf.class.getName(), FlattenUdf.class.getName(), MyTestArrayUdf.class.getName())); + staticConfigs.put(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES, Joiner.on(",") + .join(MyTestUdf.class.getName(), RegexMatchUdf.class.getName(), FlattenUdf.class.getName(), + MyTestArrayUdf.class.getName())); String avroSystemConfigPrefix = String.format(ConfigBasedSourceResolverFactory.CFG_FMT_SAMZA_PREFIX, SAMZA_SYSTEM_TEST_AVRO); diff --git a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java index 9803117144..11a49f751e 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java @@ -20,12 +20,6 @@ package org.apache.samza.tools; import com.google.common.base.Joiner; -import org.apache.samza.tools.avro.AvroSchemaGenRelConverterFactory; -import org.apache.samza.tools.avro.AvroSerDeFactory; -import org.apache.samza.tools.json.JsonRelConverterFactory; -import org.apache.samza.tools.schemas.PageViewEvent; -import org.apache.samza.tools.schemas.ProfileChangeEvent; -import org.apache.samza.tools.udf.RegexMatchUdf; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -43,6 +37,7 @@ import org.apache.samza.serializers.StringSerdeFactory; import org.apache.samza.sql.avro.ConfigBasedAvroRelSchemaProviderFactory; import org.apache.samza.sql.fn.FlattenUdf; +import org.apache.samza.sql.fn.RegexMatchUdf; import org.apache.samza.sql.impl.ConfigBasedSourceResolverFactory; import org.apache.samza.sql.impl.ConfigBasedUdfResolver; import org.apache.samza.sql.interfaces.SqlSystemStreamConfig; @@ -52,6 +47,11 @@ import org.apache.samza.sql.testutil.SqlFileParser; import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; import org.apache.samza.system.kafka.KafkaSystemFactory; +import org.apache.samza.tools.avro.AvroSchemaGenRelConverterFactory; +import org.apache.samza.tools.avro.AvroSerDeFactory; +import org.apache.samza.tools.json.JsonRelConverterFactory; +import org.apache.samza.tools.schemas.PageViewEvent; +import org.apache.samza.tools.schemas.ProfileChangeEvent; public class SamzaSqlConsole { From 267dfc6ba0eddd78d5913663496c65eaa1aa3c0b Mon Sep 17 00:00:00 2001 From: Srinivasulu Punuru Date: Wed, 10 Jan 2018 17:14:03 -0800 Subject: [PATCH 32/36] Fix for the TestSamzaSqlApplicationConfig.testConfigInit Currently testConfigInit checks for a hardcoded number for udfs. Whenever a new UDF is added, This test is going to fail if it is not updated. Changed the test to validate the number of udfs based on the config that is passed. Author: Srinivasulu Punuru Reviewers: Prateek Maheshwari Closes #403 from srinipunuru/testfix.1 --- .../org/apache/samza/sql/TestSamzaSqlApplicationConfig.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java index 0b85339928..1c5fc41f62 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java @@ -24,6 +24,7 @@ import junit.framework.Assert; import org.apache.samza.SamzaException; import org.apache.samza.config.MapConfig; +import org.apache.samza.sql.impl.ConfigBasedUdfResolver; import org.apache.samza.sql.interfaces.SqlSystemStreamConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; import org.apache.samza.sql.testutil.SamzaSqlTestConfig; @@ -36,9 +37,11 @@ public class TestSamzaSqlApplicationConfig { public void testConfigInit() { Map config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1"); + String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config"); + int numUdfs = config.get(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES).split(",").length; SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); Assert.assertEquals(1, samzaSqlApplicationConfig.getQueryInfo().size()); - Assert.assertEquals(3, samzaSqlApplicationConfig.getUdfMetadata().size()); + Assert.assertEquals(numUdfs, samzaSqlApplicationConfig.getUdfMetadata().size()); Assert.assertEquals(1, samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().size()); Assert.assertEquals(1, samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().size()); } From 3a9e8064211a79827a2a4793c1f159a689dfa256 Mon Sep 17 00:00:00 2001 From: Prateek Maheshwari Date: Wed, 17 Jan 2018 09:39:26 -0800 Subject: [PATCH 33/36] SAMZA-1500: Added metrics for RocksDB state store memory usage Approximate RocksDB memory usage = Configured Block Cache size + MemTable size + Indexes and Bloom Filters size = rocksdb.block-cache-size + rocksdb.size-all-mem-tables + rocksdb.estimate-table-readers-mem Author: Prateek Maheshwari Reviewers: Jagadish Closes #404 from prateekm/rocksdb-memory --- .../storage/kv/RocksDbOptionsHelper.java | 12 ++++++--- .../RocksDbKeyValueStorageEngineFactory.scala | 3 +++ .../storage/kv/RocksDbKeyValueStore.scala | 25 +++++++++++++------ .../storage/kv/TestRocksDbKeyValueStore.scala | 6 ++--- 4 files changed, 32 insertions(+), 14 deletions(-) diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java index 9b8f44b195..9389681eca 100644 --- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java @@ -75,12 +75,10 @@ public static Options options(Config storeConfig, SamzaContainerContext containe } options.setCompressionType(compressionType); - Long cacheSize = storeConfig.getLong("container.cache.size.bytes", 100 * 1024 * 1024L); - Long cacheSizePerContainer = cacheSize / numTasks; - + long blockCacheSize = getBlockCacheSize(storeConfig, containerContext); int blockSize = storeConfig.getInt(ROCKSDB_BLOCK_SIZE_BYTES, 4096); BlockBasedTableConfig tableOptions = new BlockBasedTableConfig(); - tableOptions.setBlockCacheSize(cacheSizePerContainer).setBlockSize(blockSize); + tableOptions.setBlockCacheSize(blockCacheSize).setBlockSize(blockSize); options.setTableFormatConfig(tableOptions); CompactionStyle compactionStyle = CompactionStyle.UNIVERSAL; @@ -110,4 +108,10 @@ public static Options options(Config storeConfig, SamzaContainerContext containe return options; } + + public static Long getBlockCacheSize(Config storeConfig, SamzaContainerContext containerContext) { + int numTasks = containerContext.taskNames.size(); + long cacheSize = storeConfig.getLong("container.cache.size.bytes", 100 * 1024 * 1024L); + return cacheSize / numTasks; + } } \ No newline at end of file diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala index a7b748f173..2b7ffb5977 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala @@ -44,6 +44,9 @@ class RocksDbKeyValueStorageEngineFactory [K, V] extends BaseKeyValueStorageEngi val storageConfig = containerContext.config.subset("stores." + storeName + ".", true) val isLoggedStore = containerContext.config.getChangelogStream(storeName).isDefined val rocksDbMetrics = new KeyValueStoreMetrics(storeName, registry) + rocksDbMetrics.newGauge("rocksdb.block-cache-size", + () => RocksDbOptionsHelper.getBlockCacheSize(storageConfig, containerContext)) + val rocksDbOptions = RocksDbOptionsHelper.options(storageConfig, containerContext) val rocksDbWriteOptions = new WriteOptions().setDisableWAL(true) val rocksDbFlushOptions = new FlushOptions().setWaitForFlush(true) diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 6aad45f573..eae7da2c1f 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -72,13 +72,24 @@ object RocksDbKeyValueStore extends Logging { RocksDB.open(options, dir.toString) } - if (storeConfig.containsKey("rocksdb.metrics.list")) { - storeConfig - .get("rocksdb.metrics.list") - .split(",") - .map(property => property.trim) - .foreach(property => metrics.newGauge(property, () => rocksDb.getProperty(property))) - } + // See https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h for available properties + val rocksDbMetrics = Set ( + "rocksdb.estimate-table-readers-mem", // indexes and bloom filters + "rocksdb.cur-size-active-mem-table", // approximate active memtable size in bytes + "rocksdb.cur-size-all-mem-tables", // approximate active and unflushed memtable size in bytes + "rocksdb.size-all-mem-tables", // approximate active, unflushed and pinned memtable size in bytes + "rocksdb.estimate-num-keys" // approximate number keys in the active and unflushed memtable and storage + ) + + val configuredMetrics = storeConfig + .get("rocksdb.metrics.list", "") + .split(",") + .map(property => property.trim) + .filter(!_.isEmpty) + .toSet + + (configuredMetrics ++ rocksDbMetrics) + .foreach(property => metrics.newGauge(property, () => rocksDb.getProperty(property))) rocksDb } catch { diff --git a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala index 418e9867ce..ca9c023ad8 100644 --- a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala @@ -44,7 +44,7 @@ class TestRocksDbKeyValueStore config, false, "someStore", - null) + new KeyValueStoreMetrics()) val key = "test".getBytes("UTF-8") rocksDB.put(key, "val".getBytes("UTF-8")) Assert.assertNotNull(rocksDB.get(key)) @@ -76,7 +76,7 @@ class TestRocksDbKeyValueStore config, false, "dbStore", - null) + new KeyValueStoreMetrics()) val key = "key".getBytes("UTF-8") rocksDB.put(key, "val".getBytes("UTF-8")) // SAMZA-836: Mysteriously,calling new FlushOptions() does not invoke the NativeLibraryLoader in rocksdbjni-3.13.1! @@ -136,7 +136,7 @@ class TestRocksDbKeyValueStore config, false, "dbStore", - null) + new KeyValueStoreMetrics()) val key = "key".getBytes("UTF-8") val key1 = "key1".getBytes("UTF-8") From fcd2b9a53367c2e0b53df7a1fb7eedc8086b285c Mon Sep 17 00:00:00 2001 From: Srinivasulu Punuru Date: Wed, 17 Jan 2018 10:06:50 -0800 Subject: [PATCH 34/36] SAMZA-1556: Adding support for multi level sources in queries Right now Samza SQL supports queries with just two levels i.e. `select * from foo.bar`. But there can be sources that are identified though multiple levels. for e.g. `select * from kafka.clusterName.topicName`. This change adds the support for sql queries with sources that have more than two levels. Author: Srinivasulu Punuru Reviewers: Miguel S, Aditya T Closes #405 from srinipunuru/multi-level.1 --- .../ConfigBasedSourceResolverFactory.java | 8 +-- .../sql/interfaces/SqlSystemStreamConfig.java | 47 ++++++++++++++++- .../samza/sql/planner/QueryPlanner.java | 38 +++++++++----- .../sql/runner/SamzaSqlApplicationConfig.java | 23 ++++----- .../sql/runner/SamzaSqlApplicationRunner.java | 3 ++ .../samza/sql/translator/QueryTranslator.java | 5 +- .../samza/sql/translator/ScanTranslator.java | 26 ++++------ .../apache/samza/sql/TestQueryTranslator.java | 2 +- .../sql/testutil/SamzaSqlTestConfig.java | 2 +- .../testutil/TestSourceResolverFactory.java | 50 +++++++++++++++++++ 10 files changed, 154 insertions(+), 50 deletions(-) create mode 100644 samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSourceResolverFactory.java diff --git a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedSourceResolverFactory.java b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedSourceResolverFactory.java index 1203b254ca..a2d8b0c04f 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedSourceResolverFactory.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedSourceResolverFactory.java @@ -29,8 +29,8 @@ /** - * Schema Resolver that uses static config to return a config corresponding to a system stream. - * Schemas are configured using the config of format {systemName}.{streamName}.schema. + * Source Resolver implementation that uses static config to return a config corresponding to a system stream. + * This Source resolver implementation supports sources of type {systemName}.{streamName} */ public class ConfigBasedSourceResolverFactory implements SourceResolverFactory { @@ -53,8 +53,10 @@ public ConfigBasedSourceResolver(Config config) { @Override public SqlSystemStreamConfig fetchSourceInfo(String source) { String[] sourceComponents = source.split("\\."); + + // This source resolver expects sources of format {systemName}.{streamName} if (sourceComponents.length != 2) { - String msg = String.format("Source %s is not of the format .", source); + String msg = String.format("Source %s is not of the format {systemName}.{streamName{", source); LOG.error(msg); throw new SamzaException(msg); } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlSystemStreamConfig.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlSystemStreamConfig.java index df21784447..d8965a43f6 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlSystemStreamConfig.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlSystemStreamConfig.java @@ -19,8 +19,13 @@ package org.apache.samza.sql.interfaces; +import com.google.common.base.Joiner; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; import org.apache.commons.lang.Validate; import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; import org.apache.samza.system.SystemStream; @@ -38,18 +43,48 @@ public class SqlSystemStreamConfig { private final String samzaRelConverterName; private final SystemStream systemStream; + + private final String source; private String relSchemaProviderName; + private Config config; + + private List sourceParts; + public SqlSystemStreamConfig(String systemName, String streamName, Config systemConfig) { + this(systemName, streamName, Arrays.asList(systemName, streamName), systemConfig); + } + + public SqlSystemStreamConfig(String systemName, String streamName, List sourceParts, + Config systemConfig) { + + HashMap streamConfigs = new HashMap<>(systemConfig); this.systemName = systemName; this.streamName = streamName; + this.source = getSourceFromSourceParts(sourceParts); + this.sourceParts = sourceParts; this.systemStream = new SystemStream(systemName, streamName); - samzaRelConverterName = systemConfig.get(CFG_SAMZA_REL_CONVERTER); - relSchemaProviderName = systemConfig.get(CFG_REL_SCHEMA_PROVIDER); + samzaRelConverterName = streamConfigs.get(CFG_SAMZA_REL_CONVERTER); Validate.notEmpty(samzaRelConverterName, String.format("%s is not set or empty for system %s", CFG_SAMZA_REL_CONVERTER, systemName)); + + relSchemaProviderName = streamConfigs.get(CFG_REL_SCHEMA_PROVIDER); + + // Removing the Samza SQL specific configs to get the remaining Samza configs. + streamConfigs.remove(CFG_SAMZA_REL_CONVERTER); + streamConfigs.remove(CFG_REL_SCHEMA_PROVIDER); + + config = new MapConfig(streamConfigs); + } + + public static String getSourceFromSourceParts(List sourceParts) { + return Joiner.on(".").join(sourceParts); + } + + public List getSourceParts() { + return sourceParts; } public String getSystemName() { @@ -71,4 +106,12 @@ public String getRelSchemaProviderName() { public SystemStream getSystemStream() { return systemStream; } + + public Config getConfig() { + return config; + } + + public String getSource() { + return source; + } } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/planner/QueryPlanner.java b/samza-sql/src/main/java/org/apache/samza/sql/planner/QueryPlanner.java index ce03ba3166..061c03fd5a 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/planner/QueryPlanner.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/planner/QueryPlanner.java @@ -23,7 +23,6 @@ import java.sql.DriverManager; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -69,11 +68,16 @@ public class QueryPlanner { private static final Logger LOG = LoggerFactory.getLogger(QueryPlanner.class); - private final Map systemStreamConfigBySource; private final Collection udfMetadata; - private final Map relSchemaProviders; - public QueryPlanner(Map relSchemaProviders, + // Mapping between the source to the RelSchemaProvider corresponding to the source. + private final Map relSchemaProviders; + + // Mapping between the source to the SqlSystemStreamConfig corresponding to the source. + private final Map systemStreamConfigBySource; + + + public QueryPlanner(Map relSchemaProviders, Map systemStreamConfigBySource, Collection udfMetadata) { this.relSchemaProviders = relSchemaProviders; this.systemStreamConfigBySource = systemStreamConfigBySource; @@ -85,16 +89,26 @@ public RelRoot plan(String query) { Connection connection = DriverManager.getConnection("jdbc:calcite:"); CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class); SchemaPlus rootSchema = calciteConnection.getRootSchema(); - Map systemSchemas = new HashMap<>(); for (SqlSystemStreamConfig ssc : systemStreamConfigBySource.values()) { - - RelSchemaProvider relSchemaProvider = relSchemaProviders.get(ssc.getSystemStream()); - SchemaPlus systemSchema = - systemSchemas.computeIfAbsent(ssc.getSystemName(), s -> rootSchema.add(s, new AbstractSchema())); - RelDataType relationalSchema = relSchemaProvider.getRelationalSchema(); - - systemSchema.add(ssc.getStreamName(), createTableFromRelSchema(relationalSchema)); + SchemaPlus previousLevelSchema = rootSchema; + List sourceParts = ssc.getSourceParts(); + RelSchemaProvider relSchemaProvider = relSchemaProviders.get(ssc.getSource()); + + for (String sourcePart : sourceParts) { + if (!sourcePart.equalsIgnoreCase(ssc.getStreamName())) { + SchemaPlus sourcePartSchema = rootSchema.getSubSchema(sourcePart); + if (sourcePartSchema == null) { + sourcePartSchema = previousLevelSchema.add(sourcePart, new AbstractSchema()); + } + previousLevelSchema = sourcePartSchema; + } else { + // If the source part is the streamName, then fetch the schema corresponding to the stream and register. + RelDataType relationalSchema = relSchemaProvider.getRelationalSchema(); + previousLevelSchema.add(ssc.getStreamName(), createTableFromRelSchema(relationalSchema)); + break; + } + } } List samzaSqlFunctions = udfMetadata.stream() diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java index f0297452a6..227a0f1a85 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java @@ -78,8 +78,8 @@ public class SamzaSqlApplicationConfig { public static final String CFG_UDF_RESOLVER = "samza.sql.udfResolver"; public static final String CFG_FMT_UDF_RESOLVER_DOMAIN = "samza.sql.udfResolver.%s."; - private final Map relSchemaProvidersBySystemStream; - private final Map samzaRelConvertersBySystemStream; + private final Map relSchemaProvidersBySource; + private final Map samzaRelConvertersBySource; private SourceResolver sourceResolver; private UdfResolver udfResolver; @@ -87,7 +87,6 @@ public class SamzaSqlApplicationConfig { private final Collection udfMetadata; private final Map inputSystemStreamConfigBySource; - private final Map outputSystemStreamConfigsBySource; private final List sql; @@ -117,17 +116,17 @@ public SamzaSqlApplicationConfig(Config staticConfig) { .collect(Collectors.toMap(Function.identity(), x -> sourceResolver.fetchSourceInfo(x))); systemStreamConfigs.addAll(outputSystemStreamConfigsBySource.values()); - relSchemaProvidersBySystemStream = systemStreamConfigs.stream() - .collect(Collectors.toMap(SqlSystemStreamConfig::getSystemStream, + relSchemaProvidersBySource = systemStreamConfigs.stream() + .collect(Collectors.toMap(SqlSystemStreamConfig::getSource, x -> initializePlugin("RelSchemaProvider", x.getRelSchemaProviderName(), staticConfig, CFG_FMT_REL_SCHEMA_PROVIDER_DOMAIN, (o, c) -> ((RelSchemaProviderFactory) o).create(x.getSystemStream(), c)))); - samzaRelConvertersBySystemStream = systemStreamConfigs.stream() - .collect(Collectors.toMap(SqlSystemStreamConfig::getSystemStream, + samzaRelConvertersBySource = systemStreamConfigs.stream() + .collect(Collectors.toMap(SqlSystemStreamConfig::getSource, x -> initializePlugin("SamzaRelConverter", x.getSamzaRelConverterName(), staticConfig, CFG_FMT_SAMZA_REL_CONVERTER_DOMAIN, (o, c) -> ((SamzaRelConverterFactory) o).create(x.getSystemStream(), - relSchemaProvidersBySystemStream.get(x.getSystemStream()), c)))); + relSchemaProvidersBySource.get(x.getSource()), c)))); } private static T initializePlugin(String pluginName, String plugin, Config staticConfig, @@ -235,11 +234,11 @@ public Map getOutputSystemStreamConfigsBySource() return outputSystemStreamConfigsBySource; } - public Map getSamzaRelConverters() { - return samzaRelConvertersBySystemStream; + public Map getSamzaRelConverters() { + return samzaRelConvertersBySource; } - public Map getRelSchemaProviders() { - return relSchemaProvidersBySystemStream; + public Map getRelSchemaProviders() { + return relSchemaProvidersBySource; } } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java index 57d889f1f3..5215f7e47a 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java @@ -85,10 +85,13 @@ public static Config computeSamzaConfigs(Boolean localRunner, Config config) { SqlSystemStreamConfig inputSystemStreamConfig = sourceResolver.fetchSourceInfo(inputSource); newConfig.put(String.format(CFG_FMT_SAMZA_STREAM_SYSTEM, inputSystemStreamConfig.getStreamName()), inputSystemStreamConfig.getSystemName()); + newConfig.putAll(inputSystemStreamConfig.getConfig()); } + SqlSystemStreamConfig outputSystemStreamConfig = sourceResolver.fetchSourceInfo(query.getOutputSource()); newConfig.put(String.format(CFG_FMT_SAMZA_STREAM_SYSTEM, outputSystemStreamConfig.getStreamName()), outputSystemStreamConfig.getSystemName()); + newConfig.putAll(outputSystemStreamConfig.getConfig()); } if (localRunner) { diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index ab17018075..87e37f4f19 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -50,7 +50,8 @@ public class QueryTranslator { public QueryTranslator(SamzaSqlApplicationConfig sqlConfig) { this.sqlConfig = sqlConfig; - scanTranslator = new ScanTranslator(sqlConfig.getSamzaRelConverters()); + scanTranslator = + new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource()); } public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamGraph streamGraph) { @@ -87,7 +88,7 @@ public RelNode visit(LogicalProject project) { SqlSystemStreamConfig outputSystemConfig = sqlConfig.getOutputSystemStreamConfigsBySource().get(queryInfo.getOutputSource()); - SamzaRelConverter samzaMsgConverter = sqlConfig.getSamzaRelConverters().get(outputSystemConfig.getSystemStream()); + SamzaRelConverter samzaMsgConverter = sqlConfig.getSamzaRelConverters().get(queryInfo.getOutputSource()); MessageStreamImpl stream = (MessageStreamImpl) context.getMessageStream(node.getId()); MessageStream> outputStream = stream.map(samzaMsgConverter::convertToSamzaMessage); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java index 202bdbd1ab..30e5a9b3d2 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java @@ -21,19 +21,14 @@ import java.util.List; import java.util.Map; - import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.rel.type.RelDataType; import org.apache.commons.lang.Validate; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.sql.data.SamzaSqlRelMessage; import org.apache.samza.sql.interfaces.SamzaRelConverter; - -import com.google.common.base.Joiner; -import org.apache.samza.system.SystemStream; +import org.apache.samza.sql.interfaces.SqlSystemStreamConfig; /** @@ -42,25 +37,22 @@ */ public class ScanTranslator { - private final Map relMsgConverters; + private final Map relMsgConverters; + private final Map systemStreamConfig; - public ScanTranslator(Map converters) { + public ScanTranslator(Map converters, Map ssc) { relMsgConverters = converters; + this.systemStreamConfig = ssc; } public void translate(final TableScan tableScan, final TranslatorContext context) { StreamGraph streamGraph = context.getStreamGraph(); List tableNameParts = tableScan.getTable().getQualifiedName(); - Validate.isTrue(tableNameParts.size() == 2, - String.format("table name %s is not of the format .", - Joiner.on(".").join(tableNameParts))); - - String streamName = tableNameParts.get(1); - String systemName = tableNameParts.get(0); - SystemStream systemStream = new SystemStream(systemName, streamName); + String sourceName = SqlSystemStreamConfig.getSourceFromSourceParts(tableNameParts); - Validate.isTrue(relMsgConverters.containsKey(systemStream), String.format("Unknown system %s", systemName)); - SamzaRelConverter converter = relMsgConverters.get(systemStream); + Validate.isTrue(relMsgConverters.containsKey(sourceName), String.format("Unknown source %s", sourceName)); + SamzaRelConverter converter = relMsgConverters.get(sourceName); + String streamName = systemStreamConfig.get(sourceName).getStreamName(); MessageStream> inputStream = streamGraph.getInputStream(streamName); MessageStream samzaSqlRelMessageStream = inputStream.map(converter::convertToRelMessage); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java index 7fa99749c5..04fdec55a7 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java @@ -39,7 +39,7 @@ public class TestQueryTranslator { public void testTranslate() { Map config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, - "Insert into testavro.outputTopic select MyTest(id) from testavro.SIMPLE1"); + "Insert into testavro.outputTopic select MyTest(id) from testavro.level1.level2.SIMPLE1"); Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java index 008eb82211..92766f66d2 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java @@ -59,7 +59,7 @@ public static Map fetchStaticConfigsWithFactories(int numberOfMe String configSourceResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_SOURCE_RESOLVER_DOMAIN, "config"); staticConfigs.put(configSourceResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY, - ConfigBasedSourceResolverFactory.class.getName()); + TestSourceResolverFactory.class.getName()); staticConfigs.put(SamzaSqlApplicationConfig.CFG_UDF_RESOLVER, "config"); String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config"); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSourceResolverFactory.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSourceResolverFactory.java new file mode 100644 index 0000000000..b9cf8032e2 --- /dev/null +++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSourceResolverFactory.java @@ -0,0 +1,50 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package org.apache.samza.sql.testutil; + +import java.util.Arrays; +import org.apache.samza.config.Config; +import org.apache.samza.sql.interfaces.SourceResolver; +import org.apache.samza.sql.interfaces.SourceResolverFactory; +import org.apache.samza.sql.interfaces.SqlSystemStreamConfig; + + +public class TestSourceResolverFactory implements SourceResolverFactory { + @Override + public SourceResolver create(Config config) { + return new TestSourceResolver(config); + } + + private class TestSourceResolver implements SourceResolver { + private final Config config; + + public TestSourceResolver(Config config) { + this.config = config; + } + + @Override + public SqlSystemStreamConfig fetchSourceInfo(String sourceName) { + String[] sourceComponents = sourceName.split("\\."); + Config systemConfigs = config.subset(sourceComponents[0] + "."); + return new SqlSystemStreamConfig(sourceComponents[0], sourceComponents[sourceComponents.length - 1], + Arrays.asList(sourceComponents), systemConfigs); + } + } +} From 95a71fb9e234c9fc7fcb8a4ec0bbab33c9555738 Mon Sep 17 00:00:00 2001 From: navina Date: Wed, 17 Jan 2018 10:11:40 -0800 Subject: [PATCH 35/36] SAMZA-1523: Cleanup table entries before shutting down the processor Modified the `TableUtils#deleteProcessorEntity` to provide an option to disable optimistic locking during a call to Azure Table Storage service. sborya PawasChhokra nickpan47 Review please? Author: navina Reviewers: Shanthoosh V, Boris S Closes #379 from navina/azure-etag-fix --- .../coordinator/AzureJobCoordinator.java | 75 ++++++++++--------- .../coordinator/data/ProcessorEntity.java | 8 ++ .../scheduler/JMVersionUpgradeScheduler.java | 9 ++- .../org/apache/samza/util/TableUtils.java | 49 +++++++++--- 4 files changed, 89 insertions(+), 52 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java index 468705b9ef..ca3384dc9a 100644 --- a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java +++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java @@ -19,22 +19,10 @@ package org.apache.samza.coordinator; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.apache.samza.AzureClient; -import org.apache.samza.config.AzureConfig; -import org.apache.samza.coordinator.data.BarrierState; import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.AzureConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.JobConfig; @@ -42,8 +30,8 @@ import org.apache.samza.container.TaskName; import org.apache.samza.container.grouper.stream.SystemStreamPartitionGrouper; import org.apache.samza.container.grouper.stream.SystemStreamPartitionGrouperFactory; -import org.apache.samza.job.model.JobModel; -import org.apache.samza.runtime.ProcessorIdGenerator; +import org.apache.samza.coordinator.data.BarrierState; +import org.apache.samza.coordinator.data.ProcessorEntity; import org.apache.samza.coordinator.scheduler.HeartbeatScheduler; import org.apache.samza.coordinator.scheduler.JMVersionUpgradeScheduler; import org.apache.samza.coordinator.scheduler.LeaderBarrierCompleteScheduler; @@ -51,6 +39,8 @@ import org.apache.samza.coordinator.scheduler.LivenessCheckScheduler; import org.apache.samza.coordinator.scheduler.RenewLeaseScheduler; import org.apache.samza.coordinator.scheduler.SchedulerStateChangeListener; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.runtime.ProcessorIdGenerator; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamMetadata; @@ -64,6 +54,18 @@ import org.slf4j.LoggerFactory; import scala.collection.JavaConverters; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; + /** * Class that provides coordination mechanism for Samza standalone in Azure. @@ -80,7 +82,6 @@ public class AzureJobCoordinator implements JobCoordinator { private final TableUtils table; private final Config config; private final String processorId; - private final AzureClient client; private final AtomicReference currentJMVersion; private final AtomicBoolean versionUpgradeDetected; private final HeartbeatScheduler heartbeat; @@ -103,7 +104,7 @@ public AzureJobCoordinator(Config config) { processorId = createProcessorId(config); currentJMVersion = new AtomicReference<>(INITIAL_STATE); AzureConfig azureConfig = new AzureConfig(config); - client = new AzureClient(azureConfig.getAzureConnectionString()); + AzureClient client = new AzureClient(azureConfig.getAzureConnectionString()); leaderBlob = new BlobUtils(client, azureConfig.getAzureContainerName(), azureConfig.getAzureBlobName(), azureConfig.getAzureBlobLength()); errorHandler = (errorMsg) -> { LOG.error(errorMsg); @@ -149,18 +150,17 @@ public void start() { public void stop() { LOG.info("Shutting down Azure job coordinator."); - if (coordinatorListener != null) { - coordinatorListener.onJobModelExpired(); - } - - // Resign leadership - if (azureLeaderElector.amILeader()) { - azureLeaderElector.resignLeadership(); - } + // Clean up resources & Resign leadership (if you are leader) + azureLeaderElector.resignLeadership(); + table.deleteProcessorEntity(currentJMVersion.get(), processorId, true); // Shutdown all schedulers shutdownSchedulers(); + if (coordinatorListener != null) { + coordinatorListener.onJobModelExpired(); + } + if (coordinatorListener != null) { coordinatorListener.onCoordinatorStop(); } @@ -217,7 +217,6 @@ private SchedulerStateChangeListener createLeaderBarrierCompleteListener(String if (!leaderBlob.publishBarrierState(state, azureLeaderElector.getLeaseId().get())) { LOG.info("Leader failed to publish the job model {}. Stopping the processor with PID: .", jobModel, processorId); stop(); - table.deleteProcessorEntity(currentJMVersion.get(), processorId); } leaderBarrierScheduler.shutdown(); }; @@ -374,7 +373,6 @@ private void doOnProcessorChange(List currentProcessorIds) { if (!jmWrite || !barrierWrite || !processorWrite) { LOG.info("Leader failed to publish the job model {}. Stopping the processor with PID: .", jobModel, processorId); stop(); - table.deleteProcessorEntity(currentJMVersion.get(), processorId); } LOG.info("pid=" + processorId + "Published new Job Model. Version = " + nextJMVersion); @@ -400,7 +398,6 @@ private void onNewJobModelAvailable(final String nextJMVersion) { if (!jobModel.getContainers().containsKey(processorId)) { LOG.info("JobModel: {} does not contain the processorId: {}. Stopping the processor.", jobModel, processorId); stop(); - table.deleteProcessorEntity(currentJMVersion.get(), processorId); } else { //Stop current work if (coordinatorListener != null) { @@ -443,18 +440,24 @@ private void onNewJobModelAvailable(final String nextJMVersion) { private void onNewJobModelConfirmed(final String nextJMVersion) { LOG.info("pid=" + processorId + "new version " + nextJMVersion + " of the job model got confirmed"); - // Delete previous value - if (table.getEntity(currentJMVersion.get(), processorId) != null) { - table.deleteProcessorEntity(currentJMVersion.get(), processorId); - } - if (table.getEntity(INITIAL_STATE, processorId) != null) { - table.deleteProcessorEntity(INITIAL_STATE, processorId); - } + String prevVersion = currentJMVersion.get(); - //Start heartbeating to new entry only when barrier reached. + //Start heart-beating to new entry only when barrier reached. //Changing the current job model version enables that since we are heartbeating to a row identified by the current job model version. currentJMVersion.getAndSet(nextJMVersion); + // Delete previous value + ProcessorEntity entity = table.getEntity(prevVersion, processorId); + if (entity != null) { + entity.setEtag("*"); + table.deleteProcessorEntity(entity); + } + entity = table.getEntity(INITIAL_STATE, processorId); + if (entity != null) { + entity.setEtag("*"); + table.deleteProcessorEntity(entity); + } + //Start the container with the new model if (coordinatorListener != null) { coordinatorListener.onNewJobModel(processorId, jobModel); diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java b/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java index 9323bde35e..27c18b35da 100644 --- a/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java +++ b/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java @@ -59,4 +59,12 @@ public void setIsLeader(boolean leader) { public boolean getIsLeader() { return isLeader; } + + public String getJobModelVersion() { + return partitionKey; + } + + public String getProcessorId() { + return rowKey; + } } \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java index ded014ff64..235b1f8d95 100644 --- a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java +++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java @@ -20,6 +20,11 @@ package org.apache.samza.coordinator.scheduler; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.samza.coordinator.data.BarrierState; +import org.apache.samza.util.BlobUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -29,10 +34,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import org.apache.samza.coordinator.data.BarrierState; -import org.apache.samza.util.BlobUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** diff --git a/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java b/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java index f49ce27b47..d5b8320168 100644 --- a/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java +++ b/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java @@ -24,17 +24,17 @@ import com.microsoft.azure.storage.table.CloudTableClient; import com.microsoft.azure.storage.table.TableOperation; import com.microsoft.azure.storage.table.TableQuery; -import java.net.URISyntaxException; -import java.util.HashSet; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.AzureClient; import org.apache.samza.AzureException; import org.apache.samza.coordinator.data.ProcessorEntity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + /** * Client side class that has a reference to Azure Table Storage. @@ -44,17 +44,15 @@ * ROW KEY = Unique entity ID for a group = Processor ID (for this case). */ public class TableUtils { - private static final Logger LOG = LoggerFactory.getLogger(TableUtils.class); private static final String PARTITION_KEY = "PartitionKey"; private static final long LIVENESS_DEBOUNCE_TIME_SEC = 30; private final String initialState; - private final CloudTableClient tableClient; private final CloudTable table; public TableUtils(AzureClient client, String tableName, String initialState) { this.initialState = initialState; - tableClient = client.getTableClient(); + CloudTableClient tableClient = client.getTableClient(); try { table = tableClient.getTableReference(tableName); table.createIfNotExists(); @@ -97,8 +95,7 @@ public void addProcessorEntity(String jmVersion, String pid, boolean isLeader) { public ProcessorEntity getEntity(String jmVersion, String pid) { try { TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class); - ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType(); - return entity; + return table.execute(retrieveEntity).getResultAsType(); } catch (StorageException e) { LOG.error("Azure storage exception while retrieving processor entity with job model version: " + jmVersion + "and pid: " + pid, e); throw new AzureException(e); @@ -112,7 +109,6 @@ public ProcessorEntity getEntity(String jmVersion, String pid) { */ public void updateHeartbeat(String jmVersion, String pid) { try { - Random rand = new Random(); TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class); ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType(); entity.updateLiveness(); @@ -145,14 +141,24 @@ public void updateIsLeader(String jmVersion, String pid, boolean isLeader) { /** * Deletes a specified row in the processor table. + * + * Note: Table service uses optimistic locking by default. Hence, if there is an update after retrieving the entity, + * then the delete operation will fail. + * * @param jmVersion Job model version of the processor row to be deleted. * @param pid Unique processor ID of the processor row to be deleted. + * @param force True, to disable optimistic locking on the table. False, otherwise. Setting to false may result in + * AzureException when there is concurrent access to the table. + * * @throws AzureException If an Azure storage service error occurred. */ - public void deleteProcessorEntity(String jmVersion, String pid) { + public void deleteProcessorEntity(String jmVersion, String pid, boolean force) { try { TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class); ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType(); + if (force) { + entity.setEtag("*"); + } TableOperation remove = TableOperation.delete(entity); table.execute(remove); } catch (StorageException e) { @@ -161,6 +167,25 @@ public void deleteProcessorEntity(String jmVersion, String pid) { } } + /** + * Deletes a specified row in the processor table. + * + * Note: Table service uses optimistic locking by default. In order to disable it, set the ETag on the ProcessorEntity + * to "*" before invoking this method. + * + * @param entity ProcessorEntity that has to be deleted + * @throws AzureException If an Azure storage service error occurred. + */ + public void deleteProcessorEntity(ProcessorEntity entity) { + try { + TableOperation remove = TableOperation.delete(entity); + table.execute(remove); + } catch (StorageException e) { + LOG.error("Azure storage exception while deleting processor entity with job model version: " + + entity.getJobModelVersion() + "and pid: " + entity.getProcessorId(), e); + throw new AzureException(e); + } + } /** * Retrieve all rows in a table with the given partition key. * @param partitionKey Job model version of the processors to be retrieved. From dd42bc3dfe9ab203d19752daf124906eaabd4559 Mon Sep 17 00:00:00 2001 From: Fred Ji Date: Thu, 30 Nov 2017 15:29:25 -0800 Subject: [PATCH 36/36] SAMZA-1407 upgrade junit version to 4.12 --- gradle/dependency-versions.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 2e45914f59..6ee117f48c 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -32,7 +32,7 @@ jerseyVersion = "2.22.1" jodaTimeVersion = "2.2" joptSimpleVersion = "3.2" - junitVersion = "4.8.1" + junitVersion = "4.12" kafkaVersion = "0.11.0.2" log4jVersion = "1.2.17" metricsVersion = "2.2.0"
  • Writing to HDFS