From 5f1e75247db7decf933b508d77ce76abdbabbdf4 Mon Sep 17 00:00:00 2001 From: Prateek Maheshwari Date: Wed, 17 Oct 2018 23:58:16 -0700 Subject: [PATCH 1/3] Javadoc cleanup for new Application, Descriptor, Context and Table APIs. Author: Prateek Maheshwari Reviewers: Cameron Lee Closes #737 from prateekm/javadoc-cleanup --- .../samza/application/SamzaApplication.java | 24 +++++-- .../samza/application/StreamApplication.java | 67 +++++++++--------- .../samza/application/TaskApplication.java | 69 ++++++++----------- .../descriptors/ApplicationDescriptor.java | 61 ++++++++++------ .../StreamApplicationDescriptor.java | 11 ++- .../TaskApplicationDescriptor.java | 22 ++++-- .../context/ApplicationContainerContext.java | 36 ++++++---- .../ApplicationContainerContextFactory.java | 25 +++---- .../samza/context/ApplicationTaskContext.java | 32 +++++---- .../ApplicationTaskContextFactory.java | 30 ++++---- .../samza/context/ContainerContext.java | 21 +++--- .../org/apache/samza/context/Context.java | 56 +++++++-------- .../org/apache/samza/context/JobContext.java | 20 +++--- .../org/apache/samza/context/TaskContext.java | 53 ++++++++------ .../descriptors/GenericInputDescriptor.java | 14 ++-- .../descriptors/GenericOutputDescriptor.java | 14 ++-- .../descriptors/GenericSystemDescriptor.java | 13 ++-- .../system/descriptors/InputDescriptor.java | 8 ++- .../system/descriptors/OutputDescriptor.java | 8 ++- .../system/descriptors/StreamDescriptor.java | 9 ++- .../system/descriptors/SystemDescriptor.java | 9 ++- .../java/org/apache/samza/table/Table.java | 21 +++++- .../table/descriptors/TableDescriptor.java | 37 ++++++---- .../org/apache/samza/task/TaskFactory.java | 4 +- .../descriptors/EventHubsInputDescriptor.java | 7 +- .../EventHubsOutputDescriptor.java | 8 ++- .../EventHubsSystemDescriptor.java | 7 +- .../application/LegacyTaskApplication.java | 4 +- .../application/MockStreamApplication.java | 2 +- .../application/TestApplicationUtil.java | 2 +- .../samza/execution/TestExecutionPlanner.java | 2 +- .../descriptors/KafkaInputDescriptor.java | 8 ++- .../descriptors/KafkaOutputDescriptor.java | 8 ++- .../descriptors/KafkaSystemDescriptor.java | 7 +- .../samza/sql/runner/SamzaSqlApplication.java | 14 ++-- .../example/AppWithGlobalConfigExample.java | 8 +-- .../samza/example/BroadcastExample.java | 10 +-- .../samza/example/KeyValueStoreExample.java | 8 +-- .../apache/samza/example/MergeExample.java | 6 +- .../example/OrderShipmentJoinExample.java | 8 +-- .../samza/example/PageViewCounterExample.java | 6 +- .../samza/example/RepartitionExample.java | 8 +-- .../samza/example/TaskApplicationExample.java | 10 +-- .../apache/samza/example/WindowExample.java | 6 +- .../TestStandaloneIntegrationApplication.java | 6 +- .../EndOfStreamIntegrationTest.java | 4 +- .../WatermarkIntegrationTest.java | 4 +- .../test/framework/BroadcastAssertApp.java | 6 +- .../test/framework/FaultInjectionTest.java | 8 +-- .../StreamApplicationIntegrationTest.java | 18 ++--- .../framework/StreamTaskIntegrationTest.java | 12 ++-- .../test/framework/TestSchedulingApp.java | 4 +- .../operator/RepartitionJoinWindowApp.java | 8 +-- .../test/operator/RepartitionWindowApp.java | 6 +- .../samza/test/operator/SessionWindowApp.java | 6 +- .../test/operator/TumblingWindowApp.java | 6 +- .../test/processor/TestStreamApplication.java | 6 +- .../samza/test/table/TestLocalTable.java | 8 +-- .../table/TestLocalTableWithSideInputs.java | 8 +-- 59 files changed, 518 insertions(+), 395 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java b/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java index 5423e2e5b5dee..849b2b346adf9 100644 --- a/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java +++ b/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java @@ -23,19 +23,29 @@ /** - * The base interface for all user-implemented applications in Samza. + * A {@link SamzaApplication} describes the inputs, outputs, state, configuration and the logic + * for processing data from one or more streaming sources. *

- * The main processing logic of the user application should be implemented in {@link SamzaApplication#describe(ApplicationDescriptor)} - * method. Sub-classes {@link StreamApplication} and {@link TaskApplication} are specific interfaces for applications - * written in high-level DAG and low-level task APIs, respectively. + * This is the base {@link SamzaApplication}. Implement a {@link StreamApplication} to describe the + * processing logic using Samza's High Level API in terms of {@link org.apache.samza.operators.MessageStream} + * operators, or a {@link TaskApplication} to describe it using Samza's Low Level API in terms of per-message + * processing logic. + *

+ * A {@link SamzaApplication} implementation must have a no-argument constructor, which will be used by the framework + * to create new instances and call {@link #describe(ApplicationDescriptor)}. + *

+ * Per container context may be managed using {@link org.apache.samza.context.ApplicationContainerContext} and + * set using {@link ApplicationDescriptor#withApplicationContainerContextFactory}. Similarly, per task context + * may be managed using {@link org.apache.samza.context.ApplicationTaskContext} and set using + * {@link ApplicationDescriptor#withApplicationTaskContextFactory}. */ @InterfaceStability.Evolving public interface SamzaApplication { /** - * Describes the user processing logic via {@link ApplicationDescriptor} + * Describes the inputs, outputs, state, configuration and processing logic using the provided {@code appDescriptor}. * - * @param appDesc the {@link ApplicationDescriptor} object to describe user application logic + * @param appDescriptor the {@link ApplicationDescriptor} to use for describing the application. */ - void describe(S appDesc); + void describe(S appDescriptor); } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java index fe770452ec36b..3749b58342795 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java @@ -21,22 +21,38 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.application.descriptors.StreamApplicationDescriptor; + /** - * Describes and initializes the transforms for processing message streams and generating results in high-level API. + * A {@link StreamApplication} describes the inputs, outputs, state, configuration and the processing logic + * in Samza's High Level API. + *

+ * A typical {@link StreamApplication} implementation consists of the following stages: + *

    + *
  1. Configuring the inputs, outputs and state (tables) using the appropriate + * {@link org.apache.samza.system.descriptors.SystemDescriptor}s, + * {@link org.apache.samza.system.descriptors.InputDescriptor}s, + * {@link org.apache.samza.system.descriptors.OutputDescriptor}s and + * {@link org.apache.samza.table.descriptors.TableDescriptor}s + *
  2. Obtaining the corresponding + * {@link org.apache.samza.operators.MessageStream}s, + * {@link org.apache.samza.operators.OutputStream}s and + * {@link org.apache.samza.table.Table}s from the provided {@link StreamApplicationDescriptor}. + *
  3. Defining the processing logic using operators and functions on the streams and tables thus obtained. + * E.g., {@link org.apache.samza.operators.MessageStream#filter(org.apache.samza.operators.functions.FilterFunction)} + *
*

- * The following example removes page views older than 1 hour from the input stream: + * The following example {@link StreamApplication} removes page views older than 1 hour from the input stream: *

{@code
  * public class PageViewFilter implements StreamApplication {
- *   public void describe(StreamAppDescriptor appDesc) {
- *     KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking");
+ *   public void describe(StreamApplicationDescriptor appDescriptor) {
+ *     KafkaSystemDescriptor trackingSystemDescriptor = new KafkaSystemDescriptor("tracking");
  *     KafkaInputDescriptor inputStreamDescriptor =
- *         trackingSystem.getInputDescriptor("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class));
- *
+ *         trackingSystemDescriptor.getInputDescriptor("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class));
  *     KafkaOutputDescriptor> outputStreamDescriptor =
- *         trackingSystem.getOutputDescriptor("recentPageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)));
+ *         trackingSystemDescriptor.getOutputDescriptor("recentPageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)));
  *
- *     MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor);
- *     OutputStream recentPageViewEvents = appDesc.getOutputStream(outputStreamDescriptor);
+ *     MessageStream pageViewEvents = appDescriptor.getInputStream(inputStreamDescriptor);
+ *     OutputStream recentPageViewEvents = appDescriptor.getOutputStream(outputStreamDescriptor);
  *
  *     pageViewEvents
  *       .filter(m -> m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis())
@@ -44,33 +60,20 @@
  *   }
  * }
  * }
- *

- * The example above can be run using an ApplicationRunner: - *

{@code
- *   public static void main(String[] args) {
- *     CommandLine cmdLine = new CommandLine();
- *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- *     PageViewFilter app = new PageViewFilter();
- *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config);
- *     runner.run();
- *     runner.waitForFinish();
- *   }
- * }
- * + *

+ * All operator function implementations used in a {@link StreamApplication} must be {@link java.io.Serializable}. Any + * context required within an operator function may be managed by implementing the + * {@link org.apache.samza.operators.functions.InitableFunction#init} and + * {@link org.apache.samza.operators.functions.ClosableFunction#close} methods in the function implementation. + *

+ * Functions may implement the {@link org.apache.samza.operators.functions.ScheduledFunction} interface + * to schedule and receive periodic callbacks from the Samza framework. *

* Implementation Notes: Currently {@link StreamApplication}s are wrapped in a {@link org.apache.samza.task.StreamTask} * during execution. The execution planner will generate a serialized DAG which will be deserialized in each * {@link org.apache.samza.task.StreamTask} instance used for processing incoming messages. Execution is synchronous - * and thread-safe within each {@link org.apache.samza.task.StreamTask}. - * - *

- * A {@link StreamApplication} implementation must have a proper fully-qualified class name and a default constructor - * with no parameters to ensure successful instantiation in both local and remote environments. - * Functions implemented for transforms in StreamApplications ({@link org.apache.samza.operators.functions.MapFunction}, - * {@link org.apache.samza.operators.functions.FilterFunction} for e.g.) are initable and closable. They are initialized - * before messages are delivered to them and closed after their execution when the {@link org.apache.samza.task.StreamTask} - * instance is closed. See {@link org.apache.samza.operators.functions.InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. - * Function implementations are required to be {@link java.io.Serializable}. + * and thread-safe within each {@link org.apache.samza.task.StreamTask}. Multiple tasks may process their + * messages concurrently depending on the job parallelism configuration. */ @InterfaceStability.Evolving public interface StreamApplication extends SamzaApplication { diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskApplication.java b/samza-api/src/main/java/org/apache/samza/application/TaskApplication.java index d84aa12b7a512..42103931c1f93 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskApplication.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplication.java @@ -23,64 +23,49 @@ /** - * Describes and initializes the transforms for processing message streams and generating results in low-level API. Your - * application is expected to implement this interface. + * A {@link TaskApplication} describes the inputs, outputs, state, configuration and the processing logic + * in Samza's Low Level API. + * A typical {@link TaskApplication} implementation consists of the following stages: + *

    + *
  1. Configuring the inputs, outputs and state (tables) using the appropriate + * {@link org.apache.samza.system.descriptors.SystemDescriptor}s, + * {@link org.apache.samza.system.descriptors.StreamDescriptor}s and + * {@link org.apache.samza.table.descriptors.TableDescriptor}s + *
  2. Adding these descriptors to the provided {@link TaskApplicationDescriptor}. + *
  3. Defining the processing logic by implementing a {@link org.apache.samza.task.StreamTask} or + * {@link org.apache.samza.task.AsyncStreamTask} that operates on each + * {@link org.apache.samza.system.IncomingMessageEnvelope} one at a time. + *
  4. Setting a {@link org.apache.samza.task.TaskFactory} using + * {@link TaskApplicationDescriptor#setTaskFactory(org.apache.samza.task.TaskFactory)} that creates instances of the + * task above. The {@link org.apache.samza.task.TaskFactory} implementation must be {@link java.io.Serializable}. + *
*

- * The following example removes page views older than 1 hour from the input stream: + * The following example {@link TaskApplication} removes page views older than 1 hour from the input stream: *

{@code
  * public class PageViewFilter implements TaskApplication {
- *   public void describe(TaskAppDescriptor appDesc) {
- *     KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor(PageViewTask.SYSTEM);
+ *   public void describe(TaskApplicationDescriptor appDescriptor) {
+ *     KafkaSystemDescriptor trackingSystemDescriptor = new KafkaSystemDescriptor("tracking");
  *     KafkaInputDescriptor inputStreamDescriptor =
- *         trackingSystem.getInputDescriptor(PageViewTask.TASK_INPUT, new JsonSerdeV2<>(PageViewEvent.class));
- *
+ *         trackingSystemDescriptor.getInputDescriptor("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class));
  *     KafkaOutputDescriptor> outputStreamDescriptor =
- *         trackingSystem.getOutputDescriptor(PageViewTask.TASK_OUTPUT, new JsonSerdeV2<>(PageViewEvent.class)));
+ *         trackingSystemDescriptor.getOutputDescriptor("recentPageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)));
  *
- *     appDesc.addInputStream(inputStreamDescriptor);
- *     appDesc.addOutputStream(outputStreamDescriptor);
- *     appDesc.setTaskFactory((StreamTaskFactory) () -> new PageViewTask());
+ *     appDescriptor.addInputStream(inputStreamDescriptor);
+ *     appDescriptor.addOutputStream(outputStreamDescriptor);
+ *     appDescriptor.setTaskFactory((StreamTaskFactory) () -> new PageViewTask());
  *   }
  * }
  *
  * public class PageViewTask implements StreamTask {
- *   final static String TASK_INPUT = "pageViewEvents";
- *   final static String TASK_OUTPUT = "recentPageViewEvents";
- *   final static String SYSTEM = "kafka";
- *
- *   public void process(IncomingMessageEnvelope message, MessageCollector collector,
- *       TaskCoordinator coordinator) {
+ *   public void process(IncomingMessageEnvelope message, MessageCollector collector, TaskCoordinator coordinator) {
  *     PageViewEvent m = (PageViewEvent) message.getValue();
  *     if (m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis()) {
- *       collector.send(new OutgoingMessageEnvelope(new SystemStream(SYSTEM, TASK_OUTPUT),
- *           message.getKey(), message.getKey(), m));
+ *       collector.send(new OutgoingMessageEnvelope(
+ *          new SystemStream("tracking", "recentPageViewEvent"), message.getKey(), message.getKey(), m));
  *     }
  *   }
  * }
  * }
- * - *

- * The example above can be run using an ApplicationRunner: - *

{@code
- *   public static void main(String[] args) {
- *     CommandLine cmdLine = new CommandLine();
- *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- *     PageViewFilter app = new PageViewFilter();
- *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config);
- *     runner.run();
- *     runner.waitForFinish();
- *   }
- * }
- * - *

- * Implementation Notes: {@link TaskApplication} allow users to instantiate {@link org.apache.samza.task.StreamTask} or - * {@link org.apache.samza.task.AsyncStreamTask} when describing the processing logic. A new {@link TaskApplicationDescriptor } - * instance will be created and described by the user-defined {@link TaskApplication} when planning the execution. - * {@link org.apache.samza.task.TaskFactory} is required to be serializable. - * - *

- * The user-implemented {@link TaskApplication} class must be a class with proper fully-qualified class name and - * a default constructor with no parameters to ensure successful instantiation in both local and remote environments. */ @InterfaceStability.Evolving public interface TaskApplication extends SamzaApplication { diff --git a/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java index b2d54cad45f73..6a4c9fde36304 100644 --- a/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/descriptors/ApplicationDescriptor.java @@ -29,32 +29,42 @@ /** - * The interface class to describe the configuration, input and output streams, and processing logic in a + * An {@link ApplicationDescriptor} contains the description of inputs, outputs, state, configuration and the + * processing logic for a {@link org.apache.samza.application.SamzaApplication}. + *

+ * This is the base {@link ApplicationDescriptor} and provides functionality common to all * {@link org.apache.samza.application.SamzaApplication}. + * {@link org.apache.samza.application.StreamApplication#describe} will provide access to a + * {@link StreamApplicationDescriptor} with additional functionality for describing High Level API applications. + * Similarly, {@link org.apache.samza.application.TaskApplication#describe} will provide access to a + * {@link TaskApplicationDescriptor} with additional functionality for describing Low Level API applications. *

- * Sub-classes {@link StreamApplicationDescriptor} and {@link TaskApplicationDescriptor} are specific interfaces for - * applications written in high-level {@link org.apache.samza.application.StreamApplication} and low-level - * {@link org.apache.samza.application.TaskApplication} APIs, respectively. - * - * @param sub-class of user application descriptor. + * Use the {@link ApplicationDescriptor} to set the container scope context factory using + * {@link ApplicationDescriptor#withApplicationContainerContextFactory}, and task scope context factory using + * {@link ApplicationDescriptor#withApplicationTaskContextFactory}. Please note that the terms {@code container} + * and {@code task} here refer to the units of physical and logical parallelism, not the programming API. */ @InterfaceStability.Evolving public interface ApplicationDescriptor { /** - * Get the {@link Config} of the application - * @return config of the application + * Get the configuration for the application. + * @return config for the application */ Config getConfig(); /** - * Sets the default SystemDescriptor to use for the application. This is equivalent to setting - * {@code job.default.system} and its properties in configuration. + * Sets the {@link SystemDescriptor} for the default system for the application. + *

+ * The default system is used by the framework for creating any internal (e.g., coordinator, changelog, checkpoint) + * streams. In an {@link org.apache.samza.application.StreamApplication}, it is also used for creating any + * intermediate streams; e.g., those created by the {@link org.apache.samza.operators.MessageStream#partitionBy} and + * {@link org.apache.samza.operators.MessageStream#broadcast} operators. *

* If the default system descriptor is set, it must be set before creating any input/output/intermediate streams. * - * @param defaultSystemDescriptor the default system descriptor to use - * @return type {@code S} of {@link ApplicationDescriptor} with {@code defaultSystemDescriptor} set as its default system + * @param defaultSystemDescriptor the {@link SystemDescriptor} for the default system for the application + * @return this {@link ApplicationDescriptor} */ S withDefaultSystem(SystemDescriptor defaultSystemDescriptor); @@ -64,10 +74,11 @@ public interface ApplicationDescriptor { * context can be accessed through the {@link org.apache.samza.context.Context}. *

* Setting this is optional. + *

+ * The provided {@code factory} instance must be {@link java.io.Serializable}. * * @param factory the {@link ApplicationContainerContextFactory} for this application - * @return type {@code S} of {@link ApplicationDescriptor} with {@code factory} set as its - * {@link ApplicationContainerContextFactory} + * @return this {@link ApplicationDescriptor} */ S withApplicationContainerContextFactory(ApplicationContainerContextFactory factory); @@ -77,31 +88,37 @@ public interface ApplicationDescriptor { * accessed through the {@link org.apache.samza.context.Context}. *

* Setting this is optional. + *

+ * The provided {@code factory} instance must be {@link java.io.Serializable}. * * @param factory the {@link ApplicationTaskContextFactory} for this application - * @return type {@code S} of {@link ApplicationDescriptor} with {@code factory} set as its - * {@link ApplicationTaskContextFactory} + * @return this {@link ApplicationDescriptor} */ S withApplicationTaskContextFactory(ApplicationTaskContextFactory factory); /** * Sets the {@link ProcessorLifecycleListenerFactory} for this application. - * - *

Setting a {@link ProcessorLifecycleListenerFactory} is optional to a user application. It allows users to + *

+ * Setting a {@link ProcessorLifecycleListenerFactory} is optional to a user application. It allows users to * plug in optional code to be invoked in different stages before/after the main processing logic is started/stopped in * the application. + *

+ * The provided {@code factory} instance must be {@link java.io.Serializable}. * * @param listenerFactory the user implemented {@link ProcessorLifecycleListenerFactory} that creates lifecycle listener * with callback methods before and after the start/stop of each StreamProcessor in the application - * @return type {@code S} of {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory} + * @return this {@link ApplicationDescriptor} */ S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); /** - * Sets a set of customized {@link MetricsReporterFactory}s in the application + * Sets the {@link org.apache.samza.metrics.MetricsReporterFactory}s for creating the + * {@link org.apache.samza.metrics.MetricsReporter}s to use for the application. + *

+ * The provided {@link MetricsReporterFactory} instances must be {@link java.io.Serializable}. * - * @param reporterFactories the map of customized {@link MetricsReporterFactory}s to be used - * @return type {@code S} of {@link ApplicationDescriptor} with {@code reporterFactories} + * @param reporterFactories a map of {@link org.apache.samza.metrics.MetricsReporter} names to their factories. + * @return this {@link ApplicationDescriptor} */ S withMetricsReporterFactories(Map reporterFactories); diff --git a/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java index 3a35054589284..4a77c6cab6269 100644 --- a/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/descriptors/StreamApplicationDescriptor.java @@ -29,7 +29,16 @@ /** - * The interface class to describe a {@link org.apache.samza.application.SamzaApplication} in high-level API in Samza. + * A {@link StreamApplicationDescriptor} contains the description of inputs, outputs, state, configuration and the + * processing logic for a Samza High Level API {@link org.apache.samza.application.StreamApplication}. + *

+ * Use the {@link StreamApplicationDescriptor} obtained from + * {@link org.apache.samza.application.StreamApplication#describe} to get the {@link MessageStream}s, + * {@link OutputStream}s and {@link Table}s corresponding to their respective {@link InputDescriptor}s, + * {@link OutputDescriptor}s and {@link TableDescriptor}s. + *

+ * Use the {@link MessageStream} API operators to describe the processing logic for the + * {@link org.apache.samza.application.StreamApplication}. */ @InterfaceStability.Evolving public interface StreamApplicationDescriptor extends ApplicationDescriptor { diff --git a/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java index 473029778fc61..b395a060756bc 100644 --- a/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/descriptors/TaskApplicationDescriptor.java @@ -26,18 +26,28 @@ /** - * The interface to describe a {@link org.apache.samza.application.SamzaApplication} that uses low-level API task - * for processing. + * A {@link TaskApplicationDescriptor} contains the description of inputs, outputs, state, configuration and the + * processing logic for a Samza Low Level API {@link org.apache.samza.application.TaskApplication}. + *

+ * Use the {@link TaskApplicationDescriptor} obtained from {@link org.apache.samza.application.TaskApplication#describe} + * to add the {@link InputDescriptor}s, {@link OutputDescriptor}s and {@link TableDescriptor}s for streams and + * tables to be used in the task implementation. + *

+ * Use {@link #setTaskFactory} to set the factory for the {@link org.apache.samza.task.StreamTask} or + * {@link org.apache.samza.task.AsyncStreamTask} implementation that contains the processing logic for + * the {@link org.apache.samza.application.TaskApplication}. */ @InterfaceStability.Evolving public interface TaskApplicationDescriptor extends ApplicationDescriptor { /** - * Sets the {@link TaskFactory} for the user application. The {@link TaskFactory#createInstance()} creates task instance - * that implements the main processing logic of the user application. + * Sets the {@link org.apache.samza.task.StreamTaskFactory} or {@link org.apache.samza.task.AsyncStreamTaskFactory} + * for the {@link org.apache.samza.task.StreamTask} or {@link org.apache.samza.task.AsyncStreamTask} implementation + * that contains the processing logic for the {@link org.apache.samza.application.TaskApplication}. + *

+ * The provided {@code taskFactory} instance must be serializable. * - * @param factory the {@link TaskFactory} including the low-level task processing logic. The only allowed task factory - * classes are {@link org.apache.samza.task.StreamTaskFactory} and {@link org.apache.samza.task.AsyncStreamTaskFactory}. + * @param factory the {@link TaskFactory} for the Low Level API Task implementation */ void setTaskFactory(TaskFactory factory); diff --git a/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContext.java b/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContext.java index aab8c7f921349..8ac34a5c2b0d9 100644 --- a/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContext.java +++ b/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContext.java @@ -18,34 +18,42 @@ */ package org.apache.samza.context; + /** - * An application should implement this to contain any runtime objects required by processing logic which can be shared - * across all tasks in a container. A single instance of this will be created in each container. Note that if the - * container moves or the container model changes (e.g. container failure/rebalancing), then this will be recreated. + * An {@link ApplicationContainerContext} instance can be used for holding per-container runtime state and objects + * and managing their lifecycle. This context is shared across all tasks in the container. + *

+ * Use {@link org.apache.samza.application.descriptors.ApplicationDescriptor#withApplicationContainerContextFactory} + * to provide the {@link ApplicationContainerContextFactory}. Use {@link Context#getApplicationContainerContext()} to + * get the created {@link ApplicationContainerContext} instance for the current container. *

- * This needs to be created by an implementation of {@link ApplicationContainerContextFactory}. The factory should - * create the runtime objects contained within this context. + * A unique instance of {@link ApplicationContainerContext} is created in each container. If the + * container moves or the container model changes (e.g. due to failure or re-balancing), a new instance is created. *

- * This is related to {@link ContainerContext} in that they are both associated with the container lifecycle. In order - * to access this in application code, use {@link Context#getApplicationContainerContext()}. The - * {@link ContainerContext} is accessible through {@link Context#getContainerContext()}. + * Use the {@link ApplicationContainerContextFactory} to create any runtime state and objects, and the + * {@link ApplicationContainerContext#start()} and {@link ApplicationContainerContext#stop()} methods to + * manage their lifecycle. *

- * If it is necessary to have a separate instance per task, then use {@link ApplicationTaskContext} instead. + * Use {@link ApplicationTaskContext} to hold unique runtime state and objects for each task within a container. + * Use {@link ContainerContext} to access framework-provided context for a container. *

- * This class does not need to be {@link java.io.Serializable} and instances are not persisted across deployments. + * Unlike its {@link ApplicationContainerContextFactory}, an implementation does not need to be + * {@link java.io.Serializable}. */ public interface ApplicationContainerContext { /** - * Lifecycle logic which will run after tasks in the container are initialized but before processing begins. + * Starts this {@link ApplicationContainerContext} before any tasks in the container are initialized and before + * processing begins. *

- * If this throws an exception, then the container will fail to start. + * If this throws an exception, the container will fail to start. */ void start(); /** - * Lifecycle logic which will run after processing ends but before tasks in the container are closed. + * Stops this {@link ApplicationContainerContext} after processing ends and after all tasks in the container + * are closed. *

- * If this throws an exception, then the container will fail to fully shut down. + * If this throws an exception, the container will fail to fully shut down. */ void stop(); } diff --git a/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java b/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java index 074b0b4025262..a8c9f7cc2115d 100644 --- a/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java +++ b/samza-api/src/main/java/org/apache/samza/context/ApplicationContainerContextFactory.java @@ -22,24 +22,25 @@ /** - * An application should implement this if it has a {@link ApplicationContainerContext} that is needed for - * initialization. + * The factory for creating {@link ApplicationContainerContext} instances for a + * {@link org.apache.samza.application.SamzaApplication} during container initialization. *

- * This will be called to create an instance of {@link ApplicationContainerContext} during the container initialization - * stage. At that stage, the framework-provided job-level and container-level contexts are available for creating the - * {@link ApplicationContainerContext}. + * Use {@link org.apache.samza.application.descriptors.ApplicationDescriptor#withApplicationContainerContextFactory} to + * provide the {@link ApplicationContainerContextFactory}. Use {@link Context#getApplicationContainerContext()} to + * get the created {@link ApplicationContainerContext} instance for the current container. *

- * This is {@link Serializable} because it is specified in the - * {@link org.apache.samza.application.descriptors.ApplicationDescriptor}. - * @param concrete type of {@link ApplicationContainerContext} returned by this factory + * The {@link ApplicationContainerContextFactory} implementation must be {@link Serializable}. + * + * @param concrete type of {@link ApplicationContainerContext} created by this factory */ public interface ApplicationContainerContextFactory extends Serializable { + /** - * Create an instance of the application-defined {@link ApplicationContainerContext}. + * Creates an instance of the application-defined {@link ApplicationContainerContext}. * - * @param jobContext framework-provided job context used for building {@link ApplicationContainerContext} - * @param containerContext framework-provided container context used for building {@link ApplicationContainerContext} - * @return new instance of the application-defined {@link ApplicationContainerContext} + * @param jobContext framework-provided job context + * @param containerContext framework-provided container context + * @return a new instance of the application-defined {@link ApplicationContainerContext} */ T create(JobContext jobContext, ContainerContext containerContext); } diff --git a/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContext.java b/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContext.java index 6afbf23ab908f..a4236bffe6cb6 100644 --- a/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContext.java +++ b/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContext.java @@ -18,34 +18,38 @@ */ package org.apache.samza.context; + /** - * An application should implement this to contain any runtime objects required by processing logic which cannot be - * shared across tasks. A new instance of this will be created for each task. + * An {@link ApplicationTaskContext} instance can be used for holding per-task runtime state and objects and managing + * their lifecycle in an {@link org.apache.samza.application.SamzaApplication} *

- * This needs to be created by an implementation of {@link ApplicationTaskContextFactory}. The factory should create - * the runtime objects contained within this context. + * Use {@link org.apache.samza.application.descriptors.ApplicationDescriptor#withApplicationTaskContextFactory} + * to provide the {@link ApplicationTaskContextFactory}. Use {@link Context#getApplicationTaskContext()} to get + * the created {@link ApplicationTaskContext} instance for the current task. *

- * This is related to {@link TaskContext} in that they are both associated with a task lifecycle. In order to access - * this in application code, use {@link Context#getApplicationTaskContext()}. The {@link TaskContext} is accessible - * through {@link Context#getTaskContext()}. + * A unique instance of {@link ApplicationTaskContext} is created for each task in a container. + * Use the {@link ApplicationTaskContextFactory} to create any runtime state and objects, and the + * {@link ApplicationTaskContext#start()} and {@link ApplicationTaskContext#stop()} methods to manage their lifecycle. *

- * If it is possible to share an instance of this across tasks in a container, then use - * {@link ApplicationContainerContext} instead. + * Use {@link ApplicationContainerContext} to hold runtime state and objects shared across all tasks within a container. + * Use {@link TaskContext} to access framework-provided context for a task. *

- * This class does not need to be {@link java.io.Serializable} and instances are not persisted across deployments. + * Unlike its {@link ApplicationTaskContextFactory}, an implementation does not need to be + * {@link java.io.Serializable}. */ public interface ApplicationTaskContext { + /** - * Lifecycle logic which will run after tasks are initialized but before processing begins. + * Starts this {@link ApplicationTaskContext} after its task is initialized but before any messages are processed. *

- * If this throws an exception, then the container will fail to start. + * If this throws an exception, the container will fail to start. */ void start(); /** - * Lifecycle logic which will run after processing ends but before tasks are closed. + * Stops this {@link ApplicationTaskContext} after processing ends but before its task is closed. *

- * If this throws an exception, then the container will fail to fully shut down. + * If this throws an exception, the container will fail to fully shut down. */ void stop(); } diff --git a/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java b/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java index 619bbc708d1d9..c00935f821cfd 100644 --- a/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java +++ b/samza-api/src/main/java/org/apache/samza/context/ApplicationTaskContextFactory.java @@ -22,27 +22,27 @@ /** - * An application should implement this if it has a {@link ApplicationTaskContext} that is needed for - * initialization. This will be used to create instance(s) of that {@link ApplicationTaskContext}. + * The factory for creating {@link ApplicationTaskContext} instances for a + * {@link org.apache.samza.application.SamzaApplication}during task initialization. *

- * This will be called to create an instance of {@link ApplicationTaskContext} during the initialization stage of each - * task. At that stage, the framework-provided job-level, container-level, and task-level contexts are available for - * creating the {@link ApplicationTaskContext}. Also, the application-defined container-level context is available. + * Use {@link org.apache.samza.application.descriptors.ApplicationDescriptor#withApplicationTaskContextFactory} to + * provide the {@link ApplicationTaskContextFactory}. Use {@link Context#getApplicationTaskContext()} to + * get the created {@link ApplicationTaskContext} instance for the current task. *

- * This is {@link Serializable} because it is specified in the - * {@link org.apache.samza.application.descriptors.ApplicationDescriptor}. - * @param concrete type of {@link ApplicationTaskContext} returned by this factory + * The {@link ApplicationTaskContextFactory} implementation must be {@link Serializable}. + * + * @param concrete type of {@link ApplicationTaskContext} created by this factory */ public interface ApplicationTaskContextFactory extends Serializable { + /** - * Create an instance of the application-defined {@link ApplicationTaskContext}. + * Creates an instance of the application-defined {@link ApplicationTaskContext}. * - * @param jobContext framework-provided job context used for building {@link ApplicationTaskContext} - * @param containerContext framework-provided container context used for building {@link ApplicationTaskContext} - * @param taskContext framework-provided task context used for building {@link ApplicationTaskContext} - * @param applicationContainerContext application-provided container context used for building - * {@link ApplicationTaskContext} - * @return new instance of the application-defined {@link ApplicationContainerContext} + * @param jobContext framework-provided job context + * @param containerContext framework-provided container context + * @param taskContext framework-provided task context + * @param applicationContainerContext application-defined container context + * @return a new instance of the application-defined {@link ApplicationTaskContext} */ T create(JobContext jobContext, ContainerContext containerContext, TaskContext taskContext, ApplicationContainerContext applicationContainerContext); diff --git a/samza-api/src/main/java/org/apache/samza/context/ContainerContext.java b/samza-api/src/main/java/org/apache/samza/context/ContainerContext.java index 51d791806604c..97c5f539a8968 100644 --- a/samza-api/src/main/java/org/apache/samza/context/ContainerContext.java +++ b/samza-api/src/main/java/org/apache/samza/context/ContainerContext.java @@ -23,24 +23,25 @@ /** - * Contains information at container granularity, provided by the Samza framework, to be used to instantiate an - * application at runtime. + * The framework-provided context for the current container. *

- * Note that application-defined container-level context is accessible through - * {@link ApplicationContainerContext}. + * Use {@link ApplicationContainerContext} for the application-defined context for the current container. */ public interface ContainerContext { + /** - * Returns the {@link ContainerModel} associated with this container. This contains information like the id and the - * associated {@link org.apache.samza.job.model.TaskModel}s. - * @return {@link ContainerModel} associated with this container + * Gets the {@link ContainerModel} for this container, which contains this container's id and + * its {@link org.apache.samza.job.model.TaskModel}. + * + * @return the {@link ContainerModel} for this container */ ContainerModel getContainerModel(); /** - * Returns the {@link MetricsRegistry} for this container. Metrics built using this registry will be associated with - * the container. - * @return {@link MetricsRegistry} for this container + * Gets the {@link MetricsRegistry} for this container, which can be used to register metrics that are + * reported per container. + * + * @return the {@link MetricsRegistry} for this container */ MetricsRegistry getContainerMetricsRegistry(); } diff --git a/samza-api/src/main/java/org/apache/samza/context/Context.java b/samza-api/src/main/java/org/apache/samza/context/Context.java index bfe66d387eab4..e1111270a2116 100644 --- a/samza-api/src/main/java/org/apache/samza/context/Context.java +++ b/samza-api/src/main/java/org/apache/samza/context/Context.java @@ -19,59 +19,59 @@ package org.apache.samza.context; /** - * Container object for all context provided to instantiate an application at runtime. + * A holder for all framework and application defined contexts at runtime. */ public interface Context { /** - * Returns the framework-provided context for the overall job that is being run. - * @return framework-provided job context + * Gets the framework-provided context for the job. + * + * @return the framework-provided job context */ JobContext getJobContext(); /** - * Returns the framework-provided context for the container that this is in. + * Gets the framework-provided context for the current container. This context is shared by all tasks within + * the container. *

- * Note that this is not the application-defined container context. Use - * {@link Context#getApplicationContainerContext()} to get the application-defined container context. - * @return framework-provided container context + * Use {@link #getApplicationContainerContext()} to get the application-defined container context. + * + * @return the framework-provided container context */ ContainerContext getContainerContext(); /** - * Returns the framework-provided context for the task that that this is in. + * Gets the framework-provided context for the current task. *

- * Note that this is not the application-defined task context. Use {@link Context#getApplicationTaskContext()} - * to get the application-defined task context. - * @return framework-provided task context + * Use {@link #getApplicationTaskContext()} to get the application-defined task context. + * + * @return the framework-provided task context */ TaskContext getTaskContext(); /** - * Returns the application-defined container context object specified by the - * {@link ApplicationContainerContextFactory}. This is shared across all tasks in the container, but not across - * containers. + * Gets the application-defined context for the current container. This context is shared by all tasks within + * the container. *

- * In order to use this in application code, it should be casted to the concrete type that corresponds to the - * {@link ApplicationContainerContextFactory}. + * Use {@link org.apache.samza.application.descriptors.ApplicationDescriptor#withApplicationContainerContextFactory} + * to provide a factory for this context. Cast the returned context to the concrete implementation type to use it. *

- * Note that this is not the framework-provided container context. Use {@link Context#getContainerContext()} to get - * the framework-provided container context. - * @return application-defined container context - * @throws IllegalStateException if no context could be built (e.g. no factory provided) + * Use {@link #getContainerContext()} to get the framework-provided container context. + * + * @return the application-defined container context + * @throws IllegalStateException if no {@link ApplicationContainerContextFactory} was was provided for the application */ ApplicationContainerContext getApplicationContainerContext(); /** - * Returns the application-defined task context object specified by the {@link ApplicationTaskContextFactory}. - * Each task will have a separate instance of this. + * Gets the application-defined task context for the current task. This context is unique to this task. *

- * In order to use this in application code, it should be casted to the concrete type that corresponds to the - * {@link ApplicationTaskContextFactory}. + * Use {@link org.apache.samza.application.descriptors.ApplicationDescriptor#withApplicationTaskContextFactory} + * to provide a factory for this context. Cast the returned context to the concrete implementation type to use it. *

- * Note that this is not the framework-provided task context. Use {@link Context#getTaskContext()} to get the - * framework-provided task context. - * @return application-defined task context - * @throws IllegalStateException if no context could be built (e.g. no factory provided) + * Use {@link Context#getTaskContext()} to get the framework-provided task context. + * + * @return the application-defined task context + * @throws IllegalStateException if no {@link ApplicationTaskContextFactory} was provided for the application */ ApplicationTaskContext getApplicationTaskContext(); } diff --git a/samza-api/src/main/java/org/apache/samza/context/JobContext.java b/samza-api/src/main/java/org/apache/samza/context/JobContext.java index 239a01111a627..8e41980b442a2 100644 --- a/samza-api/src/main/java/org/apache/samza/context/JobContext.java +++ b/samza-api/src/main/java/org/apache/samza/context/JobContext.java @@ -22,26 +22,28 @@ /** - * Contains information at job granularity, provided by the Samza framework, to be used to instantiate an application at - * runtime. + * The framework-provided context for the job. */ public interface JobContext { + /** - * Returns the final configuration for this job. - * @return configuration for this job + * Gets the final configuration for this job. + * + * @return the configuration for this job */ Config getConfig(); /** - * Returns the name of the job. - * @return name of the job - * @throws org.apache.samza.SamzaException if the job name was not configured + * Gets the name of the job. + * + * @return the name of this job */ String getJobName(); /** - * Returns the instance id for this instance of this job. - * @return instance id for the job + * Gets the id for this job. + * + * @return the id for this job */ String getJobId(); } diff --git a/samza-api/src/main/java/org/apache/samza/context/TaskContext.java b/samza-api/src/main/java/org/apache/samza/context/TaskContext.java index d29f6a55f9e0e..3a5333c552d00 100644 --- a/samza-api/src/main/java/org/apache/samza/context/TaskContext.java +++ b/samza-api/src/main/java/org/apache/samza/context/TaskContext.java @@ -28,54 +28,65 @@ /** - * Contains information at task granularity, provided by the Samza framework, to be used to instantiate an application - * at runtime. + * The framework-provided context for the current task. *

- * Note that application-defined task-level context is accessible through {@link ApplicationTaskContext}. + * Use {@link ApplicationTaskContext} for the application-defined context for the current task. */ public interface TaskContext { + /** - * Returns the {@link TaskModel} associated with this task. This contains information like the task name and - * associated {@link SystemStreamPartition}s. - * @return {@link TaskModel} associated with this task + * Gets the {@link TaskModel} for this task, which contains this task's name and its {@link SystemStreamPartition}s. + * + * @return the {@link TaskModel} for this task */ TaskModel getTaskModel(); /** - * Returns the {@link MetricsRegistry} for this task. Metrics built using this registry will be associated with the - * task. - * @return {@link MetricsRegistry} for this task + * Gets the {@link MetricsRegistry} for this task, which can be used to register metrics that are reported per task. + * + * @return the {@link MetricsRegistry} for this task */ MetricsRegistry getTaskMetricsRegistry(); /** - * Returns the {@link KeyValueStore} corresponding to the {@code storeName}. In application code, it is recommended to - * cast the resulting stores to {@link KeyValueStore}s with the correct concrete type parameters. - * @param storeName name of the {@link KeyValueStore} to get - * @return {@link KeyValueStore} corresponding to the {@code storeName} + * Gets the {@link KeyValueStore} associated with {@code storeName} for this task. + *

+ * The returned store should be cast with the concrete type parameters based on the configured store serdes. + * E.g., if using string key and integer value serde, it should be cast to a {@code KeyValueStore}. + * + * @param storeName name of the {@link KeyValueStore} to get for this task + * @return the {@link KeyValueStore} associated with {@code storeName} for this task * @throws IllegalArgumentException if there is no store associated with {@code storeName} */ KeyValueStore getStore(String storeName); /** - * Returns the {@link Table} corresponding to the {@code tableId}. In application code, it is recommended to cast this - * to the resulting tables to {@link Table}s with the correct concrete type parameters. + * Gets the {@link Table} corresponding to the {@code tableId} for this task. + * + * The returned table should be cast with the concrete type parameters based on the configured table serdes, and + * whether it is {@link org.apache.samza.table.ReadWriteTable} or {@link org.apache.samza.table.ReadableTable}. + * E.g., if using string key and integer value serde for a writable table, it should be cast to a + * {@code ReadWriteTable}. + * * @param tableId id of the {@link Table} to get - * @return {@link Table} corresponding to the {@code tableId} + * @return the {@link Table} associated with {@code tableId} for this task * @throws IllegalArgumentException if there is no table associated with {@code tableId} */ Table getTable(String tableId); /** - * Returns a task-level {@link CallbackScheduler} which can be used to delay execution of some logic. - * @return {@link CallbackScheduler} for this task + * Gets the {@link CallbackScheduler} for this task, which can be used to schedule a callback to be executed + * at a future time. + * + * @return the {@link CallbackScheduler} for this task */ CallbackScheduler getCallbackScheduler(); /** - * Set the starting offset for the given {@link SystemStreamPartition}. Offsets can only be set for a - * {@link SystemStreamPartition} assigned to this task. The {@link SystemStreamPartition}s assigned to this task can - * be accessed through {@link TaskModel#getSystemStreamPartitions()} for the {@link TaskModel} obtained by calling + * Sets the starting offset for the given {@link SystemStreamPartition}. + *

Offsets can only be set for a {@link SystemStreamPartition} assigned to this task. + * The {@link SystemStreamPartition}s assigned to this task can be accessed through + * {@link TaskModel#getSystemStreamPartitions()} for the {@link TaskModel} obtained by calling * {@link #getTaskModel()}. Trying to set the offset for any other partition will have no effect. * * NOTE: this feature is experimental, and the API may change in a future release. diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java index aa5c8d2df8bc4..e08fa09728d72 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericInputDescriptor.java @@ -21,16 +21,16 @@ import org.apache.samza.serializers.Serde; /** - * A descriptor for a generic input stream. + * A {@link GenericInputDescriptor} can be used for specifying Samza and system-specific properties of + * input streams. *

- * An instance of this descriptor may be obtained from an appropriately configured - * {@link GenericSystemDescriptor}. + * If the system provides its own system and stream descriptor implementations, use them instead. + * Otherwise, use this {@link GenericInputDescriptor} to specify Samza-specific properties of the stream, + * and {@link #withStreamConfigs} to specify additional system specific properties. *

- * If the system being used provides its own system and stream descriptor implementations, they should be used instead. - * Otherwise, this {@link GenericInputDescriptor} may be used to provide Samza-specific properties of the input stream. - * Additional system stream specific properties may be provided using {@link #withStreamConfigs} + * Use {@link GenericSystemDescriptor#getInputDescriptor} to obtain an instance of this descriptor. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. * * @param type of messages in this stream. */ diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java index 1d8152557fe06..7302c35be7e45 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericOutputDescriptor.java @@ -21,16 +21,16 @@ import org.apache.samza.serializers.Serde; /** - * A descriptor for a generic output stream. + * A {@link GenericOutputDescriptor} can be used for specifying Samza and system-specific properties of + * output streams. *

- * An instance of this descriptor may be obtained from an appropriately configured - * {@link GenericSystemDescriptor}. + * If the system provides its own system and stream descriptor implementations, use them instead. + * Otherwise, use this {@link GenericOutputDescriptor} to specify Samza-specific properties of the stream, + * and {@link #withStreamConfigs} to specify additional system specific properties. *

- * If the system being used provides its own system and stream descriptor implementations, they should be used instead. - * Otherwise, this {@link GenericOutputDescriptor} may be used to provide Samza-specific properties of the output stream. - * Additional system stream specific properties may be provided using {@link #withStreamConfigs} + * Use {@link GenericSystemDescriptor#getOutputDescriptor} to obtain an instance of this descriptor. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. * * @param type of messages in this stream. */ diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java index eb86877a3d9dd..4884cd6ec76e4 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/GenericSystemDescriptor.java @@ -22,13 +22,16 @@ import org.apache.samza.serializers.Serde; /** - * A descriptor for a generic system. + * A {@link GenericSystemDescriptor} can be used for specifying Samza and system-specific properties of an + * input/output system. It can also be used for obtaining {@link GenericInputDescriptor}s and + * {@link GenericOutputDescriptor}s, which can be used for specifying any Samza and system-specific properties + * of input/output streams. *

- * If the system being used provides its own system and stream descriptor implementations, they should be used instead. - * Otherwise, this {@link GenericSystemDescriptor} may be used to provide Samza-specific properties of the system. - * Additional system specific properties may be provided using {@link #withSystemConfigs} + * If the system provides its own system and stream descriptor implementations, use them instead. + * Otherwise, use this {@link GenericSystemDescriptor} to specify Samza-specific properties of the system, + * and {@link #withSystemConfigs} to specify additional system specific properties. *

- * System properties provided in configuration override corresponding properties configured using a descriptor. + * System properties provided in configuration override corresponding properties specified using a descriptor. */ public final class GenericSystemDescriptor extends SystemDescriptor implements SimpleInputDescriptorProvider, OutputDescriptorProvider { diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java index fd7a50c2b0a2e..2c6f88bc359d3 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/InputDescriptor.java @@ -26,9 +26,13 @@ import org.apache.samza.system.SystemStreamMetadata.OffsetType; /** - * The base descriptor for an input stream. Allows setting properties that are common to all input streams. + * An {@link InputDescriptor} can be used for specifying Samza and system-specific properties of input streams. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. + *

+ * This is the base descriptor for an input stream. Use a system-specific input descriptor (e.g. KafkaInputDescriptor) + * obtained from its system descriptor (e.g. KafkaSystemDescriptor) if one is available. Otherwise use the + * {@link GenericInputDescriptor} obtained from a {@link GenericSystemDescriptor}. * * @param type of messages in this stream. * @param type of the concrete sub-class diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java index 898be1ea62b98..264c6da6959fe 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/OutputDescriptor.java @@ -21,9 +21,13 @@ import org.apache.samza.serializers.Serde; /** - * The base descriptor for an output stream. Allows setting properties that are common to all output streams. + * An {@link OutputDescriptor} can be used for specifying Samza and system-specific properties of output streams. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. + *

+ * This is the base descriptor for an output stream. Use a system-specific input descriptor (e.g. KafkaOutputDescriptor) + * obtained from its system descriptor (e.g. KafkaSystemDescriptor) if one is available. Otherwise use the + * {@link GenericOutputDescriptor} obtained from a {@link GenericSystemDescriptor}. * * @param type of messages in this stream. * @param type of the concrete sub-class diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java index e8e586f26b44c..43cab8fbe599f 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/StreamDescriptor.java @@ -29,9 +29,14 @@ import org.apache.samza.serializers.Serde; /** - * The base descriptor for an input or output stream. Allows setting properties that are common to all streams. + * A {@link StreamDescriptor} can be used for specifying Samza and system-specific properties of input/output streams. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptors. + * Stream properties provided in configuration override corresponding properties specified using a descriptors. + *

+ * This is the base descriptor for an input/output stream. Use a system-specific input/output descriptor + * (e.g. KafkaInputDescriptor) obtained from its system descriptor (e.g. KafkaSystemDescriptor) if one is available. + * Otherwise use the {@link GenericInputDescriptor} and {@link GenericOutputDescriptor} obtained from a + * {@link GenericSystemDescriptor}. * * @param type of messages in this stream. * @param type of the concrete sub-class diff --git a/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java b/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java index 9db25440c84e5..813deb1e01e78 100644 --- a/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/system/descriptors/SystemDescriptor.java @@ -30,9 +30,14 @@ import org.slf4j.LoggerFactory; /** - * The base descriptor for a system. Allows setting properties that are common to all systems. + * A {@link SystemDescriptor} can be used for specifying Samza and system-specific properties of an input/output system. + * It can also be used for obtaining {@link InputDescriptor}s and {@link OutputDescriptor}s, which can be used for + * specifying Samza and system-specific properties of input/output streams. *

- * System properties provided in configuration override corresponding properties configured using a descriptor. + * System properties provided in configuration override corresponding properties specified using a descriptor. + *

+ * This is the base descriptor for a system. Use a system-specific descriptor (e.g. KafkaSystemDescriptor) if one + * is available. Otherwise use the {@link GenericSystemDescriptor}. *

* Systems may provide an {@link InputTransformer} to be used for input streams on the system. An * {@link InputTransformer} transforms an {@code IncomingMessageEnvelope} with deserialized key and message 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 index 767e17697f5c1..76ad460e2e9ff 100644 --- a/samza-api/src/main/java/org/apache/samza/table/Table.java +++ b/samza-api/src/main/java/org/apache/samza/table/Table.java @@ -20,9 +20,28 @@ import org.apache.samza.annotation.InterfaceStability; + /** * - * Marker interface for a table. + * A {@link Table} is an abstraction for data sources that support random access by key. It is an + * evolution of the existing {@link org.apache.samza.storage.kv.KeyValueStore} API. It offers support for + * both local and remote data sources and composition through hybrid tables. For remote data sources, + * a {@code RemoteTable} provides optimized access with caching, rate-limiting, and retry support. + *

+ * Depending on the implementation, a {@link Table} can be a {@link ReadableTable} or a {@link ReadWriteTable}. + *

+ * Use a {@link org.apache.samza.table.descriptors.TableDescriptor} to specify the properties of a {@link Table}. + * For High Level API {@link org.apache.samza.application.StreamApplication}s, use + * {@link org.apache.samza.application.descriptors.StreamApplicationDescriptor#getTable} to obtain + * the {@link org.apache.samza.table.Table} instance for the descriptor that can be used with the + * {@link org.apache.samza.operators.MessageStream} operators like + * {@link org.apache.samza.operators.MessageStream#sendTo(Table)}. + * Alternatively, use {@link org.apache.samza.context.TaskContext#getTable(String)} in + * {@link org.apache.samza.operators.functions.InitableFunction#init} to get the table instance for use within + * operator functions. + * For Low Level API {@link org.apache.samza.application.TaskApplication}s, use + * {@link org.apache.samza.context.TaskContext#getTable(String)} in + * {@link org.apache.samza.task.InitableTask#init} to get the table instance for use within the Task. * * @param the type of records in the table */ diff --git a/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java index 5d7b89e0911c8..f1118ebf68ef9 100644 --- a/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/table/descriptors/TableDescriptor.java @@ -20,25 +20,33 @@ import org.apache.samza.annotation.InterfaceStability; + /** - * User facing class to collect metadata that fully describes a - * Samza table. This interface should be implemented by concrete table implementations. + * A {@link TableDescriptor} can be used for specifying Samza and implementation-specific properties of a + * {@link org.apache.samza.table.Table}. *

- * 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",
+ * Table properties provided in configuration override corresponding properties specified using a descriptor.
+ * 

+ * This is the base descriptor for a table. Use a implementation-specific descriptor (e.g. RocksDBTableDescriptor) to + * use it in the application. For example: + *

{@code
+ * RocksDbTableDescriptor tableDescriptor = new RocksDbTableDescriptor("table",
  *         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. + * For High Level API {@link org.apache.samza.application.StreamApplication}s, use + * {@link org.apache.samza.application.descriptors.StreamApplicationDescriptor#getTable(TableDescriptor)} to obtain + * the corresponding {@link org.apache.samza.table.Table} instance that can be used with the + * {@link org.apache.samza.operators.MessageStream} operators like + * {@link org.apache.samza.operators.MessageStream#sendTo(org.apache.samza.table.Table)}. + * Alternatively, use {@link org.apache.samza.context.TaskContext#getTable(String)} in + * {@link org.apache.samza.operators.functions.InitableFunction#init} to get the table instance for use within + * operator functions. + * For Low Level API {@link org.apache.samza.application.TaskApplication}s, use + * {@link org.apache.samza.context.TaskContext#getTable(String)} in + * {@link org.apache.samza.task.InitableTask#init} to get the table instance for use within the Task. * * @param the type of the key in this table * @param the type of the value in this table @@ -48,13 +56,14 @@ public interface TableDescriptor> { /** - * Get the Id of the table - * @return Id of the table + * Get the id of the table + * @return id of the table */ String getTableId(); /** * Add a configuration entry for the table + * * @param key the key * @param value the value * @return this table descriptor instance diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskFactory.java b/samza-api/src/main/java/org/apache/samza/task/TaskFactory.java index 8443d20698194..f9349bdacaeb6 100644 --- a/samza-api/src/main/java/org/apache/samza/task/TaskFactory.java +++ b/samza-api/src/main/java/org/apache/samza/task/TaskFactory.java @@ -23,9 +23,9 @@ /** - * The interface for all task factories (i.e. {@link StreamTaskFactory} and {@link AsyncStreamTaskFactory} + * The base interface for all task factories (i.e. {@link StreamTaskFactory} and {@link AsyncStreamTaskFactory}) * - * @param the type of task instances + * @param the type of task instances created by the factory */ @InterfaceStability.Stable public interface TaskFactory extends Serializable { diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java index df22269ed8ac7..c8cc36b9ae698 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsInputDescriptor.java @@ -32,11 +32,12 @@ /** - * A descriptor for the Event Hubs output stream + * A {@link EventHubsInputDescriptor} can be used for specifying Samza and EventHubs-specific properties of EventHubs + * input streams. *

- * An instance of this descriptor may be obtained from an {@link EventHubsSystemDescriptor} + * Use {@link EventHubsSystemDescriptor#getInputDescriptor} to obtain an instance of this descriptor. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. * * @param type of messages in this stream */ diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java index 95f7e4201360d..b3e1c594e5f71 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsOutputDescriptor.java @@ -30,12 +30,14 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.system.eventhub.EventHubConfig; + /** - * A descriptor for an Event Hubs output stream + * A {@link EventHubsOutputDescriptor} can be used for specifying Samza and EventHubs-specific properties of EventHubs + * output streams. *

- * An instance of this descriptor may be obtained from and {@link EventHubsSystemDescriptor} + * Use {@link EventHubsSystemDescriptor#getOutputDescriptor} to obtain an instance of this descriptor. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. * * @param type of messages in this stream */ diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java index feffd87b95bbc..20840188c4312 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/descriptors/EventHubsSystemDescriptor.java @@ -32,9 +32,12 @@ /** - * A descriptor for a Event Hubs system. + * A {@link EventHubsSystemDescriptor} can be used for specifying Samza and EventHubs-specific properties of a EventHubs + * input/output system. It can also be used for obtaining {@link EventHubsInputDescriptor}s and + * {@link EventHubsOutputDescriptor}s, which can be used for specifying Samza and system-specific properties of + * EventHubs input/output streams. *

- * System properties provided in configuration override corresponding properties configured using a descriptor. + * System properties provided in configuration override corresponding properties specified using a descriptor. */ public class EventHubsSystemDescriptor extends SystemDescriptor { private static final String FACTORY_CLASS_NAME = EventHubSystemFactory.class.getName(); diff --git a/samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java b/samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java index 2b29a2bf7e2ff..e9cc302d78199 100644 --- a/samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java +++ b/samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java @@ -32,7 +32,7 @@ public LegacyTaskApplication(String taskClassName) { } @Override - public void describe(TaskApplicationDescriptor appDesc) { - appDesc.setTaskFactory(TaskFactoryUtil.getTaskFactory(taskClassName)); + public void describe(TaskApplicationDescriptor appDescriptor) { + appDescriptor.setTaskFactory(TaskFactoryUtil.getTaskFactory(taskClassName)); } } \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java b/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java index 8b96c8a8d997d..6baa54eab3ddc 100644 --- a/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java +++ b/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java @@ -25,7 +25,7 @@ */ public class MockStreamApplication implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appSpec) { + public void describe(StreamApplicationDescriptor appDescriptor) { } } diff --git a/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java b/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java index ab91cee966852..f620ece7f702d 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java @@ -91,7 +91,7 @@ public void testNoAppClassNoTaskClass() { */ public static class MockTaskApplication implements TaskApplication { @Override - public void describe(TaskApplicationDescriptor appSpec) { + public void describe(TaskApplicationDescriptor appDescriptor) { } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java index 6d017cb628cb1..299d63123119a 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java @@ -818,7 +818,7 @@ public void testCreateJobGraphForLegacyTaskApplication() { public static class MockTaskApplication implements SamzaApplication { @Override - public void describe(ApplicationDescriptor appDesc) { + public void describe(ApplicationDescriptor appDescriptor) { } } diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java index fb279ab305db9..d9477e561cd91 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaInputDescriptor.java @@ -27,12 +27,14 @@ import org.apache.samza.system.descriptors.InputTransformer; import org.apache.samza.serializers.Serde; + /** - * A descriptor for a kafka input stream. + * A {@link KafkaInputDescriptor} can be used for specifying Samza and Kafka-specific properties of Kafka + * input streams. *

- * An instance of this descriptor may be obtained from an appropriately configured {@link KafkaSystemDescriptor}. + * Use {@link KafkaSystemDescriptor#getInputDescriptor} to obtain an instance of this descriptor. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. * * @param type of messages in this stream. */ diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java index f13352c292b6e..dcc15a89464d0 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaOutputDescriptor.java @@ -22,12 +22,14 @@ import org.apache.samza.system.descriptors.SystemDescriptor; import org.apache.samza.serializers.Serde; + /** - * A descriptor for a kafka output stream. + * A {@link KafkaOutputDescriptor} can be used for specifying Samza and Kafka-specific properties of Kafka + * output streams. *

- * An instance of this descriptor may be obtained from an appropriately configured {@link KafkaSystemDescriptor}. + * Use {@link KafkaSystemDescriptor#getOutputDescriptor} to obtain an instance of this descriptor. *

- * Stream properties provided in configuration override corresponding properties configured using a descriptor. + * Stream properties provided in configuration override corresponding properties specified using a descriptor. * * @param type of messages in this stream. */ diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java index 6fb8c1c26cf8b..091c21a6c4171 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/descriptors/KafkaSystemDescriptor.java @@ -34,9 +34,12 @@ /** - * A descriptor for a Kafka system. + * A {@link KafkaSystemDescriptor} can be used for specifying Samza and Kafka-specific properties of a Kafka + * input/output system. It can also be used for obtaining {@link KafkaInputDescriptor}s and + * {@link KafkaOutputDescriptor}s, which can be used for specifying Samza and system-specific properties of + * Kafka input/output streams. *

- * System properties provided in configuration override corresponding properties configured using a descriptor. + * System properties provided in configuration override corresponding properties specified using a descriptor. */ @SuppressWarnings("unchecked") public class KafkaSystemDescriptor extends SystemDescriptor diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java index 8265414c3b8b2..47d6cf0e01b7f 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java @@ -47,10 +47,10 @@ public class SamzaSqlApplication implements StreamApplication { private AtomicInteger queryId = new AtomicInteger(0); @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { try { // TODO: Introduce an API to return a dsl string containing one or more sql statements. - List dslStmts = SamzaSqlDslConverter.fetchSqlFromConfig(appDesc.getConfig()); + List dslStmts = SamzaSqlDslConverter.fetchSqlFromConfig(appDescriptor.getConfig()); Map translatorContextMap = new HashMap<>(); @@ -59,21 +59,21 @@ public void describe(StreamApplicationDescriptor appDesc) { Set outputSystemStreams = new HashSet<>(); Collection relRoots = - SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, appDesc.getConfig(), + SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, appDescriptor.getConfig(), inputSystemStreams, outputSystemStreams); // 2. Populate configs SamzaSqlApplicationConfig sqlConfig = - new SamzaSqlApplicationConfig(appDesc.getConfig(), inputSystemStreams, outputSystemStreams); + new SamzaSqlApplicationConfig(appDescriptor.getConfig(), inputSystemStreams, outputSystemStreams); // 3. Translate Calcite plan to Samza stream operators - QueryTranslator queryTranslator = new QueryTranslator(appDesc, sqlConfig); + QueryTranslator queryTranslator = new QueryTranslator(appDescriptor, sqlConfig); SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(sqlConfig); Map converters = sqlConfig.getSamzaRelConverters(); for (RelRoot relRoot : relRoots) { LOG.info("Translating relRoot {} to samza stream graph", relRoot); int qId = queryId.incrementAndGet(); - TranslatorContext translatorContext = new TranslatorContext(appDesc, relRoot, executionContext, converters); + TranslatorContext translatorContext = new TranslatorContext(appDescriptor, relRoot, executionContext, converters); translatorContextMap.put(qId, translatorContext); queryTranslator.translate(relRoot, translatorContext, qId); } @@ -85,7 +85,7 @@ public void describe(StreamApplicationDescriptor appDesc) { * container, so it does not need to be serialized. Therefore, the translatorContext is recreated in each container * and does not need to be serialized. */ - appDesc.withApplicationTaskContextFactory((jobContext, + appDescriptor.withApplicationTaskContextFactory((jobContext, containerContext, taskContext, applicationContainerContext) -> diff --git a/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java b/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java index ba9c8b365baec..766b5293d64d4 100644 --- a/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java @@ -55,7 +55,7 @@ public static void main(String[] args) { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = @@ -65,15 +65,15 @@ public void describe(StreamApplicationDescriptor appDesc) { trackingSystem.getOutputDescriptor("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); - appDesc.getInputStream(inputStreamDescriptor) + appDescriptor.getInputStream(inputStreamDescriptor) .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), () -> 0, (m, c) -> c + 1, null, null) .setEarlyTrigger(Triggers.repeat(Triggers.count(5))) .setAccumulationMode(AccumulationMode.DISCARDING), "window1") .map(m -> KV.of(m.getKey().getKey(), new PageViewCount(m))) - .sendTo(appDesc.getOutputStream(outputStreamDescriptor)); + .sendTo(appDescriptor.getOutputStream(outputStreamDescriptor)); - appDesc.withMetricsReporterFactories(new HashMap<>()); + appDescriptor.withMetricsReporterFactories(new HashMap<>()); } class PageViewEvent { diff --git a/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java b/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java index 7721d44acb772..bf641cec8e153 100644 --- a/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KVSerde serde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor> pageViewEvent = @@ -62,10 +62,10 @@ public void describe(StreamApplicationDescriptor appDesc) { KafkaOutputDescriptor> outStream3 = trackingSystem.getOutputDescriptor("outStream3", serde); - MessageStream> inputStream = appDesc.getInputStream(pageViewEvent); - inputStream.filter(m -> m.key.equals("key1")).sendTo(appDesc.getOutputStream(outStream1)); - inputStream.filter(m -> m.key.equals("key2")).sendTo(appDesc.getOutputStream(outStream2)); - inputStream.filter(m -> m.key.equals("key3")).sendTo(appDesc.getOutputStream(outStream3)); + MessageStream> inputStream = appDescriptor.getInputStream(pageViewEvent); + inputStream.filter(m -> m.key.equals("key1")).sendTo(appDescriptor.getOutputStream(outStream1)); + inputStream.filter(m -> m.key.equals("key2")).sendTo(appDescriptor.getOutputStream(outStream2)); + inputStream.filter(m -> m.key.equals("key3")).sendTo(appDescriptor.getOutputStream(outStream3)); } class PageViewEvent { diff --git a/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java index 4923b7dd44e27..444039a35c9e6 100644 --- a/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java @@ -58,7 +58,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = @@ -68,9 +68,9 @@ public void describe(StreamApplicationDescriptor appDesc) { trackingSystem.getOutputDescriptor("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class))); - appDesc.withDefaultSystem(trackingSystem); - MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor); - OutputStream> pageViewEventPerMember = appDesc.getOutputStream(outputStreamDescriptor); + appDescriptor.withDefaultSystem(trackingSystem); + MessageStream pageViewEvents = appDescriptor.getInputStream(inputStreamDescriptor); + OutputStream> pageViewEventPerMember = appDescriptor.getOutputStream(outputStreamDescriptor); pageViewEvents .partitionBy(pve -> pve.memberId, pve -> pve, diff --git a/samza-test/src/main/java/org/apache/samza/example/MergeExample.java b/samza-test/src/main/java/org/apache/samza/example/MergeExample.java index ac0db36174068..e3eee23b85ed5 100644 --- a/samza-test/src/main/java/org/apache/samza/example/MergeExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/MergeExample.java @@ -49,7 +49,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KVSerde serde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); @@ -64,8 +64,8 @@ public void describe(StreamApplicationDescriptor appDesc) { trackingSystem.getOutputDescriptor("mergedStream", serde); MessageStream - .mergeAll(ImmutableList.of(appDesc.getInputStream(isd1), appDesc.getInputStream(isd2), appDesc.getInputStream(isd3))) - .sendTo(appDesc.getOutputStream(osd)); + .mergeAll(ImmutableList.of(appDescriptor.getInputStream(isd1), appDescriptor.getInputStream(isd2), appDescriptor.getInputStream(isd3))) + .sendTo(appDescriptor.getOutputStream(osd)); } class PageViewEvent { diff --git a/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java index ea38984ee3d2c..54cced117caa4 100644 --- a/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor orderStreamDescriptor = @@ -61,12 +61,12 @@ public void describe(StreamApplicationDescriptor appDesc) { trackingSystem.getOutputDescriptor("fulfilledOrders", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(FulfilledOrderRecord.class))); - appDesc.getInputStream(orderStreamDescriptor) - .join(appDesc.getInputStream(shipmentStreamDescriptor), new MyJoinFunction(), + appDescriptor.getInputStream(orderStreamDescriptor) + .join(appDescriptor.getInputStream(shipmentStreamDescriptor), new MyJoinFunction(), new StringSerde(), new JsonSerdeV2<>(OrderRecord.class), new JsonSerdeV2<>(ShipmentRecord.class), Duration.ofMinutes(1), "join") .map(fulFilledOrder -> KV.of(fulFilledOrder.orderId, fulFilledOrder)) - .sendTo(appDesc.getOutputStream(fulfilledOrdersStreamDescriptor)); + .sendTo(appDescriptor.getOutputStream(fulfilledOrdersStreamDescriptor)); } diff --git a/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java b/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java index 1476c813020be..5fe7b9c149f9d 100644 --- a/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java @@ -58,7 +58,7 @@ public static void main(String[] args) { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = @@ -68,8 +68,8 @@ public void describe(StreamApplicationDescriptor appDesc) { trackingSystem.getOutputDescriptor("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); - MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor); - OutputStream> pageViewEventPerMemberStream = appDesc.getOutputStream(outputStreamDescriptor); + MessageStream pageViewEvents = appDescriptor.getInputStream(inputStreamDescriptor); + OutputStream> pageViewEventPerMemberStream = appDescriptor.getOutputStream(outputStreamDescriptor); SupplierFunction initialValue = () -> 0; FoldLeftFunction foldLeftFn = (m, c) -> c + 1; diff --git a/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java b/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java index 2cf3ac3fb1861..19403b0a56f67 100644 --- a/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java @@ -54,7 +54,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = @@ -64,9 +64,9 @@ public void describe(StreamApplicationDescriptor appDesc) { trackingSystem.getOutputDescriptor("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class))); - appDesc.withDefaultSystem(trackingSystem); - MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor); - OutputStream> pageViewEventPerMember = appDesc.getOutputStream(outputStreamDescriptor); + appDescriptor.withDefaultSystem(trackingSystem); + MessageStream pageViewEvents = appDescriptor.getInputStream(inputStreamDescriptor); + OutputStream> pageViewEventPerMember = appDescriptor.getOutputStream(outputStreamDescriptor); pageViewEvents .partitionBy(pve -> pve.memberId, pve -> pve, diff --git a/samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java b/samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java index 8f6c6f8698479..44528e6c2aff4 100644 --- a/samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java @@ -60,18 +60,18 @@ public static void main(String[] args) { } @Override - public void describe(TaskApplicationDescriptor appDesc) { + public void describe(TaskApplicationDescriptor appDescriptor) { // add input and output streams KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor isd = ksd.getInputDescriptor("myinput", new StringSerde()); KafkaOutputDescriptor osd = ksd.getOutputDescriptor("myout", new StringSerde()); TableDescriptor td = new RocksDbTableDescriptor("mytable"); - appDesc.addInputStream(isd); - appDesc.addOutputStream(osd); - appDesc.addTable(td); + appDescriptor.addInputStream(isd); + appDescriptor.addOutputStream(osd); + appDescriptor.addTable(td); // create the task factory based on configuration - appDesc.setTaskFactory((StreamTaskFactory) () -> new MyStreamTask()); + appDescriptor.setTaskFactory((StreamTaskFactory) () -> new MyStreamTask()); } } \ No newline at end of file diff --git a/samza-test/src/main/java/org/apache/samza/example/WindowExample.java b/samza-test/src/main/java/org/apache/samza/example/WindowExample.java index 51089f78384f7..426fd8d9ac078 100644 --- a/samza-test/src/main/java/org/apache/samza/example/WindowExample.java +++ b/samza-test/src/main/java/org/apache/samza/example/WindowExample.java @@ -57,7 +57,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = @@ -67,8 +67,8 @@ public void describe(StreamApplicationDescriptor appDesc) { SupplierFunction initialValue = () -> 0; FoldLeftFunction counter = (m, c) -> c == null ? 1 : c + 1; - MessageStream inputStream = appDesc.getInputStream(inputStreamDescriptor); - OutputStream outputStream = appDesc.getOutputStream(outputStreamDescriptor); + MessageStream inputStream = appDescriptor.getInputStream(inputStreamDescriptor); + OutputStream outputStream = appDescriptor.getOutputStream(outputStreamDescriptor); // create a tumbling window that outputs the number of message collected every 10 minutes. // also emit early results if either the number of messages collected reaches 30000, or if no new messages arrive diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/TestStandaloneIntegrationApplication.java b/samza-test/src/main/java/org/apache/samza/test/integration/TestStandaloneIntegrationApplication.java index 2e51f6a244136..2002ce6987973 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/TestStandaloneIntegrationApplication.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/TestStandaloneIntegrationApplication.java @@ -37,9 +37,9 @@ public class TestStandaloneIntegrationApplication implements StreamApplication { private static final Logger LOGGER = LoggerFactory.getLogger(TestStandaloneIntegrationApplication.class); @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { String systemName = "testSystemName"; - String inputStreamName = appDesc.getConfig().get("input.stream.name"); + String inputStreamName = appDescriptor.getConfig().get("input.stream.name"); String outputStreamName = "standaloneIntegrationTestKafkaOutputTopic"; LOGGER.info("Publishing message from: {} to: {}.", inputStreamName, outputStreamName); KafkaSystemDescriptor kafkaSystemDescriptor = new KafkaSystemDescriptor(systemName); @@ -49,6 +49,6 @@ public void describe(StreamApplicationDescriptor appDesc) { kafkaSystemDescriptor.getInputDescriptor(inputStreamName, noOpSerde); KafkaOutputDescriptor> osd = kafkaSystemDescriptor.getOutputDescriptor(inputStreamName, noOpSerde); - appDesc.getInputStream(isd).sendTo(appDesc.getOutputStream(osd)); + appDescriptor.getInputStream(isd).sendTo(appDescriptor.getOutputStream(osd)); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java index 672837b616721..6f381e2c73006 100644 --- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java @@ -96,11 +96,11 @@ public void testPipeline() throws Exception { class PipelineApplication implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { DelegatingSystemDescriptor sd = new DelegatingSystemDescriptor("test"); GenericInputDescriptor> isd = sd.getInputDescriptor("PageView", KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); - appDesc.getInputStream(isd) + appDescriptor.getInputStream(isd) .map(KV::getValue) .partitionBy(pv -> pv.getMemberId(), pv -> pv, KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()), "p1") .sink((m, collector, coordinator) -> { diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java index 8431f57c1996e..74c32b487ac2c 100644 --- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java @@ -151,11 +151,11 @@ public void testWatermark() throws Exception { class TestStreamApp implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { DelegatingSystemDescriptor sd = new DelegatingSystemDescriptor("test"); GenericInputDescriptor> isd = sd.getInputDescriptor("PageView", KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); - appDesc.getInputStream(isd) + appDescriptor.getInputStream(isd) .map(KV::getValue) .partitionBy(pv -> pv.getMemberId(), pv -> pv, KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()), "p1") .sink((m, collector, coordinator) -> { diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/BroadcastAssertApp.java b/samza-test/src/test/java/org/apache/samza/test/framework/BroadcastAssertApp.java index ef17a22940c09..28d790ee0335d 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/BroadcastAssertApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/BroadcastAssertApp.java @@ -35,14 +35,14 @@ public class BroadcastAssertApp implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { - Config config = appDesc.getConfig(); + public void describe(StreamApplicationDescriptor appDescriptor) { + Config config = appDescriptor.getConfig(); String inputTopic = config.get(INPUT_TOPIC_NAME_PROP); final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); KafkaInputDescriptor isd = ksd.getInputDescriptor(inputTopic, serde); - final MessageStream broadcastPageViews = appDesc + final MessageStream broadcastPageViews = appDescriptor .getInputStream(isd) .broadcast(serde, "pv"); diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java index 649c03283046c..eca62d0daf705 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/FaultInjectionTest.java @@ -92,15 +92,15 @@ private static class FaultInjectionStreamApp implements TaskApplication { private static transient CountDownLatch containerShutdownLatch; @Override - public void describe(TaskApplicationDescriptor appDesc) { - Config config = appDesc.getConfig(); + public void describe(TaskApplicationDescriptor appDescriptor) { + Config config = appDescriptor.getConfig(); String inputTopic = config.get(INPUT_TOPIC_NAME_PROP); final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); KafkaInputDescriptor isd = ksd.getInputDescriptor(inputTopic, serde); - appDesc.addInputStream(isd); - appDesc.setTaskFactory((StreamTaskFactory) () -> new FaultInjectionTask(containerShutdownLatch)); + appDescriptor.addInputStream(isd); + appDescriptor.setTaskFactory((StreamTaskFactory) () -> new FaultInjectionTask(containerShutdownLatch)); } private static class FaultInjectionTask implements StreamTask, ClosableTask { diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java index a4421401c64e6..476c0dc3655f3 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java @@ -129,20 +129,20 @@ public void testSamzaJobFailureForStreamApplication() { private static class PageViewProfileViewJoinApplication implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { - Table> table = appDesc.getTable( + public void describe(StreamApplicationDescriptor appDescriptor) { + Table> table = appDescriptor.getTable( new RocksDbTableDescriptor("profile-view-store", KVSerde.of(new IntegerSerde(), new TestTableData.ProfileJsonSerde()))); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("test"); KafkaInputDescriptor profileISD = ksd.getInputDescriptor("Profile", new NoOpSerde<>()); - appDesc.getInputStream(profileISD).map(m -> new KV(m.getMemberId(), m)).sendTo(table); + appDescriptor.getInputStream(profileISD).map(m -> new KV(m.getMemberId(), m)).sendTo(table); KafkaInputDescriptor pageViewISD = ksd.getInputDescriptor("PageView", new NoOpSerde<>()); KafkaOutputDescriptor enrichedPageViewOSD = ksd.getOutputDescriptor("EnrichedPageView", new NoOpSerde<>()); - OutputStream outputStream = appDesc.getOutputStream(enrichedPageViewOSD); - appDesc.getInputStream(pageViewISD) + OutputStream outputStream = appDescriptor.getOutputStream(enrichedPageViewOSD); + appDescriptor.getInputStream(pageViewISD) .partitionBy(TestTableData.PageView::getMemberId, pv -> pv, KVSerde.of(new IntegerSerde(), new JsonSerdeV2<>( TestTableData.PageView.class)), "p1") .join(table, new PageViewToProfileJoinFunction()) @@ -152,22 +152,22 @@ public void describe(StreamApplicationDescriptor appDesc) { private static class PageViewFilterApplication implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("test"); KafkaInputDescriptor> isd = ksd.getInputDescriptor("PageView", KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); - MessageStream> inputStream = appDesc.getInputStream(isd); + MessageStream> inputStream = appDescriptor.getInputStream(isd); inputStream.map(KV::getValue).filter(pv -> pv.getPageKey().equals("inbox")); } } private static class PageViewRepartitionApplication implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("test"); KafkaInputDescriptor> isd = ksd.getInputDescriptor("PageView", KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); - MessageStream> inputStream = appDesc.getInputStream(isd); + MessageStream> inputStream = appDescriptor.getInputStream(isd); inputStream .map(KV::getValue) .partitionBy(PageView::getMemberId, pv -> pv, KVSerde.of(new IntegerSerde(), new JsonSerdeV2<>(PageView.class)), "p1") diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java index aa9e1079044e7..184bb125f22f8 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java @@ -207,18 +207,18 @@ public void testSyncTaskWithMultiplePartitionMultithreaded() throws Exception { static public class JoinTaskApplication implements TaskApplication { @Override - public void describe(TaskApplicationDescriptor appDesc) { - appDesc.setTaskFactory((StreamTaskFactory) () -> new StatefulStreamTask()); - appDesc.addTable(new InMemoryTableDescriptor("profile-view-store", + public void describe(TaskApplicationDescriptor appDescriptor) { + appDescriptor.setTaskFactory((StreamTaskFactory) () -> new StatefulStreamTask()); + appDescriptor.addTable(new InMemoryTableDescriptor("profile-view-store", KVSerde.of(new IntegerSerde(), new TestTableData.ProfileJsonSerde()))); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("test"); KafkaInputDescriptor profileISD = ksd.getInputDescriptor("Profile", new NoOpSerde<>()); - appDesc.addInputStream(profileISD); + appDescriptor.addInputStream(profileISD); KafkaInputDescriptor pageViewISD = ksd.getInputDescriptor("PageView", new NoOpSerde<>()); - appDesc.addInputStream(pageViewISD); + appDescriptor.addInputStream(pageViewISD); KafkaOutputDescriptor enrichedPageViewOSD = ksd.getOutputDescriptor("EnrichedPageView", new NoOpSerde<>()); - appDesc.addOutputStream(enrichedPageViewOSD); + appDescriptor.addOutputStream(enrichedPageViewOSD); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java b/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java index 20f18ee0ff6ed..b6d3ed86a1d14 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/TestSchedulingApp.java @@ -39,11 +39,11 @@ public class TestSchedulingApp implements StreamApplication { public static final String PAGE_VIEWS = "page-views"; @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("kafka"); KafkaInputDescriptor isd = ksd.getInputDescriptor(PAGE_VIEWS, serde); - final MessageStream pageViews = appDesc.getInputStream(isd); + final MessageStream pageViews = appDescriptor.getInputStream(isd); final MessageStream output = pageViews.flatMap(new FlatmapScheduledFn()); MessageStreamAssert.that("Output from scheduling function should container all complete messages", output, serde) diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java index dda31ea358e34..24726f83b86d0 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java @@ -55,10 +55,10 @@ public class RepartitionJoinWindowApp implements StreamApplication { private final List intermediateStreamIds = new ArrayList<>(); @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { // offset.default = oldest required for tests since checkpoint topic is empty on start and messages are published // before the application is run - Config config = appDesc.getConfig(); + Config config = appDescriptor.getConfig(); String inputTopic1 = config.get(INPUT_TOPIC_1_CONFIG_KEY); String inputTopic2 = config.get(INPUT_TOPIC_2_CONFIG_KEY); String outputTopic = config.get(OUTPUT_TOPIC_CONFIG_KEY); @@ -66,8 +66,8 @@ public void describe(StreamApplicationDescriptor appDesc) { KafkaInputDescriptor id1 = ksd.getInputDescriptor(inputTopic1, new JsonSerdeV2<>(PageView.class)); KafkaInputDescriptor id2 = ksd.getInputDescriptor(inputTopic2, new JsonSerdeV2<>(AdClick.class)); - MessageStream pageViews = appDesc.getInputStream(id1); - MessageStream adClicks = appDesc.getInputStream(id2); + MessageStream pageViews = appDescriptor.getInputStream(id1); + MessageStream adClicks = appDescriptor.getInputStream(id2); MessageStream> pageViewsRepartitionedByViewId = pageViews .partitionBy(PageView::getViewId, pv -> pv, diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java index fdf0761e8032c..fe8e31803683a 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java @@ -47,19 +47,19 @@ public class RepartitionWindowApp implements StreamApplication { @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KVSerde inputSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageView.class)); KVSerde outputSerde = KVSerde.of(new StringSerde(), new StringSerde()); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); KafkaInputDescriptor> id = ksd.getInputDescriptor(INPUT_TOPIC, inputSerde); KafkaOutputDescriptor> od = ksd.getOutputDescriptor(OUTPUT_TOPIC, outputSerde); - appDesc.getInputStream(id) + appDescriptor.getInputStream(id) .map(KV::getValue) .partitionBy(PageView::getUserId, m -> m, inputSerde, "p1") .window(Windows.keyedSessionWindow(m -> m.getKey(), Duration.ofSeconds(3), () -> 0, (m, c) -> c + 1, new StringSerde("UTF-8"), new IntegerSerde()), "w1") .map(wp -> KV.of(wp.getKey().getKey().toString(), String.valueOf(wp.getMessage()))) - .sendTo(appDesc.getOutputStream(od)); + .sendTo(appDescriptor.getOutputStream(od)); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java index 6dd4303f1f0bf..508e3dccf1517 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java @@ -57,15 +57,15 @@ public static void main(String[] args) { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { JsonSerdeV2 inputSerde = new JsonSerdeV2<>(PageView.class); KVSerde outputSerde = KVSerde.of(new StringSerde(), new IntegerSerde()); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); KafkaInputDescriptor id = ksd.getInputDescriptor(INPUT_TOPIC, inputSerde); KafkaOutputDescriptor> od = ksd.getOutputDescriptor(OUTPUT_TOPIC, outputSerde); - MessageStream pageViews = appDesc.getInputStream(id); - OutputStream> outputStream = appDesc.getOutputStream(od); + MessageStream pageViews = appDescriptor.getInputStream(id); + OutputStream> outputStream = appDescriptor.getOutputStream(od); pageViews .filter(m -> !FILTER_KEY.equals(m.getUserId())) diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java index 4b87169514a6d..d1bd44f5d38c4 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java @@ -59,15 +59,15 @@ public static void main(String[] args) { } @Override - public void describe(StreamApplicationDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { JsonSerdeV2 inputSerde = new JsonSerdeV2<>(PageView.class); KVSerde outputSerde = KVSerde.of(new StringSerde(), new IntegerSerde()); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); KafkaInputDescriptor id = ksd.getInputDescriptor(INPUT_TOPIC, inputSerde); KafkaOutputDescriptor> od = ksd.getOutputDescriptor(OUTPUT_TOPIC, outputSerde); - MessageStream pageViews = appDesc.getInputStream(id); - OutputStream> outputStream = appDesc.getOutputStream(od); + MessageStream pageViews = appDescriptor.getInputStream(id); + OutputStream> outputStream = appDescriptor.getOutputStream(od); pageViews .filter(m -> !FILTER_KEY.equals(m.getUserId())) diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java index 0991fa1a5a708..a2170cc9e4291 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java @@ -59,12 +59,12 @@ private TestStreamApplication(String systemName, String inputTopic, String outpu } @Override - public void describe(StreamApplicationDescriptor streamAppDesc) { + public void describe(StreamApplicationDescriptor appDescriptor) { KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(systemName); KafkaInputDescriptor isd = ksd.getInputDescriptor(inputTopic, new NoOpSerde<>()); KafkaOutputDescriptor osd = ksd.getOutputDescriptor(outputTopic, new StringSerde()); - MessageStream inputStream = streamAppDesc.getInputStream(isd); - OutputStream outputStream = streamAppDesc.getOutputStream(osd); + MessageStream inputStream = appDescriptor.getInputStream(isd); + OutputStream outputStream = appDescriptor.getOutputStream(osd); inputStream.map(new TestMapFunction(appName, processorName)).sendTo(outputStream); } 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 index a9741b49cce28..5a977e2cf7d43 100644 --- 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 @@ -416,12 +416,12 @@ public void testWithLowLevelApi() throws Exception { static public class MyTaskApplication implements TaskApplication { @Override - public void describe(TaskApplicationDescriptor appDesc) { - appDesc.setTaskFactory((StreamTaskFactory) () -> new MyStreamTask()); - appDesc.addTable(new InMemoryTableDescriptor("t1", KVSerde.of(new IntegerSerde(), new PageViewJsonSerde()))); + public void describe(TaskApplicationDescriptor appDescriptor) { + appDescriptor.setTaskFactory((StreamTaskFactory) () -> new MyStreamTask()); + appDescriptor.addTable(new InMemoryTableDescriptor("t1", KVSerde.of(new IntegerSerde(), new PageViewJsonSerde()))); DelegatingSystemDescriptor ksd = new DelegatingSystemDescriptor("test"); GenericInputDescriptor pageViewISD = ksd.getInputDescriptor("PageView", new NoOpSerde<>()); - appDesc.addInputStream(pageViewISD); + appDescriptor.addInputStream(pageViewISD); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java index 5852de514ec21..4e410d9ab1f10 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java @@ -130,14 +130,14 @@ static class PageViewProfileJoin implements StreamApplication { static final String PROFILE_TABLE = "profile-table"; @Override - public void describe(StreamApplicationDescriptor appDesc) { - Table> table = appDesc.getTable(getTableDescriptor()); + public void describe(StreamApplicationDescriptor appDescriptor) { + Table> table = appDescriptor.getTable(getTableDescriptor()); KafkaSystemDescriptor sd = new KafkaSystemDescriptor("test"); - appDesc.getInputStream(sd.getInputDescriptor(PAGEVIEW_STREAM, new NoOpSerde())) + appDescriptor.getInputStream(sd.getInputDescriptor(PAGEVIEW_STREAM, new NoOpSerde())) .partitionBy(TestTableData.PageView::getMemberId, v -> v, KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()), "partition-page-view") .join(table, new PageViewToProfileJoinFunction()) - .sendTo(appDesc.getOutputStream(sd.getOutputDescriptor(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<>()))); + .sendTo(appDescriptor.getOutputStream(sd.getOutputDescriptor(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<>()))); } protected TableDescriptor getTableDescriptor() { From 44d0685dfbd979b11209903fadaee2468b001092 Mon Sep 17 00:00:00 2001 From: Boris S Date: Thu, 18 Oct 2018 09:56:01 -0700 Subject: [PATCH 2/3] Keep a version of deprecated KafkaSystemConsumer Author: Boris S Author: Boris S Author: Boris Shkolnik Reviewers: Prateek Maheshwari Closes #740 from sborya/OldKafkaConsumer --- .../system/kafka_deprecated/BrokerProxy.scala | 332 ++++++++++ .../DefaultFetchSimpleConsumer.scala | 66 ++ .../system/kafka_deprecated/GetOffset.scala | 116 ++++ .../kafka_deprecated/KafkaSystemAdmin.scala | 609 ++++++++++++++++++ .../KafkaSystemConsumer.scala | 309 +++++++++ .../KafkaSystemConsumerMetrics.scala | 100 +++ .../kafka_deprecated/KafkaSystemFactory.scala | 178 +++++ .../KafkaSystemProducer.scala | 235 +++++++ .../KafkaSystemProducerMetrics.scala | 42 ++ .../system/kafka_deprecated/MessageSink.scala | 35 + .../kafka_deprecated/TopicMetadataCache.scala | 78 +++ .../samza/system/kafka_deprecated/Toss.scala | 28 + 12 files changed, 2128 insertions(+) create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/BrokerProxy.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/DefaultFetchSimpleConsumer.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/GetOffset.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemAdmin.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumer.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumerMetrics.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemFactory.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducer.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducerMetrics.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/MessageSink.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/TopicMetadataCache.scala create mode 100644 samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/Toss.scala diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/BrokerProxy.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/BrokerProxy.scala new file mode 100644 index 0000000000000..a39752b74de1c --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/BrokerProxy.scala @@ -0,0 +1,332 @@ +/* + * + * 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.system.kafka_deprecated + +import java.lang.Thread.UncaughtExceptionHandler +import java.nio.channels.ClosedByInterruptException +import java.util.concurrent.{ConcurrentHashMap, CountDownLatch} + +import kafka.api._ +import kafka.common.{ErrorMapping, NotLeaderForPartitionException, TopicAndPartition, UnknownTopicOrPartitionException} +import kafka.consumer.ConsumerConfig +import kafka.message.MessageSet +import org.apache.samza.SamzaException +import org.apache.samza.util.ExponentialSleepStrategy +import org.apache.samza.util.KafkaUtil +import org.apache.samza.util.Logging + +import scala.collection.JavaConverters._ +import scala.collection.concurrent + +/** + * A BrokerProxy consolidates Kafka fetches meant for a particular broker and retrieves them all at once, providing + * a way for consumers to retrieve those messages by topic and partition. + */ +class BrokerProxy( + val host: String, + val port: Int, + val system: String, + val clientID: String, + val metrics: KafkaSystemConsumerMetrics, + val messageSink: MessageSink, + val timeout: Int = ConsumerConfig.SocketTimeout, + val bufferSize: Int = ConsumerConfig.SocketBufferSize, + val fetchSize: StreamFetchSizes = new StreamFetchSizes, + val consumerMinSize:Int = ConsumerConfig.MinFetchBytes, + val consumerMaxWait:Int = ConsumerConfig.MaxFetchWaitMs, + offsetGetter: GetOffset = new GetOffset("fail")) extends Toss with Logging { + + /** + * How long should the fetcher thread sleep before checking if any TopicPartitions has been added to its purview + */ + val sleepMSWhileNoTopicPartitions = 100 + + /** What's the next offset for a particular partition? **/ + val nextOffsets:concurrent.Map[TopicAndPartition, Long] = new ConcurrentHashMap[TopicAndPartition, Long]().asScala + + /** Block on the first call to get message if the fetcher has not yet returned its initial results **/ + // TODO: It should be sufficient to just use the count down latch and await on it for each of the calls, but + // VisualVM was showing the consumer thread spending all its time in the await method rather than returning + // immediately, even though the process was proceeding normally. Hence the extra boolean. Should be investigated. + val firstCallBarrier = new CountDownLatch(1) + var firstCall = true + + var simpleConsumer = createSimpleConsumer() + + metrics.registerBrokerProxy(host, port) + + def createSimpleConsumer() = { + val hostString = "%s:%d" format (host, port) + info("Creating new SimpleConsumer for host %s for system %s" format (hostString, system)) + + val sc = new DefaultFetchSimpleConsumer(host, port, timeout, bufferSize, clientID, fetchSize, consumerMinSize, consumerMaxWait) + sc + } + + def addTopicPartition(tp: TopicAndPartition, nextOffset: Option[String]) = { + debug("Adding new topic and partition %s to queue for %s" format (tp, host)) + + if (nextOffsets.asJava.containsKey(tp)) { + toss("Already consuming TopicPartition %s" format tp) + } + + val offset = if (nextOffset.isDefined && offsetGetter.isValidOffset(simpleConsumer, tp, nextOffset.get)) { + nextOffset + .get + .toLong + } else { + warn("It appears that we received an invalid or empty offset %s for %s. Attempting to use Kafka's auto.offset.reset setting. This can result in data loss if processing continues." format (nextOffset, tp)) + + offsetGetter.getResetOffset(simpleConsumer, tp) + } + + debug("Got offset %s for new topic and partition %s." format (offset, tp)) + + nextOffsets += tp -> offset + + metrics.topicPartitions.get((host, port)).set(nextOffsets.size) + } + + def removeTopicPartition(tp: TopicAndPartition) = { + if (nextOffsets.asJava.containsKey(tp)) { + val offset = nextOffsets.remove(tp) + metrics.topicPartitions.get((host, port)).set(nextOffsets.size) + debug("Removed %s" format tp) + offset + } else { + warn("Asked to remove topic and partition %s, but not in map (keys = %s)" format (tp, nextOffsets.keys.mkString(","))) + None + } + } + + val thread = new Thread(new Runnable { + def run { + var reconnect = false + + try { + (new ExponentialSleepStrategy).run( + loop => { + if (reconnect) { + metrics.reconnects.get((host, port)).inc + simpleConsumer.close() + simpleConsumer = createSimpleConsumer() + } + + while (!Thread.currentThread.isInterrupted) { + messageSink.refreshDropped + if (nextOffsets.size == 0) { + debug("No TopicPartitions to fetch. Sleeping.") + Thread.sleep(sleepMSWhileNoTopicPartitions) + } else { + fetchMessages + + // If we got here, fetchMessages didn't throw an exception, i.e. it was successful. + // In that case, reset the loop delay, so that the next time an error occurs, + // we start with a short retry delay. + loop.reset + } + } + }, + + (exception, loop) => { + warn("Restarting consumer due to %s. Releasing ownership of all partitions, and restarting consumer. Turn on debugging to get a full stack trace." format exception) + debug("Exception detail:", exception) + abdicateAll + reconnect = true + }) + } catch { + case e: InterruptedException => info("Got interrupt exception in broker proxy thread.") + case e: ClosedByInterruptException => info("Got closed by interrupt exception in broker proxy thread.") + case e: OutOfMemoryError => throw new SamzaException("Got out of memory error in broker proxy thread.") + case e: StackOverflowError => throw new SamzaException("Got stack overflow error in broker proxy thread.") + } + + if (Thread.currentThread.isInterrupted) info("Shutting down due to interrupt.") + } + }, "BrokerProxy thread pointed at %s:%d for client %s" format (host, port, clientID)) + + private def fetchMessages(): Unit = { + val topicAndPartitionsToFetch = nextOffsets.filterKeys(messageSink.needsMoreMessages(_)).toList + + if (topicAndPartitionsToFetch.size > 0) { + metrics.brokerReads.get((host, port)).inc + val response: FetchResponse = simpleConsumer.defaultFetch(topicAndPartitionsToFetch: _*) + firstCall = false + firstCallBarrier.countDown() + + // Split response into errors and non errors, processing the errors first + val (nonErrorResponses, errorResponses) = response.data.toSet.partition(_._2.error.code() == ErrorMapping.NoError) + + handleErrors(errorResponses, response) + + nonErrorResponses.foreach { case (tp, data) => moveMessagesToTheirQueue(tp, data) } + } else { + refreshLatencyMetrics + + debug("No topic/partitions need to be fetched for %s:%s right now. Sleeping %sms." format (host, port, sleepMSWhileNoTopicPartitions)) + + metrics.brokerSkippedFetchRequests.get((host, port)).inc + + Thread.sleep(sleepMSWhileNoTopicPartitions) + } + } + + /** + * Releases ownership for a single TopicAndPartition. The + * KafkaSystemConsumer will try and find a new broker for the + * TopicAndPartition. + */ + def abdicate(tp: TopicAndPartition) = removeTopicPartition(tp) match { + // Need to be mindful of a tp that was removed by another thread + case Some(offset) => messageSink.abdicate(tp, offset) + case None => warn("Tried to abdicate for topic partition not in map. Removed in interim?") + } + + /** + * Releases all TopicAndPartition ownership for this BrokerProxy thread. The + * KafkaSystemConsumer will try and find a new broker for the + * TopicAndPartition. + */ + def abdicateAll { + info("Abdicating all topic partitions.") + val immutableNextOffsetsCopy = nextOffsets.toMap + immutableNextOffsetsCopy.keySet.foreach(abdicate(_)) + } + + 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: 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; + 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) + + // Can recover from two types of errors: not leader (go find the new leader) and offset out of range (go get the new offset) + // However, we want to bail as quickly as possible if there are non recoverable errors so that the state of the other + // topic-partitions remains the same. That way, when we've rebuilt the simple consumer, we can come around and + // 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.exception) }) + + notLeaderOrUnknownTopic.foreach(e => { + warn("Received (UnknownTopicOr|NotLeaderFor)Partition exception %s for %s. Abdicating" format(e.code, e.tp)) + abdicate(e.tp) + }) + + offsetOutOfRangeErrors.foreach(e => { + warn("Received OffsetOutOfRange exception for %s. Current offset = %s" format (e.tp, nextOffsets.getOrElse(e.tp, "not found in map, likely removed in the interim"))) + + try { + val newOffset = offsetGetter.getResetOffset(simpleConsumer, e.tp) + // Put the new offset into the map (if the tp still exists). Will catch it on the next go-around + nextOffsets.replace(e.tp, newOffset) + } catch { + // UnknownTopic or NotLeader are routine events and handled via abdication. All others, bail. + case _ @ (_:UnknownTopicOrPartitionException | _: NotLeaderForPartitionException) => warn("Received (UnknownTopicOr|NotLeaderFor)Partition exception %s for %s. Abdicating" format(e.code, e.tp)) + abdicate(e.tp) + } + }) + } + + def moveMessagesToTheirQueue(tp: TopicAndPartition, data: FetchResponsePartitionData) = { + val messageSet: MessageSet = data.messages + var nextOffset = nextOffsets(tp) + + messageSink.setIsAtHighWatermark(tp, data.hw == 0 || data.hw == nextOffset) + require(messageSet != null) + for (message <- messageSet.iterator) { + messageSink.addMessage(tp, message, data.hw) // TODO: Verify this is correct + + nextOffset = message.nextOffset + + val bytesSize = message.message.payloadSize + message.message.keySize + metrics.reads.get(tp).inc + metrics.bytesRead.get(tp).inc(bytesSize) + metrics.brokerBytesRead.get((host, port)).inc(bytesSize) + metrics.offsets.get(tp).set(nextOffset) + } + + nextOffsets.replace(tp, nextOffset) // use replace rather than put in case this tp was removed while we were fetching. + + // Update high water mark + val hw = data.hw + if (hw >= 0) { + metrics.highWatermark.get(tp).set(hw) + metrics.lag.get(tp).set(hw - nextOffset) + } else { + debug("Got a high water mark less than 0 (%d) for %s, so skipping." format (hw, tp)) + } + } + override def toString() = "BrokerProxy for %s:%d" format (host, port) + + def start { + if (!thread.isAlive) { + info("Starting " + toString) + thread.setDaemon(true) + thread.setName("Samza BrokerProxy " + thread.getName) + thread.setUncaughtExceptionHandler(new UncaughtExceptionHandler { + override def uncaughtException(t: Thread, e: Throwable) = error("Uncaught exception in broker proxy:", e) + }) + thread.start + } else { + debug("Tried to start an already started broker proxy (%s). Ignoring." format toString) + } + } + + def stop { + info("Shutting down " + toString) + + if (simpleConsumer != null) { + info("closing simple consumer...") + simpleConsumer.close + } + + thread.interrupt + thread.join + } + + private def refreshLatencyMetrics { + nextOffsets.foreach{ + case (topicAndPartition, offset) => { + val latestOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, -1, Request.OrdinaryConsumerId) + trace("latest offset of %s is %s" format (topicAndPartition, latestOffset)) + if (latestOffset >= 0) { + // only update the registered topicAndpartitions + if(metrics.highWatermark.containsKey(topicAndPartition)) { + metrics.highWatermark.get(topicAndPartition).set(latestOffset) + } + if(metrics.lag.containsKey(topicAndPartition)) { + metrics.lag.get(topicAndPartition).set(latestOffset - offset) + } + } + } + } + } +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/DefaultFetchSimpleConsumer.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/DefaultFetchSimpleConsumer.scala new file mode 100644 index 0000000000000..5f79ea568a444 --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/DefaultFetchSimpleConsumer.scala @@ -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.system.kafka_deprecated + +import kafka.consumer.SimpleConsumer +import kafka.api._ +import kafka.common.TopicAndPartition +import kafka.consumer.ConsumerConfig + +class DefaultFetchSimpleConsumer(host: scala.Predef.String, port: scala.Int, soTimeout: scala.Int, bufferSize: scala.Int, + clientId: scala.Predef.String, fetchSize: StreamFetchSizes = new StreamFetchSizes, + minBytes: Int = ConsumerConfig.MinFetchBytes, maxWait: Int = ConsumerConfig.MaxFetchWaitMs) + extends SimpleConsumer(host, port, soTimeout, bufferSize, clientId) { + + def defaultFetch(fetches: (TopicAndPartition, Long)*) = { + val fbr = new FetchRequestBuilder().maxWait(maxWait) + .minBytes(minBytes) + .clientId(clientId) + + fetches.foreach(f => fbr.addFetch(f._1.topic, f._1.partition, f._2, fetchSize.streamValue.getOrElse(f._1.topic, fetchSize.defaultValue))) + + this.fetch(fbr.build()) + } + + override def close(): Unit = super.close() + + override def send(request: TopicMetadataRequest): TopicMetadataResponse = super.send(request) + + override def fetch(request: FetchRequest): FetchResponse = super.fetch(request) + + override def getOffsetsBefore(request: OffsetRequest): OffsetResponse = super.getOffsetsBefore(request) + + override def commitOffsets(request: OffsetCommitRequest): OffsetCommitResponse = super.commitOffsets(request) + + override def fetchOffsets(request: OffsetFetchRequest): OffsetFetchResponse = super.fetchOffsets(request) + + override def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = super.earliestOrLatestOffset(topicAndPartition, earliestOrLatest, consumerId) +} + +/** + * a simple class for holding values for the stream's fetch size (fetch.message.max.bytes). + * The stream-level fetch size values are put in the streamValue map streamName -> fetchSize. + * If stream-level fetch size is not defined, use the default value. The default value is the + * Kafka's default fetch size value or the system-level fetch size value (if defined). + */ +case class StreamFetchSizes(defaultValue: Int = ConsumerConfig.MaxFetchSize, streamValue: Map[String, Int] = Map[String, Int]()) + diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/GetOffset.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/GetOffset.scala new file mode 100644 index 0000000000000..c4e7354dc37af --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/GetOffset.scala @@ -0,0 +1,116 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.samza.system.kafka_deprecated + +import org.apache.kafka.common.errors.OffsetOutOfRangeException +import kafka.api._ +import kafka.common.TopicAndPartition +import kafka.api.PartitionOffsetRequestInfo +import org.apache.samza.util.Logging +import org.apache.samza.util.KafkaUtil + +/** + * GetOffset validates offsets for topic partitions, and manages fetching new + * offsets for topics using Kafka's auto.offset.reset configuration. + */ +class GetOffset( + /** + * The default auto.offset.reset to use if a topic is not overridden in + * autoOffsetResetTopics. Any value other than "earliest" or "latest" will + * result in an exception when getRestOffset is called. + */ + default: String, + + /** + * Topic-level overrides for auto.offset.reset. Any value other than + * "earliest" or "latest" will result in an exception when getRestOffset is + * called. + */ + autoOffsetResetTopics: Map[String, String] = Map()) extends Logging with Toss { + + /** + * Checks if an offset is valid for a given topic/partition. Validity is + * defined as an offset that returns a readable non-empty message set with + * no exceptions. + */ + def isValidOffset(consumer: DefaultFetchSimpleConsumer, topicAndPartition: TopicAndPartition, offset: String) = { + info("Validating offset %s for topic and partition %s" format (offset, topicAndPartition)) + + try { + val messages = consumer.defaultFetch((topicAndPartition, offset.toLong)) + + if (messages.hasError) { + 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)) + + true + } catch { + case e: OffsetOutOfRangeException => false + } + } + + /** + * Uses a topic's auto.offset.reset setting (defined via the + * autoOffsetResetTopics map in the constructor) to fetch either the + * earliest or latest offset. If neither earliest or latest is defined for + * the topic in question, the default supplied in the constructor will be + * used. + */ + def getResetOffset(consumer: DefaultFetchSimpleConsumer, topicAndPartition: TopicAndPartition) = { + val offsetRequest = new OffsetRequest(Map(topicAndPartition -> new PartitionOffsetRequestInfo(getAutoOffset(topicAndPartition.topic), 1))) + val offsetResponse = consumer.getOffsetsBefore(offsetRequest) + val partitionOffsetResponse = offsetResponse + .partitionErrorAndOffsets + .get(topicAndPartition) + .getOrElse(toss("Unable to find offset information for %s" format topicAndPartition)) + + KafkaUtil.maybeThrowException(partitionOffsetResponse.error.exception()) + + partitionOffsetResponse + .offsets + .headOption + .getOrElse(toss("Got response, but no offsets defined for %s" format topicAndPartition)) + } + + /** + * Returns either the earliest or latest setting (a Kafka constant) for a + * given topic using the autoOffsetResetTopics map defined in the + * constructor. If the topic is not defined in autoOffsetResetTopics, the + * default value supplied in the constructor will be used. This is used in + * conjunction with getResetOffset to fetch either the earliest or latest + * offset for a topic. + */ + private def getAutoOffset(topic: String): Long = { + info("Checking if auto.offset.reset is defined for topic %s" format (topic)) + autoOffsetResetTopics.getOrElse(topic, default) match { + case OffsetRequest.LargestTimeString => + info("Got reset of type %s." format OffsetRequest.LargestTimeString) + OffsetRequest.LatestTime + case OffsetRequest.SmallestTimeString => + info("Got reset of type %s." format OffsetRequest.SmallestTimeString) + OffsetRequest.EarliestTime + case other => toss("Can't get offset value for topic %s due to invalid value: %s" format (topic, other)) + } + } +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemAdmin.scala new file mode 100644 index 0000000000000..e7ff7496befd7 --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemAdmin.scala @@ -0,0 +1,609 @@ +/* + * 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.system.kafka_deprecated + +import java.util +import java.util.{Properties, UUID} + +import com.google.common.annotations.VisibleForTesting +import kafka.admin.{AdminClient, AdminUtils} +import kafka.api._ +import kafka.common.TopicAndPartition +import kafka.consumer.{ConsumerConfig, SimpleConsumer} +import kafka.utils.ZkUtils +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.errors.TopicExistsException +import org.apache.kafka.common.TopicPartition +import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata +import org.apache.samza.system._ +import org.apache.samza.system.kafka.KafkaStreamSpec +import org.apache.samza.util.{ClientUtilTopicMetadataStore, ExponentialSleepStrategy, KafkaUtil, Logging} +import org.apache.samza.{Partition, SamzaException} + +import scala.collection.JavaConverters._ + + +object KafkaSystemAdmin extends Logging { + + @VisibleForTesting @volatile var deleteMessagesCalled = false + val CLEAR_STREAM_RETRIES = 3 + + /** + * A helper method that takes oldest, newest, and upcoming offsets for each + * system stream partition, and creates a single map from stream name to + * SystemStreamMetadata. + */ + def assembleMetadata(oldestOffsets: Map[SystemStreamPartition, String], newestOffsets: Map[SystemStreamPartition, String], upcomingOffsets: Map[SystemStreamPartition, String]): Map[String, SystemStreamMetadata] = { + val allMetadata = (oldestOffsets.keySet ++ newestOffsets.keySet ++ upcomingOffsets.keySet) + .groupBy(_.getStream) + .map { + case (streamName, systemStreamPartitions) => + val streamPartitionMetadata = systemStreamPartitions + .map(systemStreamPartition => { + val partitionMetadata = new SystemStreamPartitionMetadata( + // If the topic/partition is empty then oldest and newest will + // be stripped of their offsets, so default to null. + oldestOffsets.getOrElse(systemStreamPartition, null), + newestOffsets.getOrElse(systemStreamPartition, null), + upcomingOffsets(systemStreamPartition)) + (systemStreamPartition.getPartition, partitionMetadata) + }) + .toMap + val streamMetadata = new SystemStreamMetadata(streamName, streamPartitionMetadata.asJava) + (streamName, streamMetadata) + } + .toMap + + // This is typically printed downstream and it can be spammy, so debug level here. + debug("Got metadata: %s" format allMetadata) + + allMetadata + } +} + +/** + * A helper class that is used to construct the changelog stream specific information + * + * @param replicationFactor The number of replicas for the changelog stream + * @param kafkaProps The kafka specific properties that need to be used for changelog stream creation + */ +case class ChangelogInfo(var replicationFactor: Int, var kafkaProps: Properties) + +/** + * A Kafka-based implementation of SystemAdmin. + */ +class KafkaSystemAdmin( + /** + * The system name to use when creating SystemStreamPartitions to return in + * the getSystemStreamMetadata responser. + */ + systemName: String, + + // TODO whenever Kafka decides to make the Set[Broker] class public, let's switch to Set[Broker] here. + /** + * List of brokers that are part of the Kafka system that we wish to + * interact with. The format is host1:port1,host2:port2. + */ + brokerListString: String, + + /** + * A method that returns a ZkUtils for the Kafka system. This is invoked + * when the system admin is attempting to create a coordinator stream. + */ + connectZk: () => ZkUtils, + + /** + * Custom properties to use when the system admin tries to create a new + * coordinator stream. + */ + coordinatorStreamProperties: Properties = new Properties, + + /** + * The replication factor to use when the system admin creates a new + * coordinator stream. + */ + coordinatorStreamReplicationFactor: Int = 1, + + /** + * The timeout to use for the simple consumer when fetching metadata from + * Kafka. Equivalent to Kafka's socket.timeout.ms configuration. + */ + timeout: Int = Int.MaxValue, + + /** + * The buffer size to use for the simple consumer when fetching metadata + * from Kafka. Equivalent to Kafka's socket.receive.buffer.bytes + * configuration. + */ + bufferSize: Int = ConsumerConfig.SocketBufferSize, + + /** + * The client ID to use for the simple consumer when fetching metadata from + * Kafka. Equivalent to Kafka's client.id configuration. + */ + clientId: String = UUID.randomUUID.toString, + + /** + * Replication factor for the Changelog topic in kafka + * Kafka properties to be used during the Changelog topic creation + */ + topicMetaInformation: Map[String, ChangelogInfo] = Map[String, ChangelogInfo](), + + /** + * Kafka properties to be used during the intermediate topic creation + */ + intermediateStreamProperties: Map[String, Properties] = Map(), + + /** + * Whether deleteMessages() API can be used + */ + deleteCommittedMessages: Boolean = false) extends ExtendedSystemAdmin with Logging { + + import KafkaSystemAdmin._ + + @volatile var running = false + @volatile var adminClient: AdminClient = null + + override def start() = { + if (!running) { + running = true + adminClient = createAdminClient() + } + } + + override def stop() = { + if (running) { + running = false + adminClient.close() + adminClient = null + } + } + + private def createAdminClient(): AdminClient = { + val props = new Properties() + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerListString) + AdminClient.create(props) + } + + override def getSystemStreamPartitionCounts(streams: util.Set[String], cacheTTL: Long): util.Map[String, SystemStreamMetadata] = { + getSystemStreamPartitionCounts(streams, new ExponentialSleepStrategy(initialDelayMs = 500), cacheTTL) + } + + def getSystemStreamPartitionCounts(streams: util.Set[String], retryBackoff: ExponentialSleepStrategy, cacheTTL: Long = Long.MaxValue): util.Map[String, SystemStreamMetadata] = { + debug("Fetching system stream partition count for: %s" format streams) + var metadataTTL = cacheTTL + retryBackoff.run( + loop => { + val metadata = TopicMetadataCache.getTopicMetadata( + streams.asScala.toSet, + systemName, + getTopicMetadata, + metadataTTL) + val result = metadata.map { + case (topic, topicMetadata) => { + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) + val partitionsMap = topicMetadata.partitionsMetadata.map { + pm => + new Partition(pm.partitionId) -> new SystemStreamPartitionMetadata("", "", "") + }.toMap[Partition, SystemStreamPartitionMetadata] + (topic -> new SystemStreamMetadata(topic, partitionsMap.asJava)) + } + } + loop.done + result.asJava + }, + + (exception, loop) => { + warn("Unable to fetch last offsets for streams %s due to %s. Retrying." format (streams, exception)) + debug("Exception detail:", exception) + if (metadataTTL == Long.MaxValue) { + metadataTTL = 5000 // Revert to the default cache expiration + } + } + ).getOrElse(throw new SamzaException("Failed to get system stream metadata")) + } + + /** + * Returns the offset for the message after the specified offset for each + * SystemStreamPartition that was passed in. + */ + + override def getOffsetsAfter(offsets: java.util.Map[SystemStreamPartition, String]) = { + // This is safe to do with Kafka, even if a topic is key-deduped. If the + // offset doesn't exist on a compacted topic, Kafka will return the first + // message AFTER the offset that was specified in the fetch request. + offsets.asScala.mapValues(offset => (offset.toLong + 1).toString).asJava + } + + override def getSystemStreamMetadata(streams: java.util.Set[String]) = + getSystemStreamMetadata(streams, new ExponentialSleepStrategy(initialDelayMs = 500)).asJava + + /** + * Given a set of stream names (topics), fetch metadata from Kafka for each + * stream, and return a map from stream name to SystemStreamMetadata for + * each stream. This method will return null for oldest and newest offsets + * if a given SystemStreamPartition is empty. This method will block and + * retry indefinitely until it gets a successful response from Kafka. + */ + def getSystemStreamMetadata(streams: java.util.Set[String], retryBackoff: ExponentialSleepStrategy) = { + debug("Fetching system stream metadata for: %s" format streams) + var metadataTTL = Long.MaxValue // Trust the cache until we get an exception + retryBackoff.run( + loop => { + val metadata = TopicMetadataCache.getTopicMetadata( + streams.asScala.toSet, + systemName, + getTopicMetadata, + metadataTTL) + + debug("Got metadata for streams: %s" format metadata) + + val brokersToTopicPartitions = getTopicsAndPartitionsByBroker(metadata) + var oldestOffsets = Map[SystemStreamPartition, String]() + var newestOffsets = Map[SystemStreamPartition, String]() + var upcomingOffsets = Map[SystemStreamPartition, String]() + + // Get oldest, newest, and upcoming offsets for each topic and partition. + for ((broker, topicsAndPartitions) <- brokersToTopicPartitions) { + debug("Fetching offsets for %s:%s: %s" format (broker.host, broker.port, topicsAndPartitions)) + + val consumer = new SimpleConsumer(broker.host, broker.port, timeout, bufferSize, clientId) + try { + upcomingOffsets ++= getOffsets(consumer, topicsAndPartitions, OffsetRequest.LatestTime) + oldestOffsets ++= getOffsets(consumer, topicsAndPartitions, OffsetRequest.EarliestTime) + + // Kafka's "latest" offset is always last message in stream's offset + + // 1, so get newest message in stream by subtracting one. this is safe + // even for key-deduplicated streams, since the last message will + // never be deduplicated. + newestOffsets = upcomingOffsets.mapValues(offset => (offset.toLong - 1).toString) + // Keep only oldest/newest offsets where there is a message. Should + // return null offsets for empty streams. + upcomingOffsets.foreach { + case (topicAndPartition, offset) => + if (offset.toLong <= 0) { + debug("Stripping newest offsets for %s because the topic appears empty." format topicAndPartition) + newestOffsets -= topicAndPartition + debug("Setting oldest offset to 0 to consume from beginning") + oldestOffsets += (topicAndPartition -> "0") + } + } + } finally { + consumer.close + } + } + + val result = assembleMetadata(oldestOffsets, newestOffsets, upcomingOffsets) + loop.done + result + }, + + (exception, loop) => { + warn("Unable to fetch last offsets for streams %s due to %s. Retrying." format (streams, exception)) + debug("Exception detail:", exception) + metadataTTL = 5000 // Revert to the default cache expiration + }).getOrElse(throw new SamzaException("Failed to get system stream metadata")) + } + + /** + * Returns the newest offset for the specified SSP. + * This method is fast and targeted. It minimizes the number of kafka requests. + * It does not retry indefinitely if there is any failure. + * It returns null if the topic is empty. To get the offsets for *all* + * partitions, it would be more efficient to call getSystemStreamMetadata + */ + override def getNewestOffset(ssp: SystemStreamPartition, maxRetries: Integer) = { + debug("Fetching newest offset for: %s" format ssp) + var offset: String = null + var metadataTTL = Long.MaxValue // Trust the cache until we get an exception + var retries = maxRetries + new ExponentialSleepStrategy().run( + loop => { + val metadata = TopicMetadataCache.getTopicMetadata( + Set(ssp.getStream), + systemName, + getTopicMetadata, + metadataTTL) + debug("Got metadata for streams: %s" format metadata) + + val brokersToTopicPartitions = getTopicsAndPartitionsByBroker(metadata) + val topicAndPartition = new TopicAndPartition(ssp.getStream, ssp.getPartition.getPartitionId) + val broker = brokersToTopicPartitions.filter((e) => e._2.contains(topicAndPartition)).head._1 + + // Get oldest, newest, and upcoming offsets for each topic and partition. + debug("Fetching offset for %s:%s: %s" format (broker.host, broker.port, topicAndPartition)) + val consumer = new SimpleConsumer(broker.host, broker.port, timeout, bufferSize, clientId) + try { + offset = getOffsets(consumer, Set(topicAndPartition), OffsetRequest.LatestTime).head._2 + + // Kafka's "latest" offset is always last message in stream's offset + + // 1, so get newest message in stream by subtracting one. this is safe + // even for key-deduplicated streams, since the last message will + // never be deduplicated. + if (offset.toLong <= 0) { + debug("Stripping newest offsets for %s because the topic appears empty." format topicAndPartition) + offset = null + } else { + offset = (offset.toLong - 1).toString + } + } finally { + consumer.close + } + + debug("Got offset %s for %s." format(offset, ssp)) + loop.done + }, + + (exception, loop) => { + if (retries > 0) { + warn("Exception while trying to get offset for %s: %s. Retrying." format(ssp, exception)) + metadataTTL = 0L // Force metadata refresh + retries -= 1 + } else { + warn("Exception while trying to get offset for %s" format(ssp), exception) + loop.done + throw exception + } + }) + + offset + } + + /** + * Helper method to use topic metadata cache when fetching metadata, so we + * don't hammer Kafka more than we need to. + */ + def getTopicMetadata(topics: Set[String]) = { + new ClientUtilTopicMetadataStore(brokerListString, clientId, timeout) + .getTopicInfo(topics) + } + + /** + * Break topic metadata topic/partitions into per-broker map so that we can + * execute only one offset request per broker. + */ + private def getTopicsAndPartitionsByBroker(metadata: Map[String, TopicMetadata]) = { + val brokersToTopicPartitions = metadata + .values + // Convert the topic metadata to a Seq[(Broker, TopicAndPartition)] + .flatMap(topicMetadata => { + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) + topicMetadata + .partitionsMetadata + // Convert Seq[PartitionMetadata] to Seq[(Broker, TopicAndPartition)] + .map(partitionMetadata => { + val topicAndPartition = new TopicAndPartition(topicMetadata.topic, partitionMetadata.partitionId) + val leader = partitionMetadata + .leader + .getOrElse(throw new SamzaException("Need leaders for all partitions when fetching offsets. No leader available for TopicAndPartition: %s" format topicAndPartition)) + (leader, topicAndPartition) + }) + }) + + // Convert to a Map[Broker, Seq[(Broker, TopicAndPartition)]] + .groupBy(_._1) + // Convert to a Map[Broker, Set[TopicAndPartition]] + .mapValues(_.map(_._2).toSet) + + debug("Got topic partition data for brokers: %s" format brokersToTopicPartitions) + + brokersToTopicPartitions + } + + /** + * Use a SimpleConsumer to fetch either the earliest or latest offset from + * Kafka for each topic/partition in the topicsAndPartitions set. It is + * assumed that all topics/partitions supplied reside on the broker that the + * consumer is connected to. + */ + private def getOffsets(consumer: SimpleConsumer, topicsAndPartitions: Set[TopicAndPartition], earliestOrLatest: Long) = { + debug("Getting offsets for %s using earliest/latest value of %s." format (topicsAndPartitions, earliestOrLatest)) + + var offsets = Map[SystemStreamPartition, String]() + val partitionOffsetInfo = topicsAndPartitions + .map(topicAndPartition => (topicAndPartition, PartitionOffsetRequestInfo(earliestOrLatest, 1))) + .toMap + val brokerOffsets = consumer + .getOffsetsBefore(new OffsetRequest(partitionOffsetInfo)) + .partitionErrorAndOffsets + .mapValues(partitionErrorAndOffset => { + KafkaUtil.maybeThrowException(partitionErrorAndOffset.error.exception()) + partitionErrorAndOffset.offsets.head + }) + + for ((topicAndPartition, offset) <- brokerOffsets) { + offsets += new SystemStreamPartition(systemName, topicAndPartition.topic, new Partition(topicAndPartition.partition)) -> offset.toString + } + + debug("Got offsets for %s using earliest/latest value of %s: %s" format (topicsAndPartitions, earliestOrLatest, offsets)) + + offsets + } + + /** + * @inheritdoc + */ + override def createStream(spec: StreamSpec): Boolean = { + info("Create topic %s in system %s" format (spec.getPhysicalName, systemName)) + val kSpec = toKafkaSpec(spec) + var streamCreated = false + + new ExponentialSleepStrategy(initialDelayMs = 500).run( + loop => { + val zkClient = connectZk() + try { + AdminUtils.createTopic( + zkClient, + kSpec.getPhysicalName, + kSpec.getPartitionCount, + kSpec.getReplicationFactor, + kSpec.getProperties) + } finally { + zkClient.close + } + + streamCreated = true + loop.done + }, + + (exception, loop) => { + exception match { + case e: TopicExistsException => + streamCreated = false + loop.done + case e: Exception => + warn("Failed to create topic %s: %s. Retrying." format (spec.getPhysicalName, e)) + debug("Exception detail:", e) + } + }) + + streamCreated + } + + /** + * Converts a StreamSpec into a KafakStreamSpec. Special handling for coordinator and changelog stream. + * @param spec a StreamSpec object + * @return KafkaStreamSpec object + */ + def toKafkaSpec(spec: StreamSpec): KafkaStreamSpec = { + if (spec.isChangeLogStream) { + val topicName = spec.getPhysicalName + val topicMeta = topicMetaInformation.getOrElse(topicName, throw new StreamValidationException("Unable to find topic information for topic " + topicName)) + new KafkaStreamSpec(spec.getId, topicName, systemName, spec.getPartitionCount, topicMeta.replicationFactor, + topicMeta.kafkaProps) + } else if (spec.isCoordinatorStream){ + new KafkaStreamSpec(spec.getId, spec.getPhysicalName, systemName, 1, coordinatorStreamReplicationFactor, + coordinatorStreamProperties) + } else if (intermediateStreamProperties.contains(spec.getId)) { + KafkaStreamSpec.fromSpec(spec).copyWithProperties(intermediateStreamProperties(spec.getId)) + } else { + KafkaStreamSpec.fromSpec(spec) + } + } + + /** + * @inheritdoc + * + * Validates a stream in Kafka. Should not be called before createStream(), + * since ClientUtils.fetchTopicMetadata(), used by different Kafka clients, + * is not read-only and will auto-create a new topic. + */ + override def validateStream(spec: StreamSpec): Unit = { + val topicName = spec.getPhysicalName + info("Validating topic %s." format topicName) + + val retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy + var metadataTTL = Long.MaxValue // Trust the cache until we get an exception + retryBackoff.run( + loop => { + val metadataStore = new ClientUtilTopicMetadataStore(brokerListString, clientId, timeout) + val topicMetadataMap = TopicMetadataCache.getTopicMetadata(Set(topicName), systemName, metadataStore.getTopicInfo, metadataTTL) + val topicMetadata = topicMetadataMap(topicName) + KafkaUtil.maybeThrowException(topicMetadata.error.exception()) + + val partitionCount = topicMetadata.partitionsMetadata.length + if (partitionCount != spec.getPartitionCount) { + throw new StreamValidationException("Topic validation failed for topic %s because partition count %s did not match expected partition count of %d" format (topicName, topicMetadata.partitionsMetadata.length, spec.getPartitionCount)) + } + + info("Successfully validated topic %s." format topicName) + loop.done + }, + + (exception, loop) => { + exception match { + case e: StreamValidationException => throw e + case e: Exception => + warn("While trying to validate topic %s: %s. Retrying." format (topicName, e)) + debug("Exception detail:", e) + metadataTTL = 5000L // Revert to the default value + } + }) + } + + /** + * @inheritdoc + * + * Delete a stream in Kafka. Deleting topics works only when the broker is configured with "delete.topic.enable=true". + * Otherwise it's a no-op. + */ + override def clearStream(spec: StreamSpec): Boolean = { + info("Delete topic %s in system %s" format (spec.getPhysicalName, systemName)) + val kSpec = KafkaStreamSpec.fromSpec(spec) + var retries = CLEAR_STREAM_RETRIES + new ExponentialSleepStrategy().run( + loop => { + val zkClient = connectZk() + try { + AdminUtils.deleteTopic( + zkClient, + kSpec.getPhysicalName) + } finally { + zkClient.close + } + + loop.done + }, + + (exception, loop) => { + if (retries > 0) { + warn("Exception while trying to delete topic %s: %s. Retrying." format (spec.getPhysicalName, exception)) + retries -= 1 + } else { + warn("Fail to delete topic %s: %s" format (spec.getPhysicalName, exception)) + loop.done + throw exception + } + }) + + val topicMetadata = getTopicMetadata(Set(kSpec.getPhysicalName)).get(kSpec.getPhysicalName).get + topicMetadata.partitionsMetadata.isEmpty + } + + /** + * @inheritdoc + * + * Delete records up to (and including) the provided ssp offsets for all system stream partitions specified in the map + * This only works with Kafka cluster 0.11 or later. Otherwise it's a no-op. + */ + override def deleteMessages(offsets: util.Map[SystemStreamPartition, String]) { + if (!running) { + throw new SamzaException(s"KafkaSystemAdmin has not started yet for system $systemName") + } + if (deleteCommittedMessages) { + val nextOffsets = offsets.asScala.toSeq.map { case (systemStreamPartition, offset) => + (new TopicPartition(systemStreamPartition.getStream, systemStreamPartition.getPartition.getPartitionId), offset.toLong + 1) + }.toMap + adminClient.deleteRecordsBefore(nextOffsets) + deleteMessagesCalled = true + } + } + + /** + * Compare the two offsets. Returns x where x < 0 if offset1 < offset2; + * x == 0 if offset1 == offset2; x > 0 if offset1 > offset2. + * + * Currently it's used in the context of the broadcast streams to detect + * the mismatch between two streams when consuming the broadcast streams. + */ + override def offsetComparator(offset1: String, offset2: String): Integer = { + offset1.toLong compare offset2.toLong + } +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumer.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumer.scala new file mode 100644 index 0000000000000..b7c436839a28e --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumer.scala @@ -0,0 +1,309 @@ +/* + * 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.system.kafka_deprecated + +import kafka.common.TopicAndPartition +import org.apache.samza.util.Logging +import kafka.message.Message +import kafka.message.MessageAndOffset +import org.apache.samza.Partition +import org.apache.kafka.common.utils.Utils +import org.apache.samza.util.Clock +import kafka.serializer.DefaultDecoder +import kafka.serializer.Decoder +import org.apache.samza.util.BlockingEnvelopeMap +import org.apache.samza.system.SystemStreamPartition +import org.apache.samza.system.IncomingMessageEnvelope +import kafka.consumer.ConsumerConfig +import org.apache.samza.util.TopicMetadataStore +import kafka.api.PartitionMetadata +import kafka.api.TopicMetadata +import org.apache.samza.util.ExponentialSleepStrategy +import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConverters._ +import org.apache.samza.system.SystemAdmin + +object KafkaSystemConsumer { + + // Approximate additional shallow heap overhead per message in addition to the raw bytes + // received from Kafka 4 + 64 + 4 + 4 + 4 = 80 bytes overhead. + // As this overhead is a moving target, and not very large + // compared to the message size its being ignore in the computation for now. + val MESSAGE_SIZE_OVERHEAD = 4 + 64 + 4 + 4 + 4; + + def toTopicAndPartition(systemStreamPartition: SystemStreamPartition) = { + val topic = systemStreamPartition.getStream + val partitionId = systemStreamPartition.getPartition.getPartitionId + TopicAndPartition(topic, partitionId) + } +} + +/** + * Maintain a cache of BrokerProxies, returning the appropriate one for the + * requested topic and partition. + */ +private[kafka_deprecated] class KafkaSystemConsumer( + systemName: String, + systemAdmin: SystemAdmin, + metrics: KafkaSystemConsumerMetrics, + metadataStore: TopicMetadataStore, + clientId: String, + timeout: Int = ConsumerConfig.ConsumerTimeoutMs, + bufferSize: Int = ConsumerConfig.SocketBufferSize, + fetchSize: StreamFetchSizes = new StreamFetchSizes, + consumerMinSize: Int = ConsumerConfig.MinFetchBytes, + consumerMaxWait: Int = ConsumerConfig.MaxFetchWaitMs, + + /** + * Defines a low water mark for how many messages we buffer before we start + * executing fetch requests against brokers to get more messages. This value + * is divided equally among all registered SystemStreamPartitions. For + * example, if fetchThreshold is set to 50000, and there are 50 + * SystemStreamPartitions registered, then the per-partition threshold is + * 1000. As soon as a SystemStreamPartition's buffered message count drops + * below 1000, a fetch request will be executed to get more data for it. + * + * Increasing this parameter will decrease the latency between when a queue + * is drained of messages and when new messages are enqueued, but also leads + * to an increase in memory usage since more messages will be held in memory. + */ + fetchThreshold: Int = 50000, + /** + * Defines a low water mark for how many bytes we buffer before we start + * executing fetch requests against brokers to get more messages. This + * value is divided by 2 because the messages are buffered twice, once in + * KafkaConsumer and then in SystemConsumers. This value + * is divided equally among all registered SystemStreamPartitions. + * However this is a soft limit per partition, as the + * bytes are cached at the message boundaries, and the actual usage can be + * 1000 bytes + size of max message in the partition for a given stream. + * The bytes if the size of the bytebuffer in Message. Hence, the + * Object overhead is not taken into consideration. In this codebase + * it seems to be quite small. Hence, even for 500000 messages this is around 4MB x 2 = 8MB, + * which is not considerable. + * + * For example, + * if fetchThresholdBytes is set to 100000 bytes, and there are 50 + * SystemStreamPartitions registered, then the per-partition threshold is + * (100000 / 2) / 50 = 1000 bytes. + * As this is a soft limit, the actual usage can be 1000 bytes + size of max message. + * As soon as a SystemStreamPartition's buffered messages bytes drops + * below 1000, a fetch request will be executed to get more data for it. + * + * Increasing this parameter will decrease the latency between when a queue + * is drained of messages and when new messages are enqueued, but also leads + * to an increase in memory usage since more messages will be held in memory. + * + * The default value is -1, which means this is not used. When the value + * is > 0, then the fetchThreshold which is count based is ignored. + */ + fetchThresholdBytes: Long = -1, + /** + * if(fetchThresholdBytes > 0) true else false + */ + fetchLimitByBytesEnabled: Boolean = false, + offsetGetter: GetOffset = new GetOffset("fail"), + deserializer: Decoder[Object] = new DefaultDecoder().asInstanceOf[Decoder[Object]], + keyDeserializer: Decoder[Object] = new DefaultDecoder().asInstanceOf[Decoder[Object]], + retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy, + clock: () => Long = { System.currentTimeMillis }) extends BlockingEnvelopeMap( + metrics.registry, + new Clock { + def currentTimeMillis = clock() + }, + classOf[KafkaSystemConsumerMetrics].getName) with Toss with Logging { + + type HostPort = (String, Int) + val brokerProxies = scala.collection.mutable.Map[HostPort, BrokerProxy]() + val topicPartitionsAndOffsets: scala.collection.concurrent.Map[TopicAndPartition, String] = new ConcurrentHashMap[TopicAndPartition, String]().asScala + var perPartitionFetchThreshold = fetchThreshold + var perPartitionFetchThresholdBytes = 0L + + def start() { + if (topicPartitionsAndOffsets.size > 0) { + perPartitionFetchThreshold = fetchThreshold / topicPartitionsAndOffsets.size + // messages get double buffered, hence divide by 2 + if(fetchLimitByBytesEnabled) { + perPartitionFetchThresholdBytes = (fetchThresholdBytes / 2) / topicPartitionsAndOffsets.size + } + } + + systemAdmin.start() + refreshBrokers + } + + override def register(systemStreamPartition: SystemStreamPartition, offset: String) { + super.register(systemStreamPartition, offset) + + val topicAndPartition = KafkaSystemConsumer.toTopicAndPartition(systemStreamPartition) + val existingOffset = topicPartitionsAndOffsets.getOrElseUpdate(topicAndPartition, offset) + // register the older offset in the consumer + if (systemAdmin.offsetComparator(existingOffset, offset) >= 0) { + topicPartitionsAndOffsets.replace(topicAndPartition, offset) + } + + metrics.registerTopicAndPartition(KafkaSystemConsumer.toTopicAndPartition(systemStreamPartition)) + } + + def stop() { + systemAdmin.stop() + brokerProxies.values.foreach(_.stop) + } + + protected def createBrokerProxy(host: String, port: Int): BrokerProxy = { + info("Creating new broker proxy for host: %s and port: %s" format(host, port)) + new BrokerProxy(host, port, systemName, clientId, metrics, sink, timeout, bufferSize, fetchSize, consumerMinSize, consumerMaxWait, offsetGetter) + } + + protected def getPartitionMetadata(topicMetadata: TopicMetadata, partition: Int): Option[PartitionMetadata] = { + topicMetadata.partitionsMetadata.find(_.partitionId == partition) + } + + protected def getLeaderHostPort(partitionMetadata: Option[PartitionMetadata]): Option[(String, Int)] = { + // Whatever we do, we can't say Broker, even though we're + // manipulating it here. Broker is a private type and Scala doesn't seem + // to care about that as long as you don't explicitly declare its type. + val brokerOption = partitionMetadata.flatMap(_.leader) + + brokerOption match { + case Some(broker) => Some(broker.host, broker.port) + case _ => None + } + } + + def refreshBrokers { + var tpToRefresh = topicPartitionsAndOffsets.keySet.toList + info("Refreshing brokers for: %s" format topicPartitionsAndOffsets) + retryBackoff.run( + loop => { + val topics = tpToRefresh.map(_.topic).toSet + val topicMetadata = TopicMetadataCache.getTopicMetadata(topics, systemName, (topics: Set[String]) => metadataStore.getTopicInfo(topics)) + + // addTopicPartition one at a time, leaving the to-be-done list intact in case of exceptions. + // This avoids trying to re-add the same topic partition repeatedly + def refresh() = { + val head = tpToRefresh.head + // refreshBrokers can be called from abdicate and refreshDropped, + // both of which are triggered from BrokerProxy threads. To prevent + // accidentally creating multiple objects for the same broker, or + // accidentally not updating the topicPartitionsAndOffsets variable, + // we need to lock. + this.synchronized { + // Check if we still need this TopicAndPartition inside the + // critical section. If we don't, then notAValidEvent it. + topicPartitionsAndOffsets.get(head) match { + case Some(nextOffset) => + val partitionMetadata = getPartitionMetadata(topicMetadata(head.topic), head.partition) + getLeaderHostPort(partitionMetadata) match { + case Some((host, port)) => + debug("Got partition metadata for %s: %s" format(head, partitionMetadata.get)) + val brokerProxy = brokerProxies.getOrElseUpdate((host, port), createBrokerProxy(host, port)) + brokerProxy.addTopicPartition(head, Option(nextOffset)) + brokerProxy.start + debug("Claimed topic-partition (%s) for (%s)".format(head, brokerProxy)) + topicPartitionsAndOffsets -= head + case None => info("No metadata available for: %s. Will try to refresh and add to a consumer thread later." format head) + } + case _ => debug("Ignoring refresh for %s because we already added it from another thread." format head) + } + } + tpToRefresh.tail + } + + while (!tpToRefresh.isEmpty) { + tpToRefresh = refresh() + } + + loop.done + }, + + (exception, loop) => { + warn("While refreshing brokers for %s: %s. Retrying." format (tpToRefresh.head, exception)) + debug("Exception detail:", exception) + }) + } + + val sink = new MessageSink { + var lastDroppedRefresh = clock() + + def refreshDropped() { + if (topicPartitionsAndOffsets.size > 0 && clock() - lastDroppedRefresh > 10000) { + refreshBrokers + lastDroppedRefresh = clock() + } + } + + def setIsAtHighWatermark(tp: TopicAndPartition, isAtHighWatermark: Boolean) { + setIsAtHead(toSystemStreamPartition(tp), isAtHighWatermark) + } + + def needsMoreMessages(tp: TopicAndPartition) = { + if(fetchLimitByBytesEnabled) { + getMessagesSizeInQueue(toSystemStreamPartition(tp)) < perPartitionFetchThresholdBytes + } else { + getNumMessagesInQueue(toSystemStreamPartition(tp)) < perPartitionFetchThreshold + } + } + + def getMessageSize(message: Message): Integer = { + message.size + KafkaSystemConsumer.MESSAGE_SIZE_OVERHEAD + } + + def addMessage(tp: TopicAndPartition, msg: MessageAndOffset, highWatermark: Long) = { + trace("Incoming message %s: %s." format (tp, msg)) + + val systemStreamPartition = toSystemStreamPartition(tp) + val isAtHead = highWatermark == msg.offset + val offset = msg.offset.toString + val key = if (msg.message.key != null) { + keyDeserializer.fromBytes(Utils.readBytes(msg.message.key)) + } else { + null + } + val message = if (!msg.message.isNull) { + deserializer.fromBytes(Utils.readBytes(msg.message.payload)) + } else { + null + } + + if(fetchLimitByBytesEnabled ) { + val ime = new IncomingMessageEnvelope(systemStreamPartition, offset, key, message, getMessageSize(msg.message)) + ime.setTimestamp(if (!msg.message.isNull) msg.message.timestamp else 0L) + put(systemStreamPartition, ime) + } else { + val ime = new IncomingMessageEnvelope(systemStreamPartition, offset, key, message) + ime.setTimestamp(if (!msg.message.isNull) msg.message.timestamp else 0L) + put(systemStreamPartition, ime) + } + + setIsAtHead(systemStreamPartition, isAtHead) + } + + def abdicate(tp: TopicAndPartition, nextOffset: Long) { + info("Abdicating for %s" format (tp)) + topicPartitionsAndOffsets += tp -> nextOffset.toString + refreshBrokers + } + + private def toSystemStreamPartition(tp: TopicAndPartition) = { + new SystemStreamPartition(systemName, tp.topic, new Partition(tp.partition)) + } + } +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumerMetrics.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumerMetrics.scala new file mode 100644 index 0000000000000..08b1a0c4dc83e --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemConsumerMetrics.scala @@ -0,0 +1,100 @@ +/* + * 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.system.kafka_deprecated + +import org.apache.samza.metrics.MetricsHelper +import org.apache.samza.metrics.MetricsRegistryMap +import org.apache.samza.metrics.MetricsRegistry +import java.util.concurrent.ConcurrentHashMap +import kafka.common.TopicAndPartition +import org.apache.samza.metrics.Counter +import org.apache.samza.metrics.Gauge + +class KafkaSystemConsumerMetrics(val systemName: String = "unknown", val registry: MetricsRegistry = new MetricsRegistryMap) extends MetricsHelper { + val offsets = new ConcurrentHashMap[TopicAndPartition, Counter] + val bytesRead = new ConcurrentHashMap[TopicAndPartition, Counter] + val reads = new ConcurrentHashMap[TopicAndPartition, Counter] + val lag = new ConcurrentHashMap[TopicAndPartition, Gauge[Long]] + val highWatermark = new ConcurrentHashMap[TopicAndPartition, Gauge[Long]] + + /* + * (String, Int) = (host, port) of BrokerProxy. + */ + + val reconnects = new ConcurrentHashMap[(String, Int), Counter] + val brokerBytesRead = new ConcurrentHashMap[(String, Int), Counter] + val brokerReads = new ConcurrentHashMap[(String, Int), Counter] + val brokerSkippedFetchRequests = new ConcurrentHashMap[(String, Int), Counter] + val topicPartitions = new ConcurrentHashMap[(String, Int), Gauge[Int]] + + def registerTopicAndPartition(tp: TopicAndPartition) = { + if (!offsets.contains(tp)) { + offsets.put(tp, newCounter("%s-%s-offset-change" format (tp.topic, tp.partition))) + bytesRead.put(tp, newCounter("%s-%s-bytes-read" format (tp.topic, tp.partition))) + reads.put(tp, newCounter("%s-%s-messages-read" format (tp.topic, tp.partition))) + highWatermark.put(tp, newGauge("%s-%s-high-watermark" format (tp.topic, tp.partition), -1L)) + lag.put(tp, newGauge("%s-%s-messages-behind-high-watermark" format (tp.topic, tp.partition), 0L)) + } + } + + def registerBrokerProxy(host: String, port: Int) { + reconnects.put((host, port), newCounter("%s-%s-reconnects" format (host, port))) + brokerBytesRead.put((host, port), newCounter("%s-%s-bytes-read" format (host, port))) + brokerReads.put((host, port), newCounter("%s-%s-messages-read" format (host, port))) + brokerSkippedFetchRequests.put((host, port), newCounter("%s-%s-skipped-fetch-requests" format (host, port))) + topicPartitions.put((host, port), newGauge("%s-%s-topic-partitions" format (host, port), 0)) + } + + // java friendlier interfaces + // Gauges + def setTopicPartitionValue(host: String, port: Int, value: Int) { + topicPartitions.get((host,port)).set(value) + } + def setLagValue(topicAndPartition: TopicAndPartition, value: Long) { + lag.get((topicAndPartition)).set(value); + } + def setHighWatermarkValue(topicAndPartition: TopicAndPartition, value: Long) { + highWatermark.get((topicAndPartition)).set(value); + } + + // Counters + def incBrokerReads(host: String, port: Int) { + brokerReads.get((host,port)).inc + } + def incReads(topicAndPartition: TopicAndPartition) { + reads.get(topicAndPartition).inc; + } + def incBytesReads(topicAndPartition: TopicAndPartition, inc: Long) { + bytesRead.get(topicAndPartition).inc(inc); + } + def incBrokerBytesReads(host: String, port: Int, incBytes: Long) { + brokerBytesRead.get((host,port)).inc(incBytes) + } + def incBrokerSkippedFetchRequests(host: String, port: Int) { + brokerSkippedFetchRequests.get((host,port)).inc() + } + def setOffsets(topicAndPartition: TopicAndPartition, offset: Long) { + offsets.get(topicAndPartition).set(offset) + } + def incReconnects(host: String, port: Int) { + reconnects.get((host,port)).inc() + } + override def getPrefix = systemName + "-" +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemFactory.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemFactory.scala new file mode 100644 index 0000000000000..eecdbe4c5ed7f --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemFactory.scala @@ -0,0 +1,178 @@ +/* + * 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.system.kafka_deprecated +import java.util.Properties + +import kafka.utils.ZkUtils +import org.apache.samza.SamzaException +import org.apache.samza.config.ApplicationConfig.ApplicationMode +import org.apache.samza.util.{ClientUtilTopicMetadataStore, ExponentialSleepStrategy, Logging} +import org.apache.samza.config._ +import org.apache.samza.metrics.MetricsRegistry +import org.apache.samza.config.KafkaConfig.Config2Kafka +import org.apache.samza.config.TaskConfig.Config2Task +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.samza.system.SystemFactory +import org.apache.samza.config.StorageConfig._ +import org.apache.samza.system.SystemProducer +import org.apache.samza.system.SystemAdmin +import org.apache.samza.config.SystemConfig.Config2System +import org.apache.samza.system.SystemConsumer + +object KafkaSystemFactory extends Logging { + def getInjectedProducerProperties(systemName: String, config: Config) = if (config.isChangelogSystem(systemName)) { + warn("System name '%s' is being used as a changelog. Disabling compression since Kafka does not support compression for log compacted topics." format systemName) + Map[String, String]("compression.type" -> "none") + } else { + Map[String, String]() + } +} + +class KafkaSystemFactory extends SystemFactory with Logging { + def getConsumer(systemName: String, config: Config, registry: MetricsRegistry): SystemConsumer = { + val clientId = getClientId("samza-consumer", config) + val metrics = new KafkaSystemConsumerMetrics(systemName, registry) + + // Kind of goofy to need a producer config for consumers, but we need metadata. + val producerConfig = config.getKafkaSystemProducerConfig(systemName, clientId) + val bootstrapServers = producerConfig.bootsrapServers + val consumerConfig = config.getKafkaSystemConsumerConfig(systemName, clientId) + + val timeout = consumerConfig.socketTimeoutMs + val bufferSize = consumerConfig.socketReceiveBufferBytes + val fetchSize = new StreamFetchSizes(consumerConfig.fetchMessageMaxBytes, config.getFetchMessageMaxBytesTopics(systemName)) + val consumerMinSize = consumerConfig.fetchMinBytes + val consumerMaxWait = consumerConfig.fetchWaitMaxMs + val autoOffsetResetDefault = consumerConfig.autoOffsetReset + val autoOffsetResetTopics = config.getAutoOffsetResetTopics(systemName) + val fetchThreshold = config.getConsumerFetchThreshold(systemName).getOrElse("50000").toInt + val fetchThresholdBytes = config.getConsumerFetchThresholdBytes(systemName).getOrElse("-1").toLong + val offsetGetter = new GetOffset(autoOffsetResetDefault, autoOffsetResetTopics) + val metadataStore = new ClientUtilTopicMetadataStore(bootstrapServers, clientId, timeout) + + new KafkaSystemConsumer( + systemName = systemName, + systemAdmin = getAdmin(systemName, config), + metrics = metrics, + metadataStore = metadataStore, + clientId = clientId, + timeout = timeout, + bufferSize = bufferSize, + fetchSize = fetchSize, + consumerMinSize = consumerMinSize, + consumerMaxWait = consumerMaxWait, + fetchThreshold = fetchThreshold, + fetchThresholdBytes = fetchThresholdBytes, + fetchLimitByBytesEnabled = config.isConsumerFetchThresholdBytesEnabled(systemName), + offsetGetter = offsetGetter) + } + + def getProducer(systemName: String, config: Config, registry: MetricsRegistry): SystemProducer = { + val clientId = getClientId("samza-producer", config) + val injectedProps = KafkaSystemFactory.getInjectedProducerProperties(systemName, config) + val producerConfig = config.getKafkaSystemProducerConfig(systemName, clientId, injectedProps) + val getProducer = () => { new KafkaProducer[Array[Byte], Array[Byte]](producerConfig.getProducerProperties) } + val metrics = new KafkaSystemProducerMetrics(systemName, registry) + + // Unlike consumer, no need to use encoders here, since they come for free + // inside the producer configs. Kafka's producer will handle all of this + // for us. + + new KafkaSystemProducer( + systemName, + new ExponentialSleepStrategy(initialDelayMs = producerConfig.reconnectIntervalMs), + getProducer, + metrics, + dropProducerExceptions = config.getDropProducerError) + } + + def getAdmin(systemName: String, config: Config): SystemAdmin = { + val clientId = getClientId("samza-admin", config) + val producerConfig = config.getKafkaSystemProducerConfig(systemName, clientId) + val bootstrapServers = producerConfig.bootsrapServers + val consumerConfig = config.getKafkaSystemConsumerConfig(systemName, clientId) + val timeout = consumerConfig.socketTimeoutMs + val bufferSize = consumerConfig.socketReceiveBufferBytes + val zkConnect = Option(consumerConfig.zkConnect) + .getOrElse(throw new SamzaException("no zookeeper.connect defined in config")) + val connectZk = () => { + ZkUtils(zkConnect, 6000, 6000, false) + } + val coordinatorStreamProperties = getCoordinatorTopicProperties(config) + val coordinatorStreamReplicationFactor = config.getCoordinatorReplicationFactor.toInt + val storeToChangelog = config.getKafkaChangelogEnabledStores() + // Construct the meta information for each topic, if the replication factor is not defined, we use 2 as the number of replicas for the change log stream. + val topicMetaInformation = storeToChangelog.map{case (storeName, topicName) => + { + val replicationFactor = config.getChangelogStreamReplicationFactor(storeName).toInt + val changelogInfo = ChangelogInfo(replicationFactor, config.getChangelogKafkaProperties(storeName)) + info("Creating topic meta information for topic: %s with replication factor: %s" format (topicName, replicationFactor)) + (topicName, changelogInfo) + }} + + val deleteCommittedMessages = config.deleteCommittedMessages(systemName) + val intermediateStreamProperties: Map[String, Properties] = getIntermediateStreamProperties(config) + new KafkaSystemAdmin( + systemName, + bootstrapServers, + connectZk, + coordinatorStreamProperties, + coordinatorStreamReplicationFactor, + timeout, + bufferSize, + clientId, + topicMetaInformation, + intermediateStreamProperties, + deleteCommittedMessages) + } + + def getCoordinatorTopicProperties(config: Config) = { + val segmentBytes = config.getCoordinatorSegmentBytes + (new Properties /: Map( + "cleanup.policy" -> "compact", + "segment.bytes" -> segmentBytes)) { case (props, (k, v)) => props.put(k, v); props } + } + + def getIntermediateStreamProperties(config : Config): Map[String, Properties] = { + val appConfig = new ApplicationConfig(config) + if (appConfig.getAppMode == ApplicationMode.BATCH) { + val streamConfig = new StreamConfig(config) + streamConfig.getStreamIds().filter(streamConfig.getIsIntermediateStream(_)).map(streamId => { + val properties = new Properties() + properties.putAll(streamConfig.getStreamProperties(streamId)) + properties.putIfAbsent("retention.ms", String.valueOf(KafkaConfig.DEFAULT_RETENTION_MS_FOR_BATCH)) + (streamId, properties) + }).toMap + } else { + Map() + } + } + def getClientId(id: String, config: Config): String = getClientId( + id, + new JobConfig(config).getName.getOrElse(throw new ConfigException("Missing job name.")), + new JobConfig(config)getJobId) + + def getClientId(id: String, jobName: String, jobId: String): String = + "%s-%s-%s" format + (id.replaceAll("[^A-Za-z0-9]", "_"), + jobName.replaceAll("[^A-Za-z0-9]", "_"), + jobId.replaceAll("[^A-Za-z0-9]", "_")) + +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducer.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducer.scala new file mode 100644 index 0000000000000..af393785e812a --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducer.scala @@ -0,0 +1,235 @@ +/* + * 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.system.kafka_deprecated + + +import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.TimeUnit + +import org.apache.kafka.clients.producer.Callback +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.samza.system.OutgoingMessageEnvelope +import org.apache.samza.system.SystemProducer +import org.apache.samza.system.SystemProducerException +import org.apache.samza.util.ExponentialSleepStrategy +import org.apache.samza.util.KafkaUtil +import org.apache.samza.util.Logging +import org.apache.samza.util.TimerUtil + +class KafkaSystemProducer(systemName: String, + retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy, + getProducer: () => Producer[Array[Byte], Array[Byte]], + metrics: KafkaSystemProducerMetrics, + val clock: () => Long = () => System.nanoTime, + val dropProducerExceptions: Boolean = false) extends SystemProducer with Logging with TimerUtil { + + // Represents a fatal error that caused the producer to close. + val fatalException: AtomicReference[SystemProducerException] = new AtomicReference[SystemProducerException]() + 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 = { + producerRef.set(getProducer()) + } + + def stop() { + info("Stopping producer for system: " + this.systemName) + + 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) + } + } + + def register(source: String) { + } + + def send(source: String, envelope: OutgoingMessageEnvelope) { + trace("Enqueuing message: %s, %s." format (source, envelope)) + + val topicName = envelope.getSystemStream.getStream + if (topicName == null || topicName.isEmpty) { + throw new IllegalArgumentException("Invalid system stream: " + envelope.getSystemStream) + } + + val globalProducerException = fatalException.get() + if (globalProducerException != null) { + metrics.sendFailed.inc + throw new SystemProducerException("Producer was unable to recover from previous exception.", globalProducerException) + } + + 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 + val partitions: java.util.List[PartitionInfo] = currentProducer.partitionsFor(topicName) + val partitionKey = if (envelope.getPartitionKey != null) KafkaUtil.getIntegerPartitionKey(envelope, partitions) else null + val record = new ProducerRecord(envelope.getSystemStream.getStream, + partitionKey, + envelope.getKey.asInstanceOf[Array[Byte]], + envelope.getMessage.asInstanceOf[Array[Byte]]) + + try { + currentProducer.send(record, new Callback { + def onCompletion(metadata: RecordMetadata, exception: Exception): Unit = { + if (exception == null) { + metrics.sendSuccess.inc + } else { + val producerException = new SystemProducerException("Failed to send message for Source: %s on System:%s Topic:%s Partition:%s" + .format(source, systemName, topicName, partitionKey), exception) + + handleFatalSendException(currentProducer, producerException) + } + } + }) + metrics.sends.inc + } catch { + case originalException : Exception => + val producerException = new SystemProducerException("Failed to send message for Source: %s on System:%s Topic:%s Partition:%s" + .format(source, systemName, topicName, partitionKey), 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 = 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.") + } + + // Flush only throws InterruptedException, all other errors are handled in send() callbacks + currentProducer.flush() + + // Invariant: At this point either + // 1. The producer is fine and there are no exceptions to handle OR + // 2. The producer is closed and one or more sources have exceptions to handle + // 2a. All new sends get a ProducerClosedException or IllegalStateException (depending on kafka version) + // 2b. There are no messages in flight because the producer is closed + + // We must check for an exception AFTER flush() because when flush() returns all callbacks for messages sent + // in that flush() are guaranteed to have completed and we update the exception in the callback. + // If there is an exception, we rethrow it here to prevent the checkpoint. + val exception = fatalException.get() + if (exception != null) { + metrics.flushFailed.inc + throw new SystemProducerException("Flush failed. One or more batches of messages were not sent!", exception) + } + trace("Flushed %s." format source) + } + } + + + /** + * 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 + // be handled by the Task before we recreate the producer, and if it isn't handled, a concurrent send() from another + // Task could send on the new producer before the first Task properly handled the exception and produce out of order messages. + // So we have to handle it right here in the SystemProducer. + if (dropProducerExceptions) { + warn("Ignoring producer exception. All messages in the failed producer request will be dropped!") + + // 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) + } + } + } + + /** + * @return the current producer. Never returns null. + */ + 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 + } +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducerMetrics.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducerMetrics.scala new file mode 100644 index 0000000000000..e3ff90ab0948a --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/KafkaSystemProducerMetrics.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.kafka_deprecated + +import org.apache.samza.metrics.{MetricsHelper, MetricsRegistry, MetricsRegistryMap} + +class KafkaSystemProducerMetrics(val systemName: String = "unknown", val registry: MetricsRegistry = new MetricsRegistryMap) extends MetricsHelper { + /* Tracks the number of calls made to send in KafkaSystemProducer */ + val sends = newCounter("producer-sends") + /* Tracks the number of calls made to flush in KafkaSystemProducer */ + val flushes = newCounter("flushes") + /* Tracks how long the flush call takes to complete */ + val flushNs = newTimer("flush-ns") + /* Tracks the number of times the system producer retries a send request (due to RetriableException) */ + @Deprecated + val retries = newCounter("producer-retries") + /* Tracks the number of times flush operation failed */ + val flushFailed = newCounter("flush-failed") + /* Tracks the number of send requests that was failed by the KafkaProducer (due to unrecoverable errors) */ + val sendFailed = newCounter("producer-send-failed") + /* Tracks the number of send requests that was successfully completed by the KafkaProducer */ + val sendSuccess = newCounter("producer-send-success") + + override def getPrefix = systemName + "-" +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/MessageSink.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/MessageSink.scala new file mode 100644 index 0000000000000..d63f805450a13 --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/MessageSink.scala @@ -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.system.kafka_deprecated + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndOffset + +private[kafka_deprecated] trait MessageSink { + def setIsAtHighWatermark(tp: TopicAndPartition, isAtHighWatermark: Boolean): Unit + + def addMessage(tp: TopicAndPartition, msg: MessageAndOffset, highWatermark: Long): Unit + + def abdicate(tp: TopicAndPartition, nextOffset: Long): Unit + + def refreshDropped(): Unit + + def needsMoreMessages(tp: TopicAndPartition): Boolean +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/TopicMetadataCache.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/TopicMetadataCache.scala new file mode 100644 index 0000000000000..fbfd1c664196f --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/TopicMetadataCache.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.kafka_deprecated + +import org.apache.samza.util.Logging +import kafka.api.TopicMetadata +import org.apache.samza.util.KafkaUtil + +/** + * TopicMetadataCache is used to cache all the topic metadata for Kafka per + * (system, topic) partition. The cache access is thread safe. Each entry in + * the cache is refreshed after a specified interval. The cache uses the passed + * in getTopicInfoFromStore that retrieves the topic metadata from the store (usually zookeeper). + */ +object TopicMetadataCache extends Logging { + private case class MetadataInfo(var streamMetadata: TopicMetadata, var lastRefreshMs: Long) + private val topicMetadataMap: scala.collection.mutable.Map[(String, String), MetadataInfo] = new scala.collection.mutable.HashMap[(String, String), MetadataInfo] + private val lock = new Object + + // used to fetch the topic metadata from the store. Accepts a topic and system + type FetchTopicMetadataType = (Set[String]) => Map[String, TopicMetadata] + + def getTopicMetadata(topics: Set[String], systemName: String, getTopicInfoFromStore: FetchTopicMetadataType, cacheTimeout: Long = 5000L, getTime: () => Long = { System.currentTimeMillis }): Map[String, TopicMetadata] = { + lock synchronized { + val time = getTime() + val missingTopics = topics.filter(topic => !topicMetadataMap.contains(systemName, topic)) + val topicsWithBadOrExpiredMetadata = (topics -- missingTopics).filter(topic => { + val metadata = topicMetadataMap(systemName, topic) + hasBadErrorCode(metadata.streamMetadata) || ((time - metadata.lastRefreshMs) > cacheTimeout) + }) + val topicsToRefresh = missingTopics ++ topicsWithBadOrExpiredMetadata + + if (topicsToRefresh.size > 0) { + // Refresh topic information for any missing, expired, or bad topic metadata. + topicMetadataMap ++= getTopicInfoFromStore(missingTopics ++ topicsWithBadOrExpiredMetadata) + .map { case (topic, metadata) => ((systemName, topic), MetadataInfo(metadata, getTime())) } + .toMap + } + + // Use our new updated cache to return a map of topic -> metadata + topicMetadataMap + .filterKeys(topics.map(topic => (systemName, topic))) + .map { + case ((systemName, topic), metadata) => + (topic, metadata.streamMetadata) + }.toMap + } + } + + def clear { + topicMetadataMap.clear + } + + /** + * Helper method to check if a topic's metadata has a bad errorCode, or if a + * partition's metadata has a bad errorCode. + */ + def hasBadErrorCode(streamMetadata: TopicMetadata) = { + 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/system/kafka_deprecated/Toss.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/Toss.scala new file mode 100644 index 0000000000000..2c3ec0fab42c2 --- /dev/null +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka_deprecated/Toss.scala @@ -0,0 +1,28 @@ +/* + * + * 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.system.kafka_deprecated + +import org.apache.samza.SamzaException + +private[kafka_deprecated] trait Toss { + def toss(s:String) = throw new SamzaException(s) +} From 0c2076cf8504362ce8816728e3384c516ed6cfbb Mon Sep 17 00:00:00 2001 From: Cameron Lee Date: Thu, 18 Oct 2018 10:44:41 -0700 Subject: [PATCH 3/3] Minor: fixing javadocs in StreamProcessor Author: Cameron Lee Reviewers: Shanthoosh Venkatraman Closes #734 from cameronlee314/doc_fixes --- .../apache/samza/processor/StreamProcessor.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java index 314998949fae8..7328bc740786d 100644 --- a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java +++ b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java @@ -164,6 +164,11 @@ public State getState() { * Same as {@link #StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener, JobCoordinator)}, except * it creates a {@link JobCoordinator} instead of accepting it as an argument. * + * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer} + * @param customMetricsReporters registered with the metrics system to report metrics + * @param taskFactory task factory to instantiate the Task + * @param processorListener listener to the StreamProcessor life cycle + * * Deprecated: Use {@link #StreamProcessor(Config, Map, TaskFactory, Optional, Optional, * StreamProcessorLifecycleListenerFactory, JobCoordinator)} instead. */ @@ -181,10 +186,15 @@ public StreamProcessor(Config config, Map customMetrics *

  • Accepts a {@link ProcessorLifecycleListener} directly instead of a * {@link StreamProcessorLifecycleListenerFactory}
  • * - * Deprecated: Use {@link #StreamProcessor(Config, Map, TaskFactory, Optional, Optional, - * StreamProcessorLifecycleListenerFactory, JobCoordinator)} instead. * + * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer} + * @param customMetricsReporters registered with the metrics system to report metrics + * @param taskFactory task factory to instantiate the Task * @param processorListener listener to the StreamProcessor life cycle + * @param jobCoordinator the instance of {@link JobCoordinator} + * + * Deprecated: Use {@link #StreamProcessor(Config, Map, TaskFactory, Optional, Optional, + * StreamProcessorLifecycleListenerFactory, JobCoordinator)} instead. */ @Deprecated public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory,