From 8e6fc2dac5af11f0748fdcd32175c2ae95418fc7 Mon Sep 17 00:00:00 2001 From: prateekm Date: Sat, 9 Jun 2018 15:01:10 -0700 Subject: [PATCH 01/38] Remove all usages of StreamSpec and ApplicationRunner from the operator spec and impl layers. --- .../operators/functions/MapFunction.java | 2 +- .../samza/runtime/ApplicationRunner.java | 22 -- .../org/apache/samza/system/StreamSpec.java | 56 +-- .../apache/samza/config/TaskConfigJava.java | 3 +- .../samza/execution/ExecutionPlanner.java | 22 +- .../org/apache/samza/execution/JobGraph.java | 18 +- .../execution/JobGraphJsonGenerator.java | 12 +- .../org/apache/samza/execution/JobNode.java | 13 +- .../apache/samza/execution/StreamEdge.java | 34 +- .../apache/samza/execution/StreamManager.java | 3 +- .../samza/operators/OperatorSpecGraph.java | 15 +- .../samza/operators/StreamGraphSpec.java | 79 ++-- .../operators/impl/BroadcastOperatorImpl.java | 4 +- .../operators/impl/OperatorImplGraph.java | 161 +++++---- .../operators/impl/OutputOperatorImpl.java | 4 +- .../impl/PartitionByOperatorImpl.java | 11 +- .../operators/spec/InputOperatorSpec.java | 12 +- .../samza/operators/spec/OperatorSpecs.java | 7 +- .../operators/spec/OutputStreamImpl.java | 18 +- .../stream/IntermediateMessageStreamImpl.java | 7 +- .../runtime/AbstractApplicationRunner.java | 64 +--- .../samza/storage/ChangelogStreamManager.java | 4 +- .../apache/samza/storage/StorageRecovery.java | 3 +- .../org/apache/samza/util/StreamUtil.java | 87 +++++ .../apache/samza/config/StorageConfig.scala | 5 +- .../org/apache/samza/config/TaskConfig.scala | 4 +- .../samza/container/SamzaContainer.scala | 2 +- .../samza/job/local/ThreadJobFactory.scala | 4 +- .../MetricsSnapshotReporterFactory.scala | 5 +- .../scala/org/apache/samza/util/Util.scala | 20 -- .../samza/execution/TestExecutionPlanner.java | 48 ++- .../execution/TestJobGraphJsonGenerator.java | 54 +-- .../apache/samza/execution/TestJobNode.java | 17 +- .../samza/execution/TestStreamEdge.java | 14 +- .../samza/operators/TestJoinOperator.java | 11 +- .../operators/TestOperatorSpecGraph.java | 23 +- .../samza/operators/TestStreamGraphSpec.java | 336 +++++++----------- .../operators/impl/TestOperatorImplGraph.java | 294 +++++---------- .../impl/TestOperatorImplGraphUtil.java | 226 ++++++++++++ .../operators/impl/TestWindowOperator.java | 16 +- .../operators/spec/OperatorSpecTestUtils.java | 14 +- .../operators/spec/TestOperatorSpec.java | 18 +- .../spec/TestPartitionByOperatorSpec.java | 14 +- .../samza/task/TestTaskFactoryUtil.java | 5 +- .../TestStreamUtil.java} | 186 ++++------ .../samza/system/kafka/KafkaStreamSpec.java | 15 +- .../org/apache/samza/config/KafkaConfig.scala | 4 +- .../samza/config/RegExTopicGenerator.scala | 8 +- .../samza/system/kafka/KafkaSystemAdmin.scala | 4 +- .../system/kafka/TestKafkaStreamSpec.java | 6 +- .../kafka/TestKafkaCheckpointManager.scala | 4 +- .../sql/translator/TestQueryTranslator.java | 235 +++++++----- .../test/integration/NegateNumberTask.java | 4 +- .../performance/TestPerformanceTask.scala | 4 +- .../operator/RepartitionJoinWindowApp.java | 13 +- .../TestRepartitionJoinWindowApp.java | 7 +- 56 files changed, 1105 insertions(+), 1176 deletions(-) create mode 100644 samza-core/src/main/java/org/apache/samza/util/StreamUtil.java create mode 100644 samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java rename samza-core/src/test/java/org/apache/samza/{runtime/TestAbstractApplicationRunner.java => util/TestStreamUtil.java} (57%) diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java index fad9cf8d20..e2f5d0d0f1 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java @@ -30,7 +30,7 @@ */ @InterfaceStability.Unstable @FunctionalInterface -public interface MapFunction extends InitableFunction, ClosableFunction, Serializable { +public interface MapFunction extends InitableFunction, ClosableFunction, Serializable { /** * Transforms the provided message into another message. diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index 8339429972..45abb5d954 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -24,7 +24,6 @@ import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.system.StreamSpec; import java.lang.reflect.Constructor; @@ -124,25 +123,4 @@ public void waitForFinish() { public boolean waitForFinish(Duration timeout) { throw new UnsupportedOperationException(getClass().getName() + " does not support timed waitForFinish."); } - - /** - * Constructs a {@link StreamSpec} from the configuration for the specified streamId. - * - * The stream configurations are read from the following properties in the config: - * {@code streams.{$streamId}.*} - *
- * All properties matching this pattern are assumed to be system-specific with two exceptions. The following two - * properties are Samza properties which are used to bind the stream to a system and a physical resource on that system. - * - *
    - *
  • samza.system - The name of the System on which this stream will be used. If this property isn't defined - * the stream will be associated with the System defined in {@code job.default.system}
  • - *
  • samza.physical.name - The system-specific name for this stream. It could be a file URN, topic name, or other identifer. - * If this property isn't defined the physical.name will be set to the streamId
  • - *
- * - * @param streamId The logical identifier for the stream in Samza. - * @return The {@link StreamSpec} instance. - */ - public abstract StreamSpec getStreamSpec(String streamId); } diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java index cd86426bf2..aa71f0ed86 100644 --- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java +++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java @@ -75,25 +75,11 @@ public class StreamSpec implements Serializable { */ private final int partitionCount; - /** - * Bounded or unbounded stream - */ - private final boolean isBounded; - - /** - * broadcast stream to all tasks - */ - private final boolean isBroadcast; - /** * A set of all system-specific configurations for the stream. */ private final Map config; - @Override - public String toString() { - return String.format("StreamSpec: id=%s, systemName=%s, pName=%s, partCount=%d.", id, systemName, physicalName, partitionCount); - } /** * @param id The application-unique logical identifier for the stream. It is used to distinguish between * streams in a Samza application so it must be unique in the context of one deployable unit. @@ -107,7 +93,7 @@ public String toString() { * Samza System abstraction. See {@link SystemFactory} */ public StreamSpec(String id, String physicalName, String systemName) { - this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, false, false, Collections.emptyMap()); + this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, Collections.emptyMap()); } /** @@ -126,7 +112,7 @@ public StreamSpec(String id, String physicalName, String systemName) { * @param partitionCount The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned. */ public StreamSpec(String id, String physicalName, String systemName, int partitionCount) { - this(id, physicalName, systemName, partitionCount, false, false, Collections.emptyMap()); + this(id, physicalName, systemName, partitionCount, Collections.emptyMap()); } /** @@ -141,12 +127,10 @@ public StreamSpec(String id, String physicalName, String systemName, int partiti * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the * Samza System abstraction. See {@link SystemFactory} * - * @param isBounded The stream is bounded or not. - * * @param config A map of properties for the stream. These may be System-specfic. */ - public StreamSpec(String id, String physicalName, String systemName, boolean isBounded, Map config) { - this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, isBounded, false, config); + public StreamSpec(String id, String physicalName, String systemName, Map config) { + this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, config); } /** @@ -161,16 +145,11 @@ public StreamSpec(String id, String physicalName, String systemName, boolean isB * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the * Samza System abstraction. See {@link SystemFactory} * - * @param partitionCount The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned. - * - * @param isBounded The stream is bounded or not. - * - * @param isBroadcast This stream is broadcast or not. + * @param partitionCount The number of partitions for the stream. A value of {@code 1} indicates unpartitioned. * * @param config A map of properties for the stream. These may be System-specfic. */ - public StreamSpec(String id, String physicalName, String systemName, int partitionCount, - boolean isBounded, boolean isBroadcast, Map config) { + public StreamSpec(String id, String physicalName, String systemName, int partitionCount, Map config) { validateLogicalIdentifier("streamId", id); validateLogicalIdentifier("systemName", systemName); @@ -183,8 +162,6 @@ public StreamSpec(String id, String physicalName, String systemName, int partiti this.systemName = systemName; this.physicalName = physicalName; this.partitionCount = partitionCount; - this.isBounded = isBounded; - this.isBroadcast = isBroadcast; if (config != null) { this.config = Collections.unmodifiableMap(new HashMap<>(config)); @@ -202,15 +179,11 @@ public StreamSpec(String id, String physicalName, String systemName, int partiti * @return A copy of this StreamSpec with the specified partitionCount. */ public StreamSpec copyWithPartitionCount(int partitionCount) { - return new StreamSpec(id, physicalName, systemName, partitionCount, this.isBounded, this.isBroadcast, config); + return new StreamSpec(id, physicalName, systemName, partitionCount, config); } public StreamSpec copyWithPhysicalName(String physicalName) { - return new StreamSpec(id, physicalName, systemName, partitionCount, this.isBounded, this.isBroadcast, config); - } - - public StreamSpec copyWithBroadCast() { - return new StreamSpec(id, physicalName, systemName, partitionCount, this.isBounded, true, config); + return new StreamSpec(id, physicalName, systemName, partitionCount, config); } public String getId() { @@ -253,14 +226,6 @@ public boolean isCoordinatorStream() { return id.equals(COORDINATOR_STREAM_ID); } - public boolean isBounded() { - return isBounded; - } - - public boolean isBroadcast() { - return isBroadcast; - } - private void validateLogicalIdentifier(String identifierName, String identifierValue) { if (identifierValue == null || !identifierValue.matches("[A-Za-z0-9_-]+")) { throw new IllegalArgumentException(String.format("Identifier '%s' is '%s'. It must match the expression [A-Za-z0-9_-]+", identifierName, identifierValue)); @@ -297,4 +262,9 @@ public static StreamSpec createCheckpointStreamSpec(String physicalName, String public static StreamSpec createStreamAppenderStreamSpec(String physicalName, String systemName, int partitionCount) { return new StreamSpec(STREAM_APPENDER_ID, physicalName, systemName, partitionCount); } + + @Override + public String toString() { + return String.format("StreamSpec: id=%s, systemName=%s, pName=%s, partCount=%d.", id, systemName, physicalName, partitionCount); + } } diff --git a/samza-core/src/main/java/org/apache/samza/config/TaskConfigJava.java b/samza-core/src/main/java/org/apache/samza/config/TaskConfigJava.java index 29dd3ef658..c5b218376f 100644 --- a/samza-core/src/main/java/org/apache/samza/config/TaskConfigJava.java +++ b/samza-core/src/main/java/org/apache/samza/config/TaskConfigJava.java @@ -32,6 +32,7 @@ import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.StreamUtil; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,7 +103,7 @@ public Set getBroadcastSystemStreamPartitions() { } else { String systemStreamName = systemStreamPartition.substring(0, hashPosition); String partitionSegment = systemStreamPartition.substring(hashPosition + 1); - SystemStream systemStream = Util.getSystemStreamFromNames(systemStreamName); + SystemStream systemStream = StreamUtil.getSystemStreamFromNames(systemStreamName); if (Pattern.matches(BROADCAST_STREAM_PATTERN, partitionSegment)) { systemStreamPartitionSet.add(new SystemStreamPartition(systemStream, new Partition(Integer.valueOf(partitionSegment)))); diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 48f939c8e3..ef52e90e9c 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -34,6 +34,7 @@ import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; +import org.apache.samza.config.StreamConfig; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; @@ -43,6 +44,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.samza.util.StreamUtil.*; + /** * The ExecutionPlanner creates the physical execution graph for the StreamGraph, and @@ -93,8 +96,9 @@ private void validateConfig() { */ /* package private */ JobGraph createJobGraph(OperatorSpecGraph specGraph) { JobGraph jobGraph = new JobGraph(config, specGraph); - Set sourceStreams = new HashSet<>(specGraph.getInputOperators().keySet()); - Set sinkStreams = new HashSet<>(specGraph.getOutputStreams().keySet()); + StreamConfig streamConfig = new StreamConfig(config); + Set sourceStreams = getStreamSpecs(specGraph.getInputOperators().keySet(), streamConfig); + Set sinkStreams = getStreamSpecs(specGraph.getOutputStreams().keySet(), streamConfig); Set intStreams = new HashSet<>(sourceStreams); Set tables = new HashSet<>(specGraph.getTables().keySet()); intStreams.retainAll(sinkStreams); @@ -128,7 +132,7 @@ private void validateConfig() { */ /* package private */ void calculatePartitions(JobGraph jobGraph) { // calculate the partitions for the input streams of join operators - calculateJoinInputPartitions(jobGraph); + calculateJoinInputPartitions(jobGraph, config); // calculate the partitions for the rest of intermediate streams calculateIntStreamPartitions(jobGraph, config); @@ -172,7 +176,7 @@ private void validateConfig() { /** * Calculate the partitions for the input streams of join operators */ - /* package private */ static void calculateJoinInputPartitions(JobGraph jobGraph) { + /* package private */ static void calculateJoinInputPartitions(JobGraph jobGraph, Config config) { // mapping from a source stream to all join specs reachable from it Multimap joinSpecToStreamEdges = HashMultimap.create(); // reverse mapping of the above @@ -183,10 +187,10 @@ private void validateConfig() { Set visited = new HashSet<>(); jobGraph.getSpecGraph().getInputOperators().entrySet().forEach(entry -> { - StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(entry.getKey()); + StreamConfig streamConfig = new StreamConfig(config); + StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(getStreamSpec(entry.getKey(), streamConfig)); // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge - findReachableJoins(entry.getValue(), streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, - joinQ, visited); + findReachableJoins(entry.getValue(), streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, visited); }); // At this point, joinQ contains joinSpecs where at least one of the input stream edge partitions is known. @@ -203,7 +207,7 @@ private void validateConfig() { } else if (partitions != edgePartitions) { throw new SamzaException(String.format( "Unable to resolve input partitions of stream %s for join. Expected: %d, Actual: %d", - edge.getFormattedSystemStream(), partitions, edgePartitions)); + edge.getName(), partitions, edgePartitions)); } } } @@ -282,7 +286,7 @@ private static void calculateIntStreamPartitions(JobGraph jobGraph, Config confi private static void validatePartitions(JobGraph jobGraph) { for (StreamEdge edge : jobGraph.getIntermediateStreamEdges()) { if (edge.getPartitionCount() <= 0) { - throw new SamzaException(String.format("Failure to assign the partitions to Stream %s", edge.getFormattedSystemStream())); + throw new SamzaException(String.format("Failure to assign the partitions to Stream %s", edge.getName())); } } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 843db85b1a..2f210f2113 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -87,7 +87,7 @@ public List getJobConfigs() { @Override public List getIntermediateStreams() { return getIntermediateStreamEdges().stream() - .map(streamEdge -> streamEdge.getStreamSpec()) + .map(StreamEdge::getStreamSpec) .collect(Collectors.toList()); } @@ -187,12 +187,10 @@ StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec, boolean isIntermediate) String streamId = streamSpec.getId(); StreamEdge edge = edges.get(streamId); if (edge == null) { - edge = new StreamEdge(streamSpec, isIntermediate, config); + boolean isBroadcast = specGraph.getBroadcastStreams().contains(streamId); + edge = new StreamEdge(streamSpec, isIntermediate, isBroadcast, config); edges.put(streamId, edge); } - if (streamSpec.isBroadcast()) { - edge.setPartitionCount(1); - } return edge; } @@ -262,11 +260,11 @@ private void validateSources() { sources.forEach(edge -> { if (!edge.getSourceNodes().isEmpty()) { throw new IllegalArgumentException( - String.format("Source stream %s should not have producers.", edge.getFormattedSystemStream())); + String.format("Source stream %s should not have producers.", edge.getName())); } if (edge.getTargetNodes().isEmpty()) { throw new IllegalArgumentException( - String.format("Source stream %s should have consumers.", edge.getFormattedSystemStream())); + String.format("Source stream %s should have consumers.", edge.getName())); } }); } @@ -278,11 +276,11 @@ private void validateSinks() { sinks.forEach(edge -> { if (!edge.getTargetNodes().isEmpty()) { throw new IllegalArgumentException( - String.format("Sink stream %s should not have consumers", edge.getFormattedSystemStream())); + String.format("Sink stream %s should not have consumers", edge.getName())); } if (edge.getSourceNodes().isEmpty()) { throw new IllegalArgumentException( - String.format("Sink stream %s should have producers", edge.getFormattedSystemStream())); + String.format("Sink stream %s should have producers", edge.getName())); } }); } @@ -298,7 +296,7 @@ private void validateInternalStreams() { internalEdges.forEach(edge -> { if (edge.getSourceNodes().isEmpty() || edge.getTargetNodes().isEmpty()) { throw new IllegalArgumentException( - String.format("Internal stream %s should have both producers and consumers", edge.getFormattedSystemStream())); + String.format("Internal stream %s should have both producers and consumers", edge.getName())); } }); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 298042b133..4f2aa23f28 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -170,10 +170,10 @@ private JobNodeJson buildJobNodeJson(JobNode jobNode) { private OperatorGraphJson buildOperatorGraphJson(JobNode jobNode) { OperatorGraphJson opGraph = new OperatorGraphJson(); opGraph.inputStreams = new ArrayList<>(); - jobNode.getSpecGraph().getInputOperators().forEach((streamSpec, operatorSpec) -> { + jobNode.getSpecGraph().getInputOperators().forEach((streamId, operatorSpec) -> { StreamJson inputJson = new StreamJson(); opGraph.inputStreams.add(inputJson); - inputJson.streamId = streamSpec.getId(); + inputJson.streamId = streamId; Collection specs = operatorSpec.getRegisteredOperatorSpecs(); inputJson.nextOperatorIds = specs.stream().map(OperatorSpec::getOpId).collect(Collectors.toSet()); @@ -181,9 +181,9 @@ private OperatorGraphJson buildOperatorGraphJson(JobNode jobNode) { }); opGraph.outputStreams = new ArrayList<>(); - jobNode.getSpecGraph().getOutputStreams().keySet().forEach(streamSpec -> { + jobNode.getSpecGraph().getOutputStreams().keySet().forEach(streamId -> { StreamJson outputJson = new StreamJson(); - outputJson.streamId = streamSpec.getId(); + outputJson.streamId = streamId; opGraph.outputStreams.add(outputJson); }); return opGraph; @@ -219,10 +219,10 @@ private Map operatorToMap(OperatorSpec spec) { if (spec instanceof OutputOperatorSpec) { OutputStreamImpl outputStream = ((OutputOperatorSpec) spec).getOutputStream(); - map.put("outputStreamId", outputStream.getStreamSpec().getId()); + map.put("outputStreamId", outputStream.getStreamId()); } else if (spec instanceof PartitionByOperatorSpec) { OutputStreamImpl outputStream = ((PartitionByOperatorSpec) spec).getOutputStream(); - map.put("outputStreamId", outputStream.getStreamSpec().getId()); + map.put("outputStreamId", outputStream.getStreamId()); } if (spec instanceof StreamTableJoinOperatorSpec) { diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index db44d9f12b..94e509a43c 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -49,7 +49,6 @@ import org.apache.samza.util.MathUtil; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerializableSerde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableProvider; import org.apache.samza.table.TableProviderFactory; import org.apache.samza.table.TableSpec; @@ -135,8 +134,8 @@ public JobConfig generateConfig(String executionPlanJson) { final List inputs = new ArrayList<>(); final List broadcasts = new ArrayList<>(); for (StreamEdge inEdge : inEdges) { - String formattedSystemStream = inEdge.getFormattedSystemStream(); - if (inEdge.getStreamSpec().isBroadcast()) { + String formattedSystemStream = inEdge.getName(); + if (inEdge.isBroadcast()) { broadcasts.add(formattedSystemStream + "#0"); } else { inputs.add(formattedSystemStream); @@ -228,17 +227,17 @@ void addSerdeConfigs(Map configs) { // collect all key and msg serde instances for streams Map streamKeySerdes = new HashMap<>(); Map streamMsgSerdes = new HashMap<>(); - Map inputOperators = specGraph.getInputOperators(); + Map inputOperators = specGraph.getInputOperators(); inEdges.forEach(edge -> { String streamId = edge.getStreamSpec().getId(); - InputOperatorSpec inputOperatorSpec = inputOperators.get(edge.getStreamSpec()); + InputOperatorSpec inputOperatorSpec = inputOperators.get(streamId); streamKeySerdes.put(streamId, inputOperatorSpec.getKeySerde()); streamMsgSerdes.put(streamId, inputOperatorSpec.getValueSerde()); }); - Map outputStreams = specGraph.getOutputStreams(); + Map outputStreams = specGraph.getOutputStreams(); outEdges.forEach(edge -> { String streamId = edge.getStreamSpec().getId(); - OutputStreamImpl outputStream = outputStreams.get(edge.getStreamSpec()); + OutputStreamImpl outputStream = outputStreams.get(streamId); streamKeySerdes.put(streamId, outputStream.getKeySerde()); streamMsgSerdes.put(streamId, outputStream.getValueSerde()); }); diff --git a/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java b/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java index b4c93d9578..f2f0310d05 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java +++ b/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java @@ -29,7 +29,7 @@ import org.apache.samza.config.StreamConfig; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; -import org.apache.samza.util.Util; +import org.apache.samza.util.StreamUtil; /** @@ -41,23 +41,24 @@ public class StreamEdge { public static final int PARTITIONS_UNKNOWN = -1; private final StreamSpec streamSpec; + private final boolean isBroadcast; + private final boolean isIntermediate; private final List sourceNodes = new ArrayList<>(); private final List targetNodes = new ArrayList<>(); private final Config config; + private final String name; - private String name = ""; private int partitions = PARTITIONS_UNKNOWN; - private final boolean isIntermediate; - - StreamEdge(StreamSpec streamSpec, Config config) { - this(streamSpec, false, config); - } - StreamEdge(StreamSpec streamSpec, boolean isIntermediate, Config config) { + StreamEdge(StreamSpec streamSpec, boolean isIntermediate, boolean isBroadcast, Config config) { this.streamSpec = streamSpec; - this.name = Util.getNameFromSystemStream(getSystemStream()); this.isIntermediate = isIntermediate; + this.isBroadcast = isBroadcast; this.config = config; + if (isBroadcast) { + partitions = 1; + } + this.name = StreamUtil.getNameFromSystemStream(getSystemStream()); } void addSourceNode(JobNode sourceNode) { @@ -85,10 +86,6 @@ SystemStream getSystemStream() { return getStreamSpec().toSystemStream(); } - String getFormattedSystemStream() { - return Util.getNameFromSystemStream(getSystemStream()); - } - List getSourceNodes() { return sourceNodes; } @@ -109,10 +106,6 @@ String getName() { return name; } - void setName(String name) { - this.name = name; - } - boolean isIntermediate() { return isIntermediate; } @@ -128,12 +121,13 @@ Config generateConfig() { config.put(String.format(StreamConfig.CONSUMER_OFFSET_DEFAULT_FOR_STREAM_ID(), spec.getId()), "oldest"); config.put(String.format(StreamConfig.PRIORITY_FOR_STREAM_ID(), spec.getId()), String.valueOf(Integer.MAX_VALUE)); } - if (spec.isBounded()) { - config.put(String.format(StreamConfig.IS_BOUNDED_FOR_STREAM_ID(), spec.getId()), "true"); - } spec.getConfig().forEach((property, value) -> { config.put(String.format(StreamConfig.STREAM_ID_PREFIX(), spec.getId()) + property, value); }); return new MapConfig(config); } + + public boolean isBroadcast() { + return isBroadcast; + } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java b/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java index b0473c1235..2441226e81 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java +++ b/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java @@ -37,6 +37,7 @@ import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.util.StreamUtil; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -128,7 +129,7 @@ public void clearStreamsFromPreviousRun(Config prevConfig) { .getOrElse(defaultValue(null)); if (changelog != null) { LOGGER.info("Clear store {} changelog {}", store, changelog); - SystemStream systemStream = Util.getSystemStreamFromNames(changelog); + SystemStream systemStream = StreamUtil.getSystemStreamFromNames(changelog); StreamSpec spec = StreamSpec.createChangeLogStreamSpec(systemStream.getStream(), systemStream.getSystem(), 1); systemAdmins.getSystemAdmin(spec.getSystemName()).clearStream(spec); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java index ba51c7cb95..b6c3dae298 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java @@ -29,7 +29,6 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.serializers.SerializableSerde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableSpec; @@ -41,8 +40,9 @@ */ public class OperatorSpecGraph implements Serializable { // We use a LHM for deterministic order in initializing and closing operators. - private final Map inputOperators; - private final Map outputStreams; + private final Map inputOperators; + private final Map outputStreams; + private final Set broadcastStreams; private final Map tables; private final Set allOpSpecs; private final boolean hasWindowOrJoins; @@ -54,20 +54,25 @@ public class OperatorSpecGraph implements Serializable { OperatorSpecGraph(StreamGraphSpec graphSpec) { this.inputOperators = graphSpec.getInputOperators(); this.outputStreams = graphSpec.getOutputStreams(); + this.broadcastStreams = graphSpec.getBroadcastStreams(); this.tables = graphSpec.getTables(); this.allOpSpecs = Collections.unmodifiableSet(this.findAllOperatorSpecs()); this.hasWindowOrJoins = checkWindowOrJoins(); this.serializedOpSpecGraph = opSpecGraphSerde.toBytes(this); } - public Map getInputOperators() { + public Map getInputOperators() { return inputOperators; } - public Map getOutputStreams() { + public Map getOutputStreams() { return outputStreams; } + public Set getBroadcastStreams() { + return broadcastStreams; + } + public Map getTables() { return tables; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java index ea9690b314..a187b94172 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java @@ -34,11 +34,9 @@ import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.Table; import org.apache.samza.table.TableSpec; import org.slf4j.Logger; @@ -55,13 +53,13 @@ */ public class StreamGraphSpec implements StreamGraph { private static final Logger LOGGER = LoggerFactory.getLogger(StreamGraphSpec.class); - private static final Pattern USER_DEFINED_ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); + private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); // We use a LHM for deterministic order in initializing and closing operators. - private final Map inputOperators = new LinkedHashMap<>(); - private final Map outputStreams = new LinkedHashMap<>(); + private final Map inputOperators = new LinkedHashMap<>(); + private final Map outputStreams = new LinkedHashMap<>(); + private final Set broadcastStreams = new HashSet<>(); private final Map tables = new LinkedHashMap<>(); - private final ApplicationRunner runner; private final Config config; /** @@ -74,10 +72,7 @@ public class StreamGraphSpec implements StreamGraph { private Serde defaultSerde = new KVSerde(new NoOpSerde(), new NoOpSerde()); private ContextManager contextManager = null; - public StreamGraphSpec(ApplicationRunner runner, Config config) { - // TODO: SAMZA-1118 - Move StreamSpec and ApplicationRunner out of StreamGraphSpec once Systems - // can use streamId to send and receive messages. - this.runner = runner; + public StreamGraphSpec(Config config) { this.config = config; } @@ -91,15 +86,15 @@ public void setDefaultSerde(Serde serde) { @Override public MessageStream getInputStream(String streamId, Serde serde) { - StreamSpec streamSpec = runner.getStreamSpec(streamId); - Preconditions.checkState(streamSpec != null, "No StreamSpec found for streamId: " + streamId); + Preconditions.checkState(isValidId(streamId), + "streamId must be non-empty and must not contain spaces or special characters: " + streamId); Preconditions.checkNotNull(serde, "serde must not be null for an input stream."); - Preconditions.checkState(!inputOperators.containsKey(streamSpec), + Preconditions.checkState(!inputOperators.containsKey(streamId), "getInputStream must not be called multiple times with the same streamId: " + streamId); KV kvSerdes = getKVSerdes(streamId, serde); - if (outputStreams.containsKey(streamSpec)) { - OutputStreamImpl outputStream = outputStreams.get(streamSpec); + if (outputStreams.containsKey(streamId)) { + OutputStreamImpl outputStream = outputStreams.get(streamId); Serde keySerde = outputStream.getKeySerde(); Serde valueSerde = outputStream.getValueSerde(); Preconditions.checkState(kvSerdes.getKey().equals(keySerde) && kvSerdes.getValue().equals(valueSerde), @@ -109,10 +104,10 @@ public MessageStream getInputStream(String streamId, Serde serde) { boolean isKeyed = serde instanceof KVSerde; InputOperatorSpec inputOperatorSpec = - OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), + OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed, this.getNextOpId(OpCode.INPUT, null)); - inputOperators.put(streamSpec, inputOperatorSpec); - return new MessageStreamImpl<>(this, inputOperators.get(streamSpec)); + inputOperators.put(streamId, inputOperatorSpec); + return new MessageStreamImpl<>(this, inputOperators.get(streamId)); } @Override @@ -122,15 +117,15 @@ public MessageStream getInputStream(String streamId) { @Override public OutputStream getOutputStream(String streamId, Serde serde) { - StreamSpec streamSpec = runner.getStreamSpec(streamId); - Preconditions.checkState(streamSpec != null, "No StreamSpec found for streamId: " + streamId); + Preconditions.checkState(isValidId(streamId), + "streamId must be non-empty and must not contain spaces or special characters: " + streamId); Preconditions.checkNotNull(serde, "serde must not be null for an output stream."); - Preconditions.checkState(!outputStreams.containsKey(streamSpec), + Preconditions.checkState(!outputStreams.containsKey(streamId), "getOutputStream must not be called multiple times with the same streamId: " + streamId); KV kvSerdes = getKVSerdes(streamId, serde); - if (inputOperators.containsKey(streamSpec)) { - InputOperatorSpec inputOperatorSpec = inputOperators.get(streamSpec); + if (inputOperators.containsKey(streamId)) { + InputOperatorSpec inputOperatorSpec = inputOperators.get(streamId); Serde keySerde = inputOperatorSpec.getKeySerde(); Serde valueSerde = inputOperatorSpec.getValueSerde(); Preconditions.checkState(kvSerdes.getKey().equals(keySerde) && kvSerdes.getValue().equals(valueSerde), @@ -139,8 +134,8 @@ public OutputStream getOutputStream(String streamId, Serde serde) { } boolean isKeyed = serde instanceof KVSerde; - outputStreams.put(streamSpec, new OutputStreamImpl<>(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); - return outputStreams.get(streamSpec); + outputStreams.put(streamId, new OutputStreamImpl<>(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); + return outputStreams.get(streamId); } @Override @@ -183,8 +178,8 @@ public OperatorSpecGraph getOperatorSpecGraph() { * @return the unique ID for the next operator in the graph */ public String getNextOpId(OpCode opCode, String userDefinedId) { - if (StringUtils.isNotBlank(userDefinedId) && !USER_DEFINED_ID_PATTERN.matcher(userDefinedId).matches()) { - throw new SamzaException("Operator ID must not contain spaces and special characters: " + userDefinedId); + if (StringUtils.isNotBlank(userDefinedId) && !ID_PATTERN.matcher(userDefinedId).matches()) { + throw new SamzaException("Operator ID must not contain spaces or special characters: " + userDefinedId); } String nextOpId = String.format("%s-%s-%s-%s", @@ -234,17 +229,10 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI * @param isBroadcast whether the stream is a broadcast stream. * @param the type of messages in the intermediate {@link MessageStream} * @return the intermediate {@link MessageStreamImpl} - * - * TODO: once SAMZA-1566 is resolved, we should be able to pass in the StreamSpec directly. */ @VisibleForTesting IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serde serde, boolean isBroadcast) { - StreamSpec streamSpec = runner.getStreamSpec(streamId); - if (isBroadcast) { - streamSpec = streamSpec.copyWithBroadCast(); - } - - Preconditions.checkState(!inputOperators.containsKey(streamSpec) && !outputStreams.containsKey(streamSpec), + Preconditions.checkState(!inputOperators.containsKey(streamId) && !outputStreams.containsKey(streamId), "getIntermediateStream must not be called multiple times with the same streamId: " + streamId); if (serde == null) { @@ -252,28 +240,37 @@ IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serd serde = (Serde) defaultSerde; } + if (isBroadcast) broadcastStreams.add(streamId); boolean isKeyed = serde instanceof KVSerde; KV kvSerdes = getKVSerdes(streamId, serde); InputOperatorSpec inputOperatorSpec = - OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), + OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed, this.getNextOpId(OpCode.INPUT, null)); - inputOperators.put(streamSpec, inputOperatorSpec); - outputStreams.put(streamSpec, new OutputStreamImpl(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); - return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamSpec), outputStreams.get(streamSpec)); + inputOperators.put(streamId, inputOperatorSpec); + outputStreams.put(streamId, new OutputStreamImpl(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); + return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamId), outputStreams.get(streamId)); } - Map getInputOperators() { + Map getInputOperators() { return Collections.unmodifiableMap(inputOperators); } - Map getOutputStreams() { + Map getOutputStreams() { return Collections.unmodifiableMap(outputStreams); } + Set getBroadcastStreams() { + return Collections.unmodifiableSet(broadcastStreams); + } + Map getTables() { return Collections.unmodifiableMap(tables); } + private boolean isValidId(String id) { + return StringUtils.isNotBlank(id) && ID_PATTERN.matcher(id).matches(); + } + private KV getKVSerdes(String streamId, Serde serde) { Serde keySerde, valueSerde; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java index 8df670ebf8..99ed089b63 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java @@ -40,9 +40,9 @@ class BroadcastOperatorImpl extends OperatorImpl { private final SystemStream systemStream; private final String taskName; - BroadcastOperatorImpl(BroadcastOperatorSpec broadcastOpSpec, TaskContext context) { + BroadcastOperatorImpl(BroadcastOperatorSpec broadcastOpSpec, SystemStream systemStream, TaskContext context) { this.broadcastOpSpec = broadcastOpSpec; - this.systemStream = broadcastOpSpec.getOutputStream().getSystemStream(); + this.systemStream = systemStream; this.taskName = context.getTaskName().getTaskName(); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index 0f51798e89..2c13f92d10 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Multimap; import org.apache.samza.config.Config; +import org.apache.samza.config.StreamConfig; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.job.model.JobModel; import org.apache.samza.operators.KV; @@ -84,6 +85,7 @@ public class OperatorImplGraph { private final Map> joinOpImpls = new HashMap<>(); private final Clock clock; + private final OperatorImplGraphUtil util; /** * Constructs the DAG of {@link OperatorImpl}s corresponding to the the DAG of {@link OperatorSpec}s @@ -96,12 +98,14 @@ public class OperatorImplGraph { */ public OperatorImplGraph(OperatorSpecGraph specGraph, Config config, TaskContext context, Clock clock) { this.clock = clock; - + this.util = new OperatorImplGraphUtil(config); TaskContextImpl taskContext = (TaskContextImpl) context; - Map producerTaskCounts = hasIntermediateStreams(specGraph) ? - getProducerTaskCountForIntermediateStreams(getStreamToConsumerTasks(taskContext.getJobModel()), - getIntermediateToInputStreamsMap(specGraph)) : - Collections.EMPTY_MAP; + Map producerTaskCounts = + util.hasIntermediateStreams(specGraph) + ? util.getProducerTaskCountForIntermediateStreams( + util.getStreamToConsumerTasks(taskContext.getJobModel()), + util.getIntermediateToInputStreamsMap(specGraph)) + : Collections.EMPTY_MAP; producerTaskCounts.forEach((stream, count) -> { LOG.info("{} has {} producer tasks.", stream, count); }); @@ -113,8 +117,8 @@ public OperatorImplGraph(OperatorSpecGraph specGraph, Config config, TaskContext taskContext.registerObject(WatermarkStates.class.getName(), new WatermarkStates(context.getSystemStreamPartitions(), producerTaskCounts)); - specGraph.getInputOperators().forEach((streamSpec, inputOpSpec) -> { - SystemStream systemStream = new SystemStream(streamSpec.getSystemName(), streamSpec.getPhysicalName()); + specGraph.getInputOperators().forEach((streamId, inputOpSpec) -> { + SystemStream systemStream = util.toSystemStream(streamId); InputOperatorImpl inputOperatorImpl = (InputOperatorImpl) createAndRegisterOperatorImpl(null, inputOpSpec, systemStream, config, context); this.inputOperators.put(systemStream, inputOperatorImpl); @@ -217,9 +221,13 @@ OperatorImpl createOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec oper } else if (operatorSpec instanceof SinkOperatorSpec) { return new SinkOperatorImpl((SinkOperatorSpec) operatorSpec, config, context); } else if (operatorSpec instanceof OutputOperatorSpec) { - return new OutputOperatorImpl((OutputOperatorSpec) operatorSpec); + String streamId = ((OutputOperatorSpec) operatorSpec).getOutputStream().getStreamId(); + SystemStream systemStream = util.toSystemStream(streamId); + return new OutputOperatorImpl((OutputOperatorSpec) operatorSpec, systemStream); } else if (operatorSpec instanceof PartitionByOperatorSpec) { - return new PartitionByOperatorImpl((PartitionByOperatorSpec) operatorSpec, config, context); + String streamId = ((PartitionByOperatorSpec) operatorSpec).getOutputStream().getStreamId(); + SystemStream systemStream = util.toSystemStream(streamId); + return new PartitionByOperatorImpl((PartitionByOperatorSpec) operatorSpec, systemStream, context); } else if (operatorSpec instanceof WindowOperatorSpec) { return new WindowOperatorImpl((WindowOperatorSpec) operatorSpec, clock); } else if (operatorSpec instanceof JoinOperatorSpec) { @@ -231,7 +239,9 @@ OperatorImpl createOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec oper } else if (operatorSpec instanceof SendToTableOperatorSpec) { return new SendToTableOperatorImpl((SendToTableOperatorSpec) operatorSpec, config, context); } else if (operatorSpec instanceof BroadcastOperatorSpec) { - return new BroadcastOperatorImpl((BroadcastOperatorSpec) operatorSpec, context); + String streamId = ((BroadcastOperatorSpec) operatorSpec).getOutputStream().getStreamId(); + SystemStream systemStream = util.toSystemStream(streamId); + return new BroadcastOperatorImpl((BroadcastOperatorSpec) operatorSpec, systemStream, context); } throw new IllegalArgumentException( String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName())); @@ -323,70 +333,81 @@ public KeyValueStore> getState() { }; } - private boolean hasIntermediateStreams(OperatorSpecGraph specGraph) { - return !Collections.disjoint(specGraph.getInputOperators().keySet(), specGraph.getOutputStreams().keySet()); - } + protected static class OperatorImplGraphUtil { + private final Config config; - /** - * calculate the task count that produces to each intermediate streams - * @param streamToConsumerTasks input streams to task mapping - * @param intermediateToInputStreams intermediate stream to input streams mapping - * @return mapping from intermediate stream to task count - */ - static Map getProducerTaskCountForIntermediateStreams( - Multimap streamToConsumerTasks, - Multimap intermediateToInputStreams) { - Map result = new HashMap<>(); - intermediateToInputStreams.asMap().entrySet().forEach(entry -> { - result.put(entry.getKey(), - entry.getValue().stream() - .flatMap(systemStream -> streamToConsumerTasks.get(systemStream).stream()) - .collect(Collectors.toSet()).size()); - }); - return result; - } + OperatorImplGraphUtil(Config config) { + this.config = config; + } - /** - * calculate the mapping from input streams to consumer tasks - * @param jobModel JobModel object - * @return mapping from input stream to tasks - */ - static Multimap getStreamToConsumerTasks(JobModel jobModel) { - Multimap streamToConsumerTasks = HashMultimap.create(); - jobModel.getContainers().values().forEach(containerModel -> { - containerModel.getTasks().values().forEach(taskModel -> { - taskModel.getSystemStreamPartitions().forEach(ssp -> { - streamToConsumerTasks.put(ssp.getSystemStream(), taskModel.getTaskName().getTaskName()); - }); - }); - }); - return streamToConsumerTasks; - } + /** + * calculate the task count that produces to each intermediate streams + * @param streamToConsumerTasks input streams to task mapping + * @param intermediateToInputStreams intermediate stream to input streams mapping + * @return mapping from intermediate stream to task count + */ + Map getProducerTaskCountForIntermediateStreams( + Multimap streamToConsumerTasks, + Multimap intermediateToInputStreams) { + Map result = new HashMap<>(); + intermediateToInputStreams.asMap().entrySet().forEach(entry -> + result.put(entry.getKey(), + entry.getValue().stream() + .flatMap(systemStream -> streamToConsumerTasks.get(systemStream).stream()) + .collect(Collectors.toSet()).size())); + return result; + } - /** - * calculate the mapping from output streams to input streams - * @param specGraph the user {@link OperatorSpecGraph} - * @return mapping from output streams to input streams - */ - static Multimap getIntermediateToInputStreamsMap(OperatorSpecGraph specGraph) { - Multimap outputToInputStreams = HashMultimap.create(); - specGraph.getInputOperators().entrySet().stream() - .forEach( - entry -> computeOutputToInput(entry.getKey().toSystemStream(), entry.getValue(), outputToInputStreams)); - return outputToInputStreams; - } + /** + * calculate the mapping from input streams to consumer tasks + * @param jobModel JobModel object + * @return mapping from input stream to tasks + */ + Multimap getStreamToConsumerTasks(JobModel jobModel) { + Multimap streamToConsumerTasks = HashMultimap.create(); + jobModel.getContainers().values().forEach(containerModel -> { + containerModel.getTasks().values().forEach(taskModel -> { + taskModel.getSystemStreamPartitions().forEach(ssp -> { + streamToConsumerTasks.put(ssp.getSystemStream(), taskModel.getTaskName().getTaskName()); + }); + }); + }); + return streamToConsumerTasks; + } - private static void computeOutputToInput(SystemStream input, OperatorSpec opSpec, - Multimap outputToInputStreams) { - if (opSpec instanceof PartitionByOperatorSpec) { - PartitionByOperatorSpec spec = (PartitionByOperatorSpec) opSpec; - outputToInputStreams.put(spec.getOutputStream().getStreamSpec().toSystemStream(), input); - } else if (opSpec instanceof BroadcastOperatorSpec) { - BroadcastOperatorSpec spec = (BroadcastOperatorSpec) opSpec; - outputToInputStreams.put(spec.getOutputStream().getStreamSpec().toSystemStream(), input); - } else { - Collection nextOperators = opSpec.getRegisteredOperatorSpecs(); - nextOperators.forEach(spec -> computeOutputToInput(input, spec, outputToInputStreams)); + /** + * calculate the mapping from output streams to input streams + * @param specGraph the user {@link OperatorSpecGraph} + * @return mapping from output streams to input streams + */ + Multimap getIntermediateToInputStreamsMap(OperatorSpecGraph specGraph) { + Multimap outputToInputStreams = HashMultimap.create(); + specGraph.getInputOperators().entrySet().stream() + .forEach(entry -> + computeOutputToInput(toSystemStream(entry.getKey()), entry.getValue(), outputToInputStreams)); + return outputToInputStreams; + } + + private void computeOutputToInput(SystemStream input, OperatorSpec opSpec, + Multimap outputToInputStreams) { + if (opSpec instanceof PartitionByOperatorSpec) { + PartitionByOperatorSpec spec = (PartitionByOperatorSpec) opSpec; + outputToInputStreams.put(toSystemStream(spec.getOutputStream().getStreamId()), input); + } else if (opSpec instanceof BroadcastOperatorSpec) { + BroadcastOperatorSpec spec = (BroadcastOperatorSpec) opSpec; + outputToInputStreams.put(toSystemStream(spec.getOutputStream().getStreamId()), input); + } else { + Collection nextOperators = opSpec.getRegisteredOperatorSpecs(); + nextOperators.forEach(spec -> computeOutputToInput(input, spec, outputToInputStreams)); + } + } + + private SystemStream toSystemStream(String streamId) { + return new StreamConfig(config).streamIdToSystemStream(streamId); + } + + private boolean hasIntermediateStreams(OperatorSpecGraph specGraph) { + return !Collections.disjoint(specGraph.getInputOperators().keySet(), specGraph.getOutputStreams().keySet()); } } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java index e625484b97..22fbb1b9ab 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java @@ -42,10 +42,10 @@ class OutputOperatorImpl extends OperatorImpl { private final OutputStreamImpl outputStream; private final SystemStream systemStream; - OutputOperatorImpl(OutputOperatorSpec outputOpSpec) { + OutputOperatorImpl(OutputOperatorSpec outputOpSpec, SystemStream systemStream) { this.outputOpSpec = outputOpSpec; this.outputStream = outputOpSpec.getOutputStream(); - this.systemStream = outputStream.getSystemStream(); + this.systemStream = systemStream; } @Override diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java index dd64429664..63e269d842 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java @@ -20,10 +20,8 @@ import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; -import org.apache.samza.operators.KV; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.spec.OperatorSpec; -import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.spec.PartitionByOperatorSpec; import org.apache.samza.system.ControlMessage; import org.apache.samza.system.EndOfStreamMessage; @@ -51,10 +49,10 @@ class PartitionByOperatorImpl extends OperatorImpl { private final String taskName; private final ControlMessageSender controlMessageSender; - PartitionByOperatorImpl(PartitionByOperatorSpec partitionByOpSpec, Config config, TaskContext context) { + PartitionByOperatorImpl(PartitionByOperatorSpec partitionByOpSpec, + SystemStream systemStream, TaskContext context) { this.partitionByOpSpec = partitionByOpSpec; - OutputStreamImpl> outputStream = partitionByOpSpec.getOutputStream(); - this.systemStream = outputStream.getSystemStream(); + this.systemStream = systemStream; this.keyFunction = partitionByOpSpec.getKeyFunction(); this.valueFunction = partitionByOpSpec.getValueFunction(); this.taskName = context.getTaskName().getTaskName(); @@ -102,7 +100,6 @@ protected Collection handleWatermark(long watermark, MessageCollector coll } private void sendControlMessage(ControlMessage message, MessageCollector collector) { - SystemStream outputStream = partitionByOpSpec.getOutputStream().getSystemStream(); - controlMessageSender.send(message, outputStream, collector); + controlMessageSender.send(message, systemStream, collector); } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java index a636ac56c5..922a1f9d6b 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java @@ -22,7 +22,6 @@ import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.serializers.Serde; -import org.apache.samza.system.StreamSpec; /** * The spec for an operator that receives incoming messages from an input stream @@ -34,7 +33,7 @@ public class InputOperatorSpec extends OperatorSpec, Object> { // Object == KV | V private final boolean isKeyed; - private final StreamSpec streamSpec; + private final String streamId; /** * The following {@link Serde}s are serialized by the ExecutionPlanner when generating the configs for a stream, and deserialized @@ -43,17 +42,16 @@ public class InputOperatorSpec extends OperatorSpec, Object> { // private transient final Serde keySerde; private transient final Serde valueSerde; - public InputOperatorSpec(StreamSpec streamSpec, - Serde keySerde, Serde valueSerde, boolean isKeyed, String opId) { + public InputOperatorSpec(String streamId, Serde keySerde, Serde valueSerde, boolean isKeyed, String opId) { super(OpCode.INPUT, opId); - this.streamSpec = streamSpec; + this.streamId = streamId; this.keySerde = keySerde; this.valueSerde = valueSerde; this.isKeyed = isKeyed; } - public StreamSpec getStreamSpec() { - return this.streamSpec; + public String getStreamId() { + return this.streamId; } public Serde getKeySerde() { diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java index 6e98d5a38b..9e788dab2a 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java @@ -28,7 +28,6 @@ import org.apache.samza.operators.functions.StreamTableJoinFunction; import org.apache.samza.operators.windows.internal.WindowInternal; import org.apache.samza.serializers.Serde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableSpec; @@ -42,7 +41,7 @@ private OperatorSpecs() {} /** * Creates an {@link InputOperatorSpec} for consuming input. * - * @param streamSpec the stream spec for the input stream + * @param streamId the stream id for the input stream * @param keySerde the serde for the input key * @param valueSerde the serde for the input value * @param isKeyed whether the input stream is keyed @@ -52,8 +51,8 @@ private OperatorSpecs() {} * @return the {@link InputOperatorSpec} */ public static InputOperatorSpec createInputOperatorSpec( - StreamSpec streamSpec, Serde keySerde, Serde valueSerde, boolean isKeyed, String opId) { - return new InputOperatorSpec<>(streamSpec, keySerde, valueSerde, isKeyed, opId); + String streamId, Serde keySerde, Serde valueSerde, boolean isKeyed, String opId) { + return new InputOperatorSpec<>(streamId, keySerde, valueSerde, isKeyed, opId); } /** diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java index fe0abcb5d6..5d70e6f4ad 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java @@ -21,13 +21,10 @@ import java.io.Serializable; import org.apache.samza.operators.OutputStream; import org.apache.samza.serializers.Serde; -import org.apache.samza.system.StreamSpec; -import org.apache.samza.system.SystemStream; - public class OutputStreamImpl implements OutputStream, Serializable { - private final StreamSpec streamSpec; + private final String streamId; private final boolean isKeyed; /** @@ -37,16 +34,15 @@ public class OutputStreamImpl implements OutputStream, Serializable { private transient final Serde keySerde; private transient final Serde valueSerde; - public OutputStreamImpl(StreamSpec streamSpec, - Serde keySerde, Serde valueSerde, boolean isKeyed) { - this.streamSpec = streamSpec; + public OutputStreamImpl(String streamId, Serde keySerde, Serde valueSerde, boolean isKeyed) { + this.streamId = streamId; this.keySerde = keySerde; this.valueSerde = valueSerde; this.isKeyed = isKeyed; } - public StreamSpec getStreamSpec() { - return streamSpec; + public String getStreamId() { + return streamId; } public Serde getKeySerde() { @@ -57,10 +53,6 @@ public Serde getValueSerde() { return valueSerde; } - public SystemStream getSystemStream() { - return this.streamSpec.toSystemStream(); - } - public boolean isKeyed() { return isKeyed; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java index 272ba635b8..3bb8713b97 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java @@ -24,7 +24,6 @@ import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; -import org.apache.samza.system.StreamSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,13 +50,13 @@ public IntermediateMessageStreamImpl(StreamGraphSpec graph, InputOperatorSpec getOutputStream() { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 5043977602..45ab50833a 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -31,7 +31,6 @@ import org.apache.samza.execution.StreamManager; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmins; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,18 +59,11 @@ public abstract class AbstractApplicationRunner extends ApplicationRunner { public AbstractApplicationRunner(Config config) { super(config); - this.graphSpec = new StreamGraphSpec(this, config); + this.graphSpec = new StreamGraphSpec(config); this.systemAdmins = new SystemAdmins(config); this.streamManager = new StreamManager(systemAdmins); } - @Override - public StreamSpec getStreamSpec(String streamId) { - StreamConfig streamConfig = new StreamConfig(config); - String physicalName = streamConfig.getPhysicalName(streamId); - return getStreamSpec(streamId, physicalName); - } - @Override public void run(StreamApplication streamApp) { systemAdmins.start(); @@ -82,50 +74,6 @@ public void kill(StreamApplication streamApp) { systemAdmins.stop(); } - /** - * Constructs a {@link StreamSpec} from the configuration for the specified streamId. - * - * The stream configurations are read from the following properties in the config: - * {@code streams.{$streamId}.*} - *
- * All properties matching this pattern are assumed to be system-specific with one exception. The following - * property is a Samza property which is used to bind the stream to a system. - * - *
    - *
  • samza.system - The name of the System on which this stream will be used. If this property isn't defined - * the stream will be associated with the System defined in {@code job.default.system}
  • - *
- * - * @param streamId The logical identifier for the stream in Samza. - * @param physicalName The system-specific name for this stream. It could be a file URN, topic name, or other identifer. - * @return The {@link StreamSpec} instance. - */ - /*package private*/ StreamSpec getStreamSpec(String streamId, String physicalName) { - StreamConfig streamConfig = new StreamConfig(config); - String system = streamConfig.getSystem(streamId); - - return getStreamSpec(streamId, physicalName, system); - } - - /** - * Constructs a {@link StreamSpec} from the configuration for the specified streamId. - * - * The stream configurations are read from the following properties in the config: - * {@code streams.{$streamId}.*} - * - * @param streamId The logical identifier for the stream in Samza. - * @param physicalName The system-specific name for this stream. It could be a file URN, topic name, or other identifer. - * @param system The name of the System on which this stream will be used. - * @return The {@link StreamSpec} instance. - */ - /*package private*/ StreamSpec getStreamSpec(String streamId, String physicalName, String system) { - StreamConfig streamConfig = new StreamConfig(config); - Map properties = streamConfig.getStreamProperties(streamId); - boolean isBounded = streamConfig.getIsBounded(streamId); - - return new StreamSpec(streamId, physicalName, system, isBounded, properties); - } - public ExecutionPlan getExecutionPlan(StreamApplication app) throws Exception { return getExecutionPlan(app, null); } @@ -134,20 +82,22 @@ public ExecutionPlan getExecutionPlan(StreamApplication app) throws Exception { ExecutionPlan getExecutionPlan(StreamApplication app, String runId) throws Exception { // build stream graph app.init(graphSpec, config); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); - // create the physical execution plan + + // update application configs Map cfg = new HashMap<>(config); if (StringUtils.isNoneEmpty(runId)) { cfg.put(ApplicationConfig.APP_RUN_ID, runId); } - Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); + StreamConfig streamConfig = new StreamConfig(config); + Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); inputStreams.removeAll(specGraph.getOutputStreams().keySet()); - ApplicationMode mode = inputStreams.stream().allMatch(StreamSpec::isBounded) + ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) ? ApplicationMode.BATCH : ApplicationMode.STREAM; cfg.put(ApplicationConfig.APP_MODE, mode.name()); + // create the physical execution plan ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager); return planner.plan(specGraph); } diff --git a/samza-core/src/main/java/org/apache/samza/storage/ChangelogStreamManager.java b/samza-core/src/main/java/org/apache/samza/storage/ChangelogStreamManager.java index 6aeb2bad0f..ea55fe562c 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/ChangelogStreamManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/ChangelogStreamManager.java @@ -34,7 +34,7 @@ import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStream; -import org.apache.samza.util.Util; +import org.apache.samza.util.StreamUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -113,7 +113,7 @@ public static void createChangelogStreams(Config config, int maxChangeLogStreamP .stream() .filter(name -> StringUtils.isNotBlank(storageConfig.getChangelogStream(name))) .collect(Collectors.toMap(name -> name, - name -> Util.getSystemStreamFromNames(storageConfig.getChangelogStream(name)))); + name -> StreamUtil.getSystemStreamFromNames(storageConfig.getChangelogStream(name)))); // Get SystemAdmin for changelog store's system and attempt to create the stream JavaSystemConfig systemConfig = new JavaSystemConfig(config); diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java index db6f0d938f..47058017d2 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java @@ -48,6 +48,7 @@ import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.util.CommandLine; import org.apache.samza.util.ScalaJavaUtil; +import org.apache.samza.util.StreamUtil; import org.apache.samza.util.SystemClock; import org.apache.samza.util.Util; import org.slf4j.Logger; @@ -147,7 +148,7 @@ private void getChangeLogSystemStreamsAndStorageFactories() { log.info("stream name for " + storeName + " is " + streamName); if (streamName != null) { - changeLogSystemStreams.put(storeName, Util.getSystemStreamFromNames(streamName)); + changeLogSystemStreams.put(storeName, StreamUtil.getSystemStreamFromNames(streamName)); } String factoryClass = config.getStorageFactoryClassName(storeName); diff --git a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java new file mode 100644 index 0000000000..5170b29eed --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.util; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.ImmutableTriple; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StreamConfig; +import org.apache.samza.config.StreamConfig$; +import org.apache.samza.system.StreamSpec; +import org.apache.samza.system.SystemStream; + +public class StreamUtil { + /** + * Returns a SystemStream object based on the system stream name given. For + * example, kafka.topic would return new SystemStream("kafka", "topic"). + */ + public static SystemStream getSystemStreamFromNames(String systemStreamNames) { + int idx = systemStreamNames.indexOf('.'); + if (idx < 0) { + throw new IllegalArgumentException("No '.' in stream name '" + systemStreamNames + + "'. Stream names should be in the form 'system.stream'"); + } + return new SystemStream( + systemStreamNames.substring(0, idx), + systemStreamNames.substring(idx + 1, systemStreamNames.length())); + } + + /** + * Returns a SystemStream object based on the system stream name given. For + * example, kafka.topic would return new SystemStream("kafka", "topic"). + */ + public static String getNameFromSystemStream(SystemStream systemStream) { + return systemStream.getSystem() + "." + systemStream.getStream(); + } + + public static Set getStreamSpecs(Set streamIds, StreamConfig streamConfig) { + return streamIds.stream().map(streamId -> getStreamSpec(streamId, streamConfig)).collect(Collectors.toSet()); + } + + public static StreamSpec getStreamSpec(String streamId, StreamConfig streamConfig) { + String physicalName = streamConfig.getPhysicalName(streamId); + String system = streamConfig.getSystem(streamId); + Map streamProperties = streamConfig.getStreamProperties(streamId); + return new StreamSpec(streamId, physicalName, system, streamProperties); + } + + /** + * Converts the provided list of (streamId, system, physicalName) triplets to their corresponding + * stream.stream-id.* configurations. + * + * @param streams a list of (streamId, system, physicalName) triplets to get the stream configuration for. + * @return the configuration for the provided { @code streams} + */ + public static Config toStreamConfigs(List> streams) { + Map configsMap = new HashMap<>(); + streams.stream().forEach(triple -> { + String streamId = triple.getLeft(); + String systemName = triple.getMiddle(); + String physicalName = triple.getRight(); + configsMap.put(String.format(StreamConfig$.MODULE$.SYSTEM_FOR_STREAM_ID(), streamId), systemName); + configsMap.put(String.format(StreamConfig$.MODULE$.PHYSICAL_NAME_FOR_STREAM_ID(), streamId), physicalName); + }); + return new MapConfig(configsMap); + } +} diff --git a/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala index e4ee76721d..06dd00648e 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala @@ -22,8 +22,7 @@ package org.apache.samza.config import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ -import org.apache.samza.util.Logging -import org.apache.samza.util.Util +import org.apache.samza.util.{Logging, StreamUtil} object StorageConfig { // stream config constants @@ -94,7 +93,7 @@ class StorageConfig(config: Config) extends ScalaMapConfig(config) with Logging .map(getChangelogStream(_)) .filter(_.isDefined) // Convert "system.stream" to systemName - .map(systemStreamName => Util.getSystemStreamFromNames(systemStreamName.get).getSystem) + .map(systemStreamName => StreamUtil.getSystemStreamFromNames(systemStreamName.get).getSystem) .contains(systemName) } diff --git a/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala index 206eb8f241..7ecf77080e 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala @@ -22,7 +22,7 @@ package org.apache.samza.config import org.apache.samza.checkpoint.CheckpointManager import org.apache.samza.metrics.MetricsRegistry import org.apache.samza.system.SystemStream -import org.apache.samza.util.{Logging, Util} +import org.apache.samza.util.{Logging, StreamUtil} object TaskConfig { // task config constants @@ -78,7 +78,7 @@ class TaskConfig(config: Config) extends ScalaMapConfig(config) with Logging { def getInputStreams = getOption(TaskConfig.INPUT_STREAMS) match { case Some(streams) => if (streams.length > 0) { streams.split(",").map(systemStreamNames => { - Util.getSystemStreamFromNames(systemStreamNames.trim) + StreamUtil.getSystemStreamFromNames(systemStreamNames.trim) }).toSet } else { Set[SystemStream]() diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 5380fc98dc..31aef44f7d 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -323,7 +323,7 @@ object SamzaContainer extends Logging { .getStoreNames .filter(config.getChangelogStream(_).isDefined) .map(name => (name, config.getChangelogStream(name).get)).toMap - .mapValues(Util.getSystemStreamFromNames(_)) + .mapValues(StreamUtil.getSystemStreamFromNames(_)) info("Got change log system streams: %s" format changeLogSystemStreams) diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index 029b37533e..6f87f44084 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -28,7 +28,6 @@ import org.apache.samza.coordinator.stream.CoordinatorStreamManager import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} import org.apache.samza.operators.StreamGraphSpec -import org.apache.samza.runtime.LocalContainerRunner import org.apache.samza.storage.ChangelogStreamManager import org.apache.samza.task.TaskFactoryUtil import org.apache.samza.util.Logging @@ -72,10 +71,9 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val containerId = "0" val jmxServer = new JmxServer val streamApp = TaskFactoryUtil.createStreamApplication(config) - val appRunner = new LocalContainerRunner(jobModel, "0") val taskFactory = if (streamApp != null) { - val graphSpec = new StreamGraphSpec(appRunner, config) + val graphSpec = new StreamGraphSpec(config) streamApp.init(graphSpec, config) TaskFactoryUtil.createTaskFactory(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager) } else { diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala index 33802a17b0..e41d4a8566 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala @@ -19,7 +19,7 @@ package org.apache.samza.metrics.reporter -import org.apache.samza.util.Logging +import org.apache.samza.util.{Logging, StreamUtil, Util} import org.apache.samza.SamzaException import org.apache.samza.config.{ApplicationConfig, Config} import org.apache.samza.config.JobConfig.Config2Job @@ -30,7 +30,6 @@ import org.apache.samza.config.SerializerConfig.Config2Serializer import org.apache.samza.config.TaskConfig.Config2Task import org.apache.samza.metrics.MetricsReporter import org.apache.samza.metrics.MetricsReporterFactory -import org.apache.samza.util.Util import org.apache.samza.metrics.MetricsRegistryMap import org.apache.samza.serializers.SerdeFactory import org.apache.samza.system.SystemFactory @@ -68,7 +67,7 @@ class MetricsSnapshotReporterFactory extends MetricsReporterFactory with Logging .getMetricsReporterStream(name) .getOrElse(throw new SamzaException("No metrics stream defined in config.")) - val systemStream = Util.getSystemStreamFromNames(metricsSystemStreamName) + val systemStream = StreamUtil.getSystemStreamFromNames(metricsSystemStreamName) info("Got system stream %s." format systemStream) diff --git a/samza-core/src/main/scala/org/apache/samza/util/Util.scala b/samza-core/src/main/scala/org/apache/samza/util/Util.scala index 059eb038f5..a6353ac632 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/Util.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/Util.scala @@ -66,26 +66,6 @@ object Util extends Logging { } } - /** - * Returns a SystemStream object based on the system stream name given. For - * example, kafka.topic would return new SystemStream("kafka", "topic"). - */ - def getSystemStreamFromNames(systemStreamNames: String): SystemStream = { - val idx = systemStreamNames.indexOf('.') - if (idx < 0) { - throw new IllegalArgumentException("No '.' in stream name '" + systemStreamNames + "'. Stream names should be in the form 'system.stream'") - } - new SystemStream(systemStreamNames.substring(0, idx), systemStreamNames.substring(idx + 1, systemStreamNames.length)) - } - - /** - * Returns a SystemStream object based on the system stream name given. For - * example, kafka.topic would return new SystemStream("kafka", "topic"). - */ - def getNameFromSystemStream(systemStream: SystemStream) = { - systemStream.getSystem + "." + systemStream.getStream - } - /** * Returns the the first host address which is not the loopback address, or [[java.net.InetAddress#getLocalHost]] as a fallback * 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 83fe5ad757..02419f3081 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 @@ -19,6 +19,8 @@ package org.apache.samza.execution; +import com.google.common.collect.ImmutableList; + import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -27,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.commons.lang3.tuple.ImmutableTriple; import org.apache.samza.Partition; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -38,13 +41,13 @@ import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.Serde; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.StreamUtil; import org.junit.Before; import org.junit.Test; @@ -59,7 +62,6 @@ public class TestExecutionPlanner { private SystemAdmins systemAdmins; private StreamManager streamManager; - private ApplicationRunner runner; private Config config; private StreamSpec input1; @@ -104,7 +106,7 @@ private StreamGraphSpec createSimpleGraph() { * input1 -> partitionBy -> map -> output1 * */ - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); MessageStream> input1 = graphSpec.getInputStream("input1"); OutputStream> output1 = graphSpec.getOutputStream("output1"); input1 @@ -127,7 +129,7 @@ private StreamGraphSpec createStreamGraphWithJoin() { * */ - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); MessageStream> messageStream1 = graphSpec.>getInputStream("input1") .map(m -> m); @@ -159,7 +161,7 @@ private StreamGraphSpec createStreamGraphWithJoin() { private StreamGraphSpec createStreamGraphWithJoinAndWindow() { - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); MessageStream> messageStream1 = graphSpec.>getInputStream("input1") .map(m -> m); @@ -207,7 +209,15 @@ public void setup() { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), DEFAULT_SYSTEM); - + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of("input1", "system1", "input1"), + ImmutableTriple.of("input2", "system2", "input2"), + ImmutableTriple.of("input3", "system2", "input3"), + ImmutableTriple.of("input4", "system1", "input4"), + ImmutableTriple.of("output1", "system1", "output1"), + ImmutableTriple.of("output2", "system2", "output2") + )); + configMap.putAll(streamConfigs); config = new MapConfig(configMap); input1 = new StreamSpec("input1", "input1", "system1"); @@ -234,22 +244,6 @@ public void setup() { when(systemAdmins.getSystemAdmin("system1")).thenReturn(systemAdmin1); when(systemAdmins.getSystemAdmin("system2")).thenReturn(systemAdmin2); streamManager = new StreamManager(systemAdmins); - - runner = mock(ApplicationRunner.class); - when(runner.getStreamSpec("input1")).thenReturn(input1); - when(runner.getStreamSpec("input2")).thenReturn(input2); - when(runner.getStreamSpec("input3")).thenReturn(input3); - when(runner.getStreamSpec("input4")).thenReturn(input4); - when(runner.getStreamSpec("output1")).thenReturn(output1); - when(runner.getStreamSpec("output2")).thenReturn(output2); - - // intermediate streams used in tests - when(runner.getStreamSpec("test-app-1-partition_by-p1")) - .thenReturn(new StreamSpec("test-app-1-partition_by-p1", "test-app-1-partition_by-p1", "default-system")); - when(runner.getStreamSpec("test-app-1-partition_by-p2")) - .thenReturn(new StreamSpec("test-app-1-partition_by-p2", "test-app-1-partition_by-p2", "default-system")); - when(runner.getStreamSpec("test-app-1-partition_by-p3")) - .thenReturn(new StreamSpec("test-app-1-partition_by-p3", "test-app-1-partition_by-p3", "default-system")); } @Test @@ -288,7 +282,7 @@ public void testCalculateJoinInputPartitions() { JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); - ExecutionPlanner.calculateJoinInputPartitions(jobGraph); + ExecutionPlanner.calculateJoinInputPartitions(jobGraph, config); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -406,13 +400,13 @@ public void testCalculateIntStreamPartitions() throws Exception { @Test public void testMaxPartition() { Collection edges = new ArrayList<>(); - StreamEdge edge = new StreamEdge(input1, config); + StreamEdge edge = new StreamEdge(input1, false, false, config); edge.setPartitionCount(2); edges.add(edge); - edge = new StreamEdge(input2, config); + edge = new StreamEdge(input2, false, false, config); edge.setPartitionCount(32); edges.add(edge); - edge = new StreamEdge(input3, config); + edge = new StreamEdge(input3, false, false, config); edge.setPartitionCount(16); edges.add(edge); @@ -427,7 +421,7 @@ public void testMaxPartitionLimit() throws Exception { int partitionLimit = ExecutionPlanner.MAX_INFERRED_PARTITIONS; ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); MessageStream> input1 = graphSpec.getInputStream("input4"); OutputStream> output1 = graphSpec.getOutputStream("output1"); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index abe8969490..40cbe30ad7 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -19,9 +19,12 @@ package org.apache.samza.execution; +import com.google.common.collect.ImmutableList; + import java.time.Duration; import java.util.HashMap; import java.util.Map; +import org.apache.commons.lang3.tuple.ImmutableTriple; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -31,15 +34,14 @@ import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.LongSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.StringSerde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemAdmins; +import org.apache.samza.util.StreamUtil; import org.codehaus.jackson.map.ObjectMapper; import org.junit.Test; @@ -76,28 +78,16 @@ public void test() throws Exception { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of("input1", "system1", "input1"), + ImmutableTriple.of("input2", "system2", "input2"), + ImmutableTriple.of("input3", "system2", "input3"), + ImmutableTriple.of("output1", "system1", "output1"), + ImmutableTriple.of("output2", "system2", "output2") + )); + configMap.putAll(streamConfigs); Config config = new MapConfig(configMap); - StreamSpec input1 = new StreamSpec("input1", "input1", "system1"); - StreamSpec input2 = new StreamSpec("input2", "input2", "system2"); - StreamSpec input3 = new StreamSpec("input3", "input3", "system2"); - - StreamSpec output1 = new StreamSpec("output1", "output1", "system1"); - StreamSpec output2 = new StreamSpec("output2", "output2", "system2"); - - ApplicationRunner runner = mock(ApplicationRunner.class); - when(runner.getStreamSpec("input1")).thenReturn(input1); - when(runner.getStreamSpec("input2")).thenReturn(input2); - when(runner.getStreamSpec("input3")).thenReturn(input3); - when(runner.getStreamSpec("output1")).thenReturn(output1); - when(runner.getStreamSpec("output2")).thenReturn(output2); - - // intermediate streams used in tests - when(runner.getStreamSpec("test-app-1-partition_by-p1")) - .thenReturn(new StreamSpec("test-app-1-partition_by-p1", "test-app-1-partition_by-p1", "default-system")); - when(runner.getStreamSpec("test-app-1-partition_by-p2")) - .thenReturn(new StreamSpec("test-app-1-partition_by-p2", "test-app-1-partition_by-p2", "default-system")); - // set up external partition count Map system1Map = new HashMap<>(); system1Map.put("input1", 64); @@ -114,7 +104,7 @@ public void test() throws Exception { when(systemAdmins.getSystemAdmin("system2")).thenReturn(systemAdmin2); StreamManager streamManager = new StreamManager(systemAdmins); - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); graphSpec.setDefaultSerde(KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); MessageStream> messageStream1 = graphSpec.>getInputStream("input1") @@ -163,19 +153,13 @@ public void test2() throws Exception { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of("PageView", "hdfs", "hdfs:/user/dummy/PageViewEvent"), + ImmutableTriple.of("PageViewCount", "kafka", "PageViewCount") + )); + configMap.putAll(streamConfigs); Config config = new MapConfig(configMap); - StreamSpec input = new StreamSpec("PageView", "hdfs:/user/dummy/PageViewEvent", "hdfs"); - StreamSpec output = new StreamSpec("PageViewCount", "PageViewCount", "kafka"); - - ApplicationRunner runner = mock(ApplicationRunner.class); - when(runner.getStreamSpec("PageView")).thenReturn(input); - when(runner.getStreamSpec("PageViewCount")).thenReturn(output); - - // intermediate streams used in tests - when(runner.getStreamSpec("test-app-1-partition_by-keyed-by-country")) - .thenReturn(new StreamSpec("test-app-1-partition_by-keyed-by-country", "test-app-1-partition_by-keyed-by-country", "kafka")); - // set up external partition count Map system1Map = new HashMap<>(); system1Map.put("hdfs:/user/dummy/PageViewEvent", 512); @@ -189,7 +173,7 @@ public void test2() throws Exception { when(systemAdmins.getSystemAdmin("kafka")).thenReturn(systemAdmin2); StreamManager streamManager = new StreamManager(systemAdmins); - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); MessageStream> inputStream = graphSpec.getInputStream("PageView"); inputStream .partitionBy(kv -> kv.getValue().getCountry(), kv -> kv.getValue(), "keyed-by-country") diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index c43e242f51..cefe128070 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -29,7 +29,6 @@ import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; @@ -48,7 +47,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -56,22 +54,17 @@ public class TestJobNode { @Test public void testAddSerdeConfigs() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); StreamSpec input1Spec = new StreamSpec("input1", "input1", "input-system"); StreamSpec input2Spec = new StreamSpec("input2", "input2", "input-system"); StreamSpec outputSpec = new StreamSpec("output", "output", "output-system"); StreamSpec partitionBySpec = new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); - doReturn(input1Spec).when(mockRunner).getStreamSpec("input1"); - doReturn(input2Spec).when(mockRunner).getStreamSpec("input2"); - doReturn(outputSpec).when(mockRunner).getStreamSpec("output"); - doReturn(partitionBySpec).when(mockRunner).getStreamSpec("jobName-jobId-partition_by-p1"); Config mockConfig = mock(Config.class); when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); graphSpec.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>())); MessageStream> input1 = graphSpec.getInputStream("input1"); MessageStream> input2 = graphSpec.getInputStream("input2"); @@ -86,10 +79,10 @@ public void testAddSerdeConfigs() { JobNode jobNode = new JobNode("jobName", "jobId", graphSpec.getOperatorSpecGraph(), mockConfig); Config config = new MapConfig(); - StreamEdge input1Edge = new StreamEdge(input1Spec, config); - StreamEdge input2Edge = new StreamEdge(input2Spec, config); - StreamEdge outputEdge = new StreamEdge(outputSpec, config); - StreamEdge repartitionEdge = new StreamEdge(partitionBySpec, true, config); + StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, config); + StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, config); + StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, config); + StreamEdge repartitionEdge = new StreamEdge(partitionBySpec, true, false, config); jobNode.addInEdge(input1Edge); jobNode.addInEdge(input2Edge); jobNode.addOutEdge(outputEdge); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java b/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java index 8ad5b7e5b0..ee1749c07c 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java @@ -37,7 +37,7 @@ public class TestStreamEdge { @Test public void testGetStreamSpec() { - StreamEdge edge = new StreamEdge(spec, false, new MapConfig()); + StreamEdge edge = new StreamEdge(spec, false, false, new MapConfig()); assertEquals(edge.getStreamSpec(), spec); assertEquals(edge.getStreamSpec().getPartitionCount(), 1 /*StreamSpec.DEFAULT_PARTITION_COUNT*/); @@ -50,15 +50,15 @@ public void testGetStreamSpec_Batch() { Map config = new HashMap<>(); config.put(ApplicationConfig.APP_MODE, ApplicationConfig.ApplicationMode.BATCH.name()); config.put(ApplicationConfig.APP_RUN_ID, "123"); - StreamEdge edge = new StreamEdge(spec, true, new MapConfig(config)); + StreamEdge edge = new StreamEdge(spec, true, false, new MapConfig(config)); assertEquals(edge.getStreamSpec().getPhysicalName(), spec.getPhysicalName() + "-123"); } @Test public void testGenerateConfig() { // an example unbounded IO stream - StreamSpec spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", false, Collections.singletonMap("property1", "haha")); - StreamEdge edge = new StreamEdge(spec, false, new MapConfig()); + StreamSpec spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", Collections.singletonMap("property1", "haha")); + StreamEdge edge = new StreamEdge(spec, false, false, new MapConfig()); Config config = edge.generateConfig(); StreamConfig streamConfig = new StreamConfig(config); assertEquals(streamConfig.getSystem(spec.getId()), "system-1"); @@ -68,14 +68,14 @@ public void testGenerateConfig() { assertEquals(streamConfig.getStreamProperties(spec.getId()).get("property1"), "haha"); // bounded stream - spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", true, Collections.singletonMap("property1", "haha")); - edge = new StreamEdge(spec, false, new MapConfig()); + spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", Collections.singletonMap("property1", "haha")); + edge = new StreamEdge(spec, false, false, new MapConfig()); config = edge.generateConfig(); streamConfig = new StreamConfig(config); assertEquals(streamConfig.getIsBounded(spec.getId()), true); // intermediate stream - edge = new StreamEdge(spec, true, new MapConfig()); + edge = new StreamEdge(spec, true, false, new MapConfig()); config = edge.generateConfig(); streamConfig = new StreamConfig(config); assertEquals(streamConfig.getIsIntermediateStream(spec.getId()), true); diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java index 602b595443..70547274d7 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java @@ -28,12 +28,10 @@ import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TestInMemoryStore; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.OutgoingMessageEnvelope; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.task.MessageCollector; @@ -74,7 +72,6 @@ public class TestJoinOperator { @Before public void setUp() { Map mapConfig = new HashMap<>(); - mapConfig.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner"); mapConfig.put("job.default.system", "insystem"); mapConfig.put("job.name", "jobName"); mapConfig.put("job.id", "jobId"); @@ -101,7 +98,7 @@ public void join() throws Exception { public void joinWithSelfThrowsException() throws Exception { config.put("streams.instream.system", "insystem"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(ApplicationRunner.class), config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); IntegerSerde integerSerde = new IntegerSerde(); KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); MessageStream> inStream = graphSpec.getInputStream("instream", kvSerde); @@ -320,11 +317,7 @@ private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec } private StreamGraphSpec getTestJoinStreamGraph(TestJoinFunction joinFn) throws IOException { - ApplicationRunner runner = mock(ApplicationRunner.class); - when(runner.getStreamSpec("instream")).thenReturn(new StreamSpec("instream", "instream", "insystem")); - when(runner.getStreamSpec("instream2")).thenReturn(new StreamSpec("instream2", "instream2", "insystem")); - - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); IntegerSerde integerSerde = new IntegerSerde(); KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); MessageStream> inStream = graphSpec.getInputStream("instream", kvSerde); diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java index 2be88cac1d..4cfc66a6ad 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java @@ -38,7 +38,6 @@ import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; import org.apache.samza.serializers.NoOpSerde; -import org.apache.samza.system.StreamSpec; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -59,8 +58,8 @@ public class TestOperatorSpecGraph { private StreamGraphSpec mockGraph; - private Map inputOpSpecMap; - private Map outputStrmMap; + private Map inputOpSpecMap; + private Map outputStrmMap; private Set allOpSpecs; @Before @@ -72,26 +71,26 @@ public void setUp() { * 1) input1 --> filter --> sendTo * 2) input2 --> map --> sink */ - StreamSpec testInputSpec = new StreamSpec("test-input-1", "test-input-1", "kafka"); - InputOperatorSpec testInput = new InputOperatorSpec(testInputSpec, new NoOpSerde(), new NoOpSerde(), true, "test-input-1"); + String inputStreamId1 = "test-input-1"; + String outputStreamId = "test-output-1"; + InputOperatorSpec testInput = new InputOperatorSpec(inputStreamId1, new NoOpSerde(), new NoOpSerde(), true, inputStreamId1); StreamOperatorSpec filterOp = OperatorSpecs.createFilterOperatorSpec(m -> true, "test-filter-2"); - StreamSpec testOutputSpec = new StreamSpec("test-output-1", "test-output-1", "kafka"); - OutputStreamImpl outputStream1 = new OutputStreamImpl(testOutputSpec, null, null, true); + OutputStreamImpl outputStream1 = new OutputStreamImpl(outputStreamId, null, null, true); OutputOperatorSpec outputSpec = OperatorSpecs.createSendToOperatorSpec(outputStream1, "test-output-3"); testInput.registerNextOperatorSpec(filterOp); filterOp.registerNextOperatorSpec(outputSpec); - StreamSpec testInputSpec2 = new StreamSpec("test-input-2", "test-input-2", "kafka"); - InputOperatorSpec testInput2 = new InputOperatorSpec(testInputSpec2, new NoOpSerde(), new NoOpSerde(), true, "test-input-4"); + String streamId2 = "test-input-2"; + InputOperatorSpec testInput2 = new InputOperatorSpec(streamId2, new NoOpSerde(), new NoOpSerde(), true, "test-input-4"); StreamOperatorSpec testMap = OperatorSpecs.createMapOperatorSpec(m -> m, "test-map-5"); SinkOperatorSpec testSink = OperatorSpecs.createSinkOperatorSpec((m, mc, tc) -> { }, "test-sink-6"); testInput2.registerNextOperatorSpec(testMap); testMap.registerNextOperatorSpec(testSink); this.inputOpSpecMap = new LinkedHashMap<>(); - inputOpSpecMap.put(testInputSpec, testInput); - inputOpSpecMap.put(testInputSpec2, testInput2); + inputOpSpecMap.put(inputStreamId1, testInput); + inputOpSpecMap.put(streamId2, testInput2); this.outputStrmMap = new LinkedHashMap<>(); - outputStrmMap.put(testOutputSpec, outputStream1); + outputStrmMap.put(outputStreamId, outputStream1); when(mockGraph.getInputOperators()).thenReturn(Collections.unmodifiableMap(inputOpSpecMap)); when(mockGraph.getOutputStreams()).thenReturn(Collections.unmodifiableMap(outputStrmMap)); this.allOpSpecs = new HashSet() { { diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java index e476abc3c7..dfb4b706ec 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java @@ -31,11 +31,9 @@ import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableSpec; import org.junit.Test; @@ -53,82 +51,71 @@ public class TestStreamGraphSpec { @Test public void testGetInputStreamWithValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - MessageStream inputStream = graphSpec.getInputStream("test-stream-1", mockValueSerde); + MessageStream inputStream = graphSpec.getInputStream(streamId, mockValueSerde); InputOperatorSpec inputOpSpec = (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec); - assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); } @Test public void testGetInputStreamWithKeyValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + String streamId = "test-stream-1"; KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - MessageStream inputStream = graphSpec.getInputStream("test-stream-1", mockKVSerde); + MessageStream inputStream = graphSpec.getInputStream(streamId, mockKVSerde); InputOperatorSpec inputOpSpec = (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec); - assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + assertEquals(streamId, inputOpSpec.getStreamId()); assertEquals(mockKeySerde, inputOpSpec.getKeySerde()); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); } @Test(expected = NullPointerException.class) public void testGetInputStreamWithNullSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); graphSpec.getInputStream("test-stream-1", null); } @Test public void testGetInputStreamWithDefaultValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); Serde mockValueSerde = mock(Serde.class); graphSpec.setDefaultSerde(mockValueSerde); - MessageStream inputStream = graphSpec.getInputStream("test-stream-1"); + MessageStream inputStream = graphSpec.getInputStream(streamId); InputOperatorSpec inputOpSpec = (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec); - assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); } @Test public void testGetInputStreamWithDefaultKeyValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -136,63 +123,56 @@ public void testGetInputStreamWithDefaultKeyValueSerde() { doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); graphSpec.setDefaultSerde(mockKVSerde); - MessageStream inputStream = graphSpec.getInputStream("test-stream-1"); + MessageStream inputStream = graphSpec.getInputStream(streamId); InputOperatorSpec inputOpSpec = (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec); - assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + assertEquals(streamId, inputOpSpec.getStreamId()); assertEquals(mockKeySerde, inputOpSpec.getKeySerde()); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); } @Test public void testGetInputStreamWithDefaultDefaultSerde() { - // default default serde == user hasn't provided a default serde - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; - MessageStream inputStream = graphSpec.getInputStream("test-stream-1"); + // default default serde == user hasn't provided a default serde + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + MessageStream inputStream = graphSpec.getInputStream(streamId); InputOperatorSpec inputOpSpec = (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec); - assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde); } @Test public void testGetInputStreamWithRelaxedTypes() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - MessageStream inputStream = graphSpec.getInputStream("test-stream-1"); + MessageStream inputStream = graphSpec.getInputStream(streamId); InputOperatorSpec inputOpSpec = (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec); - assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + assertEquals(streamId, inputOpSpec.getStreamId()); } @Test public void testMultipleGetInputStreams() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec1 = mock(StreamSpec.class); - StreamSpec mockStreamSpec2 = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec1); - when(mockRunner.getStreamSpec("test-stream-2")).thenReturn(mockStreamSpec2); + String streamId1 = "test-stream-1"; + String streamId2 = "test-stream-2"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - MessageStream inputStream1 = graphSpec.getInputStream("test-stream-1"); - MessageStream inputStream2 = graphSpec.getInputStream("test-stream-2"); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + MessageStream inputStream1 = graphSpec.getInputStream(streamId1); + MessageStream inputStream2 = graphSpec.getInputStream(streamId2); InputOperatorSpec inputOpSpec1 = (InputOperatorSpec) ((MessageStreamImpl) inputStream1).getOperatorSpec(); @@ -200,98 +180,83 @@ public void testMultipleGetInputStreams() { (InputOperatorSpec) ((MessageStreamImpl) inputStream2).getOperatorSpec(); assertEquals(graphSpec.getInputOperators().size(), 2); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec1), inputOpSpec1); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec2), inputOpSpec2); + assertEquals(graphSpec.getInputOperators().get(streamId1), inputOpSpec1); + assertEquals(graphSpec.getInputOperators().get(streamId2), inputOpSpec2); } @Test(expected = IllegalStateException.class) public void testGetSameInputStreamTwice() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class)); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - graphSpec.getInputStream("test-stream-1"); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + graphSpec.getInputStream(streamId); // should throw exception - graphSpec.getInputStream("test-stream-1"); + graphSpec.getInputStream(streamId); } @Test public void testGetOutputStreamWithValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); Serde mockValueSerde = mock(Serde.class); OutputStream outputStream = - graphSpec.getOutputStream("test-stream-1", mockValueSerde); + graphSpec.getOutputStream(streamId, mockValueSerde); OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl); - assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); } @Test public void testGetOutputStreamWithKeyValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); graphSpec.setDefaultSerde(mockKVSerde); - OutputStream outputStream = graphSpec.getOutputStream("test-stream-1", mockKVSerde); + OutputStream outputStream = graphSpec.getOutputStream(streamId, mockKVSerde); OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl); - assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + assertEquals(streamId, outputStreamImpl.getStreamId()); assertEquals(mockKeySerde, outputStreamImpl.getKeySerde()); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); } @Test(expected = NullPointerException.class) public void testGetOutputStreamWithNullSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - graphSpec.getOutputStream("test-stream-1", null); + graphSpec.getOutputStream(streamId, null); } @Test public void testGetOutputStreamWithDefaultValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); + String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); graphSpec.setDefaultSerde(mockValueSerde); - OutputStream outputStream = graphSpec.getOutputStream("test-stream-1"); + OutputStream outputStream = graphSpec.getOutputStream(streamId); OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl); - assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); } @Test public void testGetOutputStreamWithDefaultKeyValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); + String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); @@ -299,89 +264,75 @@ public void testGetOutputStreamWithDefaultKeyValueSerde() { doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); graphSpec.setDefaultSerde(mockKVSerde); - OutputStream outputStream = graphSpec.getOutputStream("test-stream-1"); + OutputStream outputStream = graphSpec.getOutputStream(streamId); OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl); - assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + assertEquals(streamId, outputStreamImpl.getStreamId()); assertEquals(mockKeySerde, outputStreamImpl.getKeySerde()); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); } @Test public void testGetOutputStreamWithDefaultDefaultSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec); + String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - OutputStream outputStream = graphSpec.getOutputStream("test-stream-1"); + OutputStream outputStream = graphSpec.getOutputStream(streamId); OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl); - assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertTrue(outputStreamImpl.getValueSerde() instanceof NoOpSerde); } @Test(expected = IllegalStateException.class) public void testSetDefaultSerdeAfterGettingStreams() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class)); + String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - graphSpec.getInputStream("test-stream-1"); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + graphSpec.getInputStream(streamId); graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception } @Test(expected = IllegalStateException.class) public void testSetDefaultSerdeAfterGettingOutputStream() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class)); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - graphSpec.getOutputStream("test-stream-1"); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + graphSpec.getOutputStream(streamId); graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception } @Test(expected = IllegalStateException.class) public void testSetDefaultSerdeAfterGettingIntermediateStream() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class)); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - graphSpec.getIntermediateStream("test-stream-1", null); + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + graphSpec.getIntermediateStream(streamId, null); graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception } @Test(expected = IllegalStateException.class) public void testGetSameOutputStreamTwice() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class)); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - graphSpec.getOutputStream("test-stream-1"); - graphSpec.getOutputStream("test-stream-1"); // should throw exception + String streamId = "test-stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + graphSpec.getOutputStream(streamId); + graphSpec.getOutputStream(streamId); // should throw exception } @Test public void testGetIntermediateStreamWithValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - Config mockConfig = mock(Config.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - String mockStreamName = "mockStreamName"; - when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + String streamId = "stream-1"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); Serde mockValueSerde = mock(Serde.class); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(mockStreamName, mockValueSerde); + graphSpec.getIntermediateStream(streamId, mockValueSerde); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream()); - assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde); @@ -390,13 +341,8 @@ public void testGetIntermediateStreamWithValueSerde() { @Test public void testGetIntermediateStreamWithKeyValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - Config mockConfig = mock(Config.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - String mockStreamName = "mockStreamName"; - when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + String streamId = "streamId"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -404,11 +350,11 @@ public void testGetIntermediateStreamWithKeyValueSerde() { doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(mockStreamName, mockKVSerde); + graphSpec.getIntermediateStream(streamId, mockKVSerde); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream()); - assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde()); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); assertEquals(mockKeySerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde()); @@ -417,22 +363,17 @@ public void testGetIntermediateStreamWithKeyValueSerde() { @Test public void testGetIntermediateStreamWithDefaultValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - Config mockConfig = mock(Config.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - String mockStreamName = "mockStreamName"; - when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec); - - StreamGraphSpec graph = new StreamGraphSpec(mockRunner, mockConfig); + String streamId = "streamId"; + StreamGraphSpec graph = new StreamGraphSpec(mock(Config.class)); Serde mockValueSerde = mock(Serde.class); graph.setDefaultSerde(mockValueSerde); IntermediateMessageStreamImpl intermediateStreamImpl = - graph.getIntermediateStream(mockStreamName, null); + graph.getIntermediateStream(streamId, null); - assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream()); - assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec()); + assertEquals(graph.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(graph.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde); @@ -441,13 +382,10 @@ public void testGetIntermediateStreamWithDefaultValueSerde() { @Test public void testGetIntermediateStreamWithDefaultKeyValueSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - String mockStreamName = "mockStreamName"; - when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + String streamId = "streamId"; + + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -456,11 +394,11 @@ public void testGetIntermediateStreamWithDefaultKeyValueSerde() { doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); graphSpec.setDefaultSerde(mockKVSerde); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(mockStreamName, null); + graphSpec.getIntermediateStream(streamId, null); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream()); - assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde()); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); assertEquals(mockKeySerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde()); @@ -469,19 +407,16 @@ public void testGetIntermediateStreamWithDefaultKeyValueSerde() { @Test public void testGetIntermediateStreamWithDefaultDefaultSerde() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); - StreamSpec mockStreamSpec = mock(StreamSpec.class); - String mockStreamName = "mockStreamName"; - when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + String streamId = "streamId"; + + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(mockStreamName, null); + graphSpec.getIntermediateStream(streamId, null); - assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream()); - assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec()); + assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde); assertTrue(intermediateStreamImpl.getOutputStream().getValueSerde() instanceof NoOpSerde); assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde); @@ -490,22 +425,18 @@ public void testGetIntermediateStreamWithDefaultDefaultSerde() { @Test(expected = IllegalStateException.class) public void testGetSameIntermediateStreamTwice() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class)); - - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class)); graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class)); } @Test public void testGetNextOpIdIncrementsId() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); assertEquals("jobName-1234-merge-0", graphSpec.getNextOpId(OpCode.MERGE, null)); assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OpCode.JOIN, "customName")); assertEquals("jobName-1234-map-2", graphSpec.getNextOpId(OpCode.MAP, null)); @@ -513,24 +444,22 @@ public void testGetNextOpIdIncrementsId() { @Test(expected = SamzaException.class) public void testGetNextOpIdRejectsDuplicates() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OpCode.JOIN, "customName")); graphSpec.getNextOpId(OpCode.JOIN, "customName"); // should throw } @Test public void testUserDefinedIdValidation() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); // null and empty userDefinedIDs should fall back to autogenerated IDs. try { @@ -562,36 +491,29 @@ public void testUserDefinedIdValidation() { @Test public void testGetInputStreamPreservesInsertionOrder() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); - - StreamSpec testStreamSpec1 = new StreamSpec("test-stream-1", "physical-stream-1", "test-system"); - when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(testStreamSpec1); - - StreamSpec testStreamSpec2 = new StreamSpec("test-stream-2", "physical-stream-2", "test-system"); - when(mockRunner.getStreamSpec("test-stream-2")).thenReturn(testStreamSpec2); - - StreamSpec testStreamSpec3 = new StreamSpec("test-stream-3", "physical-stream-3", "test-system"); - when(mockRunner.getStreamSpec("test-stream-3")).thenReturn(testStreamSpec3); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); + String testStreamId1 = "test-stream-1"; + String testStreamId2 = "test-stream-2"; + String testStreamId3 = "test-stream-3"; + graphSpec.getInputStream("test-stream-1"); graphSpec.getInputStream("test-stream-2"); graphSpec.getInputStream("test-stream-3"); List inputSpecs = new ArrayList<>(graphSpec.getInputOperators().values()); assertEquals(inputSpecs.size(), 3); - assertEquals(inputSpecs.get(0).getStreamSpec(), testStreamSpec1); - assertEquals(inputSpecs.get(1).getStreamSpec(), testStreamSpec2); - assertEquals(inputSpecs.get(2).getStreamSpec(), testStreamSpec3); + assertEquals(inputSpecs.get(0).getStreamId(), testStreamId1); + assertEquals(inputSpecs.get(1).getStreamId(), testStreamId2); + assertEquals(inputSpecs.get(2).getStreamId(), testStreamId3); } @Test public void testGetTable() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); Config mockConfig = mock(Config.class); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); when(mockTableDescriptor.getTableSpec()).thenReturn( diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index b87e5ed665..d393e527a1 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -19,8 +19,7 @@ package org.apache.samza.operators.impl; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; +import com.google.common.collect.ImmutableList; import java.io.Serializable; import java.time.Duration; import java.util.ArrayList; @@ -32,7 +31,7 @@ import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; -import org.apache.samza.Partition; +import org.apache.commons.lang3.tuple.ImmutableTriple; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -54,21 +53,19 @@ import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.impl.store.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec.OpCode; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.KeyValueStore; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; -import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.task.MessageCollector; import org.apache.samza.task.TaskContext; import java.util.List; import org.apache.samza.task.TaskCoordinator; import org.apache.samza.util.Clock; +import org.apache.samza.util.StreamUtil; import org.apache.samza.util.SystemClock; import org.junit.After; import org.junit.Test; @@ -76,7 +73,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -217,7 +213,7 @@ public void tearDown() { @Test public void testEmptyChain() { - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(ApplicationRunner.class), mock(Config.class)); + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); OperatorImplGraph opGraph = new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mock(TaskContextImpl.class), mock(Clock.class)); assertEquals(0, opGraph.getAllInputOperators().size()); @@ -225,13 +221,30 @@ public void testEmptyChain() { @Test public void testLinearChain() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system")); - when(mockRunner.getStreamSpec(eq("output"))).thenReturn(mock(StreamSpec.class)); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); - - MessageStream inputStream = graphSpec.getInputStream("input"); - OutputStream outputStream = graphSpec.getOutputStream("output"); + String inputStreamId = "input"; + String inputSystem = "input-system"; + String inputPhysicalName = "input-stream"; + String outputStreamId = "output"; + String outputSystem = "output-system"; + String outputPhysicalName = "output-stream"; + String intermediateSystem = "intermediate-system"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystem); + + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of(inputStreamId, inputSystem, inputPhysicalName), + ImmutableTriple.of(outputStreamId, outputSystem, outputPhysicalName) + )); + configs.putAll(streamConfigs); + Config config = new MapConfig(configs); + + StreamGraphSpec graphSpec = new StreamGraphSpec(config); + + MessageStream inputStream = graphSpec.getInputStream(inputStreamId); + OutputStream outputStream = graphSpec.getOutputStream(outputStreamId); inputStream .filter(mock(FilterFunction.class)) @@ -242,9 +255,9 @@ public void testLinearChain() { when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); when(mockTaskContext.getTaskName()).thenReturn(new TaskName("task 0")); OperatorImplGraph opImplGraph = - new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mockTaskContext, mock(Clock.class)); + new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), config, mockTaskContext, mock(Clock.class)); - InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream")); + InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream(inputSystem, inputPhysicalName)); assertEquals(1, inputOpImpl.registeredOperators.size()); OperatorImpl filterOpImpl = (StreamOperatorImpl) inputOpImpl.registeredOperators.iterator().next(); @@ -262,18 +275,31 @@ public void testLinearChain() { @Test public void testPartitionByChain() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system")); - when(mockRunner.getStreamSpec(eq("output"))).thenReturn(new StreamSpec("output", "output-stream", "output-system")); - when(mockRunner.getStreamSpec(eq("jobName-jobId-partition_by-p1"))) - .thenReturn(new StreamSpec("intermediate", "intermediate-stream", "intermediate-system")); - Config mockConfig = mock(Config.class); - when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); - when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); - MessageStream inputStream = graphSpec.getInputStream("input"); + String inputStreamId = "input"; + String inputSystem = "input-system"; + String inputPhysicalName = "input-stream"; + String outputStreamId = "output"; + String outputSystem = "output-system"; + String outputPhysicalName = "output-stream"; + String intermediateStreamId = "jobName-jobId-partition_by-p1"; + String intermediateSystem = "intermediate-system"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystem); + + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of(inputStreamId, inputSystem, inputPhysicalName), + ImmutableTriple.of(outputStreamId, outputSystem, outputPhysicalName) + )); + configs.putAll(streamConfigs); + Config config = new MapConfig(configs); + + StreamGraphSpec graphSpec = new StreamGraphSpec(config); + MessageStream inputStream = graphSpec.getInputStream(inputStreamId); OutputStream> outputStream = graphSpec - .getOutputStream("output", KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class))); + .getOutputStream(outputStreamId, KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class))); inputStream .partitionBy(Object::hashCode, Object::toString, @@ -291,12 +317,12 @@ public void testPartitionByChain() { when(taskModel.getSystemStreamPartitions()).thenReturn(Collections.emptySet()); when(mockTaskContext.getJobModel()).thenReturn(jobModel); SamzaContainerContext containerContext = - new SamzaContainerContext("0", mockConfig, Collections.singleton(new TaskName("task 0")), new MetricsRegistryMap()); + new SamzaContainerContext("0", config, Collections.singleton(new TaskName("task 0")), new MetricsRegistryMap()); when(mockTaskContext.getSamzaContainerContext()).thenReturn(containerContext); OperatorImplGraph opImplGraph = - new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mockConfig, mockTaskContext, mock(Clock.class)); + new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), config, mockTaskContext, mock(Clock.class)); - InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream")); + InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream(inputSystem, inputPhysicalName)); assertEquals(1, inputOpImpl.registeredOperators.size()); OperatorImpl partitionByOpImpl = (PartitionByOperatorImpl) inputOpImpl.registeredOperators.iterator().next(); @@ -304,7 +330,7 @@ public void testPartitionByChain() { assertEquals(OpCode.PARTITION_BY, partitionByOpImpl.getOperatorSpec().getOpCode()); InputOperatorImpl repartitionedInputOpImpl = - opImplGraph.getInputOperator(new SystemStream("intermediate-system", "intermediate-stream")); + opImplGraph.getInputOperator(new SystemStream(intermediateSystem, intermediateStreamId)); assertEquals(1, repartitionedInputOpImpl.registeredOperators.size()); OperatorImpl sendToOpImpl = (OutputOperatorImpl) repartitionedInputOpImpl.registeredOperators.iterator().next(); @@ -314,18 +340,19 @@ public void testPartitionByChain() { @Test public void testBroadcastChain() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system")); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String inputStreamId = "input"; + Config config = StreamUtil.toStreamConfigs( + ImmutableList.of(ImmutableTriple.of("input", "input-system", "input-stream"))); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream inputStream = graphSpec.getInputStream("input"); + MessageStream inputStream = graphSpec.getInputStream(inputStreamId); inputStream.filter(mock(FilterFunction.class)); inputStream.map(mock(MapFunction.class)); TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); OperatorImplGraph opImplGraph = - new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mockTaskContext, mock(Clock.class)); + new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), config, mockTaskContext, mock(Clock.class)); InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream")); assertEquals(2, inputOpImpl.registeredOperators.size()); @@ -337,12 +364,10 @@ public void testBroadcastChain() { @Test public void testMergeChain() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec(eq("input"))) - .thenReturn(new StreamSpec("input", "input-stream", "input-system")); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class)); + String inputStreamId = "input"; + StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - MessageStream inputStream = graphSpec.getInputStream("input"); + MessageStream inputStream = graphSpec.getInputStream(inputStreamId); MessageStream stream1 = inputStream.filter(mock(FilterFunction.class)); MessageStream stream2 = inputStream.map(mock(MapFunction.class)); MessageStream mergedStream = stream1.merge(Collections.singleton(stream2)); @@ -372,20 +397,26 @@ public void testMergeChain() { @Test public void testJoinChain() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec(eq("input1"))).thenReturn(new StreamSpec("input1", "input-stream1", "input-system")); - when(mockRunner.getStreamSpec(eq("input2"))).thenReturn(new StreamSpec("input2", "input-stream2", "input-system")); - Config mockConfig = mock(Config.class); - when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); - when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + String inputStreamId1 = "input1"; + String inputStreamId2 = "input2"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of("input1", "input-system", "input-stream1"), + ImmutableTriple.of("input2", "input-system", "input-stream2") + )); + configs.putAll(streamConfigs); + Config config = new MapConfig(configs); + StreamGraphSpec graphSpec = new StreamGraphSpec(config); Integer joinKey = new Integer(1); Function keyFn = (Function & Serializable) m -> joinKey; JoinFunction testJoinFunction = new TestJoinFunction("jobName-jobId-join-j1", (BiFunction & Serializable) (m1, m2) -> KV.of(m1, m2), keyFn, keyFn); - MessageStream inputStream1 = graphSpec.getInputStream("input1", new NoOpSerde<>()); - MessageStream inputStream2 = graphSpec.getInputStream("input2", new NoOpSerde<>()); + MessageStream inputStream1 = graphSpec.getInputStream(inputStreamId1, new NoOpSerde<>()); + MessageStream inputStream2 = graphSpec.getInputStream(inputStreamId2, new NoOpSerde<>()); inputStream1.join(inputStream2, testJoinFunction, mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j1"); @@ -398,7 +429,7 @@ public void testJoinChain() { KeyValueStore mockRightStore = mock(KeyValueStore.class); when(mockTaskContext.getStore(eq("jobName-jobId-join-j1-R"))).thenReturn(mockRightStore); OperatorImplGraph opImplGraph = - new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mockConfig, mockTaskContext, mock(Clock.class)); + new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), config, mockTaskContext, mock(Clock.class)); // verify that join function is initialized once. assertEquals(TestJoinFunction.getInstanceByTaskName(mockTaskName, "jobName-jobId-join-j1").numInitCalled, 1); @@ -434,18 +465,17 @@ public void testJoinChain() { @Test public void testOperatorGraphInitAndClose() { - ApplicationRunner mockRunner = mock(ApplicationRunner.class); - when(mockRunner.getStreamSpec("input1")).thenReturn(new StreamSpec("input1", "input-stream1", "input-system")); - when(mockRunner.getStreamSpec("input2")).thenReturn(new StreamSpec("input2", "input-stream2", "input-system")); + String inputStreamId1 = "input1"; + String inputStreamId2 = "input2"; Config mockConfig = mock(Config.class); TaskName mockTaskName = mock(TaskName.class); TaskContextImpl mockContext = mock(TaskContextImpl.class); when(mockContext.getTaskName()).thenReturn(mockTaskName); when(mockContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); - MessageStream inputStream1 = graphSpec.getInputStream("input1"); - MessageStream inputStream2 = graphSpec.getInputStream("input2"); + MessageStream inputStream1 = graphSpec.getInputStream(inputStreamId1); + MessageStream inputStream2 = graphSpec.getInputStream(inputStreamId2); Function mapFn = (Function & Serializable) m -> m; inputStream1.map(new TestMapFunction("1", mapFn)) @@ -472,152 +502,4 @@ public void testOperatorGraphInitAndClose() { assertEquals(closedOperators.get(2), "2"); assertEquals(closedOperators.get(3), "1"); } - - @Test - public void testGetStreamToConsumerTasks() { - String system = "test-system"; - String stream0 = "test-stream-0"; - String stream1 = "test-stream-1"; - - SystemStreamPartition ssp0 = new SystemStreamPartition(system, stream0, new Partition(0)); - SystemStreamPartition ssp1 = new SystemStreamPartition(system, stream0, new Partition(1)); - SystemStreamPartition ssp2 = new SystemStreamPartition(system, stream1, new Partition(0)); - - TaskName task0 = new TaskName("Task 0"); - TaskName task1 = new TaskName("Task 1"); - Set ssps = new HashSet<>(); - ssps.add(ssp0); - ssps.add(ssp2); - TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0)); - ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0)); - TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1)); - ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1)); - - Map cms = new HashMap<>(); - cms.put(cm0.getProcessorId(), cm0); - cms.put(cm1.getProcessorId(), cm1); - - JobModel jobModel = new JobModel(new MapConfig(), cms, null); - Multimap streamToTasks = OperatorImplGraph.getStreamToConsumerTasks(jobModel); - assertEquals(streamToTasks.get(ssp0.getSystemStream()).size(), 2); - assertEquals(streamToTasks.get(ssp2.getSystemStream()).size(), 1); - } - - @Test - public void testGetOutputToInputStreams() { - Map configMap = new HashMap<>(); - configMap.put(JobConfig.JOB_NAME(), "test-app"); - configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); - Config config = new MapConfig(configMap); - - /** - * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value. - * - * input1 -> map -> join -> partitionBy (10) -> output1 - * | - * input2 -> filter -| - * | - * input3 -> filter -> partitionBy -> map -> join -> output2 - * - */ - StreamSpec input1 = new StreamSpec("input1", "input1", "system1"); - StreamSpec input2 = new StreamSpec("input2", "input2", "system2"); - StreamSpec input3 = new StreamSpec("input3", "input3", "system2"); - - StreamSpec output1 = new StreamSpec("output1", "output1", "system1"); - StreamSpec output2 = new StreamSpec("output2", "output2", "system2"); - - ApplicationRunner runner = mock(ApplicationRunner.class); - when(runner.getStreamSpec("input1")).thenReturn(input1); - when(runner.getStreamSpec("input2")).thenReturn(input2); - when(runner.getStreamSpec("input3")).thenReturn(input3); - when(runner.getStreamSpec("output1")).thenReturn(output1); - when(runner.getStreamSpec("output2")).thenReturn(output2); - - // intermediate streams used in tests - StreamSpec int1 = new StreamSpec("test-app-1-partition_by-p2", "test-app-1-partition_by-p2", "default-system"); - StreamSpec int2 = new StreamSpec("test-app-1-partition_by-p1", "test-app-1-partition_by-p1", "default-system"); - when(runner.getStreamSpec("test-app-1-partition_by-p2")).thenReturn(int1); - when(runner.getStreamSpec("test-app-1-partition_by-p1")).thenReturn(int2); - - StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config); - MessageStream messageStream1 = graphSpec.getInputStream("input1").map(m -> m); - MessageStream messageStream2 = graphSpec.getInputStream("input2").filter(m -> true); - MessageStream messageStream3 = - graphSpec.getInputStream("input3") - .filter(m -> true) - .partitionBy(m -> "hehe", m -> m, "p1") - .map(m -> m); - OutputStream outputStream1 = graphSpec.getOutputStream("output1"); - OutputStream outputStream2 = graphSpec.getOutputStream("output2"); - - messageStream1 - .join(messageStream2, mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") - .partitionBy(m -> "haha", m -> m, "p2") - .sendTo(outputStream1); - messageStream3 - .join(messageStream2, mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") - .sendTo(outputStream2); - - Multimap outputToInput = - OperatorImplGraph.getIntermediateToInputStreamsMap(graphSpec.getOperatorSpecGraph()); - Collection inputs = outputToInput.get(int1.toSystemStream()); - assertEquals(inputs.size(), 2); - assertTrue(inputs.contains(input1.toSystemStream())); - assertTrue(inputs.contains(input2.toSystemStream())); - - inputs = outputToInput.get(int2.toSystemStream()); - assertEquals(inputs.size(), 1); - assertEquals(inputs.iterator().next(), input3.toSystemStream()); - } - - @Test - public void testGetProducerTaskCountForIntermediateStreams() { - /** - * the task assignment looks like the following: - * - * input1 -----> task0, task1 -----> int1 - * ^ - * input2 ------> task1, task2--------| - * v - * input3 ------> task1 -----------> int2 - * - */ - - SystemStream input1 = new SystemStream("system1", "intput1"); - SystemStream input2 = new SystemStream("system2", "intput2"); - SystemStream input3 = new SystemStream("system2", "intput3"); - - SystemStream int1 = new SystemStream("system1", "int1"); - SystemStream int2 = new SystemStream("system1", "int2"); - - - String task0 = "Task 0"; - String task1 = "Task 1"; - String task2 = "Task 2"; - - Multimap streamToConsumerTasks = HashMultimap.create(); - streamToConsumerTasks.put(input1, task0); - streamToConsumerTasks.put(input1, task1); - streamToConsumerTasks.put(input2, task1); - streamToConsumerTasks.put(input2, task2); - streamToConsumerTasks.put(input3, task1); - streamToConsumerTasks.put(int1, task0); - streamToConsumerTasks.put(int1, task1); - streamToConsumerTasks.put(int2, task0); - - Multimap intermediateToInputStreams = HashMultimap.create(); - intermediateToInputStreams.put(int1, input1); - intermediateToInputStreams.put(int1, input2); - - intermediateToInputStreams.put(int2, input2); - intermediateToInputStreams.put(int2, input3); - - Map counts = OperatorImplGraph.getProducerTaskCountForIntermediateStreams( - streamToConsumerTasks, intermediateToInputStreams); - assertTrue(counts.get(int1) == 3); - assertTrue(counts.get(int2) == 2); - } } diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java new file mode 100644 index 0000000000..abd2a08654 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.impl; + +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Multimap; + +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.tuple.ImmutableTriple; +import org.apache.samza.Partition; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraphSpec; +import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.serializers.Serde; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.StreamUtil; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + +public class TestOperatorImplGraphUtil { + @Test + public void testGetStreamToConsumerTasks() { + String system = "test-system"; + String streamId0 = "test-stream-0"; + String streamId1 = "test-stream-1"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "test-app"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); + + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of(streamId0, system, streamId0), + ImmutableTriple.of(streamId1, system, streamId1) + )); + configs.putAll(streamConfigs); + Config config = new MapConfig(configs); + + SystemStreamPartition ssp0 = new SystemStreamPartition(system, streamId0, new Partition(0)); + SystemStreamPartition ssp1 = new SystemStreamPartition(system, streamId0, new Partition(1)); + SystemStreamPartition ssp2 = new SystemStreamPartition(system, streamId1, new Partition(0)); + + TaskName task0 = new TaskName("Task 0"); + TaskName task1 = new TaskName("Task 1"); + Set ssps = new HashSet<>(); + ssps.add(ssp0); + ssps.add(ssp2); + TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0)); + ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0)); + TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1)); + ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1)); + + Map cms = new HashMap<>(); + cms.put(cm0.getProcessorId(), cm0); + cms.put(cm1.getProcessorId(), cm1); + + JobModel jobModel = new JobModel(config, cms, null); + OperatorImplGraph.OperatorImplGraphUtil util = new OperatorImplGraph.OperatorImplGraphUtil(config); + Multimap streamToTasks = util.getStreamToConsumerTasks(jobModel); + assertEquals(streamToTasks.get(ssp0.getSystemStream()).size(), 2); + assertEquals(streamToTasks.get(ssp2.getSystemStream()).size(), 1); + } + + @Test + public void testGetOutputToInputStreams() { + String inputStreamId1 = "input1"; + String inputStreamId2 = "input2"; + String inputStreamId3 = "input3"; + String inputSystem = "input-system"; + + String outputStreamId1 = "output1"; + String outputStreamId2 = "output2"; + String outputSystem = "output-system"; + + String intStreamId1 = "test-app-1-partition_by-p1"; + String intStreamId2 = "test-app-1-partition_by-p2"; + String intSystem = "test-system"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "test-app"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intSystem); + + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of(inputStreamId1, inputSystem, inputStreamId1), + ImmutableTriple.of(inputStreamId2, inputSystem, inputStreamId2), + ImmutableTriple.of(inputStreamId3, inputSystem, inputStreamId3), + ImmutableTriple.of(outputStreamId1, outputSystem, outputStreamId1), + ImmutableTriple.of(outputStreamId2, outputSystem, outputStreamId2) + )); + configs.putAll(streamConfigs); + Config config = new MapConfig(configs); + + StreamGraphSpec graphSpec = new StreamGraphSpec(config); + MessageStream messageStream1 = graphSpec.getInputStream(inputStreamId1).map(m -> m); + MessageStream messageStream2 = graphSpec.getInputStream(inputStreamId2).filter(m -> true); + MessageStream messageStream3 = + graphSpec.getInputStream(inputStreamId3) + .filter(m -> true) + .partitionBy(m -> "m", m -> m, "p1") + .map(m -> m); + OutputStream outputStream1 = graphSpec.getOutputStream(outputStreamId1); + OutputStream outputStream2 = graphSpec.getOutputStream(outputStreamId2); + + messageStream1 + .join(messageStream2, mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") + .partitionBy(m -> "m", m -> m, "p2") + .sendTo(outputStream1); + messageStream3 + .join(messageStream2, mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") + .sendTo(outputStream2); + + OperatorImplGraph.OperatorImplGraphUtil util = new OperatorImplGraph.OperatorImplGraphUtil(config); + Multimap outputToInput = + util.getIntermediateToInputStreamsMap(graphSpec.getOperatorSpecGraph()); + Collection inputs = outputToInput.get(new SystemStream(intSystem, intStreamId2)); + assertEquals(inputs.size(), 2); + assertTrue(inputs.contains(new SystemStream(inputSystem, inputStreamId1))); + assertTrue(inputs.contains(new SystemStream(inputSystem, inputStreamId2))); + + inputs = outputToInput.get(new SystemStream(intSystem, intStreamId1)); + assertEquals(inputs.size(), 1); + assertEquals(inputs.iterator().next(), new SystemStream(inputSystem, inputStreamId3)); + } + + @Test + public void testGetProducerTaskCountForIntermediateStreams() { + String inputStreamId1 = "input1"; + String inputStreamId2 = "input2"; + String inputStreamId3 = "input3"; + String inputSystem1 = "system1"; + String inputSystem2 = "system2"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "test-app"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), inputSystem1); + + Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( + ImmutableTriple.of(inputStreamId1, inputSystem1, inputStreamId1), + ImmutableTriple.of(inputStreamId2, inputSystem2, inputStreamId2), + ImmutableTriple.of(inputStreamId3, inputSystem2, inputStreamId3) + )); + configs.putAll(streamConfigs); + Config config = new MapConfig(configs); + + SystemStream input1 = new SystemStream("system1", "intput1"); + SystemStream input2 = new SystemStream("system2", "intput2"); + SystemStream input3 = new SystemStream("system2", "intput3"); + + SystemStream int1 = new SystemStream("system1", "int1"); + SystemStream int2 = new SystemStream("system1", "int2"); + + + /** + * the task assignment looks like the following: + * + * input1 -----> task0, task1 -----> int1 + * ^ + * input2 ------> task1, task2--------| + * v + * input3 ------> task1 -----------> int2 + * + */ + String task0 = "Task 0"; + String task1 = "Task 1"; + String task2 = "Task 2"; + + Multimap streamToConsumerTasks = HashMultimap.create(); + streamToConsumerTasks.put(input1, task0); + streamToConsumerTasks.put(input1, task1); + streamToConsumerTasks.put(input2, task1); + streamToConsumerTasks.put(input2, task2); + streamToConsumerTasks.put(input3, task1); + streamToConsumerTasks.put(int1, task0); + streamToConsumerTasks.put(int1, task1); + streamToConsumerTasks.put(int2, task0); + + Multimap intermediateToInputStreams = HashMultimap.create(); + intermediateToInputStreams.put(int1, input1); + intermediateToInputStreams.put(int1, input2); + + intermediateToInputStreams.put(int2, input2); + intermediateToInputStreams.put(int2, input3); + + OperatorImplGraph.OperatorImplGraphUtil util = new OperatorImplGraph.OperatorImplGraphUtil(config); + Map counts = util.getProducerTaskCountForIntermediateStreams( + streamToConsumerTasks, intermediateToInputStreams); + assertTrue(counts.get(int1) == 3); + assertTrue(counts.get(int2) == 2); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java index 9741fc4b69..0ef6680c66 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java @@ -42,13 +42,11 @@ import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.OutgoingMessageEnvelope; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.task.MessageCollector; @@ -80,7 +78,6 @@ public class TestWindowOperator { private final List integers = ImmutableList.of(1, 2, 1, 2, 1, 2, 1, 2, 3); private Config config; private TaskContextImpl taskContext; - private ApplicationRunner runner; @Before public void setup() throws Exception { @@ -88,19 +85,16 @@ public void setup() throws Exception { when(config.get(JobConfig.JOB_NAME())).thenReturn("jobName"); when(config.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); taskContext = mock(TaskContextImpl.class); - runner = mock(ApplicationRunner.class); Serde storeKeySerde = new TimeSeriesKeySerde(new IntegerSerde()); Serde storeValSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); when(taskContext.getSystemStreamPartitions()).thenReturn(ImmutableSet - .of(new SystemStreamPartition("kafka", "integers", new Partition(0)))); + .of(new SystemStreamPartition("kafka", "integTestExecutionPlannerers", new Partition(0)))); when(taskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); when(taskContext.getStore("jobName-jobId-window-w1")) .thenReturn(new TestInMemoryStore<>(storeKeySerde, storeValSerde)); - when(runner.getStreamSpec("integers")).thenReturn(new StreamSpec("integers", "integers", "kafka")); Map mapConfig = new HashMap<>(); - mapConfig.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner"); mapConfig.put("job.default.system", "kafka"); mapConfig.put("job.name", "jobName"); mapConfig.put("job.id", "jobId"); @@ -552,7 +546,7 @@ public void testEndOfStreamFlushesWithNoTriggerFirings() throws Exception { private StreamGraphSpec getKeyedTumblingWindowStreamGraph(AccumulationMode mode, Duration duration, Trigger> earlyTrigger) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(runner, config); + StreamGraphSpec graph = new StreamGraphSpec(config); KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); graph.getInputStream("integers", kvSerde) @@ -568,7 +562,7 @@ private StreamGraphSpec getKeyedTumblingWindowStreamGraph(AccumulationMode mode, private StreamGraphSpec getTumblingWindowStreamGraph(AccumulationMode mode, Duration duration, Trigger> earlyTrigger) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(runner, config); + StreamGraphSpec graph = new StreamGraphSpec(config); KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); graph.getInputStream("integers", kvSerde) @@ -582,7 +576,7 @@ private StreamGraphSpec getTumblingWindowStreamGraph(AccumulationMode mode, } private StreamGraphSpec getKeyedSessionWindowStreamGraph(AccumulationMode mode, Duration duration) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(runner, config); + StreamGraphSpec graph = new StreamGraphSpec(config); KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); graph.getInputStream("integers", kvSerde) @@ -597,7 +591,7 @@ private StreamGraphSpec getKeyedSessionWindowStreamGraph(AccumulationMode mode, private StreamGraphSpec getAggregateTumblingWindowStreamGraph(AccumulationMode mode, Duration timeDuration, Trigger earlyTrigger) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(runner, config); + StreamGraphSpec graph = new StreamGraphSpec(config); MessageStream> integers = graph.getInputStream("integers", KVSerde.of(new IntegerSerde(), new IntegerSerde())); diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java b/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java index b39b0d070b..7704a5b034 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java @@ -31,7 +31,6 @@ import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.serializers.SerializableSerde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableSpec; import static org.junit.Assert.*; @@ -105,8 +104,8 @@ private static void assertClonedTableImpl(TableImpl oTable, TableImpl nTable) { assertEquals(oTable.getTableSpec(), nTable.getTableSpec()); } - private static void assertClonedOutputs(Map originalOutputs, - Map clonedOutputs) { + private static void assertClonedOutputs(Map originalOutputs, + Map clonedOutputs) { assertEquals(originalOutputs.size(), clonedOutputs.size()); assertEquals(originalOutputs.keySet(), clonedOutputs.keySet()); Iterator oIter = originalOutputs.values().iterator(); @@ -117,12 +116,11 @@ private static void assertClonedOutputs(Map origin private static void assertClonedOutputImpl(OutputStreamImpl oOutput, OutputStreamImpl nOutput) { assertNotEquals(oOutput, nOutput); assertEquals(oOutput.isKeyed(), nOutput.isKeyed()); - assertEquals(oOutput.getSystemStream(), nOutput.getSystemStream()); - assertEquals(oOutput.getStreamSpec(), nOutput.getStreamSpec()); + assertEquals(oOutput.getStreamId(), nOutput.getStreamId()); } - private static void assertClonedInputs(Map originalInputs, - Map clonedInputs) { + private static void assertClonedInputs(Map originalInputs, + Map clonedInputs) { assertEquals(originalInputs.size(), clonedInputs.size()); assertEquals(originalInputs.keySet(), clonedInputs.keySet()); Iterator oIter = originalInputs.values().iterator(); @@ -134,7 +132,7 @@ private static void assertClonedInputOp(InputOperatorSpec originalInput, InputOp assertNotEquals(originalInput, clonedInput); assertEquals(originalInput.getOpId(), clonedInput.getOpId()); assertEquals(originalInput.getOpCode(), clonedInput.getOpCode()); - assertEquals(originalInput.getStreamSpec(), clonedInput.getStreamSpec()); + assertEquals(originalInput.getStreamId(), clonedInput.getStreamId()); assertEquals(originalInput.isKeyed(), clonedInput.isKeyed()); } diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java index cb221b0640..b27c944c21 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java @@ -40,7 +40,6 @@ import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.StringSerde; -import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableSpec; import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; @@ -50,7 +49,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; /** @@ -230,9 +228,8 @@ public byte[] toBytes(Object object) { } }; - StreamSpec mockStreamSpec = mock(StreamSpec.class); InputOperatorSpec inputOperatorSpec = new InputOperatorSpec<>( - mockStreamSpec, new StringSerde("UTF-8"), objSerde, true, "op0"); + "mockStreamId", new StringSerde("UTF-8"), objSerde, true, "op0"); InputOperatorSpec inputOpCopy = (InputOperatorSpec) OperatorSpecTestUtils.copyOpSpec(inputOperatorSpec); assertNotEquals("Expected deserialized copy of operator spec should not be the same as the original operator spec", inputOperatorSpec, inputOpCopy); @@ -254,9 +251,8 @@ public byte[] toBytes(Object object) { return new byte[0]; } }; - StreamSpec mockStreamSpec = mock(StreamSpec.class); - OutputStreamImpl> outputStrmImpl = new OutputStreamImpl<>(mockStreamSpec, new StringSerde("UTF-8"), objSerde, true); - OutputOperatorSpec> outputOperatorSpec = new OutputOperatorSpec>(outputStrmImpl, "op0"); + OutputStreamImpl> outputStrmImpl = new OutputStreamImpl<>("mockStreamId", new StringSerde("UTF-8"), objSerde, true); + OutputOperatorSpec> outputOperatorSpec = new OutputOperatorSpec<>(outputStrmImpl, "op0"); OutputOperatorSpec> outputOpCopy = (OutputOperatorSpec>) OperatorSpecTestUtils .copyOpSpec(outputOperatorSpec); assertNotEquals("Expected deserialized copy of operator spec should not be the same as the original operator spec", outputOperatorSpec, outputOpCopy); @@ -276,10 +272,10 @@ public void testSinkOperatorSpec() { public void testJoinOperatorSpec() { InputOperatorSpec leftOpSpec = new InputOperatorSpec<>( - new StreamSpec("test-input-1", "test-input-1", "kafka"), new NoOpSerde<>(), + "test-input-1", new NoOpSerde<>(), new NoOpSerde<>(), false, "op0"); InputOperatorSpec rightOpSpec = new InputOperatorSpec<>( - new StreamSpec("test-input-2", "test-input-2", "kafka"), new NoOpSerde<>(), + "test-input-2", new NoOpSerde<>(), new NoOpSerde<>(), false, "op1"); Serde objSerde = new Serde() { @@ -341,14 +337,14 @@ public void testSendToTableOperatorSpec() { @Test public void testBroadcastOperatorSpec() { OutputStreamImpl outputStream = - new OutputStreamImpl<>(new StreamSpec("output-0", "outputStream-0", "kafka"), new StringSerde("UTF-8"), new JsonSerdeV2(), true); + new OutputStreamImpl<>("output-0", new StringSerde("UTF-8"), new JsonSerdeV2(), true); BroadcastOperatorSpec broadcastOpSpec = new BroadcastOperatorSpec<>(outputStream, "broadcast-1"); BroadcastOperatorSpec broadcastOpCopy = (BroadcastOperatorSpec) OperatorSpecTestUtils .copyOpSpec(broadcastOpSpec); assertNotEquals(broadcastOpCopy, broadcastOpSpec); assertEquals(broadcastOpCopy.getOpId(), broadcastOpSpec.getOpId()); assertTrue(broadcastOpCopy.getOutputStream() != broadcastOpSpec.getOutputStream()); - assertEquals(broadcastOpCopy.getOutputStream().getSystemStream(), broadcastOpSpec.getOutputStream().getSystemStream()); + assertEquals(broadcastOpCopy.getOutputStream().getStreamId(), broadcastOpSpec.getOutputStream().getStreamId()); assertEquals(broadcastOpCopy.getOutputStream().isKeyed(), broadcastOpSpec.getOutputStream().isKeyed()); } diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java index 00ec1761c8..9bbcbfaa01 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java @@ -29,9 +29,7 @@ import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.NoOpSerde; -import org.apache.samza.system.StreamSpec; import org.junit.Before; import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; @@ -45,7 +43,6 @@ */ public class TestPartitionByOperatorSpec { - private final ApplicationRunner mockRunner = mock(ApplicationRunner.class); private final Config mockConfig = mock(Config.class); private final String testInputId = "test-input-1"; private final String testJobName = "testJob"; @@ -93,12 +90,7 @@ public Long getOutputWatermark() { public void setup() { when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn(testJobName); when(mockConfig.get(JobConfig.JOB_ID(), "1")).thenReturn(testJobId); - StreamSpec inputSpec1 = new StreamSpec(testInputId, testInputId, "kafka"); - when(mockRunner.getStreamSpec(testInputId)).thenReturn(inputSpec1); - String intermediateStreamName = String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName); - StreamSpec intermediateSpec1 = new StreamSpec(intermediateStreamName, intermediateStreamName, "kafka"); - when(mockRunner.getStreamSpec(intermediateStreamName)).thenReturn(intermediateSpec1); - graphSpec = new StreamGraphSpec(mockRunner, mockConfig); + graphSpec = new StreamGraphSpec(mockConfig); } @Test @@ -109,7 +101,7 @@ public void testPartitionBy() { MessageStream> reparStream = inputStream.partitionBy(keyFn, valueFn, testReparStreamName); InputOperatorSpec inputOpSpec = (InputOperatorSpec) Whitebox.getInternalState(reparStream, "operatorSpec"); - assertEquals(inputOpSpec.getStreamSpec().getId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); + assertEquals(inputOpSpec.getStreamId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde); assertTrue(inputOpSpec.isKeyed()); @@ -121,7 +113,7 @@ public void testPartitionBy() { assertEquals(reparOpSpec.getOpId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); assertEquals(reparOpSpec.getKeyFunction(), keyFn); assertEquals(reparOpSpec.getValueFunction(), valueFn); - assertEquals(reparOpSpec.getOutputStream().getStreamSpec(), new StreamSpec(reparOpSpec.getOpId(), reparOpSpec.getOpId(), "kafka")); + assertEquals(reparOpSpec.getOutputStream().getStreamId(), reparOpSpec.getOpId()); assertNull(reparOpSpec.getTimerFn()); assertNull(reparOpSpec.getWatermarkFn()); } diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index e2077727c7..0b91315f4f 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -27,7 +27,6 @@ import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.testUtils.TestAsyncStreamTask; import org.apache.samza.testUtils.TestStreamTask; import org.junit.Test; @@ -46,8 +45,6 @@ */ public class TestTaskFactoryUtil { - private final ApplicationRunner mockRunner = mock(ApplicationRunner.class); - @Test public void testStreamTaskClass() { Config config = new MapConfig(new HashMap() { @@ -81,7 +78,7 @@ public void testCreateStreamApplication() throws Exception { }); StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config); assertNotNull(streamApp); - StreamGraphSpec graph = new StreamGraphSpec(mockRunner, config); + StreamGraphSpec graph = new StreamGraphSpec(config); streamApp.init(graph, config); Object retFactory = TaskFactoryUtil.createTaskFactory(graph.getOperatorSpecGraph(), null); assertTrue(retFactory instanceof StreamTaskFactory); diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/util/TestStreamUtil.java similarity index 57% rename from samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java rename to samza-core/src/test/java/org/apache/samza/util/TestStreamUtil.java index b8d3f150c6..ac075cd713 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/util/TestStreamUtil.java @@ -16,22 +16,22 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime; +package org.apache.samza.util; import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; -import org.apache.samza.job.ApplicationStatus; import org.apache.samza.system.StreamSpec; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; -public class TestAbstractApplicationRunner { + +public class TestStreamUtil { private static final String STREAM_ID = "t3st-Stream_Id"; private static final String STREAM_ID_INVALID = "test#Str3amId!"; @@ -40,26 +40,19 @@ public class TestAbstractApplicationRunner { private static final String TEST_PHYSICAL_NAME_SPECIAL_CHARS = "test://Physical.Name?"; private static final String TEST_SYSTEM = "t3st-System_Name"; - private static final String TEST_SYSTEM2 = "testSystemName2"; private static final String TEST_SYSTEM_INVALID = "test:System!Name@"; private static final String TEST_DEFAULT_SYSTEM = "testDefaultSystemName"; - @Test(expected = NullPointerException.class) - public void testConfigValidation() { - new TestAbstractApplicationRunnerImpl(null); - } - // The physical name should be pulled from the StreamConfig.PHYSICAL_NAME property value. @Test - public void testgetStreamWithPhysicalNameInConfig() { + public void testGetStreamWithPhysicalNameInConfig() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), TEST_SYSTEM); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName()); } @@ -67,80 +60,74 @@ public void testgetStreamWithPhysicalNameInConfig() { // The streamId should be used as the physicalName when the physical name is not specified. // NOTE: its either this, set to null, or exception. This seems better for backward compatibility and API brevity. @Test - public void testgetStreamWithoutPhysicalNameInConfig() { + public void testGetStreamWithoutPhysicalNameInConfig() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.SYSTEM(), TEST_SYSTEM); + StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(STREAM_ID, spec.getPhysicalName()); } // If the system is specified at the stream scope, use it @Test - public void testgetStreamWithSystemAtStreamScopeInConfig() { + public void testGetStreamWithSystemAtStreamScopeInConfig() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), TEST_SYSTEM); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_SYSTEM, spec.getSystemName()); } // If system isn't specified at stream scope, use the default system @Test - public void testgetStreamWithSystemAtDefaultScopeInConfig() { + public void testGetStreamWithSystemAtDefaultScopeInConfig() { Config config = addConfigs(buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME), - JobConfig.JOB_DEFAULT_SYSTEM(), TEST_DEFAULT_SYSTEM); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME), + JobConfig.JOB_DEFAULT_SYSTEM(), TEST_DEFAULT_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_DEFAULT_SYSTEM, spec.getSystemName()); } // Stream scope should override default scope @Test - public void testgetStreamWithSystemAtBothScopesInConfig() { + public void testGetStreamWithSystemAtBothScopesInConfig() { Config config = addConfigs(buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), TEST_SYSTEM), - JobConfig.JOB_DEFAULT_SYSTEM(), TEST_DEFAULT_SYSTEM); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), TEST_SYSTEM), + JobConfig.JOB_DEFAULT_SYSTEM(), TEST_DEFAULT_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_SYSTEM, spec.getSystemName()); } // System is required. Throw if it cannot be determined. @Test(expected = IllegalArgumentException.class) - public void testgetStreamWithOutSystemInConfig() { + public void testGetStreamWithOutSystemInConfig() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_SYSTEM, spec.getSystemName()); } // The properties in the config "streams.{streamId}.*" should be passed through to the spec. @Test - public void testgetStreamPropertiesPassthrough() { + public void testGetStreamPropertiesPassthrough() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), TEST_SYSTEM, - "systemProperty1", "systemValue1", - "systemProperty2", "systemValue2", - "systemProperty3", "systemValue3"); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), TEST_SYSTEM, + "systemProperty1", "systemValue1", + "systemProperty2", "systemValue2", + "systemProperty3", "systemValue3"); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); Map properties = spec.getConfig(); assertEquals(3, properties.size()); @@ -156,14 +143,13 @@ public void testgetStreamPropertiesPassthrough() { @Test public void testGetStreamSamzaPropertiesOmitted() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), TEST_SYSTEM, - "systemProperty1", "systemValue1", - "systemProperty2", "systemValue2", - "systemProperty3", "systemValue3"); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), TEST_SYSTEM, + "systemProperty1", "systemValue1", + "systemProperty2", "systemValue2", + "systemProperty3", "systemValue3"); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); Map properties = spec.getConfig(); assertEquals(3, properties.size()); @@ -185,8 +171,7 @@ public void testStreamConfigOverrides() { sysStreamPrefix + "systemProperty4", "systemValue4", sysStreamPrefix + "systemProperty2", "systemValue8"); - AbstractApplicationRunner env = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = env.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); Map properties = spec.getConfig(); assertEquals(4, properties.size()); @@ -203,10 +188,9 @@ public void testStreamConfigOverridesWithSystemDefaults() { "segment.bytes", "5309"), String.format("systems.%s.default.stream.replication.factor", TEST_SYSTEM), "4", // System default property String.format("systems.%s.default.stream.segment.bytest", TEST_SYSTEM), "867" - ); + ); - AbstractApplicationRunner env = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = env.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); Map properties = spec.getConfig(); assertEquals(3, properties.size()); @@ -218,11 +202,10 @@ public void testStreamConfigOverridesWithSystemDefaults() { @Test public void testGetStreamPhysicalNameArgSimple() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2, // This should be ignored because of the explicit arg - StreamConfig.SYSTEM(), TEST_SYSTEM); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2, // This should be ignored because of the explicit arg + StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(STREAM_ID, spec.getId()); assertEquals(TEST_PHYSICAL_NAME2, spec.getPhysicalName()); @@ -233,11 +216,10 @@ public void testGetStreamPhysicalNameArgSimple() { @Test public void testGetStreamPhysicalNameArgSpecialCharacters() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME_SPECIAL_CHARS, - StreamConfig.SYSTEM(), TEST_SYSTEM); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME_SPECIAL_CHARS, + StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_PHYSICAL_NAME_SPECIAL_CHARS, spec.getPhysicalName()); } @@ -245,11 +227,10 @@ public void testGetStreamPhysicalNameArgSpecialCharacters() { @Test public void testGetStreamPhysicalNameArgNull() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), null, - StreamConfig.SYSTEM(), TEST_SYSTEM); + StreamConfig.PHYSICAL_NAME(), null, + StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertNull(spec.getPhysicalName()); } @@ -257,11 +238,10 @@ public void testGetStreamPhysicalNameArgNull() { @Test public void testGetStreamSystemNameArgValid() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, // This should be ignored because of the explicit arg - StreamConfig.SYSTEM(), TEST_SYSTEM); // This too + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, // This should be ignored because of the explicit arg + StreamConfig.SYSTEM(), TEST_SYSTEM); // This too - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - StreamSpec spec = runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(STREAM_ID, spec.getId()); assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName()); @@ -272,11 +252,10 @@ public void testGetStreamSystemNameArgValid() { @Test(expected = IllegalArgumentException.class) public void testGetStreamSystemNameArgInvalid() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), TEST_SYSTEM_INVALID); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), TEST_SYSTEM_INVALID); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - runner.getStreamSpec(STREAM_ID); + StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); } // Empty strings are NOT allowed for system name, because it's used as an identifier in the config. @@ -286,19 +265,17 @@ public void testGetStreamSystemNameArgEmpty() { StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, StreamConfig.SYSTEM(), ""); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - runner.getStreamSpec(STREAM_ID); + StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); } // Null is not allowed IllegalArgumentException system name. @Test(expected = IllegalArgumentException.class) public void testGetStreamSystemNameArgNull() { Config config = buildStreamConfig(STREAM_ID, - StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, - StreamConfig.SYSTEM(), null); + StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, + StreamConfig.SYSTEM(), null); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - runner.getStreamSpec(STREAM_ID); + StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); } // Special characters are NOT allowed for streamId, because it's used as an identifier in the config. @@ -307,8 +284,7 @@ public void testGetStreamStreamIdInvalid() { Config config = buildStreamConfig(STREAM_ID_INVALID, StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - runner.getStreamSpec(STREAM_ID_INVALID); + StreamUtil.getStreamSpec(STREAM_ID_INVALID, new StreamConfig(config)); } // Empty strings are NOT allowed for streamId, because it's used as an identifier in the config. @@ -317,8 +293,7 @@ public void testGetStreamStreamIdEmpty() { Config config = buildStreamConfig("", StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - runner.getStreamSpec(""); + StreamUtil.getStreamSpec("", new StreamConfig(config)); } // Null is not allowed for streamId. @@ -327,8 +302,7 @@ public void testGetStreamStreamIdNull() { Config config = buildStreamConfig(null, StreamConfig.SYSTEM(), TEST_SYSTEM); - AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config); - runner.getStreamSpec(null); + StreamUtil.getStreamSpec(null, new StreamConfig(config)); } @@ -360,32 +334,4 @@ private Config addConfigs(Config original, String... kvs) { result.putAll(buildConfig(kvs)); return new MapConfig(result); } - - private class TestAbstractApplicationRunnerImpl extends AbstractApplicationRunner { - - public TestAbstractApplicationRunnerImpl(Config config) { - super(config); - } - - @Override - public void runTask() { - throw new UnsupportedOperationException("runTask is not supported in this test"); - } - - @Override - public void run(StreamApplication streamApp) { - // do nothing. We're only testing the stream creation methods at this point. - } - - @Override - public void kill(StreamApplication streamApp) { - // do nothing. We're only testing the stream creation methods at this point. - } - - @Override - public ApplicationStatus status(StreamApplication streamApp) { - // do nothing. We're only testing the stream creation methods at this point. - return null; - } - } } diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java index 217248d6a5..113dced455 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java @@ -110,7 +110,6 @@ public static KafkaStreamSpec fromSpec(StreamSpec originalSpec) { originalSpec.getSystemName(), originalSpec.getPartitionCount(), replicationFactor, - originalSpec.isBroadcast(), mapToProperties(filterUnsupportedProperties(originalSpec.getConfig()))); } @@ -125,7 +124,7 @@ public static KafkaStreamSpec fromSpec(StreamSpec originalSpec) { * @param partitionCount The number of partitions. */ public KafkaStreamSpec(String id, String topicName, String systemName, int partitionCount) { - this(id, topicName, systemName, partitionCount, DEFAULT_REPLICATION_FACTOR, false, new Properties()); + this(id, topicName, systemName, partitionCount, DEFAULT_REPLICATION_FACTOR, new Properties()); } /** @@ -146,13 +145,11 @@ public KafkaStreamSpec(String id, String topicName, String systemName, int parti * * @param replicationFactor The number of topic replicas in the Kafka cluster for durability. * - * @param isBroadcast The stream is broadcast or not. - * * @param properties A set of properties for the stream. These may be System-specfic. */ public KafkaStreamSpec(String id, String topicName, String systemName, int partitionCount, int replicationFactor, - Boolean isBroadcast, Properties properties) { - super(id, topicName, systemName, partitionCount, false, isBroadcast, propertiesToMap(properties)); + Properties properties) { + super(id, topicName, systemName, partitionCount, propertiesToMap(properties)); if (partitionCount < 1) { throw new IllegalArgumentException("Parameter 'partitionCount' must be > 0"); @@ -168,12 +165,12 @@ public KafkaStreamSpec(String id, String topicName, String systemName, int parti @Override public StreamSpec copyWithPartitionCount(int partitionCount) { return new KafkaStreamSpec(getId(), getPhysicalName(), getSystemName(), partitionCount, getReplicationFactor(), - isBroadcast(), getProperties()); + getProperties()); } public KafkaStreamSpec copyWithReplicationFactor(int replicationFactor) { return new KafkaStreamSpec(getId(), getPhysicalName(), getSystemName(), getPartitionCount(), replicationFactor, - isBroadcast(), getProperties()); + getProperties()); } /** @@ -183,7 +180,7 @@ public KafkaStreamSpec copyWithReplicationFactor(int replicationFactor) { */ public KafkaStreamSpec copyWithProperties(Properties properties) { return new KafkaStreamSpec(getId(), getPhysicalName(), getSystemName(), getPartitionCount(), getReplicationFactor(), - isBroadcast(), properties); + properties); } public int getReplicationFactor() { diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala index 07f4710c2a..26664ea9fc 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.samza.SamzaException import org.apache.samza.config.ApplicationConfig.ApplicationMode import org.apache.samza.config.SystemConfig.Config2System -import org.apache.samza.util.{Logging, Util} +import org.apache.samza.util.{Logging, StreamUtil} import scala.collection.JavaConverters._ @@ -237,7 +237,7 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) { val storeName = if (matcher.find()) matcher.group(1) else throw new SamzaException("Unable to find store name in the changelog configuration: " + changelogConfig + " with SystemStream: " + cn) storageConfig.getChangelogStream(storeName).foreach(changelogName => { - val systemStream = Util.getSystemStreamFromNames(changelogName) + val systemStream = StreamUtil.getSystemStreamFromNames(changelogName) val factoryName = config.getSystemFactory(systemStream.getSystem).getOrElse(new SamzaException("Unable to determine factory for system: " + systemStream.getSystem)) storeToChangelog += storeName -> systemStream.getStream }) diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala b/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala index 6dc2f82a87..ce4544b25e 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala @@ -21,11 +21,11 @@ package org.apache.samza.config import org.I0Itec.zkclient.ZkClient import kafka.utils.ZkUtils -import org.apache.samza.config.KafkaConfig.{ Config2Kafka, REGEX_RESOLVED_STREAMS } +import org.apache.samza.config.KafkaConfig.{Config2Kafka, REGEX_RESOLVED_STREAMS} import org.apache.samza.SamzaException -import org.apache.samza.util.Util +import org.apache.samza.util.{Logging, StreamUtil} + import collection.JavaConverters._ -import org.apache.samza.util.Logging import scala.collection._ import org.apache.samza.config.TaskConfig.Config2Task import org.apache.samza.system.SystemStream @@ -87,7 +87,7 @@ class RegExTopicGenerator extends ConfigRewriter with Logging { info("Generated config values for %d new topics" format newInputStreams.size) val inputStreams = TaskConfig.INPUT_STREAMS -> (existingInputStreams ++ newInputStreams) - .map(Util.getNameFromSystemStream) + .map(StreamUtil.getNameFromSystemStream) .toArray .sortWith(_ < _) .mkString(",") diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala index a63db03fc3..6ab4d323b2 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala @@ -486,10 +486,10 @@ class KafkaSystemAdmin( 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, - spec.isBroadcast, topicMeta.kafkaProps) + topicMeta.kafkaProps) } else if (spec.isCoordinatorStream){ new KafkaStreamSpec(spec.getId, spec.getPhysicalName, systemName, 1, coordinatorStreamReplicationFactor, - spec.isBroadcast, coordinatorStreamProperties) + coordinatorStreamProperties) } else if (intermediateStreamProperties.contains(spec.getId)) { KafkaStreamSpec.fromSpec(spec).copyWithProperties(intermediateStreamProperties(spec.getId)) } else { diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java index c00ed2d186..14d2fe6839 100644 --- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java +++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java @@ -21,20 +21,20 @@ import com.google.common.collect.ImmutableMap; import java.util.Map; import java.util.Properties; -import org.apache.samza.runtime.TestAbstractApplicationRunner; import org.apache.samza.system.StreamSpec; +import org.apache.samza.util.TestStreamUtil; import org.junit.Test; import static org.junit.Assert.*; /** - * See also the general StreamSpec tests in {@link TestAbstractApplicationRunner} + * See also the general StreamSpec tests in {@link TestStreamUtil} */ public class TestKafkaStreamSpec { @Test public void testUnsupportedConfigStrippedFromProperties() { - StreamSpec original = new StreamSpec("dummyId","dummyPhysicalName", "dummySystemName", false, ImmutableMap.of("segment.bytes", "4", "replication.factor", "7")); + StreamSpec original = new StreamSpec("dummyId","dummyPhysicalName", "dummySystemName", ImmutableMap.of("segment.bytes", "4", "replication.factor", "7")); // First verify the original assertEquals("7", original.get("replication.factor")); diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala index 71718b01a7..8d92f4d332 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala @@ -113,7 +113,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { Mockito.doThrow(new RuntimeException()).when(mockKafkaProducer).flush(taskName.getTaskName) val props = new org.apache.samza.config.KafkaConfig(config).getCheckpointTopicProperties() - val spec = new KafkaStreamSpec("id", checkpointTopic, checkpointSystemName, 1, 1, false, props) + val spec = new KafkaStreamSpec("id", checkpointTopic, checkpointSystemName, 1, 1, props) val checkPointManager = new KafkaCheckpointManager(spec, new MockSystemFactory, false, config, new NoOpMetricsRegistry) checkPointManager.MaxRetryDurationMs = 1 @@ -193,7 +193,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { val systemFactory = Util.getObj(systemFactoryClassName, classOf[SystemFactory]) - val spec = new KafkaStreamSpec("id", cpTopic, checkpointSystemName, 1, 1, false, props) + val spec = new KafkaStreamSpec("id", cpTopic, checkpointSystemName, 1, 1, props) new KafkaCheckpointManager(spec, systemFactory, failOnTopicValidation, config, new NoOpMetricsRegistry, serde) } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index 65b8c8c352..ede79951a3 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -26,11 +26,11 @@ import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StreamConfig; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory; @@ -86,17 +86,25 @@ public void testTranslate() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + + StreamConfig streamConfig = new StreamConfig(samzaConfig); + String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); + String inputSystem = streamConfig.getSystem(inputStreamId); + String inputPhysicalName = streamConfig.getPhysicalName(inputStreamId); + String outputStreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); + String outputSystem = streamConfig.getSystem(outputStreamId); + String outputPhysicalName = streamConfig.getPhysicalName(outputStreamId); + Assert.assertEquals(1, specGraph.getOutputStreams().size()); - Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("outputTopic", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", outputSystem); + Assert.assertEquals("outputTopic", outputPhysicalName); Assert.assertEquals(1, specGraph.getInputOperators().size()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("SIMPLE1", - specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName()); + + Assert.assertEquals("testavro", inputSystem); + Assert.assertEquals("SIMPLE1", inputPhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } @@ -130,17 +138,24 @@ public void testTranslateComplex() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + + StreamConfig streamConfig = new StreamConfig(samzaConfig); + String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); + String inputSystem = streamConfig.getSystem(inputStreamId); + String inputPhysicalName = streamConfig.getPhysicalName(inputStreamId); + String outputStreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); + String outputSystem = streamConfig.getSystem(outputStreamId); + String outputPhysicalName = streamConfig.getPhysicalName(outputStreamId); + Assert.assertEquals(1, specGraph.getOutputStreams().size()); - Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("outputTopic", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", outputSystem); + Assert.assertEquals("outputTopic", outputPhysicalName); Assert.assertEquals(1, specGraph.getInputOperators().size()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("COMPLEX1", - specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", inputSystem); + Assert.assertEquals("COMPLEX1", inputPhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } @@ -155,17 +170,24 @@ public void testTranslateSubQuery() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + + StreamConfig streamConfig = new StreamConfig(samzaConfig); + String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); + String inputSystem = streamConfig.getSystem(inputStreamId); + String inputPhysicalName = streamConfig.getPhysicalName(inputStreamId); + String outputStreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); + String outputSystem = streamConfig.getSystem(outputStreamId); + String outputPhysicalName = streamConfig.getPhysicalName(outputStreamId); + Assert.assertEquals(1, specGraph.getOutputStreams().size()); - Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("outputTopic", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", outputSystem); + Assert.assertEquals("outputTopic", outputPhysicalName); Assert.assertEquals(1, specGraph.getInputOperators().size()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("COMPLEX1", - specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", inputSystem); + Assert.assertEquals("COMPLEX1", inputPhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } @@ -184,7 +206,7 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -203,7 +225,7 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -222,7 +244,7 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -241,7 +263,7 @@ public void testTranslateStreamTableJoinWithThetaCondition() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -258,7 +280,7 @@ public void testTranslateStreamTableCrossJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -277,7 +299,7 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -297,7 +319,7 @@ public void testTranslateStreamTableJoinWithSubQuery() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -316,7 +338,7 @@ public void testTranslateTableTableJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -335,7 +357,7 @@ public void testTranslateStreamStreamJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -354,7 +376,7 @@ public void testTranslateJoinWithIncorrectLeftJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -373,7 +395,7 @@ public void testTranslateJoinWithIncorrectRightJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -396,7 +418,7 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -415,7 +437,7 @@ public void testTranslateStreamTableInnerJoinWithUdf() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } @@ -434,29 +456,40 @@ public void testTranslateStreamTableInnerJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + StreamConfig streamConfig = new StreamConfig(samzaConfig); + String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); + String input1System = streamConfig.getSystem(input1StreamId); + String input1PhysicalName = streamConfig.getPhysicalName(input1StreamId); + String input2StreamId = specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get(); + String input2System = streamConfig.getSystem(input2StreamId); + String input2PhysicalName = streamConfig.getPhysicalName(input2StreamId); + String input3StreamId = specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get(); + String input3System = streamConfig.getSystem(input3StreamId); + String input3PhysicalName = streamConfig.getPhysicalName(input3StreamId); + String output1StreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); + String output1System = streamConfig.getSystem(output1StreamId); + String output1PhysicalName = streamConfig.getPhysicalName(output1StreamId); + String output2StreamId = specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get(); + String output2System = streamConfig.getSystem(output2StreamId); + String output2PhysicalName = streamConfig.getPhysicalName(output2StreamId); + Assert.assertEquals(2, specGraph.getOutputStreams().size()); - Assert.assertEquals("kafka", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("sql-job-1-partition_by-stream_1", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName()); - Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName()); - Assert.assertEquals("enrichedPageViewTopic", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName()); + Assert.assertEquals("kafka", output1System); + Assert.assertEquals("sql-job-1-partition_by-stream_1", output1PhysicalName); + Assert.assertEquals("testavro", output2System); + Assert.assertEquals("enrichedPageViewTopic", output2PhysicalName); Assert.assertEquals(3, specGraph.getInputOperators().size()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("PAGEVIEW", - specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName()); - Assert.assertEquals("PROFILE", - specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName()); - Assert.assertEquals("kafka", - specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName()); - Assert.assertEquals("sql-job-1-partition_by-stream_1", - specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", input1System); + Assert.assertEquals("PAGEVIEW", input1PhysicalName); + Assert.assertEquals("testavro", input2System); + Assert.assertEquals("PROFILE", input2PhysicalName); + Assert.assertEquals("kafka", input3System); + Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } @@ -476,32 +509,41 @@ public void testTranslateStreamTableLeftJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + StreamConfig streamConfig = new StreamConfig(samzaConfig); + String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); + String input1System = streamConfig.getSystem(input1StreamId); + String input1PhysicalName = streamConfig.getPhysicalName(input1StreamId); + String input2StreamId = specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get(); + String input2System = streamConfig.getSystem(input2StreamId); + String input2PhysicalName = streamConfig.getPhysicalName(input2StreamId); + String input3StreamId = specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get(); + String input3System = streamConfig.getSystem(input3StreamId); + String input3PhysicalName = streamConfig.getPhysicalName(input3StreamId); + String output1StreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); + String output1System = streamConfig.getSystem(output1StreamId); + String output1PhysicalName = streamConfig.getPhysicalName(output1StreamId); + String output2StreamId = specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get(); + String output2System = streamConfig.getSystem(output2StreamId); + String output2PhysicalName = streamConfig.getPhysicalName(output2StreamId); + Assert.assertEquals(2, specGraph.getOutputStreams().size()); - Assert.assertEquals("kafka", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("sql-job-1-partition_by-stream_1", - specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName()); - Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName()); - Assert.assertEquals("enrichedPageViewTopic", - specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName()); + Assert.assertEquals("kafka", output1System); + Assert.assertEquals("sql-job-1-partition_by-stream_1", output1PhysicalName); + Assert.assertEquals("testavro", output2System); + Assert.assertEquals("enrichedPageViewTopic", output2PhysicalName); Assert.assertEquals(3, specGraph.getInputOperators().size()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("PAGEVIEW", - specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName()); - Assert.assertEquals("PROFILE", - specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName()); - Assert.assertEquals("kafka", - specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName()); - Assert.assertEquals("sql-job-1-partition_by-stream_1", - specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", input1System); + Assert.assertEquals("PAGEVIEW", input1PhysicalName); + Assert.assertEquals("testavro", input2System); + Assert.assertEquals("PROFILE", input2PhysicalName); + Assert.assertEquals("kafka", input3System); + Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } @@ -521,32 +563,41 @@ public void testTranslateStreamTableRightJoin() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + StreamConfig streamConfig = new StreamConfig(samzaConfig); + String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); + String input1System = streamConfig.getSystem(input1StreamId); + String input1PhysicalName = streamConfig.getPhysicalName(input1StreamId); + String input2StreamId = specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get(); + String input2System = streamConfig.getSystem(input2StreamId); + String input2PhysicalName = streamConfig.getPhysicalName(input2StreamId); + String input3StreamId = specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get(); + String input3System = streamConfig.getSystem(input3StreamId); + String input3PhysicalName = streamConfig.getPhysicalName(input3StreamId); + String output1StreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); + String output1System = streamConfig.getSystem(output1StreamId); + String output1PhysicalName = streamConfig.getPhysicalName(output1StreamId); + String output2StreamId = specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get(); + String output2System = streamConfig.getSystem(output2StreamId); + String output2PhysicalName = streamConfig.getPhysicalName(output2StreamId); + Assert.assertEquals(2, specGraph.getOutputStreams().size()); - Assert.assertEquals("kafka", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("sql-job-1-partition_by-stream_1", - specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName()); - Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName()); - Assert.assertEquals("enrichedPageViewTopic", - specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName()); + Assert.assertEquals("kafka", output1System); + Assert.assertEquals("sql-job-1-partition_by-stream_1", output1PhysicalName); + Assert.assertEquals("testavro", output2System); + Assert.assertEquals("enrichedPageViewTopic", output2PhysicalName); Assert.assertEquals(3, specGraph.getInputOperators().size()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName()); - Assert.assertEquals("PROFILE", - specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName()); - Assert.assertEquals("testavro", - specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName()); - Assert.assertEquals("PAGEVIEW", - specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName()); - Assert.assertEquals("kafka", - specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName()); - Assert.assertEquals("sql-job-1-partition_by-stream_1", - specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName()); + Assert.assertEquals("testavro", input1System); + Assert.assertEquals("PROFILE", input1PhysicalName); + Assert.assertEquals("testavro", input2System); + Assert.assertEquals("PAGEVIEW", input2PhysicalName); + Assert.assertEquals("kafka", input3System); + Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } @@ -566,7 +617,7 @@ public void testTranslateGroupBy() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); @@ -590,7 +641,7 @@ public void testTranslateGroupByWithSumAggregator() { QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec - graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig); + graphSpec = new StreamGraphSpec(samzaConfig); translator.translate(queryInfo, graphSpec); } } diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/NegateNumberTask.java b/samza-test/src/main/java/org/apache/samza/test/integration/NegateNumberTask.java index 617cea625e..4309d92f4c 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/NegateNumberTask.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/NegateNumberTask.java @@ -30,7 +30,7 @@ import org.apache.samza.task.TaskContext; import org.apache.samza.task.TaskCoordinator; import org.apache.samza.task.TaskCoordinator.RequestScope; -import org.apache.samza.util.Util; +import org.apache.samza.util.StreamUtil; /** * A simple test job that reads strings, converts them to integers, multiplies @@ -59,7 +59,7 @@ public void init(Config config, TaskContext context) throws Exception { if (outputSystemStreamString == null) { throw new ConfigException("Missing required configuration: task.outputs"); } - outputSystemStream = Util.getSystemStreamFromNames(outputSystemStreamString); + outputSystemStream = StreamUtil.getSystemStreamFromNames(outputSystemStreamString); } @Override diff --git a/samza-test/src/main/scala/org/apache/samza/test/performance/TestPerformanceTask.scala b/samza-test/src/main/scala/org/apache/samza/test/performance/TestPerformanceTask.scala index d1f1d8466e..99d047ddda 100644 --- a/samza-test/src/main/scala/org/apache/samza/test/performance/TestPerformanceTask.scala +++ b/samza-test/src/main/scala/org/apache/samza/test/performance/TestPerformanceTask.scala @@ -27,7 +27,7 @@ import org.apache.samza.task.StreamTask import org.apache.samza.task.TaskCoordinator import org.apache.samza.task.TaskCoordinator.RequestScope import org.apache.samza.config.Config -import org.apache.samza.util.{Util, Logging} +import org.apache.samza.util.{Logging, StreamUtil} import org.apache.samza.system.SystemStream import org.apache.samza.system.OutgoingMessageEnvelope @@ -85,7 +85,7 @@ class TestPerformanceTask extends StreamTask with InitableTask with Logging { def init(config: Config, context: TaskContext) { logInterval = config.getInt("task.log.interval", 10000) maxMessages = config.getInt("task.max.messages", 10000000) - outputSystemStream = Option(config.get("task.outputs", null)).map(Util.getSystemStreamFromNames(_)) + outputSystemStream = Option(config.get("task.outputs", null)).map(StreamUtil.getSystemStreamFromNames) } def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) { 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 2171d0707a..ec9c05dd2b 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 @@ -34,7 +34,6 @@ import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.system.OutgoingMessageEnvelope; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; import org.apache.samza.task.TaskCoordinator; import org.apache.samza.test.operator.data.AdClick; @@ -54,7 +53,7 @@ public class RepartitionJoinWindowApp implements StreamApplication { public static final String INPUT_TOPIC_NAME_2_PROP = "inputTopicName2"; public static final String OUTPUT_TOPIC_NAME_PROP = "outputTopicName"; - private final List intermediateStreams = new ArrayList<>(); + private final List intermediateStreamIds = new ArrayList<>(); public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); @@ -106,14 +105,14 @@ public void init(StreamGraph graph, Config config) { }); - intermediateStreams.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamSpec()); - intermediateStreams.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamSpec()); - intermediateStreams.add(((IntermediateMessageStreamImpl) userPageAdClicksByUserId).getStreamSpec()); + intermediateStreamIds.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamId()); + intermediateStreamIds.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamId()); + intermediateStreamIds.add(((IntermediateMessageStreamImpl) userPageAdClicksByUserId).getStreamId()); } - public List getIntermediateStreams() { - return intermediateStreams; + List getIntermediateStreamIds() { + return intermediateStreamIds; } private static class UserPageViewAdClicksJoiner implements JoinFunction { diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java index 5424888ad0..faa48593f2 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java @@ -24,7 +24,6 @@ import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.samza.Partition; -import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; import org.apache.samza.system.kafka.KafkaSystemAdmin; @@ -122,14 +121,14 @@ public void testRepartitionJoinWindowAppAndDeleteMessagesOnCommit() throws Excep // Verify that messages in the intermediate stream will be deleted in 10 seconds long startTimeMs = System.currentTimeMillis(); - for (StreamSpec spec: app.getIntermediateStreams()) { + for (String streamId: app.getIntermediateStreamIds()) { long remainingMessageNum = -1; while (remainingMessageNum != 0 && System.currentTimeMillis() - startTimeMs < 10000) { remainingMessageNum = 0; SystemStreamMetadata metadatas = systemAdmin.getSystemStreamMetadata( - new HashSet<>(Arrays.asList(spec.getPhysicalName())), new ExponentialSleepStrategy.Mock(3) - ).get(spec.getPhysicalName()).get(); + new HashSet<>(Arrays.asList(streamId)), new ExponentialSleepStrategy.Mock(3) + ).get(streamId).get(); for (Map.Entry entry : metadatas.getSystemStreamPartitionMetadata().entrySet()) { SystemStreamPartitionMetadata metadata = entry.getValue(); From 7d7aa50882d656143ea6fbc8064aca390024dd31 Mon Sep 17 00:00:00 2001 From: Prateek Maheshwari Date: Wed, 13 Jun 2018 12:56:51 -0700 Subject: [PATCH 02/38] Updated with Cameron and Daniel's feedback. --- .../operators/impl/OperatorImplGraph.java | 155 ++++++------ .../org/apache/samza/util/StreamUtil.java | 47 ++-- .../samza/execution/TestExecutionPlanner.java | 20 +- .../execution/TestJobGraphJsonGenerator.java | 25 +- .../operators/impl/TestOperatorImplGraph.java | 198 +++++++++++++-- .../impl/TestOperatorImplGraphUtil.java | 226 ------------------ .../samza/testUtils/StreamTestUtils.java | 39 +++ 7 files changed, 316 insertions(+), 394 deletions(-) delete mode 100644 samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java create mode 100644 samza-core/src/test/java/org/apache/samza/testUtils/StreamTestUtils.java diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index 2c13f92d10..21bbfc8eae 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -85,7 +85,6 @@ public class OperatorImplGraph { private final Map> joinOpImpls = new HashMap<>(); private final Clock clock; - private final OperatorImplGraphUtil util; /** * Constructs the DAG of {@link OperatorImpl}s corresponding to the the DAG of {@link OperatorSpec}s @@ -98,13 +97,13 @@ public class OperatorImplGraph { */ public OperatorImplGraph(OperatorSpecGraph specGraph, Config config, TaskContext context, Clock clock) { this.clock = clock; - this.util = new OperatorImplGraphUtil(config); + StreamConfig streamConfig = new StreamConfig(config); TaskContextImpl taskContext = (TaskContextImpl) context; Map producerTaskCounts = - util.hasIntermediateStreams(specGraph) - ? util.getProducerTaskCountForIntermediateStreams( - util.getStreamToConsumerTasks(taskContext.getJobModel()), - util.getIntermediateToInputStreamsMap(specGraph)) + hasIntermediateStreams(specGraph) + ? getProducerTaskCountForIntermediateStreams( + getStreamToConsumerTasks(taskContext.getJobModel()), + getIntermediateToInputStreamsMap(specGraph, streamConfig)) : Collections.EMPTY_MAP; producerTaskCounts.forEach((stream, count) -> { LOG.info("{} has {} producer tasks.", stream, count); @@ -118,7 +117,7 @@ public OperatorImplGraph(OperatorSpecGraph specGraph, Config config, TaskContext new WatermarkStates(context.getSystemStreamPartitions(), producerTaskCounts)); specGraph.getInputOperators().forEach((streamId, inputOpSpec) -> { - SystemStream systemStream = util.toSystemStream(streamId); + SystemStream systemStream = streamConfig.streamIdToSystemStream(streamId); InputOperatorImpl inputOperatorImpl = (InputOperatorImpl) createAndRegisterOperatorImpl(null, inputOpSpec, systemStream, config, context); this.inputOperators.put(systemStream, inputOperatorImpl); @@ -214,6 +213,7 @@ private OperatorImpl createAndRegisterOperatorImpl(OperatorSpec prevOperatorSpec */ OperatorImpl createOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec operatorSpec, Config config, TaskContext context) { + StreamConfig streamConfig = new StreamConfig(config); if (operatorSpec instanceof InputOperatorSpec) { return new InputOperatorImpl((InputOperatorSpec) operatorSpec); } else if (operatorSpec instanceof StreamOperatorSpec) { @@ -222,11 +222,11 @@ OperatorImpl createOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec oper return new SinkOperatorImpl((SinkOperatorSpec) operatorSpec, config, context); } else if (operatorSpec instanceof OutputOperatorSpec) { String streamId = ((OutputOperatorSpec) operatorSpec).getOutputStream().getStreamId(); - SystemStream systemStream = util.toSystemStream(streamId); + SystemStream systemStream = streamConfig.streamIdToSystemStream(streamId); return new OutputOperatorImpl((OutputOperatorSpec) operatorSpec, systemStream); } else if (operatorSpec instanceof PartitionByOperatorSpec) { String streamId = ((PartitionByOperatorSpec) operatorSpec).getOutputStream().getStreamId(); - SystemStream systemStream = util.toSystemStream(streamId); + SystemStream systemStream = streamConfig.streamIdToSystemStream(streamId); return new PartitionByOperatorImpl((PartitionByOperatorSpec) operatorSpec, systemStream, context); } else if (operatorSpec instanceof WindowOperatorSpec) { return new WindowOperatorImpl((WindowOperatorSpec) operatorSpec, clock); @@ -240,7 +240,7 @@ OperatorImpl createOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec oper return new SendToTableOperatorImpl((SendToTableOperatorSpec) operatorSpec, config, context); } else if (operatorSpec instanceof BroadcastOperatorSpec) { String streamId = ((BroadcastOperatorSpec) operatorSpec).getOutputStream().getStreamId(); - SystemStream systemStream = util.toSystemStream(streamId); + SystemStream systemStream = streamConfig.streamIdToSystemStream(streamId); return new BroadcastOperatorImpl((BroadcastOperatorSpec) operatorSpec, systemStream, context); } throw new IllegalArgumentException( @@ -333,81 +333,74 @@ public KeyValueStore> getState() { }; } - protected static class OperatorImplGraphUtil { - private final Config config; - - OperatorImplGraphUtil(Config config) { - this.config = config; - } - - /** - * calculate the task count that produces to each intermediate streams - * @param streamToConsumerTasks input streams to task mapping - * @param intermediateToInputStreams intermediate stream to input streams mapping - * @return mapping from intermediate stream to task count - */ - Map getProducerTaskCountForIntermediateStreams( - Multimap streamToConsumerTasks, - Multimap intermediateToInputStreams) { - Map result = new HashMap<>(); - intermediateToInputStreams.asMap().entrySet().forEach(entry -> - result.put(entry.getKey(), - entry.getValue().stream() - .flatMap(systemStream -> streamToConsumerTasks.get(systemStream).stream()) - .collect(Collectors.toSet()).size())); - return result; - } - - /** - * calculate the mapping from input streams to consumer tasks - * @param jobModel JobModel object - * @return mapping from input stream to tasks - */ - Multimap getStreamToConsumerTasks(JobModel jobModel) { - Multimap streamToConsumerTasks = HashMultimap.create(); - jobModel.getContainers().values().forEach(containerModel -> { - containerModel.getTasks().values().forEach(taskModel -> { - taskModel.getSystemStreamPartitions().forEach(ssp -> { - streamToConsumerTasks.put(ssp.getSystemStream(), taskModel.getTaskName().getTaskName()); - }); - }); - }); - return streamToConsumerTasks; - } + /** + * calculate the task count that produces to each intermediate streams + * @param streamToConsumerTasks input streams to task mapping + * @param intermediateToInputStreams intermediate stream to input streams mapping + * @return mapping from intermediate stream to task count + */ + static Map getProducerTaskCountForIntermediateStreams( + Multimap streamToConsumerTasks, + Multimap intermediateToInputStreams) { + Map result = new HashMap<>(); + intermediateToInputStreams.asMap().entrySet().forEach(entry -> + result.put(entry.getKey(), + entry.getValue().stream() + .flatMap(systemStream -> streamToConsumerTasks.get(systemStream).stream()) + .collect(Collectors.toSet()).size())); + return result; + } - /** - * calculate the mapping from output streams to input streams - * @param specGraph the user {@link OperatorSpecGraph} - * @return mapping from output streams to input streams - */ - Multimap getIntermediateToInputStreamsMap(OperatorSpecGraph specGraph) { - Multimap outputToInputStreams = HashMultimap.create(); - specGraph.getInputOperators().entrySet().stream() - .forEach(entry -> - computeOutputToInput(toSystemStream(entry.getKey()), entry.getValue(), outputToInputStreams)); - return outputToInputStreams; - } + /** + * calculate the mapping from input streams to consumer tasks + * @param jobModel JobModel object + * @return mapping from input stream to tasks + */ + static Multimap getStreamToConsumerTasks(JobModel jobModel) { + Multimap streamToConsumerTasks = HashMultimap.create(); + jobModel.getContainers().values().forEach(containerModel -> { + containerModel.getTasks().values().forEach(taskModel -> { + taskModel.getSystemStreamPartitions().forEach(ssp -> { + streamToConsumerTasks.put(ssp.getSystemStream(), taskModel.getTaskName().getTaskName()); + }); + }); + }); + return streamToConsumerTasks; + } - private void computeOutputToInput(SystemStream input, OperatorSpec opSpec, - Multimap outputToInputStreams) { - if (opSpec instanceof PartitionByOperatorSpec) { - PartitionByOperatorSpec spec = (PartitionByOperatorSpec) opSpec; - outputToInputStreams.put(toSystemStream(spec.getOutputStream().getStreamId()), input); - } else if (opSpec instanceof BroadcastOperatorSpec) { - BroadcastOperatorSpec spec = (BroadcastOperatorSpec) opSpec; - outputToInputStreams.put(toSystemStream(spec.getOutputStream().getStreamId()), input); - } else { - Collection nextOperators = opSpec.getRegisteredOperatorSpecs(); - nextOperators.forEach(spec -> computeOutputToInput(input, spec, outputToInputStreams)); - } - } + /** + * calculate the mapping from output streams to input streams + * @param specGraph the user {@link OperatorSpecGraph} + * @return mapping from output streams to input streams + */ + static Multimap getIntermediateToInputStreamsMap( + OperatorSpecGraph specGraph, StreamConfig streamConfig) { + Multimap outputToInputStreams = HashMultimap.create(); + specGraph.getInputOperators().entrySet().stream() + .forEach(entry -> { + SystemStream systemStream = streamConfig.streamIdToSystemStream(entry.getKey()); + computeOutputToInput(systemStream, entry.getValue(), outputToInputStreams, streamConfig); + }); + return outputToInputStreams; + } - private SystemStream toSystemStream(String streamId) { - return new StreamConfig(config).streamIdToSystemStream(streamId); + private static void computeOutputToInput(SystemStream input, OperatorSpec opSpec, + Multimap outputToInputStreams, StreamConfig streamConfig) { + if (opSpec instanceof PartitionByOperatorSpec) { + PartitionByOperatorSpec spec = (PartitionByOperatorSpec) opSpec; + SystemStream systemStream = streamConfig.streamIdToSystemStream(spec.getOutputStream().getStreamId()); + outputToInputStreams.put(systemStream, input); + } else if (opSpec instanceof BroadcastOperatorSpec) { + BroadcastOperatorSpec spec = (BroadcastOperatorSpec) opSpec; + SystemStream systemStream = streamConfig.streamIdToSystemStream(spec.getOutputStream().getStreamId()); + outputToInputStreams.put(systemStream, input); + } else { + Collection nextOperators = opSpec.getRegisteredOperatorSpecs(); + nextOperators.forEach(spec -> computeOutputToInput(input, spec, outputToInputStreams, streamConfig)); } + } - private boolean hasIntermediateStreams(OperatorSpecGraph specGraph) { - return !Collections.disjoint(specGraph.getInputOperators().keySet(), specGraph.getOutputStreams().keySet()); - } + private boolean hasIntermediateStreams(OperatorSpecGraph specGraph) { + return !Collections.disjoint(specGraph.getInputOperators().keySet(), specGraph.getOutputStreams().keySet()); } } diff --git a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java index 5170b29eed..dfdc3971e4 100644 --- a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java @@ -18,38 +18,38 @@ */ package org.apache.samza.util; -import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.commons.lang3.tuple.ImmutableTriple; -import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; -import org.apache.samza.config.StreamConfig$; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; public class StreamUtil { /** * Returns a SystemStream object based on the system stream name given. For - * example, kafka.topic would return new SystemStream("kafka", "topic"). + * example, kafka.topic would return SystemStream("kafka", "topic"). + * + * @param systemStreamName name of the system stream + * @return the {@link SystemStream} for the {@code systemStreamName} */ - public static SystemStream getSystemStreamFromNames(String systemStreamNames) { - int idx = systemStreamNames.indexOf('.'); + public static SystemStream getSystemStreamFromNames(String systemStreamName) { + int idx = systemStreamName.indexOf('.'); if (idx < 0) { - throw new IllegalArgumentException("No '.' in stream name '" + systemStreamNames + + throw new IllegalArgumentException("No '.' in stream name '" + systemStreamName + "'. Stream names should be in the form 'system.stream'"); } return new SystemStream( - systemStreamNames.substring(0, idx), - systemStreamNames.substring(idx + 1, systemStreamNames.length())); + systemStreamName.substring(0, idx), + systemStreamName.substring(idx + 1, systemStreamName.length())); } /** - * Returns a SystemStream object based on the system stream name given. For - * example, kafka.topic would return new SystemStream("kafka", "topic"). + * Returns the period separated system stream name for the provided {@code systemStream}. For + * example, SystemStream("kafka", "topic") would return "kafka.topic". + * + * @param systemStream the {@link SystemStream} to get the name for + * @return the system stream name */ public static String getNameFromSystemStream(SystemStream systemStream) { return systemStream.getSystem() + "." + systemStream.getStream(); @@ -65,23 +65,4 @@ public static StreamSpec getStreamSpec(String streamId, StreamConfig streamConfi Map streamProperties = streamConfig.getStreamProperties(streamId); return new StreamSpec(streamId, physicalName, system, streamProperties); } - - /** - * Converts the provided list of (streamId, system, physicalName) triplets to their corresponding - * stream.stream-id.* configurations. - * - * @param streams a list of (streamId, system, physicalName) triplets to get the stream configuration for. - * @return the configuration for the provided { @code streams} - */ - public static Config toStreamConfigs(List> streams) { - Map configsMap = new HashMap<>(); - streams.stream().forEach(triple -> { - String streamId = triple.getLeft(); - String systemName = triple.getMiddle(); - String physicalName = triple.getRight(); - configsMap.put(String.format(StreamConfig$.MODULE$.SYSTEM_FOR_STREAM_ID(), streamId), systemName); - configsMap.put(String.format(StreamConfig$.MODULE$.PHYSICAL_NAME_FOR_STREAM_ID(), streamId), physicalName); - }); - return new MapConfig(configsMap); - } } 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 02419f3081..4d25ebb9c0 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 @@ -19,8 +19,6 @@ package org.apache.samza.execution; -import com.google.common.collect.ImmutableList; - import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -29,7 +27,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.commons.lang3.tuple.ImmutableTriple; import org.apache.samza.Partition; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -47,7 +44,7 @@ import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.util.StreamUtil; +import org.apache.samza.testUtils.StreamTestUtils; import org.junit.Before; import org.junit.Test; @@ -209,15 +206,12 @@ public void setup() { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), DEFAULT_SYSTEM); - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of("input1", "system1", "input1"), - ImmutableTriple.of("input2", "system2", "input2"), - ImmutableTriple.of("input3", "system2", "input3"), - ImmutableTriple.of("input4", "system1", "input4"), - ImmutableTriple.of("output1", "system1", "output1"), - ImmutableTriple.of("output2", "system2", "output2") - )); - configMap.putAll(streamConfigs); + StreamTestUtils.addStreamConfigs(configMap, "input1", "system1", "input1"); + StreamTestUtils.addStreamConfigs(configMap, "input2", "system2", "input2"); + StreamTestUtils.addStreamConfigs(configMap, "input3", "system2", "input3"); + StreamTestUtils.addStreamConfigs(configMap, "input4", "system1", "input4"); + StreamTestUtils.addStreamConfigs(configMap, "output1", "system1", "output1"); + StreamTestUtils.addStreamConfigs(configMap, "output2", "system2", "output2"); config = new MapConfig(configMap); input1 = new StreamSpec("input1", "input1", "system1"); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index 40cbe30ad7..b0f3843eaa 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -19,12 +19,9 @@ package org.apache.samza.execution; -import com.google.common.collect.ImmutableList; - import java.time.Duration; import java.util.HashMap; import java.util.Map; -import org.apache.commons.lang3.tuple.ImmutableTriple; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -41,7 +38,7 @@ import org.apache.samza.serializers.StringSerde; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemAdmins; -import org.apache.samza.util.StreamUtil; +import org.apache.samza.testUtils.StreamTestUtils; import org.codehaus.jackson.map.ObjectMapper; import org.junit.Test; @@ -78,14 +75,11 @@ public void test() throws Exception { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of("input1", "system1", "input1"), - ImmutableTriple.of("input2", "system2", "input2"), - ImmutableTriple.of("input3", "system2", "input3"), - ImmutableTriple.of("output1", "system1", "output1"), - ImmutableTriple.of("output2", "system2", "output2") - )); - configMap.putAll(streamConfigs); + StreamTestUtils.addStreamConfigs(configMap, "input1", "system1", "input1"); + StreamTestUtils.addStreamConfigs(configMap, "input2", "system2", "input2"); + StreamTestUtils.addStreamConfigs(configMap, "input3", "system2", "input3"); + StreamTestUtils.addStreamConfigs(configMap, "output1", "system1", "output1"); + StreamTestUtils.addStreamConfigs(configMap, "output2", "system2", "output2"); Config config = new MapConfig(configMap); // set up external partition count @@ -153,11 +147,8 @@ public void test2() throws Exception { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of("PageView", "hdfs", "hdfs:/user/dummy/PageViewEvent"), - ImmutableTriple.of("PageViewCount", "kafka", "PageViewCount") - )); - configMap.putAll(streamConfigs); + StreamTestUtils.addStreamConfigs(configMap, "PageView", "hdfs", "hdfs:/user/dummy/PageViewEvent"); + StreamTestUtils.addStreamConfigs(configMap, "PageViewCount", "kafka", "PageViewCount"); Config config = new MapConfig(configMap); // set up external partition count diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index d393e527a1..c73638e7b2 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -19,7 +19,9 @@ package org.apache.samza.operators.impl; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + import java.io.Serializable; import java.time.Duration; import java.util.ArrayList; @@ -27,14 +29,16 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; -import org.apache.commons.lang3.tuple.ImmutableTriple; +import org.apache.samza.Partition; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StreamConfig; import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; @@ -44,8 +48,8 @@ import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.ClosableFunction; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.InitableFunction; @@ -60,12 +64,12 @@ import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.task.MessageCollector; import org.apache.samza.task.TaskContext; -import java.util.List; import org.apache.samza.task.TaskCoordinator; +import org.apache.samza.testUtils.StreamTestUtils; import org.apache.samza.util.Clock; -import org.apache.samza.util.StreamUtil; import org.apache.samza.util.SystemClock; import org.junit.After; import org.junit.Test; @@ -233,12 +237,8 @@ public void testLinearChain() { configs.put(JobConfig.JOB_NAME(), "jobName"); configs.put(JobConfig.JOB_ID(), "jobId"); configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystem); - - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of(inputStreamId, inputSystem, inputPhysicalName), - ImmutableTriple.of(outputStreamId, outputSystem, outputPhysicalName) - )); - configs.putAll(streamConfigs); + StreamTestUtils.addStreamConfigs(configs, inputStreamId, inputSystem, inputPhysicalName); + StreamTestUtils.addStreamConfigs(configs, outputStreamId, outputSystem, outputPhysicalName); Config config = new MapConfig(configs); StreamGraphSpec graphSpec = new StreamGraphSpec(config); @@ -288,12 +288,8 @@ public void testPartitionByChain() { configs.put(JobConfig.JOB_NAME(), "jobName"); configs.put(JobConfig.JOB_ID(), "jobId"); configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystem); - - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of(inputStreamId, inputSystem, inputPhysicalName), - ImmutableTriple.of(outputStreamId, outputSystem, outputPhysicalName) - )); - configs.putAll(streamConfigs); + StreamTestUtils.addStreamConfigs(configs, inputStreamId, inputSystem, inputPhysicalName); + StreamTestUtils.addStreamConfigs(configs, outputStreamId, outputSystem, outputPhysicalName); Config config = new MapConfig(configs); StreamGraphSpec graphSpec = new StreamGraphSpec(config); @@ -341,8 +337,9 @@ public void testPartitionByChain() { @Test public void testBroadcastChain() { String inputStreamId = "input"; - Config config = StreamUtil.toStreamConfigs( - ImmutableList.of(ImmutableTriple.of("input", "input-system", "input-stream"))); + HashMap configMap = new HashMap<>(); + StreamTestUtils.addStreamConfigs(configMap, "input", "input-system", "input-stream"); + Config config = new MapConfig(configMap); StreamGraphSpec graphSpec = new StreamGraphSpec(config); MessageStream inputStream = graphSpec.getInputStream(inputStreamId); @@ -403,11 +400,8 @@ public void testJoinChain() { HashMap configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), "jobName"); configs.put(JobConfig.JOB_ID(), "jobId"); - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of("input1", "input-system", "input-stream1"), - ImmutableTriple.of("input2", "input-system", "input-stream2") - )); - configs.putAll(streamConfigs); + StreamTestUtils.addStreamConfigs(configs, "input1", "input-system", "input-stream1"); + StreamTestUtils.addStreamConfigs(configs, "input2", "input-system", "input-stream2"); Config config = new MapConfig(configs); StreamGraphSpec graphSpec = new StreamGraphSpec(config); @@ -502,4 +496,160 @@ public void testOperatorGraphInitAndClose() { assertEquals(closedOperators.get(2), "2"); assertEquals(closedOperators.get(3), "1"); } + + @Test + public void testGetStreamToConsumerTasks() { + String system = "test-system"; + String streamId0 = "test-stream-0"; + String streamId1 = "test-stream-1"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "test-app"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); + StreamTestUtils.addStreamConfigs(configs, streamId0, system, streamId0); + StreamTestUtils.addStreamConfigs(configs, streamId1, system, streamId1); + Config config = new MapConfig(configs); + + SystemStreamPartition ssp0 = new SystemStreamPartition(system, streamId0, new Partition(0)); + SystemStreamPartition ssp1 = new SystemStreamPartition(system, streamId0, new Partition(1)); + SystemStreamPartition ssp2 = new SystemStreamPartition(system, streamId1, new Partition(0)); + + TaskName task0 = new TaskName("Task 0"); + TaskName task1 = new TaskName("Task 1"); + Set ssps = new HashSet<>(); + ssps.add(ssp0); + ssps.add(ssp2); + TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0)); + ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0)); + TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1)); + ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1)); + + Map cms = new HashMap<>(); + cms.put(cm0.getProcessorId(), cm0); + cms.put(cm1.getProcessorId(), cm1); + + JobModel jobModel = new JobModel(config, cms, null); + Multimap streamToTasks = OperatorImplGraph.getStreamToConsumerTasks(jobModel); + assertEquals(streamToTasks.get(ssp0.getSystemStream()).size(), 2); + assertEquals(streamToTasks.get(ssp2.getSystemStream()).size(), 1); + } + + @Test + public void testGetOutputToInputStreams() { + String inputStreamId1 = "input1"; + String inputStreamId2 = "input2"; + String inputStreamId3 = "input3"; + String inputSystem = "input-system"; + + String outputStreamId1 = "output1"; + String outputStreamId2 = "output2"; + String outputSystem = "output-system"; + + String intStreamId1 = "test-app-1-partition_by-p1"; + String intStreamId2 = "test-app-1-partition_by-p2"; + String intSystem = "test-system"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "test-app"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intSystem); + StreamTestUtils.addStreamConfigs(configs, inputStreamId1, inputSystem, inputStreamId1); + StreamTestUtils.addStreamConfigs(configs, inputStreamId2, inputSystem, inputStreamId2); + StreamTestUtils.addStreamConfigs(configs, inputStreamId3, inputSystem, inputStreamId3); + StreamTestUtils.addStreamConfigs(configs, outputStreamId1, outputSystem, outputStreamId1); + StreamTestUtils.addStreamConfigs(configs, outputStreamId2, outputSystem, outputStreamId2); + Config config = new MapConfig(configs); + + StreamGraphSpec graphSpec = new StreamGraphSpec(config); + MessageStream messageStream1 = graphSpec.getInputStream(inputStreamId1).map(m -> m); + MessageStream messageStream2 = graphSpec.getInputStream(inputStreamId2).filter(m -> true); + MessageStream messageStream3 = + graphSpec.getInputStream(inputStreamId3) + .filter(m -> true) + .partitionBy(m -> "m", m -> m, "p1") + .map(m -> m); + OutputStream outputStream1 = graphSpec.getOutputStream(outputStreamId1); + OutputStream outputStream2 = graphSpec.getOutputStream(outputStreamId2); + + messageStream1 + .join(messageStream2, mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") + .partitionBy(m -> "m", m -> m, "p2") + .sendTo(outputStream1); + messageStream3 + .join(messageStream2, mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") + .sendTo(outputStream2); + + Multimap outputToInput = + OperatorImplGraph.getIntermediateToInputStreamsMap(graphSpec.getOperatorSpecGraph(), new StreamConfig(config)); + Collection inputs = outputToInput.get(new SystemStream(intSystem, intStreamId2)); + assertEquals(inputs.size(), 2); + assertTrue(inputs.contains(new SystemStream(inputSystem, inputStreamId1))); + assertTrue(inputs.contains(new SystemStream(inputSystem, inputStreamId2))); + + inputs = outputToInput.get(new SystemStream(intSystem, intStreamId1)); + assertEquals(inputs.size(), 1); + assertEquals(inputs.iterator().next(), new SystemStream(inputSystem, inputStreamId3)); + } + + @Test + public void testGetProducerTaskCountForIntermediateStreams() { + String inputStreamId1 = "input1"; + String inputStreamId2 = "input2"; + String inputStreamId3 = "input3"; + String inputSystem1 = "system1"; + String inputSystem2 = "system2"; + + HashMap configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "test-app"); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), inputSystem1); + StreamTestUtils.addStreamConfigs(configs, inputStreamId1, inputSystem1, inputStreamId1); + StreamTestUtils.addStreamConfigs(configs, inputStreamId2, inputSystem2, inputStreamId2); + StreamTestUtils.addStreamConfigs(configs, inputStreamId3, inputSystem2, inputStreamId3); + Config config = new MapConfig(configs); + + SystemStream input1 = new SystemStream("system1", "intput1"); + SystemStream input2 = new SystemStream("system2", "intput2"); + SystemStream input3 = new SystemStream("system2", "intput3"); + + SystemStream int1 = new SystemStream("system1", "int1"); + SystemStream int2 = new SystemStream("system1", "int2"); + + + /** + * the task assignment looks like the following: + * + * input1 -----> task0, task1 -----> int1 + * ^ + * input2 ------> task1, task2--------| + * v + * input3 ------> task1 -----------> int2 + * + */ + String task0 = "Task 0"; + String task1 = "Task 1"; + String task2 = "Task 2"; + + Multimap streamToConsumerTasks = HashMultimap.create(); + streamToConsumerTasks.put(input1, task0); + streamToConsumerTasks.put(input1, task1); + streamToConsumerTasks.put(input2, task1); + streamToConsumerTasks.put(input2, task2); + streamToConsumerTasks.put(input3, task1); + streamToConsumerTasks.put(int1, task0); + streamToConsumerTasks.put(int1, task1); + streamToConsumerTasks.put(int2, task0); + + Multimap intermediateToInputStreams = HashMultimap.create(); + intermediateToInputStreams.put(int1, input1); + intermediateToInputStreams.put(int1, input2); + + intermediateToInputStreams.put(int2, input2); + intermediateToInputStreams.put(int2, input3); + + Map counts = OperatorImplGraph.getProducerTaskCountForIntermediateStreams( + streamToConsumerTasks, intermediateToInputStreams); + assertTrue(counts.get(int1) == 3); + assertTrue(counts.get(int2) == 2); + } } diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java deleted file mode 100644 index abd2a08654..0000000000 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraphUtil.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.impl; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Multimap; - -import java.time.Duration; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang3.tuple.ImmutableTriple; -import org.apache.samza.Partition; -import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; -import org.apache.samza.container.TaskName; -import org.apache.samza.job.model.ContainerModel; -import org.apache.samza.job.model.JobModel; -import org.apache.samza.job.model.TaskModel; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.operators.functions.JoinFunction; -import org.apache.samza.serializers.Serde; -import org.apache.samza.system.SystemStream; -import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.util.StreamUtil; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; - -public class TestOperatorImplGraphUtil { - @Test - public void testGetStreamToConsumerTasks() { - String system = "test-system"; - String streamId0 = "test-stream-0"; - String streamId1 = "test-stream-1"; - - HashMap configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), "test-app"); - configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); - - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of(streamId0, system, streamId0), - ImmutableTriple.of(streamId1, system, streamId1) - )); - configs.putAll(streamConfigs); - Config config = new MapConfig(configs); - - SystemStreamPartition ssp0 = new SystemStreamPartition(system, streamId0, new Partition(0)); - SystemStreamPartition ssp1 = new SystemStreamPartition(system, streamId0, new Partition(1)); - SystemStreamPartition ssp2 = new SystemStreamPartition(system, streamId1, new Partition(0)); - - TaskName task0 = new TaskName("Task 0"); - TaskName task1 = new TaskName("Task 1"); - Set ssps = new HashSet<>(); - ssps.add(ssp0); - ssps.add(ssp2); - TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0)); - ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0)); - TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1)); - ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1)); - - Map cms = new HashMap<>(); - cms.put(cm0.getProcessorId(), cm0); - cms.put(cm1.getProcessorId(), cm1); - - JobModel jobModel = new JobModel(config, cms, null); - OperatorImplGraph.OperatorImplGraphUtil util = new OperatorImplGraph.OperatorImplGraphUtil(config); - Multimap streamToTasks = util.getStreamToConsumerTasks(jobModel); - assertEquals(streamToTasks.get(ssp0.getSystemStream()).size(), 2); - assertEquals(streamToTasks.get(ssp2.getSystemStream()).size(), 1); - } - - @Test - public void testGetOutputToInputStreams() { - String inputStreamId1 = "input1"; - String inputStreamId2 = "input2"; - String inputStreamId3 = "input3"; - String inputSystem = "input-system"; - - String outputStreamId1 = "output1"; - String outputStreamId2 = "output2"; - String outputSystem = "output-system"; - - String intStreamId1 = "test-app-1-partition_by-p1"; - String intStreamId2 = "test-app-1-partition_by-p2"; - String intSystem = "test-system"; - - HashMap configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), "test-app"); - configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intSystem); - - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of(inputStreamId1, inputSystem, inputStreamId1), - ImmutableTriple.of(inputStreamId2, inputSystem, inputStreamId2), - ImmutableTriple.of(inputStreamId3, inputSystem, inputStreamId3), - ImmutableTriple.of(outputStreamId1, outputSystem, outputStreamId1), - ImmutableTriple.of(outputStreamId2, outputSystem, outputStreamId2) - )); - configs.putAll(streamConfigs); - Config config = new MapConfig(configs); - - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream messageStream1 = graphSpec.getInputStream(inputStreamId1).map(m -> m); - MessageStream messageStream2 = graphSpec.getInputStream(inputStreamId2).filter(m -> true); - MessageStream messageStream3 = - graphSpec.getInputStream(inputStreamId3) - .filter(m -> true) - .partitionBy(m -> "m", m -> m, "p1") - .map(m -> m); - OutputStream outputStream1 = graphSpec.getOutputStream(outputStreamId1); - OutputStream outputStream2 = graphSpec.getOutputStream(outputStreamId2); - - messageStream1 - .join(messageStream2, mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") - .partitionBy(m -> "m", m -> m, "p2") - .sendTo(outputStream1); - messageStream3 - .join(messageStream2, mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") - .sendTo(outputStream2); - - OperatorImplGraph.OperatorImplGraphUtil util = new OperatorImplGraph.OperatorImplGraphUtil(config); - Multimap outputToInput = - util.getIntermediateToInputStreamsMap(graphSpec.getOperatorSpecGraph()); - Collection inputs = outputToInput.get(new SystemStream(intSystem, intStreamId2)); - assertEquals(inputs.size(), 2); - assertTrue(inputs.contains(new SystemStream(inputSystem, inputStreamId1))); - assertTrue(inputs.contains(new SystemStream(inputSystem, inputStreamId2))); - - inputs = outputToInput.get(new SystemStream(intSystem, intStreamId1)); - assertEquals(inputs.size(), 1); - assertEquals(inputs.iterator().next(), new SystemStream(inputSystem, inputStreamId3)); - } - - @Test - public void testGetProducerTaskCountForIntermediateStreams() { - String inputStreamId1 = "input1"; - String inputStreamId2 = "input2"; - String inputStreamId3 = "input3"; - String inputSystem1 = "system1"; - String inputSystem2 = "system2"; - - HashMap configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), "test-app"); - configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), inputSystem1); - - Config streamConfigs = StreamUtil.toStreamConfigs(ImmutableList.of( - ImmutableTriple.of(inputStreamId1, inputSystem1, inputStreamId1), - ImmutableTriple.of(inputStreamId2, inputSystem2, inputStreamId2), - ImmutableTriple.of(inputStreamId3, inputSystem2, inputStreamId3) - )); - configs.putAll(streamConfigs); - Config config = new MapConfig(configs); - - SystemStream input1 = new SystemStream("system1", "intput1"); - SystemStream input2 = new SystemStream("system2", "intput2"); - SystemStream input3 = new SystemStream("system2", "intput3"); - - SystemStream int1 = new SystemStream("system1", "int1"); - SystemStream int2 = new SystemStream("system1", "int2"); - - - /** - * the task assignment looks like the following: - * - * input1 -----> task0, task1 -----> int1 - * ^ - * input2 ------> task1, task2--------| - * v - * input3 ------> task1 -----------> int2 - * - */ - String task0 = "Task 0"; - String task1 = "Task 1"; - String task2 = "Task 2"; - - Multimap streamToConsumerTasks = HashMultimap.create(); - streamToConsumerTasks.put(input1, task0); - streamToConsumerTasks.put(input1, task1); - streamToConsumerTasks.put(input2, task1); - streamToConsumerTasks.put(input2, task2); - streamToConsumerTasks.put(input3, task1); - streamToConsumerTasks.put(int1, task0); - streamToConsumerTasks.put(int1, task1); - streamToConsumerTasks.put(int2, task0); - - Multimap intermediateToInputStreams = HashMultimap.create(); - intermediateToInputStreams.put(int1, input1); - intermediateToInputStreams.put(int1, input2); - - intermediateToInputStreams.put(int2, input2); - intermediateToInputStreams.put(int2, input3); - - OperatorImplGraph.OperatorImplGraphUtil util = new OperatorImplGraph.OperatorImplGraphUtil(config); - Map counts = util.getProducerTaskCountForIntermediateStreams( - streamToConsumerTasks, intermediateToInputStreams); - assertTrue(counts.get(int1) == 3); - assertTrue(counts.get(int2) == 2); - } -} diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/StreamTestUtils.java b/samza-core/src/test/java/org/apache/samza/testUtils/StreamTestUtils.java new file mode 100644 index 0000000000..9b1fa4de67 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/testUtils/StreamTestUtils.java @@ -0,0 +1,39 @@ +/* + * 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.testUtils; + +import java.util.Map; +import org.apache.samza.config.StreamConfig$; + +public class StreamTestUtils { + + /** + * Adds the stream.stream-id.* configurations for the provided {@code streamId} to the provided {@code configs} Map. + * + * @param configs the configs Map to add the stream configurations to + * @param streamId the id of the stream + * @param systemName the system for the stream + * @param physicalName the physical name for the stream + */ + public static void addStreamConfigs(Map configs, + String streamId, String systemName, String physicalName) { + configs.put(String.format(StreamConfig$.MODULE$.SYSTEM_FOR_STREAM_ID(), streamId), systemName); + configs.put(String.format(StreamConfig$.MODULE$.PHYSICAL_NAME_FOR_STREAM_ID(), streamId), physicalName); + } +} \ No newline at end of file From 7e71dc7e04c5e1a0357663a76068d3dead4ebfd8 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Wed, 27 Jun 2018 15:22:17 -0700 Subject: [PATCH 03/38] Merge with master --- .../application/ApplicationRunnable.java | 54 ++++++ .../samza/application/StreamApplication.java | 170 ++++++++++++++++-- .../samza/application/StreamApplications.java | 45 +++++ .../samza/runtime/ApplicationRunner.java | 98 ++++------ .../StreamApplicationInternal.java | 36 ++++ .../org/apache/samza/execution/JobGraph.java | 6 - .../runtime/AbstractApplicationRunner.java | 25 ++- .../runtime/ApplicationRunnerCommandLine.java | 40 +++++ .../runtime/ApplicationRunnerOperation.java | 2 +- .../samza/runtime/LocalApplicationRunner.java | 54 ++++-- .../samza/runtime/LocalContainerRunner.java | 69 +++++-- .../runtime/RemoteApplicationRunner.java | 64 ++++--- .../apache/samza/task/TaskFactoryUtil.java | 34 +--- .../org/apache/samza/job/JobRunner.scala | 4 +- .../samza/job/local/ThreadJobFactory.scala | 17 +- .../samza/execution/TestExecutionPlanner.java | 7 +- .../TestLocalContainerRunner.java} | 14 +- .../samza/task/TestTaskFactoryUtil.java | 133 -------------- samza-shell/src/main/bash/run-app.sh | 2 +- .../example/AppWithGlobalConfigExample.java | 27 ++- .../samza/example/BroadcastExample.java | 26 ++- .../samza/example/KeyValueStoreExample.java | 21 +-- .../apache/samza/example/MergeExample.java | 23 +-- .../example/OrderShipmentJoinExample.java | 23 +-- .../samza/example/PageViewCounterExample.java | 22 +-- .../samza/example/RepartitionExample.java | 21 +-- .../apache/samza/example/WindowExample.java | 21 +-- .../EndOfStreamIntegrationTest.java | 15 +- .../WatermarkIntegrationTest.java | 20 +-- .../operator/RepartitionJoinWindowApp.java | 26 ++- .../test/operator/RepartitionWindowApp.java | 28 ++- .../samza/test/operator/SessionWindowApp.java | 21 +-- ...reamApplicationIntegrationTestHarness.java | 66 ++++--- .../TestRepartitionJoinWindowApp.java | 52 +++++- .../operator/TestRepartitionWindowApp.java | 20 ++- .../test/operator/TumblingWindowApp.java | 25 ++- .../test/processor/TestStreamApplication.java | 90 ++++------ .../TestZkLocalApplicationRunner.java | 164 ++++++++--------- .../src/test/resources/test-config.prop | 26 +++ 39 files changed, 901 insertions(+), 710 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/StreamApplications.java create mode 100644 samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java create mode 100644 samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java rename samza-core/src/test/java/org/apache/samza/{testUtils/TestStreamApplication.java => runtime/TestLocalContainerRunner.java} (68%) create mode 100644 samza-test/src/test/resources/test-config.prop diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java new file mode 100644 index 0000000000..42e9cc3fe9 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java @@ -0,0 +1,54 @@ +/* + * 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.application; + +import org.apache.samza.job.ApplicationStatus; + + +/** + * Interface method for all runtime instance of applications + */ +public interface ApplicationRunnable { + /** + * Deploy and run the Samza jobs to execute this application. + * It is non-blocking so it doesn't wait for the application running. + * + */ + void run(); + + /** + * Kill the Samza jobs represented by this application + * It is non-blocking so it doesn't wait for the application stopping. + * + */ + void kill(); + + /** + * Get the collective status of the Samza jobs represented by this application. + * Returns {@link ApplicationStatus} running if all jobs are running. + * + * @return the status of the application + */ + ApplicationStatus status(); + + /** + * Method to wait for the runner in the current JVM process to finish. + */ + void waitForFinish(); +} 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 0b2142b81d..da96606c3a 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 @@ -20,13 +20,19 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; -import org.apache.samza.operators.ContextManager; +import org.apache.samza.config.MapConfig; +import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.StreamGraph; +import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.InitableFunction; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.serializers.Serde; import org.apache.samza.task.StreamTask; -import org.apache.samza.task.TaskContext; + +import java.util.Map; + /** * Describes and initializes the transforms for processing message streams and generating results. @@ -73,25 +79,157 @@ * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ @InterfaceStability.Unstable -public interface StreamApplication { +public class StreamApplication implements ApplicationRunnable { + + /*package private*/ + final ApplicationRunner runner; + final Config config; + final StreamGraph graph; + + StreamApplication(ApplicationRunner runner, Config config) { + this.runner = runner; + this.config = config; + this.graph = runner.createGraph(); + } + + @Override + public final void run() { + this.runner.run(this); + } + + @Override + public final void kill() { + this.runner.kill(this); + } + + @Override + public final ApplicationStatus status() { + return this.runner.status(this); + } + + @Override + public final void waitForFinish() { + this.runner.waitForFinish(this); + } + + public static class AppConfig extends MapConfig { + + public static final String APP_NAME = "app.name"; + public static final String APP_ID = "app.id"; + public static final String APP_CLASS = "app.class"; + public static final String RUNNER_CONFIG = "app.runner.class"; + private static final String DEFAULT_RUNNER_CLASS = "org.apache.samza.runtime.RemoteApplicationRunner"; + + public static final String JOB_NAME = "job.name"; + public static final String JOB_ID = "job.id"; + + public AppConfig(Config config) { + super(config); + } + + public String getAppName() { + return get(APP_NAME, get(JOB_NAME)); + } + + public String getAppId() { + return get(APP_ID, get(JOB_ID, "1")); + } + + public String getAppClass() { + return get(APP_CLASS, null); + } + + public String getApplicationRunnerClass() { + return get(RUNNER_CONFIG, DEFAULT_RUNNER_CLASS); + } + + /** + * Returns full application id + * + * @return full app id + */ + public String getGlobalAppId() { + return String.format("app-%s-%s", getAppName(), getAppId()); + } + + } /** - * Describes and initializes the transforms for processing message streams and generating results. + * Set {@link MetricsReporter}s for this {@link StreamApplication} + * + * @param metricsReporters the map of {@link MetricsReporter}s to be added + * @return this {@link StreamApplication} instance + */ + public StreamApplication withMetricsReporters(Map metricsReporters) { + this.runner.addMetricsReporters(metricsReporters); + return this; + } + + /** + * Return the globally unique application ID for this {@link StreamApplication} + * + * @return the globally unique appplication ID + */ + public String getGlobalAppId() { + return new AppConfig(config).getGlobalAppId(); + } + + /** + * Gets the input {@link MessageStream} corresponding to the {@code streamId}. + *

+ * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param the type of input messages + * @param streamId the input stream name + * @param serde the {@link Serde} object used to deserialize input messages + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + public final MessageStream openInput(String streamId, Serde serde) { + return this.graph.getInputStream(streamId, serde); + } + + /** + * Gets the input {@link MessageStream} corresponding to the {@code streamId}. *

- * The {@link StreamGraph} provides access to input and output streams. Input {@link MessageStream}s can be - * transformed into other {@link MessageStream}s or sent to an {@link OutputStream} using the {@link MessageStream} - * operators. + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param the type of message in the input {@link MessageStream} + * @param streamId the input stream name + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + public final MessageStream openInput(String streamId) { + return this.graph.getInputStream(streamId); + } + + /** + * Gets the {@link OutputStream} corresponding to the {@code streamId}. *

- * Most operators accept custom functions for doing the transformations. These functions are {@link InitableFunction}s - * and are provided the {@link Config} and {@link TaskContext} during their own initialization. The config and the - * context can be used, for example, to create custom metrics or access durable state stores. + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param the type of message in the {@link OutputStream} + * @param output the output stream name + * @param serde the {@link Serde} object used to serialize output messages + * @return the output {@link OutputStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + public final OutputStream openOutput(String output, Serde serde) { + return this.graph.getOutputStream(output, serde); + } + + /** + * Gets the {@link OutputStream} corresponding to the {@code streamId}. *

- * A shared context between {@link InitableFunction}s for different operators within a task instance can be set - * up by providing a {@link ContextManager} using {@link StreamGraph#withContextManager}. + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. * - * @param graph the {@link StreamGraph} to get input/output streams from - * @param config the configuration for the application + * @param the type of message in the {@link OutputStream} + * @param output the output stream name + * @return the output {@link OutputStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} */ - void init(StreamGraph graph, Config config); + public final OutputStream openOutput(String output) { + return this.graph.getOutputStream(output); + } } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java new file mode 100644 index 0000000000..ea6094399b --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java @@ -0,0 +1,45 @@ +/* + * 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.application; + +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.config.Config; +import org.apache.samza.runtime.ApplicationRunner; + +/** + * This class defines the methods to create different types of Samza application instances: 1) high-level end-to-end + * {@link StreamApplication}; 2) task-level StreamTaskApplication; 3) task-level AsyncStreamTaskApplication + */ +public class StreamApplications { + // The static map of all created application instances from the user program + private static final Map USER_APPS = new HashMap<>(); + + private StreamApplications() { + + } + + public static StreamApplication createStreamApp(Config config) { + ApplicationRunner runner = ApplicationRunner.fromConfig(config); + StreamApplication appRuntime = new StreamApplication(runner, config); + USER_APPS.put(appRuntime.getGlobalAppId(), appRuntime); + return appRuntime; + } + +} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index 8339429972..31a1e9c2f0 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -21,24 +21,23 @@ import java.time.Duration; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplication.AppConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.system.StreamSpec; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.StreamGraph; import java.lang.reflect.Constructor; +import java.util.Map; +import org.apache.samza.system.StreamSpec; /** - * The primary means of managing execution of the {@link org.apache.samza.application.StreamApplication} at runtime. + * The primary means of managing execution of the {@link StreamApplication} at runtime. */ @InterfaceStability.Unstable -public abstract class ApplicationRunner { - - private static final String RUNNER_CONFIG = "app.runner.class"; - private static final String DEFAULT_RUNNER_CLASS = "org.apache.samza.runtime.RemoteApplicationRunner"; - - protected final Config config; +public interface ApplicationRunner { /** * Static method to load the {@link ApplicationRunner} @@ -46,29 +45,21 @@ public abstract class ApplicationRunner { * @param config configuration passed in to initialize the Samza processes * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications */ - public static ApplicationRunner fromConfig(Config config) { + static ApplicationRunner fromConfig(Config config) { + AppConfig appCfg = new AppConfig(config); try { - Class runnerClass = Class.forName(config.get(RUNNER_CONFIG, DEFAULT_RUNNER_CLASS)); + Class runnerClass = Class.forName(appCfg.getApplicationRunnerClass()); if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { Constructor constructor = runnerClass.getConstructor(Config.class); // *sigh* return (ApplicationRunner) constructor.newInstance(config); } } catch (Exception e) { - throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", config.get( - RUNNER_CONFIG)), e); + throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", + appCfg.getApplicationRunnerClass()), e); } throw new ConfigException(String.format( "Class %s does not extend ApplicationRunner properly", - config.get(RUNNER_CONFIG))); - } - - - public ApplicationRunner(Config config) { - if (config == null) { - throw new NullPointerException("Parameter 'config' cannot be null."); - } - - this.config = config; + appCfg.getApplicationRunnerClass())); } /** @@ -78,52 +69,39 @@ public ApplicationRunner(Config config) { * * NOTE. this interface will most likely change in the future. */ - @InterfaceStability.Evolving - public abstract void runTask(); + @Deprecated + void runTask(); + void run(StreamApplication app); - /** - * Deploy and run the Samza jobs to execute {@link StreamApplication}. - * It is non-blocking so it doesn't wait for the application running. - * - * @param streamApp the user-defined {@link StreamApplication} object - */ - public abstract void run(StreamApplication streamApp); + void kill(StreamApplication app); - /** - * Kill the Samza jobs represented by {@link StreamApplication} - * It is non-blocking so it doesn't wait for the application stopping. - * - * @param streamApp the user-defined {@link StreamApplication} object - */ - public abstract void kill(StreamApplication streamApp); + ApplicationStatus status(StreamApplication app); + + void waitForFinish(StreamApplication app); /** - * Get the collective status of the Samza jobs represented by {@link StreamApplication}. - * Returns {@link ApplicationRunner} running if all jobs are running. + * Waits for {@code timeout} duration for the application to finish. * - * @param streamApp the user-defined {@link StreamApplication} object - * @return the status of the application + * @param timeout time to wait for the application to finish + * @return true - application finished before timeout + * false - otherwise */ - public abstract ApplicationStatus status(StreamApplication streamApp); + boolean waitForFinish(StreamApplication app, Duration timeout); /** - * Waits until the application finishes. + * Create an empty {@link StreamGraph} object to instantiate the user defined operator DAG. + * + * @return the empty {@link StreamGraph} object to be instantiated */ - public void waitForFinish() { - throw new UnsupportedOperationException(getClass().getName() + " does not support waitForFinish."); - } + StreamGraph createGraph(); /** - * Waits for {@code timeout} duration for the application to finish. + * Method to add a set of customized {@link MetricsReporter}s in the application * - * @param timeout time to wait for the application to finish - * @return true - application finished before timeout - * false - otherwise + * @param metricsReporters the map of customized {@link MetricsReporter}s objects to be used */ - public boolean waitForFinish(Duration timeout) { - throw new UnsupportedOperationException(getClass().getName() + " does not support timed waitForFinish."); - } + void addMetricsReporters(Map metricsReporters); /** * Constructs a {@link StreamSpec} from the configuration for the specified streamId. @@ -133,16 +111,6 @@ public boolean waitForFinish(Duration timeout) { *
* All properties matching this pattern are assumed to be system-specific with two exceptions. The following two * properties are Samza properties which are used to bind the stream to a system and a physical resource on that system. - * - *

    - *
  • samza.system - The name of the System on which this stream will be used. If this property isn't defined - * the stream will be associated with the System defined in {@code job.default.system}
  • - *
  • samza.physical.name - The system-specific name for this stream. It could be a file URN, topic name, or other identifer. - * If this property isn't defined the physical.name will be set to the streamId
  • - *
- * - * @param streamId The logical identifier for the stream in Samza. - * @return The {@link StreamSpec} instance. */ - public abstract StreamSpec getStreamSpec(String streamId); + StreamSpec getStreamSpec(String streamId); } diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java new file mode 100644 index 0000000000..ee83e3843d --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java @@ -0,0 +1,36 @@ +/* + * 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.application; + +import org.apache.samza.operators.StreamGraphSpec; + + +public class StreamApplicationInternal { + + private final StreamApplication app; + + public StreamApplicationInternal(StreamApplication app) { + this.app = app; + } + + public StreamGraphSpec getStreamGraphSpec() { + return (StreamGraphSpec) this.app.graph; + } + +} diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 843db85b1a..37f6650d76 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -196,12 +196,6 @@ StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec, boolean isIntermediate) return edge; } - /** - * Get the {@link StreamEdge} for a {@link StreamSpec}. Create one if it does not exist. - * @param streamSpec spec of the StreamEdge - * @return stream edge - */ - /** * Returns the job nodes to be executed in the topological order * @return unmodifiable list of {@link JobNode} diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 5043977602..aaa6511b20 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -29,7 +29,9 @@ import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.ExecutionPlanner; import org.apache.samza.execution.StreamManager; +import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmins; @@ -47,19 +49,23 @@ /** * Defines common, core behavior for implementations of the {@link ApplicationRunner} API. */ -public abstract class AbstractApplicationRunner extends ApplicationRunner { +public abstract class AbstractApplicationRunner implements ApplicationRunner { private static final Logger log = LoggerFactory.getLogger(AbstractApplicationRunner.class); private final StreamManager streamManager; private final SystemAdmins systemAdmins; + protected final Config config; + protected final Map metricsReporters = new HashMap<>(); + /** * The {@link ApplicationRunner} is supposed to run a single {@link StreamApplication} instance in the full life-cycle */ + // TODO: need to revisit after refactory and merge the supported application instances protected final StreamGraphSpec graphSpec; public AbstractApplicationRunner(Config config) { - super(config); + this.config = config; this.graphSpec = new StreamGraphSpec(this, config); this.systemAdmins = new SystemAdmins(config); this.streamManager = new StreamManager(systemAdmins); @@ -132,9 +138,8 @@ public ExecutionPlan getExecutionPlan(StreamApplication app) throws Exception { /* package private */ ExecutionPlan getExecutionPlan(StreamApplication app, String runId) throws Exception { - // build stream graph - app.init(graphSpec, config); - + // get the already initialized operatorSpec + // TODO: revisit later to see where to get the graphSpec OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); // create the physical execution plan Map cfg = new HashMap<>(config); @@ -157,6 +162,11 @@ StreamManager getStreamManager() { return streamManager; } + @Override + public final StreamGraph createGraph() { + return this.graphSpec; + } + /** * Write the execution plan JSON to a file * @param planJson JSON representation of the plan @@ -178,4 +188,9 @@ final void writePlanJsonFile(String planJson) { } } + @Override + public final void addMetricsReporters(Map metricsReporters) { + this.metricsReporters.putAll(metricsReporters); + } + } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java new file mode 100644 index 0000000000..1e672a0376 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.runtime; + +import joptsimple.OptionSet; +import joptsimple.OptionSpec; +import org.apache.samza.util.CommandLine; + + +/** + * The class defines the basic command line arguments for Samza command line scripts. + */ +public class ApplicationRunnerCommandLine extends CommandLine { + public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; run, status, kill.") + .withRequiredArg() + .ofType(String.class) + .describedAs("operation=run") + .defaultsTo("run"); + + public ApplicationRunnerOperation getOperation(OptionSet options) { + String rawOp = options.valueOf(operationOpt).toString(); + return ApplicationRunnerOperation.fromString(rawOp); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java index 1fd60fc83a..1602681645 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java @@ -20,7 +20,7 @@ package org.apache.samza.runtime; /** - * Operation to perform in the {@link ApplicationRunnerMain} + * Operation to perform in the {@link RemoteApplicationRunner} */ public enum ApplicationRunnerOperation { RUN("run"), KILL("kill"), STATUS("status"); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index d64e57a332..8791b6c11f 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import java.time.Duration; -import java.util.HashMap; import java.util.List; import java.util.Set; import java.util.UUID; @@ -33,6 +32,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationInternal; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -41,19 +41,24 @@ import org.apache.samza.coordinator.CoordinationUtils; import org.apache.samza.coordinator.DistributedLockWithState; import org.apache.samza.execution.ExecutionPlan; +import org.apache.samza.execution.ExecutionPlanner; +import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.system.StreamSpec; +import org.apache.samza.system.SystemAdmins; import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.samza.util.ScalaJavaUtil.defaultValue; + /** - * This class implements the {@link ApplicationRunner} that runs the applications in standalone environment + * This class implements the {@link StreamApplication} that runs the applications in standalone environment */ public class LocalApplicationRunner extends AbstractApplicationRunner { @@ -66,6 +71,8 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private final AtomicInteger numProcessorsToStart = new AtomicInteger(); private final AtomicReference failure = new AtomicReference<>(); + private StreamManager streamManager; + private ExecutionPlanner planner; private ApplicationStatus appStatus = ApplicationStatus.New; final class LocalStreamProcessorLifeCycleListener implements StreamProcessorLifecycleListener { @@ -129,11 +136,12 @@ public LocalApplicationRunner(Config config) { } @Override + @Deprecated public void runTask() { - JobConfig jobConfig = new JobConfig(this.config); + JobConfig jobConfig = new JobConfig(config); // validation - String taskName = new TaskConfig(config).getTaskClass().getOrElse(null); + String taskName = new TaskConfig(config).getTaskClass().getOrElse(defaultValue(null)); if (taskName == null) { throw new SamzaException("Neither APP nor task.class are defined defined"); } @@ -148,12 +156,14 @@ public void runTask() { } @Override - public void run(StreamApplication app) { + public void run(StreamApplication userApp) { + StreamApplicationInternal app = new StreamApplicationInternal(userApp); + this.streamManager = new StreamManager(new SystemAdmins(config)); + this.planner = new ExecutionPlanner(config, streamManager); try { - super.run(app); + super.run(userApp); // 1. initialize and plan ExecutionPlan plan = getExecutionPlan(app); - String executionPlanJson = plan.getPlanAsJson(); writePlanJsonFile(executionPlanJson); LOG.info("Execution Plan: \n" + executionPlanJson); @@ -185,14 +195,19 @@ public void run(StreamApplication app) { } } + @VisibleForTesting + ExecutionPlan getExecutionPlan(StreamApplicationInternal app) throws Exception { + return this.planner.plan(app.getStreamGraphSpec().getOperatorSpecGraph()); + } + @Override - public void kill(StreamApplication streamApp) { + public void kill(StreamApplication app) { processors.forEach(StreamProcessor::stop); - super.kill(streamApp); + super.kill(app); } @Override - public ApplicationStatus status(StreamApplication streamApp) { + public ApplicationStatus status(StreamApplication app) { return appStatus; } @@ -200,8 +215,8 @@ public ApplicationStatus status(StreamApplication streamApp) { * Waits until the application finishes. */ @Override - public void waitForFinish() { - waitForFinish(Duration.ofMillis(0)); + public void waitForFinish(StreamApplication app) { + waitForFinish(app, Duration.ofMillis(0)); } /** @@ -213,7 +228,7 @@ public void waitForFinish() { * false - otherwise */ @Override - public boolean waitForFinish(Duration timeout) { + public boolean waitForFinish(StreamApplication app, Duration timeout) { long timeoutInMs = timeout.toMillis(); boolean finished = true; @@ -244,7 +259,7 @@ public boolean waitForFinish(Duration timeout) { * @param intStreams list of intermediate {@link StreamSpec}s * @throws TimeoutException exception for latch timeout */ - /* package private */ void createStreams(String planId, List intStreams) throws TimeoutException { + /* package private */ void createStreams(String planId, List intStreams) throws TimeoutException { if (intStreams.isEmpty()) { LOG.info("Set of intermediate streams is empty. Nothing to create."); return; @@ -282,12 +297,17 @@ public boolean waitForFinish(Duration timeout) { } } + @VisibleForTesting + StreamManager getStreamManager() { + return streamManager; + } + /** * Create {@link StreamProcessor} based on {@link StreamApplication} and the config * @param config config * @return {@link StreamProcessor]} */ - /* package private */ + @VisibleForTesting StreamProcessor createStreamProcessor( Config config, StreamProcessorLifecycleListener listener) { @@ -313,10 +333,10 @@ StreamProcessor createStreamProcessor( private StreamProcessor getStreamProcessorInstance(Config config, Object taskFactory, StreamProcessorLifecycleListener listener) { if (taskFactory instanceof StreamTaskFactory) { return new StreamProcessor( - config, new HashMap<>(), (StreamTaskFactory) taskFactory, listener); + config, this.metricsReporters, (StreamTaskFactory) taskFactory, listener); } else if (taskFactory instanceof AsyncStreamTaskFactory) { return new StreamProcessor( - config, new HashMap<>(), (AsyncStreamTaskFactory) taskFactory, listener); + config, this.metricsReporters, (AsyncStreamTaskFactory) taskFactory, listener); } else { throw new SamzaException(String.format("%s is not a valid task factory", taskFactory.getClass().getCanonicalName())); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 66176d77f5..683dcfc543 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -19,6 +19,11 @@ package org.apache.samza.runtime; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.lang.reflect.Method; +import java.time.Duration; import java.util.HashMap; import java.util.Random; import org.apache.log4j.MDC; @@ -37,9 +42,12 @@ import org.apache.samza.job.model.JobModel; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.ScalaJavaUtil; +import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.samza.util.ScalaJavaUtil.defaultValue; + /** * LocalContainerRunner is the local runner for Yarn {@link SamzaContainer}s. It is an intermediate step to * have a local runner for yarn before we consolidate the Yarn container and coordination into a @@ -101,7 +109,7 @@ public void onContainerFailed(Throwable t) { startContainerHeartbeatMonitor(); container.run(); stopContainerHeartbeatMonitor(); - + if (containerRunnerException != null) { log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); System.exit(1); @@ -110,24 +118,53 @@ public void onContainerFailed(Throwable t) { private Object getTaskFactory(StreamApplication streamApp) { if (streamApp != null) { - streamApp.init(graphSpec, config); + // TODO: should already be initialized. + // streamApp.init(graphSpec, config); return TaskFactoryUtil.createTaskFactory(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager()); } return TaskFactoryUtil.createTaskFactory(config); } @Override - public void kill(StreamApplication streamApp) { + public void kill(StreamApplication userApp) { // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted throw new UnsupportedOperationException(); } @Override - public ApplicationStatus status(StreamApplication streamApp) { + public ApplicationStatus status(StreamApplication userApp) { // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted throw new UnsupportedOperationException(); } + @Override + public void waitForFinish(StreamApplication userApp) { + + } + + @Override + public boolean waitForFinish(StreamApplication app, Duration timeout) { + return false; + } + + private static File writeConfigToTmpFile(Config config) throws IOException { + File tmpFile = File.createTempFile("config", ""); + FileWriter fileWriter = new FileWriter(tmpFile); + try { + config.forEach((k, v) -> { + try { + fileWriter.write(String.format("%s=%s\n", k, v)); + } catch (IOException e) { + throw new SamzaException("Failed to create a temporary config file for user application", e); + } + }); + } finally { + fileWriter.close(); + } + return tmpFile; + } + + // only invoked by legacy applications w/o user-defined main public static void main(String[] args) throws Exception { Thread.setDefaultUncaughtExceptionHandler( new SamzaUncaughtExceptionHandler(() -> { @@ -149,16 +186,27 @@ public static void main(String[] args) throws Exception { throw new SamzaException("can not find the job name"); } String jobName = jobConfig.getName().get(); - String jobId = jobConfig.getJobId().getOrElse(ScalaJavaUtil.defaultValue("1")); + String jobId = jobConfig.getJobId().getOrElse(defaultValue("1")); MDC.put("containerName", "samza-container-" + containerId); MDC.put("jobName", jobName); MDC.put("jobId", jobId); - StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config); - LocalContainerRunner localContainerRunner = new LocalContainerRunner(jobModel, containerId); - localContainerRunner.run(streamApp); - - System.exit(0); + StreamApplication.AppConfig appConfig = new StreamApplication.AppConfig(config); + + if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { + // add configuration-factory and configuration-path to the command line options and invoke the user defined main class + // write the complete configuration to a local file in property file format + config.put(StreamApplication.AppConfig.RUNNER_CONFIG, LocalContainerRunner.class.getName()); + File tmpFile = writeConfigToTmpFile(config); + Class cls = Class.forName(appConfig.getAppClass()); + Method mainMethod = cls.getMethod("main", String[].class); + String[] params = new String[] {"--configuration-path", String.format("%s", tmpFile.getAbsoluteFile())}; + mainMethod.invoke(null, (Object) params); + } else { + LocalContainerRunner localContainerRunner = new LocalContainerRunner(jobModel, containerId); + // run with app = null force to load the task class from configuration + localContainerRunner.run(null); + } } private void startContainerHeartbeatMonitor() { @@ -187,4 +235,5 @@ private void stopContainerHeartbeatMonitor() { containerHeartbeatMonitor.stop(); } } + } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 202fa7600c..13686147b7 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -22,19 +22,21 @@ import java.time.Duration; import org.apache.samza.SamzaException; import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.application.StreamApplicationInternal; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.coordinator.stream.CoordinatorStreamSystemConsumer; import org.apache.samza.execution.ExecutionPlan; +import org.apache.samza.execution.ExecutionPlanner; +import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.JobRunner; import org.apache.samza.metrics.MetricsRegistryMap; +import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.system.SystemAdmins; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.UUID; - import static org.apache.samza.job.ApplicationStatus.*; @@ -46,37 +48,41 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; + private final StreamManager streamManager; + private final ExecutionPlanner planner; + public RemoteApplicationRunner(Config config) { super(config); + this.streamManager = new StreamManager(new SystemAdmins(config)); + this.planner = new ExecutionPlanner(config, this.streamManager); } - @Override + @Deprecated public void runTask() { throw new UnsupportedOperationException("Running StreamTask is not implemented for RemoteReplicationRunner"); } - /** - * Run the {@link StreamApplication} on the remote cluster - * @param app a StreamApplication - */ @Override - public void run(StreamApplication app) { - try { - super.run(app); - // TODO: run.id needs to be set for standalone: SAMZA-1531 - // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision - String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The run id for this run is {}", runId); + public void waitForFinish(StreamApplication app) { + throw new UnsupportedOperationException("waitForFinish is not supported in RemoteApplicationRunner"); + } + @Override + public boolean waitForFinish(StreamApplication app, Duration timeout) { + throw new UnsupportedOperationException("waitForFinish is not supported in RemoteApplicationRunner"); + } + + @Override + public void run(StreamApplication userApp) { + StreamApplicationInternal app = new StreamApplicationInternal(userApp); + try { // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(app, runId); + OperatorSpecGraph specGraph = app.getStreamGraphSpec().getOperatorSpecGraph(); + ExecutionPlan plan = getExecutionPlan(specGraph); writePlanJsonFile(plan.getPlanAsJson()); // 2. create the necessary streams - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - getStreamManager().clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - getStreamManager().createStreams(plan.getIntermediateStreams()); + this.streamManager.createStreams(plan.getIntermediateStreams()); // 3. submit jobs for remote execution plan.getJobConfigs().forEach(jobConfig -> { @@ -89,30 +95,38 @@ public void run(StreamApplication app) { } } + private ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) throws Exception { + return this.planner.plan(specGraph); + } + @Override - public void kill(StreamApplication app) { + public void kill(StreamApplication userApp) { + StreamApplicationInternal app = new StreamApplicationInternal(userApp); try { - ExecutionPlan plan = getExecutionPlan(app); + OperatorSpecGraph specGraph = app.getStreamGraphSpec().getOperatorSpecGraph(); + ExecutionPlan plan = getExecutionPlan(specGraph); plan.getJobConfigs().forEach(jobConfig -> { LOG.info("Killing job {}", jobConfig.getName()); JobRunner runner = new JobRunner(jobConfig); runner.kill(); }); - super.kill(app); + super.kill(userApp); } catch (Throwable t) { throw new SamzaException("Failed to kill application", t); } } @Override - public ApplicationStatus status(StreamApplication app) { + public ApplicationStatus status(StreamApplication userApp) { + StreamApplicationInternal app = new StreamApplicationInternal(userApp); try { boolean hasNewJobs = false; boolean hasRunningJobs = false; ApplicationStatus unsuccessfulFinishStatus = null; - ExecutionPlan plan = getExecutionPlan(app); + OperatorSpecGraph specGraph = app.getStreamGraphSpec().getOperatorSpecGraph(); + ExecutionPlan plan = getExecutionPlan(specGraph); for (JobConfig jobConfig : plan.getJobConfigs()) { ApplicationStatus status = getApplicationStatus(jobConfig); diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index 38ae854ebb..fb6438ec3a 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -19,10 +19,8 @@ package org.apache.samza.task; import org.apache.samza.SamzaException; -import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.TaskConfig; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.OperatorSpecGraph; @@ -32,7 +30,6 @@ import java.util.concurrent.ExecutorService; import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; -import static org.apache.samza.util.ScalaJavaUtil.defaultValue; /** * This class provides utility functions to load task factory classes based on config, and to wrap {@link StreamTaskFactory} in {@link AsyncStreamTaskFactory} @@ -42,7 +39,7 @@ public class TaskFactoryUtil { private static final Logger log = LoggerFactory.getLogger(TaskFactoryUtil.class); /** - * This method creates a task factory class based on the {@link StreamApplication} + * This method creates a task factory class based on the {@link OperatorSpecGraph} * * @param specGraph the {@link OperatorSpecGraph} * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph} @@ -160,33 +157,4 @@ private static void validateFactory(Object factory) { } } - /** - * Returns {@link StreamApplication} if it's configured, otherwise null. - * @param config Config - * throws {@link ConfigException} if there is misconfiguration of StreamApp. - * @return {@link StreamApplication} instance - */ - public static StreamApplication createStreamApplication(Config config) { - ApplicationConfig appConfig = new ApplicationConfig(config); - if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { - TaskConfig taskConfig = new TaskConfig(config); - String taskClassName = taskConfig.getTaskClass().getOrElse(defaultValue(null)); - if (taskClassName != null && !taskClassName.isEmpty()) { - throw new ConfigException("High level StreamApplication API cannot be used together with low-level API using task.class."); - } - - String appClassName = appConfig.getAppClass(); - try { - Class builderClass = Class.forName(appClassName); - return (StreamApplication) builderClass.newInstance(); - } catch (Throwable t) { - String errorMsg = String.format("Failed to create StreamApplication class from the config. %s = %s", - ApplicationConfig.APP_CLASS, appConfig.getAppClass()); - log.error(errorMsg, t); - throw new ConfigException(errorMsg, t); - } - } else { - return null; - } - } } diff --git a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala index c6e14f2061..22b80cda5b 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala @@ -27,10 +27,8 @@ import org.apache.samza.config.Config import org.apache.samza.config.JobConfig.Config2Job import org.apache.samza.coordinator.stream.{CoordinatorStreamSystemConsumer, CoordinatorStreamSystemProducer} import org.apache.samza.coordinator.stream.messages.{Delete, SetConfig} -import org.apache.samza.job.ApplicationStatus.{Running, SuccessfulFinish} import org.apache.samza.metrics.MetricsRegistryMap -import org.apache.samza.runtime.ApplicationRunnerMain.ApplicationRunnerCommandLine -import org.apache.samza.runtime.ApplicationRunnerOperation +import org.apache.samza.runtime.{ApplicationRunnerCommandLine, ApplicationRunnerOperation} import org.apache.samza.system.StreamSpec import org.apache.samza.util.{CoordinatorStreamUtil, Logging, Util} diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index 029b37533e..76354962a6 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -22,12 +22,12 @@ package org.apache.samza.job.local import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ -import org.apache.samza.container.{SamzaContainer, SamzaContainerListener, TaskName} +import org.apache.samza.container.TaskName +import org.apache.samza.container.{SamzaContainer, SamzaContainerListener} import org.apache.samza.coordinator.JobModelManager import org.apache.samza.coordinator.stream.CoordinatorStreamManager import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} -import org.apache.samza.operators.StreamGraphSpec import org.apache.samza.runtime.LocalContainerRunner import org.apache.samza.storage.ChangelogStreamManager import org.apache.samza.task.TaskFactoryUtil @@ -71,16 +71,11 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val containerId = "0" val jmxServer = new JmxServer - val streamApp = TaskFactoryUtil.createStreamApplication(config) - val appRunner = new LocalContainerRunner(jobModel, "0") - val taskFactory = if (streamApp != null) { - val graphSpec = new StreamGraphSpec(appRunner, config) - streamApp.init(graphSpec, config) - TaskFactoryUtil.createTaskFactory(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager) - } else { - TaskFactoryUtil.createTaskFactory(config) - } + // TODO: ThreadJobFactory does not support launch StreamApplication. Launching user-defined StreamApplication is via new + // user program w/ main(). + val appRunner = new LocalContainerRunner(jobModel, "0") + val taskFactory = TaskFactoryUtil.createTaskFactory(config) // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job. config.getTaskOpts match { 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 83fe5ad757..5d7eff8b9b 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 @@ -48,8 +48,11 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestExecutionPlanner { diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/TestStreamApplication.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java similarity index 68% rename from samza-core/src/test/java/org/apache/samza/testUtils/TestStreamApplication.java rename to samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java index a1cba7dd4d..45bd6cebcb 100644 --- a/samza-core/src/test/java/org/apache/samza/testUtils/TestStreamApplication.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java @@ -16,18 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.testUtils; - -import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.application.StreamApplication; +package org.apache.samza.runtime; /** - * Test implementation class for {@link StreamApplication} + * TODO: adding tests to unit test {@link LocalContainerRunner} */ -public class TestStreamApplication implements StreamApplication { - @Override - public void init(StreamGraph graph, Config config) { - - } +public class TestLocalContainerRunner { } diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index e2077727c7..712e8d2ffe 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -72,123 +72,6 @@ public void testStreamTaskClass() { } } - @Test - public void testCreateStreamApplication() throws Exception { - Config config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.TestStreamApplication"); - } - }); - StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config); - assertNotNull(streamApp); - StreamGraphSpec graph = new StreamGraphSpec(mockRunner, config); - streamApp.init(graph, config); - Object retFactory = TaskFactoryUtil.createTaskFactory(graph.getOperatorSpecGraph(), null); - assertTrue(retFactory instanceof StreamTaskFactory); - assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof StreamOperatorTask); - - config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.InvalidStreamApplication"); - } - }); - try { - TaskFactoryUtil.createStreamApplication(config); - fail("Should have failed w/ no.such.class"); - } catch (ConfigException ce) { - // expected - } - - config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, "no.such.class"); - } - }); - try { - TaskFactoryUtil.createStreamApplication(config); - fail("Should have failed w/ no.such.class"); - } catch (ConfigException ce) { - // expected - } - - config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, ""); - } - }); - streamApp = TaskFactoryUtil.createStreamApplication(config); - assertNull(streamApp); - - config = new MapConfig(new HashMap<>()); - streamApp = TaskFactoryUtil.createStreamApplication(config); - assertNull(streamApp); - } - - @Test - public void testCreateStreamApplicationWithTaskClass() throws Exception { - Config config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.TestStreamApplication"); - } - }); - StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config); - assertNotNull(streamApp); - - config = new MapConfig(new HashMap() { - { - this.put("task.class", "org.apache.samza.testUtils.TestAsyncStreamTask"); - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.TestStreamApplication"); - } - }); - try { - TaskFactoryUtil.createStreamApplication(config); - fail("should have failed with invalid config"); - } catch (ConfigException ce) { - // expected - } - - config = new MapConfig(new HashMap() { - { - this.put("task.class", "no.such.class"); - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.TestStreamApplication"); - } - }); - try { - TaskFactoryUtil.createStreamApplication(config); - fail("should have failed with invalid config"); - } catch (ConfigException ce) { - // expected - } - - - config = new MapConfig(new HashMap() { - { - this.put("task.class", ""); - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.TestStreamApplication"); - } - }); - streamApp = TaskFactoryUtil.createStreamApplication(config); - assertNotNull(streamApp); - - } - - @Test - public void testStreamTaskClassWithInvalidStreamApplication() throws Exception { - - Config config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.InvalidStreamApplication"); - } - }); - try { - TaskFactoryUtil.createStreamApplication(config); - fail("Should have failed w/ no.such.class"); - } catch (ConfigException ce) { - // expected - } - - } - @Test public void testAsyncStreamTask() { Config config = new MapConfig(new HashMap() { @@ -213,22 +96,6 @@ public void testAsyncStreamTask() { } } - @Test - public void testAsyncStreamTaskWithInvalidStreamGraphBuilder() throws Exception { - - Config config = new MapConfig(new HashMap() { - { - this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.InvalidStreamApplication"); - } - }); - try { - TaskFactoryUtil.createStreamApplication(config); - fail("Should have failed w/ no.such.class"); - } catch (ConfigException cfe) { - // expected - } - } - @Test public void testFinalizeTaskFactory() throws NoSuchFieldException, IllegalAccessException { Object mockFactory = mock(Object.class); diff --git a/samza-shell/src/main/bash/run-app.sh b/samza-shell/src/main/bash/run-app.sh index 3880e3c04e..f8849c0fd2 100755 --- a/samza-shell/src/main/bash/run-app.sh +++ b/samza-shell/src/main/bash/run-app.sh @@ -27,4 +27,4 @@ mkdir -p $EXECUTION_PLAN_DIR [[ $JAVA_OPTS != *-Dlog4j.configuration* ]] && export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$(dirname $0)/log4j-console.xml" -exec $(dirname $0)/run-class.sh org.apache.samza.runtime.ApplicationRunnerMain "$@" +exec $(dirname $0)/run-class.sh "$@" 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 c029eb449f..4973761c54 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 @@ -22,14 +22,14 @@ import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; -import org.apache.samza.operators.StreamGraph; +import java.util.HashMap; +import org.apache.samza.application.StreamApplications; +import org.apache.samza.serializers.JsonSerdeV2; +import org.apache.samza.serializers.KVSerde; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; -import org.apache.samza.serializers.JsonSerdeV2; -import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; @@ -37,27 +37,24 @@ /** * Example code to implement window-based counter */ -public class AppWithGlobalConfigExample implements StreamApplication { +public class AppWithGlobalConfigExample { // local execution mode public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - AppWithGlobalConfigExample app = new AppWithGlobalConfigExample(); - runner.run(app); - runner.waitForFinish(); - } - @Override - public void init(StreamGraph graph, Config config) { - graph.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) + StreamApplication app = StreamApplications.createStreamApp(config).withMetricsReporters(new HashMap<>()); + app.openInput("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) .map(KV::getValue) .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), "w1") + .setAccumulationMode(AccumulationMode.DISCARDING), "window1") .map(m -> KV.of(m.getKey().getKey(), new PageViewCount(m))) - .sendTo(graph.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); + .sendTo(app.openOutput("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); + + app.run(); + app.waitForFinish(); } 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 9ca4f351b9..d6c238dece 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 @@ -19,11 +19,10 @@ package org.apache.samza.example; +import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.operators.MessageStream; @@ -34,28 +33,23 @@ /** * Example implementation of a task that splits its input into multiple output streams. */ -public class BroadcastExample implements StreamApplication { +public class BroadcastExample { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = new BroadcastExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); - MessageStream> inputStream = graph.getInputStream("pageViewEventStream", pgeMsgSerde); + StreamApplication app = StreamApplications.createStreamApp(config); + MessageStream> inputStream = app.openInput("pageViewEventStream", pgeMsgSerde); + + inputStream.filter(m -> m.key.equals("key1")).sendTo(app.openOutput("outStream1", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key2")).sendTo(app.openOutput("outStream2", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key3")).sendTo(app.openOutput("outStream3", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key1")).sendTo(graph.getOutputStream("outStream1", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key2")).sendTo(graph.getOutputStream("outStream2", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key3")).sendTo(graph.getOutputStream("outStream3", pgeMsgSerde)); + app.run(); + app.waitForFinish(); } 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 9edaabefd1..e47b3e6519 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 @@ -19,14 +19,13 @@ package org.apache.samza.example; +import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FlatMapFunction; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -43,25 +42,18 @@ /** * Example code using {@link KeyValueStore} to implement event-time window */ -public class KeyValueStoreExample implements StreamApplication { +public class KeyValueStoreExample { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - KeyValueStoreExample app = new KeyValueStoreExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); + StreamApplication app = StreamApplications.createStreamApp(config); - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { MessageStream pageViewEvents = - graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + app.openInput("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = - graph.getOutputStream("pageViewEventPerMember", + app.openOutput("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class))); pageViewEvents @@ -71,6 +63,9 @@ public void init(StreamGraph graph, Config config) { .flatMap(new MyStatsCounter()) .map(stats -> KV.of(stats.memberId, stats)) .sendTo(pageViewEventPerMember); + + app.run(); + app.waitForFinish(); } static class MyStatsCounter implements FlatMapFunction { 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 ff983a43b3..150fe82e7d 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 @@ -20,39 +20,32 @@ package org.apache.samza.example; import com.google.common.collect.ImmutableList; +import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; -public class MergeExample implements StreamApplication { +public class MergeExample { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - MergeExample app = new MergeExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication app = StreamApplications.createStreamApp(config); KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); - MessageStream.mergeAll(ImmutableList.of(graph.getInputStream("viewStream1", pgeMsgSerde), - graph.getInputStream("viewStream2", pgeMsgSerde), graph.getInputStream("viewStream3", pgeMsgSerde))) - .sendTo(graph.getOutputStream("mergedStream", pgeMsgSerde)); + MessageStream.mergeAll(ImmutableList.of(app.openInput("viewStream1", pgeMsgSerde), + app.openInput("viewStream2", pgeMsgSerde), app.openInput("viewStream3", pgeMsgSerde))) + .sendTo(app.openOutput("mergedStream", pgeMsgSerde)); + app.run(); + app.waitForFinish(); } 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 1c0bc2532a..ac951c5542 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 @@ -19,13 +19,12 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.JoinFunction; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -37,28 +36,20 @@ /** * Simple 2-way stream-to-stream join example */ -public class OrderShipmentJoinExample implements StreamApplication { +public class OrderShipmentJoinExample { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - OrderShipmentJoinExample app = new OrderShipmentJoinExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication app = StreamApplications.createStreamApp(config); MessageStream orders = - graph.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class)); + app.openInput("orders", new JsonSerdeV2<>(OrderRecord.class)); MessageStream shipments = - graph.getInputStream("shipments", new JsonSerdeV2<>(ShipmentRecord.class)); + app.openInput("shipments", new JsonSerdeV2<>(ShipmentRecord.class)); OutputStream> fulfilledOrders = - graph.getOutputStream("fulfilledOrders", + app.openOutput("fulfilledOrders", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(FulfilledOrderRecord.class))); orders @@ -68,6 +59,8 @@ public void init(StreamGraph graph, Config config) { .map(fulFilledOrder -> KV.of(fulFilledOrder.orderId, fulFilledOrder)) .sendTo(fulfilledOrders); + app.run(); + app.waitForFinish(); } static class MyJoinFunction implements JoinFunction { 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 2581506646..c511de5ab8 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 @@ -20,47 +20,37 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FoldLeftFunction; import org.apache.samza.operators.functions.SupplierFunction; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; - /** * Example code to implement window-based counter */ -public class PageViewCounterExample implements StreamApplication { +public class PageViewCounterExample { // local execution mode public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - PageViewCounterExample app = new PageViewCounterExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication appRuntime = StreamApplications.createStreamApp(config); MessageStream pageViewEvents = null; - pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + pageViewEvents = appRuntime.openInput("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMemberStream = - graph.getOutputStream("pageViewEventPerMemberStream", + appRuntime.openOutput("pageViewEventPerMemberStream", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); SupplierFunction initialValue = () -> 0; @@ -72,6 +62,8 @@ public void init(StreamGraph graph, Config config) { .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane))) .sendTo(pageViewEventPerMemberStream); + appRuntime.run(); + appRuntime.waitForFinish(); } class PageViewEvent { 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 7f283466c4..8761aab39a 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 @@ -19,14 +19,13 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -38,26 +37,18 @@ /** * Example {@link StreamApplication} code to test the API methods with re-partition operator */ -public class RepartitionExample implements StreamApplication { +public class RepartitionExample { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - RepartitionExample app = new RepartitionExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication app = StreamApplications.createStreamApp(config); MessageStream pageViewEvents = - graph.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); + app.openInput("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = - graph.getOutputStream("pageViewEventPerMember", + app.openOutput("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class))); pageViewEvents @@ -68,6 +59,8 @@ public void init(StreamGraph graph, Config config) { .map(windowPane -> KV.of(windowPane.getKey().getKey(), new MyStreamOutput(windowPane))) .sendTo(pageViewEventPerMember); + app.run(); + app.waitForFinish(); } class PageViewEvent { 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 4950695226..f1f2375f58 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 @@ -20,16 +20,15 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FoldLeftFunction; import org.apache.samza.operators.functions.SupplierFunction; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.util.CommandLine; @@ -41,26 +40,18 @@ * Example implementation of a simple user-defined task w/ a window operator. * */ -public class WindowExample implements StreamApplication { +public class WindowExample { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - WindowExample app = new WindowExample(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication app = StreamApplications.createStreamApp(config); SupplierFunction initialValue = () -> 0; FoldLeftFunction counter = (m, c) -> c == null ? 1 : c + 1; - MessageStream inputStream = graph.getInputStream("inputStream", new JsonSerdeV2()); - OutputStream outputStream = graph.getOutputStream("outputStream", new IntegerSerde()); + MessageStream inputStream = app.openInput("inputStream", new JsonSerdeV2()); + OutputStream outputStream = app.openOutput("outputStream", new IntegerSerde()); // 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 @@ -72,6 +63,8 @@ public void init(StreamGraph graph, Config config) { .map(WindowPane::getMessage) .sendTo(outputStream); + app.run(); + app.waitForFinish(); } class PageViewEvent { 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 3301af8bad..69fd816ce6 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 @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Random; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; @@ -32,7 +33,6 @@ import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory; import org.apache.samza.operators.KV; import org.apache.samza.operators.functions.MapFunction; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory; import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; import org.apache.samza.test.controlmessages.TestData.PageView; @@ -43,6 +43,7 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; +import static junit.framework.Assert.*; /** * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input. @@ -90,18 +91,16 @@ public void testPipeline() throws Exception { configs.put("serializers.registry.int.class", "org.apache.samza.serializers.IntegerSerdeFactory"); configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName()); - final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - final StreamApplication app = (streamGraph, cfg) -> { - streamGraph.>getInputStream("PageView") + final StreamApplication app = StreamApplications.createStreamApp(new MapConfig(configs)); + + app.>openInput("PageView") .map(Values.create()) .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") .sink((m, collector, coordinator) -> { received.add(m.getValue()); }); - }; - - runner.run(app); - runner.waitForFinish(); + app.run(); + app.waitForFinish(); assertEquals(received.size(), count * partitionCount); } 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 d4dc4ed126..c6628c7b19 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 @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; +import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -142,22 +143,18 @@ public void testWatermark() throws Exception { configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName()); List received = new ArrayList<>(); - final StreamApplication app = (streamGraph, cfg) -> { - streamGraph.>getInputStream("PageView") + final StreamApplication app = StreamApplications.createStreamApp(new MapConfig(configs)); + app.>openInput("PageView") .map(EndOfStreamIntegrationTest.Values.create()) .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") .sink((m, collector, coordinator) -> { received.add(m.getValue()); }); - }; + app.run(); + Map tasks = getTaskOperationGraphs(app); - LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - runner.run(app); - // processors are only available when the app is running - Map tasks = getTaskOperationGraphs(runner); + app.waitForFinish(); - runner.waitForFinish(); - // wait for the completion to ensure that all tasks are actually initialized and the OperatorImplGraph is initialized StreamOperatorTask task0 = tasks.get("Partition 0"); OperatorImplGraph graph = TestStreamOperatorTask.getOperatorImplGraph(task0); OperatorImpl pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY); @@ -177,7 +174,10 @@ public void testWatermark() throws Exception { assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3); } - Map getTaskOperationGraphs(LocalApplicationRunner runner) throws Exception { + Map getTaskOperationGraphs(StreamApplication app) throws Exception { + Field appRunnerField = StreamApplication.class.getDeclaredField("runner"); + appRunnerField.setAccessible(true); + LocalApplicationRunner runner = (LocalApplicationRunner) appRunnerField.get(app); StreamProcessor processor = TestLocalApplicationRunner.getProcessors(runner).iterator().next(); SamzaContainer container = TestStreamProcessorUtil.getContainer(processor); Map taskInstances = JavaConverters.mapAsJavaMapConverter(container.getTaskInstances()).asJava(); 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 2171d0707a..54d57d048d 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 @@ -22,14 +22,13 @@ import java.util.ArrayList; import java.util.List; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -48,33 +47,26 @@ /** * A {@link StreamApplication} that demonstrates a partitionBy, stream-stream join and a windowed count. */ -public class RepartitionJoinWindowApp implements StreamApplication { +public class RepartitionJoinWindowApp { public static final String INPUT_TOPIC_NAME_1_PROP = "inputTopicName1"; public static final String INPUT_TOPIC_NAME_2_PROP = "inputTopicName2"; public static final String OUTPUT_TOPIC_NAME_PROP = "outputTopicName"; - private final List intermediateStreams = new ArrayList<>(); + private static final List intermediateStreams = new ArrayList<>(); public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - RepartitionJoinWindowApp application = new RepartitionJoinWindowApp(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(application); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { String inputTopicName1 = config.get(INPUT_TOPIC_NAME_1_PROP); String inputTopicName2 = config.get(INPUT_TOPIC_NAME_2_PROP); String outputTopic = config.get(OUTPUT_TOPIC_NAME_PROP); - MessageStream pageViews = graph.getInputStream(inputTopicName1, new JsonSerdeV2<>(PageView.class)); - MessageStream adClicks = graph.getInputStream(inputTopicName2, new JsonSerdeV2<>(AdClick.class)); + StreamApplication app = StreamApplications.createStreamApp(config); + + MessageStream pageViews = app.openInput(inputTopicName1, new JsonSerdeV2<>(PageView.class)); + MessageStream adClicks = app.openInput(inputTopicName2, new JsonSerdeV2<>(AdClick.class)); MessageStream> pageViewsRepartitionedByViewId = pageViews .partitionBy(PageView::getViewId, pv -> pv, @@ -105,11 +97,13 @@ public void init(StreamGraph graph, Config config) { messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue())); }); - + intermediateStreams.clear(); intermediateStreams.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamSpec()); intermediateStreams.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamSpec()); intermediateStreams.add(((IntermediateMessageStreamImpl) userPageAdClicksByUserId).getStreamSpec()); + app.run(); + app.waitForFinish(); } public List getIntermediateStreams() { 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 e233793f12..724969e687 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 @@ -23,9 +23,8 @@ import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; +import org.apache.samza.application.StreamApplications; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; @@ -38,7 +37,7 @@ /** * A {@link StreamApplication} that demonstrates a repartition followed by a windowed count. */ -public class RepartitionWindowApp implements StreamApplication { +public class RepartitionWindowApp { private static final Logger LOG = LoggerFactory.getLogger(RepartitionWindowApp.class); @@ -48,25 +47,20 @@ public class RepartitionWindowApp implements StreamApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - RepartitionWindowApp reparApp = new RepartitionWindowApp(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(reparApp); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication reparApp = StreamApplications.createStreamApp(config); KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageView.class)); - graph.getInputStream(INPUT_TOPIC, pgeMsgSerde) + reparApp.openInput(INPUT_TOPIC, pgeMsgSerde) .map(KV::getValue) - .partitionBy(PageView::getUserId, m -> m, pgeMsgSerde, "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(graph.getOutputStream(OUTPUT_TOPIC)); + .partitionBy(PageView::getUserId, m -> m, pgeMsgSerde, "inputByUID") + .window(Windows.keyedSessionWindow(m -> m.getKey(), Duration.ofSeconds(3), () -> 0, (m, c) -> c + 1, + new StringSerde("UTF-8"), new IntegerSerde()), "countWindow") + .map(wp -> KV.of(wp.getKey().getKey().toString(), wp.getMessage())) + .sendTo(reparApp.openOutput(OUTPUT_TOPIC)); + reparApp.run(); + reparApp.waitForFinish(); } } 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 3224d2413a..f38db675db 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 @@ -21,13 +21,12 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; @@ -40,7 +39,7 @@ /** * A {@link StreamApplication} that demonstrates a filter followed by a session window. */ -public class SessionWindowApp implements StreamApplication { +public class SessionWindowApp { private static final String INPUT_TOPIC = "page-views"; private static final String OUTPUT_TOPIC = "page-view-counts"; @@ -50,19 +49,11 @@ public class SessionWindowApp implements StreamApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - SessionWindowApp app = new SessionWindowApp(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); + StreamApplication app = StreamApplications.createStreamApp(config); - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { - - MessageStream pageViews = graph.getInputStream(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); + MessageStream pageViews = app.openInput(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); OutputStream> outputStream = - graph.getOutputStream(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); + app.openOutput(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); pageViews .filter(m -> !FILTER_KEY.equals(m.getUserId())) @@ -71,5 +62,7 @@ public void init(StreamGraph graph, Config config) { .map(m -> KV.of(m.getKey().getKey(), m.getMessage().size())) .sendTo(outputStream); + app.run(); + app.waitForFinish(); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java index 15953478ce..439cf60a99 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java @@ -18,6 +18,18 @@ */ package org.apache.samza.test.operator; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; import kafka.utils.TestUtils; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -27,29 +39,17 @@ import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.Config; import org.apache.samza.config.KafkaConfig; -import org.apache.samza.config.MapConfig; import org.apache.samza.execution.TestStreamManager; import org.apache.samza.runtime.AbstractApplicationRunner; -import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.system.kafka.KafkaSystemAdmin; import org.apache.samza.test.harness.AbstractIntegrationTestHarness; -import org.apache.samza.test.framework.StreamAssert; import scala.Option; import scala.Option$; -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Properties; - /** - * Harness for writing integration tests for {@link StreamApplication}s. + * Harness for writing integration tests for {@link org.apache.samza.application.StreamApplication}s. * *

This provides the following features for its sub-classes: *

    @@ -75,7 +75,7 @@ * State persistence: {@link #tearDown()} clears all associated state (including topics and metadata) in Kafka and * Zookeeper. Hence, the state is not durable across invocations of {@link #tearDown()}
    * - * Execution model: {@link StreamApplication}s are run as their own {@link org.apache.samza.job.local.ThreadJob}s. + * Execution model: {@link org.apache.samza.application.StreamApplication}s are run as their own {@link org.apache.samza.job.local.ThreadJob}s. * Similarly, embedded Kafka servers and Zookeeper servers are run as their own threads. * {@link #produceMessage(String, int, String, String)} and {@link #consumeMessages(Collection, int)} are blocking calls. * @@ -212,29 +212,33 @@ public List> consumeMessages(Collection t } /** - * Executes the provided {@link StreamApplication} as a {@link org.apache.samza.job.local.ThreadJob}. The - * {@link StreamApplication} runs in its own separate thread. + * Executes the provided {@link org.apache.samza.application.StreamApplication} as a {@link org.apache.samza.job.local.ThreadJob}. The + * {@link org.apache.samza.application.StreamApplication} runs in its own separate thread. * - * @param streamApplication the application to run * @param appName the name of the application * @param overriddenConfigs configs to override */ - public void runApplication(StreamApplication streamApplication, String appName, Map overriddenConfigs) { + public void runApplication(String userAppClass, String appName, Config overriddenConfigs) + throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException, IllegalAccessException, + IOException { Map configs = new HashMap<>(); - configs.put("job.factory.class", "org.apache.samza.job.local.ThreadJobFactory"); + configs.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner"); +// configs.put("job.factory.class", "org.apache.samza.job.local.ThreadJobFactory"); configs.put("job.name", appName); - configs.put("app.class", streamApplication.getClass().getCanonicalName()); configs.put("serializers.registry.json.class", "org.apache.samza.serializers.JsonSerdeFactory"); configs.put("serializers.registry.string.class", "org.apache.samza.serializers.StringSerdeFactory"); configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory"); configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect()); configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl()); configs.put("systems.kafka.samza.key.serde", "string"); - configs.put("systems.kafka.samza.msg.serde", "string"); + configs.put("systems.kafka.samza.msg.serde", "json"); configs.put("systems.kafka.samza.offset.default", "oldest"); + configs.put("systems.kafka.default.stream.replication.factor", "1"); configs.put("job.coordinator.system", "kafka"); configs.put("job.default.system", "kafka"); configs.put("job.coordinator.replication.factor", "1"); +// configs.put("job.coordinator.factory", "org.apache.samza.zk.ZkJobCoordinatorFactory"); +// configs.put("job.coordinator.zk.connect", zkConnect()); configs.put("task.window.ms", "1000"); configs.put("task.checkpoint.factory", TestStreamManager.MockCheckpointManagerFactory.class.getName()); @@ -253,11 +257,21 @@ public void runApplication(StreamApplication streamApplication, String appName, configs.putAll(overriddenConfigs); } - app = streamApplication; - runner = (AbstractApplicationRunner) ApplicationRunner.fromConfig(new MapConfig(configs)); - runner.run(streamApplication); + Class cls = Class.forName(userAppClass); + Method mainMethod = cls.getMethod("main", String[].class); + String[] params = getCommandLineConfigs(configs); + mainMethod.invoke(null, (Object) params); + } - StreamAssert.waitForComplete(); + private String[] getCommandLineConfigs(Map configs) { + String[] cliParams = new String[configs.size() * 2 + 1]; + int i = 0; + cliParams[i++] = "--config-path=./src/test/resources/test-config.prop"; + for (Map.Entry entry : configs.entrySet()) { + cliParams[i++] = "--config"; + cliParams[i++] = String.format("%s=%s", entry.getKey(), entry.getValue()); + } + return cliParams; } public void setNumEmptyPolls(int numEmptyPolls) { diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java index 5424888ad0..b1c202e869 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java @@ -29,13 +29,16 @@ import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; import org.apache.samza.system.kafka.KafkaSystemAdmin; import org.apache.samza.util.ExponentialSleepStrategy; +import org.apache.samza.SamzaException; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.TaskConfig; import org.junit.Assert; import org.junit.Test; import java.util.Collections; import java.util.List; - /** * Test driver for {@link RepartitionJoinWindowApp}. */ @@ -81,13 +84,26 @@ public void testRepartitionJoinWindowAppWithoutDeletionOnCommit() throws Excepti configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_2_PROP, inputTopicName2); configs.put(RepartitionJoinWindowApp.OUTPUT_TOPIC_NAME_PROP, outputTopicName); - runApplication(app, appName, configs); + Thread runThread = new Thread(() -> { + try { + // run the application + runApplication(app.getClass().getName(), appName, new MapConfig(configs)); + } catch (Exception e) { + throw new SamzaException("Exception in running RepartitionJoinWindowApp", e); + } + }); + + runThread.start(); // consume and validate result List> messages = consumeMessages(Collections.singletonList(outputTopicName), 2); Assert.assertEquals(2, messages.size()); Assert.assertFalse(KafkaSystemAdmin.deleteMessagesCalled()); + + runThread.interrupt(); + runThread.join(); + } @Test @@ -102,12 +118,24 @@ public void testRepartitionJoinWindowAppAndDeleteMessagesOnCommit() throws Excep RepartitionJoinWindowApp app = new RepartitionJoinWindowApp(); final String appName = "UserPageAdClickCounter2"; Map configs = new HashMap<>(); + configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory"); + configs.put(TaskConfig.GROUPER_FACTORY(), "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"); configs.put("systems.kafka.samza.delete.committed.messages", "true"); configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_1_PROP, inputTopicName1); configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_2_PROP, inputTopicName2); configs.put(RepartitionJoinWindowApp.OUTPUT_TOPIC_NAME_PROP, outputTopicName); - runApplication(app, appName, configs); + Thread runThread = new Thread(() -> { + try { + // run the application + runApplication(app.getClass().getName(), appName, new MapConfig(configs)); + } catch (Exception e) { + throw new SamzaException("Exception in running RepartitionJoinWindowApp", e); + } + }); + + runThread.start(); // consume and validate result List> messages = consumeMessages(Collections.singletonList(outputTopicName), 2); @@ -139,11 +167,13 @@ public void testRepartitionJoinWindowAppAndDeleteMessagesOnCommit() throws Excep Assert.assertEquals(0, remainingMessageNum); } + runThread.interrupt(); + runThread.join(); } @Test - public void testBroadcastApp() { + public void testBroadcastApp() throws InterruptedException { String inputTopicName1 = "page-views"; String inputTopicName2 = "ad-clicks"; String outputTopicName = "user-ad-click-counts"; @@ -151,6 +181,18 @@ public void testBroadcastApp() { configs.put(BroadcastAssertApp.INPUT_TOPIC_NAME_PROP, inputTopicName1); initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - runApplication(new BroadcastAssertApp(), "BroadcastTest", configs); + Thread runThread = new Thread(() -> { + try { + // run the application + runApplication(BroadcastAssertApp.class.getName(), "BroadcastTest", new MapConfig(configs)); + } catch (Exception e) { + throw new SamzaException("Exception in running RepartitionJoinWindowApp", e); + } + }); + + runThread.start(); + + runThread.interrupt(); + runThread.join(); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java index fbc315f4ce..ae7b469153 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java @@ -18,9 +18,12 @@ */ package org.apache.samza.test.operator; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.samza.SamzaException; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; @@ -29,9 +32,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Collections; -import java.util.List; - import static org.apache.samza.test.operator.RepartitionWindowApp.*; /** @@ -67,8 +67,16 @@ public void testRepartitionedSessionWindowCounter() throws Exception { configs.put(String.format("streams.%s.samza.msg.serde", INPUT_TOPIC), "string"); configs.put(String.format("streams.%s.samza.key.serde", INPUT_TOPIC), "string"); - // run the application - runApplication(new RepartitionWindowApp(), APP_NAME, new MapConfig(configs)); + Thread runThread = new Thread(() -> { + try { + // run the application + runApplication(RepartitionWindowApp.class.getName(), APP_NAME, new MapConfig(configs)); + } catch (Exception e) { + throw new SamzaException("Exception in running RepatitionWindowApp", e); + } + }); + + runThread.start(); // consume and validate result List> messages = consumeMessages(Collections.singletonList(OUTPUT_TOPIC), 2); @@ -86,5 +94,7 @@ public void testRepartitionedSessionWindowCounter() throws Exception { } } + runThread.interrupt(); + runThread.join(); } } 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 40a3f91ff4..bbb842b896 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 @@ -20,12 +20,13 @@ package org.apache.samza.test.operator; import java.time.Duration; +import java.io.IOException; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.windows.Windows; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.IntegerSerde; @@ -37,34 +38,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * A {@link StreamApplication} that demonstrates a filter followed by a tumbling window. */ -public class TumblingWindowApp implements StreamApplication { +public class TumblingWindowApp { private static final String INPUT_TOPIC = "page-views"; private static final String OUTPUT_TOPIC = "page-view-counts"; private static final Logger LOG = LoggerFactory.getLogger(TumblingWindowApp.class); private static final String FILTER_KEY = "badKey"; - public static void main(String[] args) { + public static void main(String[] args) throws IOException { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - TumblingWindowApp app = new TumblingWindowApp(); - LocalApplicationRunner runner = new LocalApplicationRunner(config); - - runner.run(app); - runner.waitForFinish(); - } - - @Override - public void init(StreamGraph graph, Config config) { + StreamApplication app = StreamApplications.createStreamApp(config); MessageStream pageViews = - graph.getInputStream(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); + app.openInput(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); OutputStream> outputStream = - graph.getOutputStream(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); + app.openOutput(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); pageViews .filter(m -> !FILTER_KEY.equals(m.getUserId())) @@ -73,5 +65,8 @@ public void init(StreamGraph graph, Config config) { .map(m -> KV.of(m.getKey().getKey(), m.getMessage().size())) .sendTo(outputStream); + app.run(); + app.waitForFinish(); } + } 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 db1235129b..e82e7062dc 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 @@ -24,12 +24,11 @@ import java.io.Serializable; import java.util.concurrent.CountDownLatch; import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.StringSerde; @@ -38,54 +37,7 @@ /** * Test class to create an {@link StreamApplication} instance */ -public class TestStreamApplication implements StreamApplication, Serializable { - - private final String inputTopic; - private final String outputTopic; - private final String appName; - private final String processorName; - - private TestStreamApplication(String inputTopic, String outputTopic, String appName, String processorName) { - this.inputTopic = inputTopic; - this.outputTopic = outputTopic; - this.appName = appName; - this.processorName = processorName; - } - - @Override - public void init(StreamGraph graph, Config config) { - MessageStream inputStream = graph.getInputStream(inputTopic, new NoOpSerde()); - OutputStream outputStream = graph.getOutputStream(outputTopic, new StringSerde()); - inputStream.map(new MapFunction() { - transient CountDownLatch latch1; - transient CountDownLatch latch2; - transient StreamApplicationCallback callback; - - @Override - public String apply(String message) { - TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message); - if (callback != null) { - callback.onMessage(incomingMessage); - } - if (latch1 != null) { - latch1.countDown(); - } - if (latch2 != null) { - latch2.countDown(); - } - return incomingMessage.toString(); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - SharedContextFactories.SharedContextFactory contextFactory = - SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName); - this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch"); - this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch"); - this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback"); - } - }).sendTo(outputStream); - } +public class TestStreamApplication implements Serializable { public interface StreamApplicationCallback { void onMessage(TestKafkaEvent m); @@ -130,11 +82,43 @@ public static StreamApplication getInstance( StreamApplicationCallback callback, CountDownLatch kafkaEventsConsumedLatch, Config config) { - String appName = String.format("%s-%s", config.get(ApplicationConfig.APP_NAME), config.get(ApplicationConfig.APP_ID)); + StreamApplication app = StreamApplications.createStreamApp(config); + String appName = app.getGlobalAppId(); String processorName = config.get(JobConfig.PROCESSOR_ID()); registerLatches(processedMessageLatch, kafkaEventsConsumedLatch, callback, appName, processorName); - - StreamApplication app = new TestStreamApplication(inputTopic, outputTopic, appName, processorName); + MessageStream inputStream = null; + inputStream = app.openInput(inputTopic, new NoOpSerde()); + OutputStream outputStream = app.openOutput(outputTopic, new StringSerde()); + inputStream + .map(new MapFunction() { + transient CountDownLatch latch1; + transient CountDownLatch latch2; + transient StreamApplicationCallback callback; + + @Override + public String apply(String message) { + TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message); + if (callback != null) { + callback.onMessage(incomingMessage); + } + if (latch1 != null) { + latch1.countDown(); + } + if (latch2 != null) { + latch2.countDown(); + } + return incomingMessage.toString(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + SharedContextFactories.SharedContextFactory contextFactory = SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName); + this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch"); + this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch"); + this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback"); + } + }) + .sendTo(outputStream); return app; } diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index ea440523e7..405a549969 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -71,10 +71,10 @@ import static org.junit.Assert.assertTrue; /** - * Integration tests for {@link LocalApplicationRunner}. + * Integration tests for {@link org.apache.samza.runtime.LocalApplicationRunner}. * * Brings up embedded ZooKeeper, Kafka broker and launches multiple {@link StreamApplication} through - * {@link LocalApplicationRunner} to verify the guarantees made in stand alone execution environment. + * {@link org.apache.samza.runtime.LocalApplicationRunner} to verify the guarantees made in stand alone execution environment. */ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarness { @@ -158,6 +158,7 @@ public void setUp() { } public void tearDown() { + SharedContextFactories.clearAll(); for (String kafkaTopic : ImmutableList.of(inputKafkaTopic, outputKafkaTopic)) { LOGGER.info("Deleting kafka topic: {}.", kafkaTopic); AdminUtils.deleteTopic(zkUtils(), kafkaTopic); @@ -196,6 +197,7 @@ private Map buildStreamApplicationConfigMap(String systemName, S .put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, TEST_JOB_COORDINATOR_FACTORY) .put(ApplicationConfig.APP_NAME, appName) .put(ApplicationConfig.APP_ID, appId) + .put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner") .put(String.format("systems.%s.samza.factory", systemName), TEST_SYSTEM_FACTORY) .put(JobConfig.JOB_NAME(), appName) .put(JobConfig.JOB_ID(), appId) @@ -231,8 +233,8 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). final JobModel[] previousJobModel = new JobModel[1]; final String[] previousJobModelVersion = new String[1]; - AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); - final CountDownLatch secondProcessorRegistered = new CountDownLatch(1); + + CountDownLatch secondProcessorRegistered = new CountDownLatch(1); zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { // When streamApp2 with id: PROCESSOR_IDS[1] is registered, start processing message in streamApp1. @@ -243,33 +245,34 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum // Set up stream app 2. CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS); - Config testAppConfig2 = new MapConfig(applicationConfig2, testConfig); - LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(testAppConfig2); - StreamApplication streamApp2 = TestStreamApplication.getInstance(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic, - processedMessagesLatch, null, null, testAppConfig2); + Config localTestConfig2 = new MapConfig(applicationConfig2, testConfig); + StreamApplication + streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null, localTestConfig2); - // Callback handler for streamApp1. - StreamApplicationCallback streamApplicationCallback = message -> { + // Set up stream app 1. + Config localTestConfig1 = new MapConfig(applicationConfig1, testConfig); + AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); + CountDownLatch processedLatch = new CountDownLatch(1); + StreamApplicationCallback callback = m -> { if (hasSecondProcessorJoined.compareAndSet(false, true)) { - previousJobModelVersion[0] = zkUtils.getJobModelVersion(); - previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); - localApplicationRunner2.run(streamApp2); + processedLatch.countDown(); try { - // Wait for streamApp2 to register with zookeeper. secondProcessorRegistered.await(); - } catch (InterruptedException e) { + } catch (InterruptedException ie) { + throw new SamzaException(ie); } } }; CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2); + StreamApplication + streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, callback, kafkaEventsConsumedLatch, localTestConfig1); + streamApp1.run(); - // Set up stream app 1. - Config testAppConfig1 = new MapConfig(applicationConfig1, testConfig); - LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(testAppConfig1); - StreamApplication streamApp1 = TestStreamApplication.getInstance(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic, - null, streamApplicationCallback, kafkaEventsConsumedLatch, testAppConfig1); - localApplicationRunner1.run(streamApp1); + processedLatch.await(); + previousJobModelVersion[0] = zkUtils.getJobModelVersion(); + previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); + streamApp2.run(); kafkaEventsConsumedLatch.await(); @@ -283,12 +286,11 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum // ProcessedMessagesLatch shouldn't have changed. Should retain it's initial value. assertEquals(NUM_KAFKA_EVENTS, processedMessagesLatch.getCount()); - // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665 - // localApplicationRunner1.kill(streamApp1); - // localApplicationRunner2.kill(streamApp2); + streamApp1.kill(); + streamApp2.kill(); - // localApplicationRunner1.waitForFinish(); - // localApplicationRunner2.waitForFinish(); + streamApp1.waitForFinish(); + streamApp2.waitForFinish(); } /** @@ -407,17 +409,16 @@ public void shouldReElectLeaderWhenLeaderDies() throws InterruptedException { CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); CountDownLatch processedMessagesLatch3 = new CountDownLatch(1); - StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); - StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); - StreamApplication streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch, applicationConfig3); - - // Create LocalApplicationRunners - LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1); - LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2); + StreamApplication + streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); + StreamApplication + streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); + StreamApplication + streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch, applicationConfig3); - // Run stream applications. - applicationRunner1.run(streamApp1); - applicationRunner2.run(streamApp2); + streamApp1.run(); + streamApp2.run(); + streamApp3.run(); // Wait until all processors have processed a message. processedMessagesLatch1.await(); @@ -436,20 +437,16 @@ public void shouldReElectLeaderWhenLeaderDies() throws InterruptedException { assertEquals(PROCESSOR_IDS[0], processorIdsFromZK.get(0)); // Kill the leader. Since streamApp1 is the first to join the cluster, it's the leader. - applicationRunner1.kill(streamApp1); - applicationRunner1.waitForFinish(); - - // How do you know here that leader has been reelected. + streamApp1.kill(); + streamApp1.waitForFinish(); kafkaEventsConsumedLatch.await(); publishKafkaEvents(inputKafkaTopic, 0, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); - LocalApplicationRunner applicationRunner3 = new LocalApplicationRunner(applicationConfig3); - applicationRunner3.run(streamApp3); processedMessagesLatch3.await(); // Verifications after killing the leader. - assertEquals(ApplicationStatus.SuccessfulFinish, applicationRunner1.status(streamApp1)); + assertEquals(ApplicationStatus.SuccessfulFinish, streamApp1.status()); processorIdsFromZK = zkUtils.getActiveProcessorsIDs(ImmutableList.of(PROCESSOR_IDS[1], PROCESSOR_IDS[2])); assertEquals(2, processorIdsFromZK.size()); assertEquals(PROCESSOR_IDS[1], processorIdsFromZK.get(0)); @@ -458,12 +455,11 @@ public void shouldReElectLeaderWhenLeaderDies() throws InterruptedException { assertEquals(Sets.newHashSet("0000000001", "0000000002"), jobModel.getContainers().keySet()); assertEquals(2, jobModel.getContainers().size()); - // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665 - // applicationRunner2.kill(streamApp2); - // applicationRunner3.kill(streamApp3); + streamApp2.kill(); + streamApp3.kill(); - // applicationRunner2.waitForFinish(); - // applicationRunner3.waitForFinish(); + streamApp2.waitForFinish(); + streamApp3.waitForFinish(); } @Test @@ -476,38 +472,35 @@ public void shouldFailWhenNewProcessorJoinsWithSameIdAsExistingProcessor() throw CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); - StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); - StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); - - // Create LocalApplicationRunners - LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1); - LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2); + StreamApplication + streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); + StreamApplication + streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); // Run stream applications. - applicationRunner1.run(streamApp1); - applicationRunner2.run(streamApp2); + streamApp1.run(); + streamApp2.run(); // Wait for message processing to start in both the processors. processedMessagesLatch1.await(); processedMessagesLatch2.await(); - MapConfig appConfig = new ApplicationConfig(new MapConfig(applicationConfig2, ImmutableMap.of(ZkConfig.ZK_SESSION_TIMEOUT_MS, "10"))); - LocalApplicationRunner applicationRunner3 = new LocalApplicationRunner(appConfig); - // Create a stream app with same processor id as SP2 and run it. It should fail. - StreamApplication streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, null, kafkaEventsConsumedLatch, applicationConfig2); + publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[2]); + kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); + StreamApplication + streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, null, kafkaEventsConsumedLatch, applicationConfig2); // Fail when the duplicate processor joins. expectedException.expect(SamzaException.class); - try { - applicationRunner3.run(streamApp3); - } finally { - // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665 - // applicationRunner1.kill(streamApp1); - // applicationRunner2.kill(streamApp2); - - // applicationRunner1.waitForFinish(); - // applicationRunner2.waitForFinish(); - } + streamApp3.run(); + + streamApp1.kill(); + streamApp2.kill(); + streamApp3.kill(); + + streamApp1.waitForFinish(); + streamApp2.waitForFinish(); + streamApp3.waitForFinish(); } @Test @@ -523,11 +516,8 @@ public void testRollingUpgradeOfStreamApplicationsShouldGenerateSameJobModel() t configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]); Config applicationConfig2 = new MapConfig(configMap); - LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1); - LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2); - List messagesProcessed = new ArrayList<>(); - StreamApplicationCallback streamApplicationCallback = m -> messagesProcessed.add(m); + StreamApplicationCallback streamApplicationCallback = messagesProcessed::add; // Create StreamApplication from configuration. CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); @@ -538,8 +528,8 @@ public void testRollingUpgradeOfStreamApplicationsShouldGenerateSameJobModel() t StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); // Run stream application. - applicationRunner1.run(streamApp1); - applicationRunner2.run(streamApp2); + streamApp1.run(); + streamApp2.run(); processedMessagesLatch1.await(); processedMessagesLatch2.await(); @@ -548,14 +538,19 @@ public void testRollingUpgradeOfStreamApplicationsShouldGenerateSameJobModel() t String jobModelVersion = zkUtils.getJobModelVersion(); JobModel jobModel = zkUtils.getJobModel(jobModelVersion); - applicationRunner1.kill(streamApp1); - applicationRunner1.waitForFinish(); + streamApp1.kill(); + streamApp1.waitForFinish(); + + int lastProcessedMessageId = -1; + for (TestStreamApplication.TestKafkaEvent message : messagesProcessed) { + lastProcessedMessageId = Math.max(lastProcessedMessageId, Integer.parseInt(message.getEventData())); + } + messagesProcessed.clear(); - LocalApplicationRunner applicationRunner4 = new LocalApplicationRunner(applicationConfig1); processedMessagesLatch1 = new CountDownLatch(1); publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, streamApplicationCallback, kafkaEventsConsumedLatch, applicationConfig1); - applicationRunner4.run(streamApp1); + streamApp1.run(); processedMessagesLatch1.await(); @@ -566,12 +561,11 @@ public void testRollingUpgradeOfStreamApplicationsShouldGenerateSameJobModel() t assertEquals(Integer.parseInt(jobModelVersion) + 1, Integer.parseInt(newJobModelVersion)); assertEquals(jobModel.getContainers(), newJobModel.getContainers()); - // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665 - // applicationRunner2.kill(streamApp2); - // applicationRunner4.kill(streamApp1); + streamApp1.kill(); + streamApp2.kill(); - // applicationRunner2.waitForFinish(); - // applicationRunner4.waitForFinish(); + streamApp1.waitForFinish(); + streamApp2.waitForFinish(); } } diff --git a/samza-test/src/test/resources/test-config.prop b/samza-test/src/test/resources/test-config.prop new file mode 100644 index 0000000000..be16c865ba --- /dev/null +++ b/samza-test/src/test/resources/test-config.prop @@ -0,0 +1,26 @@ +# +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# + +job.factory.class=org.apache.samza.job.MockJobFactory +job.name=test-job +foo=bar +systems.coordinator.samza.factory=org.apache.samza.coordinator.stream.MockCoordinatorStreamSystemFactory +job.coordinator.system=coordinator \ No newline at end of file From d43e923192b8b78a9eb6ba4ba15a05151ec457f2 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Tue, 10 Jul 2018 21:30:12 -0700 Subject: [PATCH 04/38] WIP: proto-type with ApplicationRunnable and no ApplicationRunner exposed to user --- .../application/ApplicationRunnable.java | 98 +++-- .../samza/application/StreamApplication.java | 172 +-------- .../samza/application/StreamApplications.java | 89 ++++- .../application/internal/ApplicationSpec.java | 57 +++ .../internal/StreamApplicationSpec.java | 17 + .../internal/TaskApplicationSpec.java | 22 ++ .../apache/samza/operators/StreamGraph.java | 8 + .../samza/runtime/ApplicationRunners.java | 57 +++ .../{ => internal}/ApplicationRunner.java | 63 +--- .../samza/task/AsyncStreamTaskFactory.java | 3 +- .../apache/samza/task/StreamTaskFactory.java | 3 +- .../org/apache/samza/task/TaskFactory.java | 12 + .../StreamApplicationRuntime.java} | 12 +- .../internal/TaskApplicationRuntime.java | 20 ++ .../samza/operators/StreamGraphSpec.java | 2 +- .../runtime/AbstractApplicationRunner.java | 67 +++- .../samza/runtime/ApplicationRunnerMain.java | 41 +-- .../samza/runtime/LocalApplicationRunner.java | 248 ++++++++----- .../samza/runtime/LocalContainerRunner.java | 250 ++++++++----- .../runtime/RemoteApplicationRunner.java | 335 ++++++++++-------- .../samza/execution/TestExecutionPlanner.java | 2 +- .../execution/TestJobGraphJsonGenerator.java | 2 +- .../apache/samza/execution/TestJobNode.java | 2 +- .../samza/operators/TestJoinOperator.java | 2 +- .../samza/operators/TestStreamGraphSpec.java | 2 +- .../operators/impl/TestOperatorImplGraph.java | 2 +- .../operators/impl/TestWindowOperator.java | 2 +- .../spec/TestPartitionByOperatorSpec.java | 2 +- .../samza/task/TestTaskFactoryUtil.java | 5 +- .../sql/runner/SamzaSqlApplicationRunner.java | 52 +-- .../LocalApplicationRunnerMain.java | 7 +- .../EndOfStreamIntegrationTest.java | 1 - .../test/operator/TumblingWindowApp.java | 1 - 33 files changed, 991 insertions(+), 667 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java rename samza-api/src/main/java/org/apache/samza/runtime/{ => internal}/ApplicationRunner.java (51%) create mode 100644 samza-api/src/main/java/org/apache/samza/task/TaskFactory.java rename samza-core/src/main/java/org/apache/samza/application/{StreamApplicationInternal.java => internal/StreamApplicationRuntime.java} (77%) create mode 100644 samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java index 42e9cc3fe9..34f1c60b17 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java @@ -1,54 +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.application; +import java.time.Duration; +import java.util.Map; +import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.StreamGraph; +import org.apache.samza.operators.functions.InitableFunction; +import org.apache.samza.task.StreamTask; /** - * Interface method for all runtime instance of applications + * Describes and initializes the transforms for processing message streams and generating results. + *

    + * The following example removes page views older than 1 hour from the input stream: + *

    {@code
    + * public class PageViewCounter implements StreamApplication {
    + *   public void init(StreamGraph graph, Config config) {
    + *     MessageStream pageViewEvents =
    + *       graph.getInputStream("pageViewEvents", (k, m) -> (PageViewEvent) m);
    + *     OutputStream recentPageViewEvents =
    + *       graph.getOutputStream("recentPageViewEvents", m -> m.memberId, m -> m);
    + *
    + *     pageViewEvents
    + *       .filter(m -> m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis())
    + *       .sendTo(filteredPageViewEvents);
    + *   }
    + * }
    + * }
    + *

    + * 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));
    + *     PageViewCounter app = new PageViewCounter();
    + *     LocalApplicationRunner runner = new LocalApplicationRunner(config);
    + *     runner.run(app);
    + *     runner.waitForFinish();
    + *   }
    + * }
    + * + *

    + * Implementation Notes: Currently StreamApplications are wrapped in a {@link StreamTask} during execution. + * A new StreamApplication instance will be created and initialized with a user-defined {@link StreamGraph} + * when planning the execution. The {@link StreamGraph} and the functions implemented for transforms are required to + * be serializable. The execution planner will generate a serialized DAG which will be deserialized in each {@link StreamTask} + * instance used for processing incoming messages. Execution is synchronous and thread-safe within each {@link StreamTask}. + * + *

    + * 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 StreamTask} instance is closed. + * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ +@InterfaceStability.Evolving public interface ApplicationRunnable { - /** - * Deploy and run the Samza jobs to execute this application. - * It is non-blocking so it doesn't wait for the application running. - * - */ + void run(); - /** - * Kill the Samza jobs represented by this application - * It is non-blocking so it doesn't wait for the application stopping. - * - */ void kill(); - /** - * Get the collective status of the Samza jobs represented by this application. - * Returns {@link ApplicationStatus} running if all jobs are running. - * - * @return the status of the application - */ ApplicationStatus status(); + void waitForFinish(); + + boolean waitForFinish(Duration timeout); + /** - * Method to wait for the runner in the current JVM process to finish. + * Set {@link MetricsReporter}s for this {@link StreamApplications.ApplicationRuntimeInstance} + * + * @param metricsReporters the map of {@link MetricsReporter}s to be added + * @return this {@link StreamApplications.ApplicationRuntimeInstance} instance */ - void waitForFinish(); + ApplicationRunnable withMetricsReporters(Map metricsReporters); + } 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 da96606c3a..7155db9e41 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 @@ -20,19 +20,13 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; -import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.InitableFunction; -import org.apache.samza.runtime.ApplicationRunner; -import org.apache.samza.serializers.Serde; import org.apache.samza.task.StreamTask; - -import java.util.Map; - +import org.apache.samza.task.TaskContext; /** * Describes and initializes the transforms for processing message streams and generating results. @@ -78,158 +72,26 @@ * before messages are delivered to them and closed after their execution when the {@link StreamTask} instance is closed. * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ -@InterfaceStability.Unstable -public class StreamApplication implements ApplicationRunnable { - - /*package private*/ - final ApplicationRunner runner; - final Config config; - final StreamGraph graph; - - StreamApplication(ApplicationRunner runner, Config config) { - this.runner = runner; - this.config = config; - this.graph = runner.createGraph(); - } - - @Override - public final void run() { - this.runner.run(this); - } - - @Override - public final void kill() { - this.runner.kill(this); - } - - @Override - public final ApplicationStatus status() { - return this.runner.status(this); - } - - @Override - public final void waitForFinish() { - this.runner.waitForFinish(this); - } - - public static class AppConfig extends MapConfig { - - public static final String APP_NAME = "app.name"; - public static final String APP_ID = "app.id"; - public static final String APP_CLASS = "app.class"; - public static final String RUNNER_CONFIG = "app.runner.class"; - private static final String DEFAULT_RUNNER_CLASS = "org.apache.samza.runtime.RemoteApplicationRunner"; - - public static final String JOB_NAME = "job.name"; - public static final String JOB_ID = "job.id"; - - public AppConfig(Config config) { - super(config); - } - - public String getAppName() { - return get(APP_NAME, get(JOB_NAME)); - } - - public String getAppId() { - return get(APP_ID, get(JOB_ID, "1")); - } - - public String getAppClass() { - return get(APP_CLASS, null); - } - - public String getApplicationRunnerClass() { - return get(RUNNER_CONFIG, DEFAULT_RUNNER_CLASS); - } - - /** - * Returns full application id - * - * @return full app id - */ - public String getGlobalAppId() { - return String.format("app-%s-%s", getAppName(), getAppId()); - } - - } +@InterfaceStability.Evolving +public interface StreamApplication { /** - * Set {@link MetricsReporter}s for this {@link StreamApplication} - * - * @param metricsReporters the map of {@link MetricsReporter}s to be added - * @return this {@link StreamApplication} instance - */ - public StreamApplication withMetricsReporters(Map metricsReporters) { - this.runner.addMetricsReporters(metricsReporters); - return this; - } - - /** - * Return the globally unique application ID for this {@link StreamApplication} - * - * @return the globally unique appplication ID - */ - public String getGlobalAppId() { - return new AppConfig(config).getGlobalAppId(); - } - - /** - * Gets the input {@link MessageStream} corresponding to the {@code streamId}. - *

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param the type of input messages - * @param streamId the input stream name - * @param serde the {@link Serde} object used to deserialize input messages - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - public final MessageStream openInput(String streamId, Serde serde) { - return this.graph.getInputStream(streamId, serde); - } - - /** - * Gets the input {@link MessageStream} corresponding to the {@code streamId}. + * Describes and initializes the transforms for processing message streams and generating results. *

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param the type of message in the input {@link MessageStream} - * @param streamId the input stream name - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - public final MessageStream openInput(String streamId) { - return this.graph.getInputStream(streamId); - } - - /** - * Gets the {@link OutputStream} corresponding to the {@code streamId}. + * The {@link StreamGraph} provides access to input and output streams. Input {@link MessageStream}s can be + * transformed into other {@link MessageStream}s or sent to an {@link OutputStream} using the {@link MessageStream} + * operators. *

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param the type of message in the {@link OutputStream} - * @param output the output stream name - * @param serde the {@link Serde} object used to serialize output messages - * @return the output {@link OutputStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - public final OutputStream openOutput(String output, Serde serde) { - return this.graph.getOutputStream(output, serde); - } - - /** - * Gets the {@link OutputStream} corresponding to the {@code streamId}. + * Most operators accept custom functions for doing the transformations. These functions are {@link InitableFunction}s + * and are provided the {@link Config} and {@link TaskContext} during their own initialization. The config and the + * context can be used, for example, to create custom metrics or access durable state stores. *

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * A shared context between {@link InitableFunction}s for different operators within a task instance can be set + * up by providing a {@link ContextManager} using {@link StreamGraph#withContextManager}. * - * @param the type of message in the {@link OutputStream} - * @param output the output stream name - * @return the output {@link OutputStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + * @param graph the {@link StreamGraph} to get input/output streams from + * @param config the configuration for the application */ - public final OutputStream openOutput(String output) { - return this.graph.getOutputStream(output); - } + void init(StreamGraph graph, Config config); } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java index ea6094399b..e8916fc8cf 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java @@ -18,28 +18,105 @@ */ package org.apache.samza.application; +import java.time.Duration; import java.util.HashMap; import java.util.Map; +import org.apache.samza.application.internal.ApplicationSpec; +import org.apache.samza.application.internal.StreamApplicationSpec; +import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.Config; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.StreamGraph; +import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; +import org.apache.samza.task.TaskFactory; + /** * This class defines the methods to create different types of Samza application instances: 1) high-level end-to-end * {@link StreamApplication}; 2) task-level StreamTaskApplication; 3) task-level AsyncStreamTaskApplication */ -public class StreamApplications { +public final class StreamApplications { // The static map of all created application instances from the user program - private static final Map USER_APPS = new HashMap<>(); + private static final Map USER_APPS = new HashMap<>(); private StreamApplications() { } - public static StreamApplication createStreamApp(Config config) { - ApplicationRunner runner = ApplicationRunner.fromConfig(config); - StreamApplication appRuntime = new StreamApplication(runner, config); + public static final ApplicationRunnable createRunnable(StreamApplication userApp, Config config) { + StreamGraph graph = StreamGraph.createInstance(); + userApp.init(graph, config); + ApplicationRuntimeInstance appRuntime = new ApplicationRuntimeInstance(graph, config); + USER_APPS.put(appRuntime.getGlobalAppId(), appRuntime); + return appRuntime; + } + + public static final ApplicationRunnable createRunnable(TaskFactory taskFactory, Config config) { + ApplicationRuntimeInstance appRuntime = new ApplicationRuntimeInstance(taskFactory, config); USER_APPS.put(appRuntime.getGlobalAppId(), appRuntime); return appRuntime; } + static final class ApplicationRuntimeInstance implements ApplicationRunnable { + + /*package private*/ + final ApplicationRunner runner; + final ApplicationSpec appSpec; + + private ApplicationRuntimeInstance(ApplicationSpec appSpec, Config config) { + this.appSpec = appSpec; + this.runner = ApplicationRunners.fromConfig(config); + } + + ApplicationRuntimeInstance(StreamGraph graph, Config config) { + this(new StreamApplicationSpec(graph, config), config); + } + + ApplicationRuntimeInstance(TaskFactory taskFactory, Config config) { + this(new TaskApplicationSpec(taskFactory, config), config); + } + + @Override + public final void run() { + this.runner.run(this.appSpec); + } + + @Override + public final void kill() { + this.runner.kill(this.appSpec); + } + + @Override + public final ApplicationStatus status() { + return this.runner.status(this.appSpec); + } + + @Override + public final void waitForFinish() { + this.runner.waitForFinish(this.appSpec); + } + + @Override + public final boolean waitForFinish(Duration timeout) { + return this.runner.waitForFinish(this.appSpec, timeout); + } + + /** + * Set {@link MetricsReporter}s for this {@link ApplicationRuntimeInstance} + * + * @param metricsReporters the map of {@link MetricsReporter}s to be added + * @return this {@link ApplicationRuntimeInstance} instance + */ + @Override + public final ApplicationRuntimeInstance withMetricsReporters(Map metricsReporters) { + this.runner.addMetricsReporters(metricsReporters); + return this; + } + + String getGlobalAppId() { + return this.appSpec.getGlobalAppId(); + } + } } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java new file mode 100644 index 0000000000..e092f80a0b --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java @@ -0,0 +1,57 @@ +package org.apache.samza.application.internal; + +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; + + +/** + * Created by yipan on 7/10/18. + */ +public abstract class ApplicationSpec { + final Config config; + + protected ApplicationSpec(Config config) { + this.config = config; + } + + public static class AppConfig extends MapConfig { + + public static final String APP_NAME = "app.name"; + public static final String APP_ID = "app.id"; + public static final String APP_CLASS = "app.class"; + + public static final String JOB_NAME = "job.name"; + public static final String JOB_ID = "job.id"; + + public AppConfig(Config config) { + super(config); + } + + public String getAppName() { + return get(APP_NAME, get(JOB_NAME)); + } + + public String getAppId() { + return get(APP_ID, get(JOB_ID, "1")); + } + + public String getAppClass() { + return get(APP_CLASS, null); + } + + /** + * Returns full application id + * + * @return full app id + */ + public String getGlobalAppId() { + return String.format("app-%s-%s", getAppName(), getAppId()); + } + + } + + public String getGlobalAppId() { + return new AppConfig(config).getGlobalAppId(); + } + +} \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java new file mode 100644 index 0000000000..4fddd1109f --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java @@ -0,0 +1,17 @@ +package org.apache.samza.application.internal; + +import org.apache.samza.config.Config; +import org.apache.samza.operators.StreamGraph; + + +/** + * Created by yipan on 7/10/18. + */ +public class StreamApplicationSpec extends ApplicationSpec { + final StreamGraph graph; + + public StreamApplicationSpec(StreamGraph graph, Config config) { + super(config); + this.graph = graph; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java new file mode 100644 index 0000000000..5de551c79a --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java @@ -0,0 +1,22 @@ +package org.apache.samza.application.internal; + +import org.apache.samza.config.Config; +import org.apache.samza.task.TaskFactory; + + +/** + * Created by yipan on 7/10/18. + */ +public class TaskApplicationSpec extends ApplicationSpec { + + final TaskFactory taskFactory; + + public TaskApplicationSpec(TaskFactory taskFactory, Config config) { + super(config); + this.taskFactory = taskFactory; + } + + public TaskFactory getTaskFactory() { + return this.taskFactory; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java index 6871bc76d4..f8e32b98e8 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java @@ -18,6 +18,7 @@ */ package org.apache.samza.operators; +import org.apache.samza.SamzaException; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.serializers.Serde; import org.apache.samza.table.Table; @@ -146,4 +147,11 @@ public interface StreamGraph { */ StreamGraph withContextManager(ContextManager contextManager); + static StreamGraph createInstance() { + try { + return (StreamGraph) Class.forName("org.apache.samza.operators.StreamGraphSpec").newInstance(); + } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) { + throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); + } + } } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java new file mode 100644 index 0000000000..d247199fe4 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java @@ -0,0 +1,57 @@ +package org.apache.samza.runtime; + +import java.lang.reflect.Constructor; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.runtime.internal.ApplicationRunner; + + +/** + * Created by yipan on 7/10/18. + */ +public class ApplicationRunners { + + static class AppRunnerConfig { + private static final String APP_RUNNER_CFG = "app.runner.class"; + private static final String DEFAULT_APP_RUNNER = "org.apache.samza.runtime.RemoteApplicationRunner"; + + private final Config config; + + AppRunnerConfig(Config config) { + this.config = config; + } + + String getAppRunnerClass() { + return this.config.getOrDefault(APP_RUNNER_CFG, DEFAULT_APP_RUNNER); + } + + static String getAppRunnerCfg() { + return APP_RUNNER_CFG; + } + + } + + /** + * Static method to load the {@link ApplicationRunner} + * + * @param config configuration passed in to initialize the Samza processes + * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications + */ + public static ApplicationRunner fromConfig(Config config) { + AppRunnerConfig appRunnerCfg = new AppRunnerConfig(config); + try { + Class runnerClass = Class.forName(appRunnerCfg.getAppRunnerClass()); + if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { + Constructor constructor = runnerClass.getConstructor(Config.class); // *sigh* + return (ApplicationRunner) constructor.newInstance(config); + } + } catch (Exception e) { + throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", + appRunnerCfg.getAppRunnerClass()), e); + } + throw new ConfigException(String.format( + "Class %s does not extend ApplicationRunner properly", + appRunnerCfg.getAppRunnerClass())); + } + +} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java similarity index 51% rename from samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java rename to samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java index 31a1e9c2f0..c73143aa9f 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java @@ -16,69 +16,31 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime; +package org.apache.samza.runtime.internal; import java.time.Duration; +import java.util.Map; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplication.AppConfig; -import org.apache.samza.config.Config; -import org.apache.samza.config.ConfigException; +import org.apache.samza.application.internal.ApplicationSpec; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.operators.StreamGraph; - -import java.lang.reflect.Constructor; -import java.util.Map; import org.apache.samza.system.StreamSpec; /** * The primary means of managing execution of the {@link StreamApplication} at runtime. */ -@InterfaceStability.Unstable +@InterfaceStability.Evolving public interface ApplicationRunner { - /** - * Static method to load the {@link ApplicationRunner} - * - * @param config configuration passed in to initialize the Samza processes - * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications - */ - static ApplicationRunner fromConfig(Config config) { - AppConfig appCfg = new AppConfig(config); - try { - Class runnerClass = Class.forName(appCfg.getApplicationRunnerClass()); - if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { - Constructor constructor = runnerClass.getConstructor(Config.class); // *sigh* - return (ApplicationRunner) constructor.newInstance(config); - } - } catch (Exception e) { - throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", - appCfg.getApplicationRunnerClass()), e); - } - throw new ConfigException(String.format( - "Class %s does not extend ApplicationRunner properly", - appCfg.getApplicationRunnerClass())); - } + void run(ApplicationSpec appRunnable); - /** - * Deploy and run the Samza jobs to execute {@link org.apache.samza.task.StreamTask}. - * It is non-blocking so it doesn't wait for the application running. - * This method assumes you task.class is specified in the configs. - * - * NOTE. this interface will most likely change in the future. - */ - @Deprecated - void runTask(); - - void run(StreamApplication app); + void kill(ApplicationSpec appRunnable); - void kill(StreamApplication app); + ApplicationStatus status(ApplicationSpec appRunnable); - ApplicationStatus status(StreamApplication app); - - void waitForFinish(StreamApplication app); + void waitForFinish(ApplicationSpec appRunnable); /** * Waits for {@code timeout} duration for the application to finish. @@ -87,14 +49,7 @@ static ApplicationRunner fromConfig(Config config) { * @return true - application finished before timeout * false - otherwise */ - boolean waitForFinish(StreamApplication app, Duration timeout); - - /** - * Create an empty {@link StreamGraph} object to instantiate the user defined operator DAG. - * - * @return the empty {@link StreamGraph} object to be instantiated - */ - StreamGraph createGraph(); + boolean waitForFinish(ApplicationSpec appRunnable, Duration timeout); /** * Method to add a set of customized {@link MetricsReporter}s in the application diff --git a/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java b/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java index e5ce9c4b5e..ebd64b2fb0 100644 --- a/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java +++ b/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java @@ -23,6 +23,5 @@ * Build {@link AsyncStreamTask} instances. * Implementations should return a new instance for each {@link #createInstance()} invocation. */ -public interface AsyncStreamTaskFactory { - AsyncStreamTask createInstance(); +public interface AsyncStreamTaskFactory extends TaskFactory { } diff --git a/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java b/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java index 52adef6b93..02087eee09 100644 --- a/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java +++ b/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java @@ -26,6 +26,5 @@ * Implementations should return a new instance for each {@link #createInstance()} invocation. */ @InterfaceStability.Stable -public interface StreamTaskFactory { - StreamTask createInstance(); +public interface StreamTaskFactory extends TaskFactory { } 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 new file mode 100644 index 0000000000..65d03dca36 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/task/TaskFactory.java @@ -0,0 +1,12 @@ +package org.apache.samza.task; + +import org.apache.samza.annotation.InterfaceStability; + + +/** + * Created by yipan on 7/10/18. + */ +@InterfaceStability.Stable +public interface TaskFactory { + T createInstance(); +} diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java b/samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java similarity index 77% rename from samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java rename to samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java index ee83e3843d..382a269b2a 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationInternal.java +++ b/samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java @@ -16,21 +16,21 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application; +package org.apache.samza.application.internal; import org.apache.samza.operators.StreamGraphSpec; -public class StreamApplicationInternal { +public class StreamApplicationRuntime { - private final StreamApplication app; + private final StreamApplicationSpec appSpec; - public StreamApplicationInternal(StreamApplication app) { - this.app = app; + public StreamApplicationRuntime(StreamApplicationSpec app) { + this.appSpec = app; } public StreamGraphSpec getStreamGraphSpec() { - return (StreamGraphSpec) this.app.graph; + return (StreamGraphSpec) this.appSpec.graph; } } diff --git a/samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java b/samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java new file mode 100644 index 0000000000..da4435030c --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java @@ -0,0 +1,20 @@ +package org.apache.samza.application.internal; + +import org.apache.samza.task.TaskFactory; + + +/** + * Created by yipan on 7/10/18. + */ +public class TaskApplicationRuntime { + private final TaskApplicationSpec appSpec; + + public TaskApplicationRuntime(TaskApplicationSpec app) { + this.appSpec = app; + } + + public TaskFactory getTaskFactory() { + return (TaskFactory) this.appSpec.taskFactory; + } + +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java index ea9690b314..b03835eb5e 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java @@ -34,7 +34,7 @@ import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 01ec87c16e..a331db851d 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -18,15 +18,18 @@ */ package org.apache.samza.runtime; -import com.google.common.annotations.VisibleForTesting; import java.io.File; import java.io.PrintWriter; +import java.time.Duration; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.ApplicationRunnable; +import org.apache.samza.application.internal.ApplicationSpec; +import org.apache.samza.application.internal.StreamApplicationSpec; +import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ApplicationConfig.ApplicationMode; import org.apache.samza.config.Config; @@ -36,10 +39,11 @@ import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.ExecutionPlanner; import org.apache.samza.execution.StreamManager; +import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.StreamGraphSpec; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.system.StreamSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,15 +58,8 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { protected final Config config; protected final Map metricsReporters = new HashMap<>(); - /** - * The {@link ApplicationRunner} is supposed to run a single {@link StreamApplication} instance in the full life-cycle - */ - // TODO: need to revisit after refactory and merge the supported application instances - protected final StreamGraphSpec graphSpec; - public AbstractApplicationRunner(Config config) { this.config = config; - this.graphSpec = new StreamGraphSpec(this, config); } @Override @@ -116,12 +113,12 @@ public StreamSpec getStreamSpec(String streamId) { return new StreamSpec(streamId, physicalName, system, isBounded, properties); } - public ExecutionPlan getExecutionPlan(StreamApplication app, StreamManager streamManager) throws Exception { - return getExecutionPlan(app, null, streamManager); + public ExecutionPlan getExecutionPlan(StreamGraphSpec graphSpec, StreamManager streamManager) throws Exception { + return getExecutionPlan(graphSpec, null, streamManager); } /* package private */ - ExecutionPlan getExecutionPlan(StreamApplication app, String runId, StreamManager streamManager) throws Exception { + ExecutionPlan getExecutionPlan(StreamGraphSpec graphSpec, String runId, StreamManager streamManager) throws Exception { // build stream graph OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); // create the physical execution plan @@ -140,11 +137,6 @@ ExecutionPlan getExecutionPlan(StreamApplication app, String runId, StreamManage return planner.plan(specGraph); } - @Override - public final StreamGraph createGraph() { - return this.graphSpec; - } - /** * Write the execution plan JSON to a file * @param planJson JSON representation of the plan @@ -176,4 +168,43 @@ StreamManager buildAndStartStreamManager() { streamManager.start(); return streamManager; } + + private ApplicationRunnable getRunnable(ApplicationSpec appSpec) { + if (appSpec instanceof StreamApplicationSpec) { + return getStreamAppRunnable((StreamApplicationSpec) appSpec); + } + if (appSpec instanceof TaskApplicationSpec) { + return getTaskAppRunnable((TaskApplicationSpec) appSpec); + } + throw new IllegalArgumentException(String.format("The specified application %s is not valid. Only StreamApplication and Task applications are supported.", appSpec.getClass().getName()); + } + + protected abstract ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec); + + protected abstract ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec); + + @Override + public final void run(ApplicationSpec appSpec) { + getRunnable(appSpec).run(); + } + + @Override + public final ApplicationStatus status(ApplicationSpec appSpec) { + return getRunnable(appSpec).status(); + } + + @Override + public final void kill(ApplicationSpec appSpec) { + getRunnable(appSpec).kill(); + } + + @Override + public final void waitForFinish(ApplicationSpec appSpec) { + getRunnable(appSpec).waitForFinish(); + } + + @Override + public final boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { + return getRunnable(appSpec).waitForFinish(timeout); + } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 13e6d38a63..cf92f0406c 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -21,9 +21,13 @@ import joptsimple.OptionSet; import joptsimple.OptionSpec; +import org.apache.samza.application.ApplicationRunnable; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; -import org.apache.samza.job.JobRunner$; +import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.task.TaskFactory; +import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.CommandLine; import org.apache.samza.util.Util; @@ -58,25 +62,22 @@ public static void main(String[] args) throws Exception { Config config = Util.rewriteConfig(orgConfig); ApplicationRunnerOperation op = cmdLine.getOperation(options); - if (config.containsKey(STREAM_APPLICATION_CLASS_CONFIG)) { - ApplicationRunner runner = ApplicationRunner.fromConfig(config); - StreamApplication app = - (StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(); - switch (op) { - case RUN: - runner.run(app); - break; - case KILL: - runner.kill(app); - break; - case STATUS: - System.out.println(runner.status(app)); - break; - default: - throw new IllegalArgumentException("Unrecognized operation: " + op); - } - } else { - JobRunner$.MODULE$.main(args); + ApplicationRunnable appRunnable = config.containsKey(STREAM_APPLICATION_CLASS_CONFIG) ? + StreamApplications.createRunnable((StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(), config) : + StreamApplications.createRunnable((TaskFactory) TaskFactoryUtil.createTaskFactory(config), config); + + switch (op) { + case RUN: + appRunnable.run(); + break; + case KILL: + appRunnable.kill(); + break; + case STATUS: + System.out.println(appRunnable.status()); + break; + default: + throw new IllegalArgumentException("Unrecognized operation: " + op); } } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 8d0f2ee794..62b351f131 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -33,13 +33,14 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationRunnable; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationInternal; +import org.apache.samza.application.internal.StreamApplicationRuntime; +import org.apache.samza.application.internal.StreamApplicationSpec; +import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; -import org.apache.samza.config.TaskConfig; import org.apache.samza.coordinator.CoordinationUtils; import org.apache.samza.coordinator.DistributedLockWithState; import org.apache.samza.execution.ExecutionPlan; @@ -49,6 +50,7 @@ import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.StreamProcessorLifecycleListener; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.system.StreamSpec; import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTaskFactory; @@ -56,8 +58,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.samza.util.ScalaJavaUtil.*; - /** * This class implements the {@link ApplicationRunner} that runs the applications in standalone environment */ @@ -134,124 +134,180 @@ public LocalApplicationRunner(Config config) { this(config, new HashMap<>()); } - public LocalApplicationRunner(Config config, Map customMetricsReporters) { - super(config); - this.uid = UUID.randomUUID().toString(); - this.customMetricsReporters = customMetricsReporters; - } - @Override - @Deprecated - public void runTask() { - JobConfig jobConfig = new JobConfig(config); - - // validation - String taskName = new TaskConfig(config).getTaskClass().getOrElse(defaultValue(null)); - if (taskName == null) { - throw new SamzaException("Neither APP nor task.class are defined defined"); - } - LOG.info("LocalApplicationRunner will run " + taskName); - LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); - - StreamProcessor processor = createStreamProcessor(jobConfig, listener); - - numProcessorsToStart.set(1); - listener.setProcessor(processor); - processor.start(); + protected ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec) { + return new TaskAppRunnable(appSpec); } @Override - public void run(StreamApplication userApp) { - StreamApplicationInternal app = new StreamApplicationInternal(userApp); - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); + protected ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec) { + return new StreamAppRunnable(appSpec); + } - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(userApp, streamManager); + public LocalApplicationRunner(Config config, Map customMetricsReporters) { + super(config); + this.uid = UUID.randomUUID().toString(); + this.customMetricsReporters = customMetricsReporters; + } - String executionPlanJson = plan.getPlanAsJson(); - writePlanJsonFile(executionPlanJson); - LOG.info("Execution Plan: \n" + executionPlanJson); + class StreamAppRunnable implements ApplicationRunnable { + final StreamApplicationSpec streamApp; - // 2. create the necessary streams - // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 - String planId = String.valueOf(executionPlanJson.hashCode()); - createStreams(planId, plan.getIntermediateStreams(), streamManager); + StreamAppRunnable(StreamApplicationSpec streamApp) { + this.streamApp = streamApp; + } - // 3. create the StreamProcessors - if (plan.getJobConfigs().isEmpty()) { - throw new SamzaException("No jobs to run."); - } - plan.getJobConfigs().forEach(jobConfig -> { + @Override + public void run() { + StreamApplicationRuntime app = new StreamApplicationRuntime(streamApp); + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(app.getStreamGraphSpec(), streamManager); + + String executionPlanJson = plan.getPlanAsJson(); + writePlanJsonFile(executionPlanJson); + LOG.info("Execution Plan: \n" + executionPlanJson); + + // 2. create the necessary streams + // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 + String planId = String.valueOf(executionPlanJson.hashCode()); + createStreams(planId, plan.getIntermediateStreams(), streamManager); + + // 3. create the StreamProcessors + if (plan.getJobConfigs().isEmpty()) { + throw new SamzaException("No jobs to run."); + } + plan.getJobConfigs().forEach(jobConfig -> { LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); - StreamProcessor processor = createStreamProcessor(jobConfig, graphSpec, listener); + StreamProcessor processor = createStreamProcessor(jobConfig, app.getStreamGraphSpec(), listener); listener.setProcessor(processor); processors.add(processor); }); - numProcessorsToStart.set(processors.size()); + numProcessorsToStart.set(processors.size()); + + // 4. start the StreamProcessors + processors.forEach(StreamProcessor::start); + } catch (Throwable throwable) { + appStatus = ApplicationStatus.unsuccessfulFinish(throwable); + shutdownLatch.countDown(); + throw new SamzaException(String.format("Failed to start application: %s.", app), throwable); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + } - // 4. start the StreamProcessors - processors.forEach(StreamProcessor::start); - } catch (Throwable throwable) { - appStatus = ApplicationStatus.unsuccessfulFinish(throwable); - shutdownLatch.countDown(); - throw new SamzaException(String.format("Failed to start application: %s.", app), throwable); - } finally { - if (streamManager != null) { - streamManager.stop(); + @Override + public void kill() { + processors.forEach(StreamProcessor::stop); + } + + @Override + public ApplicationStatus status() { + return appStatus; + } + + @Override + public void waitForFinish() { + waitForFinish(Duration.ofMillis(0)); + } + + @Override + public boolean waitForFinish(Duration timeout) { + long timeoutInMs = timeout.toMillis(); + boolean finished = true; + + try { + if (timeoutInMs < 1) { + shutdownLatch.await(); + } else { + finished = shutdownLatch.await(timeoutInMs, TimeUnit.MILLISECONDS); + + if (!finished) { + LOG.warn("Timed out waiting for application to finish."); + } + } + } catch (Exception e) { + LOG.error("Error waiting for application to finish", e); + throw new SamzaException(e); } + + return finished; } - } - @Override - public void kill(StreamApplication streamApp) { - processors.forEach(StreamProcessor::stop); + @Override + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + throw new UnsupportedOperationException("MetricsReporters should be set to the ApplicationRunner"); + } } - @Override - public ApplicationStatus status(StreamApplication streamApp) { - return appStatus; - } + class TaskAppRunnable implements ApplicationRunnable { + final TaskApplicationSpec appSpec; + StreamProcessor sp; - /** - * Waits until the application finishes. - */ - @Override - public void waitForFinish(StreamApplication app) { - waitForFinish(app, Duration.ofMillis(0)); - } + TaskAppRunnable(TaskApplicationSpec appSpec) { + this.appSpec = appSpec; + } - /** - * Waits for {@code timeout} duration for the application to finish. - * If timeout < 1, blocks the caller indefinitely. - * - * @param timeout time to wait for the application to finish - * @return true - application finished before timeout - * false - otherwise - */ - @Override - public boolean waitForFinish(StreamApplication app, Duration timeout) { - long timeoutInMs = timeout.toMillis(); - boolean finished = true; + @Override + public void run() { + LOG.info("LocalApplicationRunner will run task " + appSpec.getGlobalAppId()); + LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); - try { - if (timeoutInMs < 1) { - shutdownLatch.await(); - } else { - finished = shutdownLatch.await(timeoutInMs, TimeUnit.MILLISECONDS); + sp = getStreamProcessorInstance(config, appSpec.getTaskFactory(), listener); - if (!finished) { - LOG.warn("Timed out waiting for application to finish."); + numProcessorsToStart.set(1); + listener.setProcessor(sp); + sp.start(); + } + + @Override + public void kill() { + sp.stop(); + } + + @Override + public ApplicationStatus status() { + return appStatus; + } + + @Override + public void waitForFinish() { + waitForFinish(Duration.ofMillis(0)); + } + + @Override + public boolean waitForFinish(Duration timeout) { + long timeoutInMs = timeout.toMillis(); + boolean finished = true; + + try { + if (timeoutInMs < 1) { + shutdownLatch.await(); + } else { + finished = shutdownLatch.await(timeoutInMs, TimeUnit.MILLISECONDS); + + if (!finished) { + LOG.warn("Timed out waiting for application to finish."); + } } + } catch (Exception e) { + LOG.error("Error waiting for application to finish", e); + throw new SamzaException(e); } - } catch (Exception e) { - LOG.error("Error waiting for application to finish", e); - throw new SamzaException(e); + + return finished; } - return finished; + @Override + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + throw new UnsupportedOperationException("MetricsReporters should be set to the ApplicationRunner"); + } } /** diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 683dcfc543..d5fabd5b82 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -22,13 +22,19 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; -import java.lang.reflect.Method; import java.time.Duration; import java.util.HashMap; +import java.util.Map; import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationRunnable; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.ApplicationSpec; +import org.apache.samza.application.internal.StreamApplicationRuntime; +import org.apache.samza.application.internal.StreamApplicationSpec; +import org.apache.samza.application.internal.TaskApplicationRuntime; +import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -36,17 +42,19 @@ import org.apache.samza.container.ContainerHeartbeatMonitor; import org.apache.samza.container.SamzaContainer; import org.apache.samza.container.SamzaContainer$; -import org.apache.samza.util.SamzaUncaughtExceptionHandler; import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.model.JobModel; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.StreamGraphSpec; +import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; +import org.apache.samza.util.SamzaUncaughtExceptionHandler; import org.apache.samza.util.ScalaJavaUtil; -import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.samza.util.ScalaJavaUtil.defaultValue; +import static org.apache.samza.util.ScalaJavaUtil.*; /** * LocalContainerRunner is the local runner for Yarn {@link SamzaContainer}s. It is an intermediate step to @@ -72,96 +80,165 @@ public LocalContainerRunner(JobModel jobModel, String containerId) { } @Override - public void runTask() { - throw new UnsupportedOperationException("Running StreamTask is not implemented for LocalContainerRunner"); + protected ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec) { + return new TaskAppRunnable(appSpec); } @Override - public void run(StreamApplication streamApp) { - Object taskFactory = getTaskFactory(streamApp); - - container = SamzaContainer$.MODULE$.apply( - containerId, - jobModel, - config, - ScalaJavaUtil.toScalaMap(new HashMap<>()), - taskFactory); - - container.setContainerListener( - new SamzaContainerListener() { - @Override - public void onContainerStart() { - log.info("Container Started"); - } + protected ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec) { + return new StreamAppRunnable(appSpec); + } - @Override - public void onContainerStop(boolean invokedExternally) { - log.info("Container Stopped"); - } + class TaskAppRunnable implements ApplicationRunnable { + final TaskApplicationRuntime taskApp; - @Override - public void onContainerFailed(Throwable t) { - log.info("Container Failed"); - containerRunnerException = t; - } - }); + TaskAppRunnable(TaskApplicationSpec taskApp) { + this.taskApp = new TaskApplicationRuntime(taskApp); + } + + @Override + public void run() { + Object taskFactory = this.taskApp.getTaskFactory(); + + container = SamzaContainer$.MODULE$.apply( + containerId, + jobModel, + config, + ScalaJavaUtil.toScalaMap(new HashMap<>()), + taskFactory); + + container.setContainerListener( + new SamzaContainerListener() { + @Override + public void onContainerStart() { + log.info("Container Started"); + } + + @Override + public void onContainerStop(boolean invokedExternally) { + log.info("Container Stopped"); + } + + @Override + public void onContainerFailed(Throwable t) { + log.info("Container Failed"); + containerRunnerException = t; + } + }); - startContainerHeartbeatMonitor(); - container.run(); - stopContainerHeartbeatMonitor(); + startContainerHeartbeatMonitor(); + container.run(); + stopContainerHeartbeatMonitor(); - if (containerRunnerException != null) { - log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); - System.exit(1); + if (containerRunnerException != null) { + log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); + System.exit(1); + } } - } - private Object getTaskFactory(StreamApplication streamApp) { - if (streamApp != null) { - // TODO: should already be initialized. - // streamApp.init(graphSpec, config); - return TaskFactoryUtil.createTaskFactory(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager()); + @Override + public void kill() { + // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted + throw new UnsupportedOperationException(); } - return TaskFactoryUtil.createTaskFactory(config); - } - @Override - public void kill(StreamApplication userApp) { - // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted - throw new UnsupportedOperationException(); - } + @Override + public ApplicationStatus status() { + // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted + throw new UnsupportedOperationException(); + } - @Override - public ApplicationStatus status(StreamApplication userApp) { - // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted - throw new UnsupportedOperationException(); - } + @Override + public void waitForFinish() { - @Override - public void waitForFinish(StreamApplication userApp) { + } - } + @Override + public boolean waitForFinish(Duration timeout) { + return false; + } - @Override - public boolean waitForFinish(StreamApplication app, Duration timeout) { - return false; + @Override + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted + throw new UnsupportedOperationException(); + } } - private static File writeConfigToTmpFile(Config config) throws IOException { - File tmpFile = File.createTempFile("config", ""); - FileWriter fileWriter = new FileWriter(tmpFile); - try { - config.forEach((k, v) -> { - try { - fileWriter.write(String.format("%s=%s\n", k, v)); - } catch (IOException e) { - throw new SamzaException("Failed to create a temporary config file for user application", e); - } - }); - } finally { - fileWriter.close(); + class StreamAppRunnable implements ApplicationRunnable { + final StreamApplicationRuntime streamApp; + + StreamAppRunnable(StreamApplicationSpec streamApp) { + this.streamApp = new StreamApplicationRuntime(streamApp); + } + + @Override + public void run() { + Object taskFactory = TaskFactoryUtil.createTaskFactory(streamApp.getStreamGraphSpec().getOperatorSpecGraph(), + streamApp.getStreamGraphSpec().getContextManager()); + + container = SamzaContainer$.MODULE$.apply( + containerId, + jobModel, + config, + ScalaJavaUtil.toScalaMap(new HashMap<>()), + taskFactory); + + container.setContainerListener( + new SamzaContainerListener() { + @Override + public void onContainerStart() { + log.info("Container Started"); + } + + @Override + public void onContainerStop(boolean invokedExternally) { + log.info("Container Stopped"); + } + + @Override + public void onContainerFailed(Throwable t) { + log.info("Container Failed"); + containerRunnerException = t; + } + }); + + startContainerHeartbeatMonitor(); + container.run(); + stopContainerHeartbeatMonitor(); + + if (containerRunnerException != null) { + log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); + System.exit(1); + } + } + + @Override + public void kill() { + // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted + throw new UnsupportedOperationException(); + } + + @Override + public ApplicationStatus status() { + // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted + throw new UnsupportedOperationException(); + } + + @Override + public void waitForFinish() { + + } + + @Override + public boolean waitForFinish(Duration timeout) { + return false; + } + + @Override + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + throw new UnsupportedOperationException(); } - return tmpFile; } // only invoked by legacy applications w/o user-defined main @@ -191,22 +268,23 @@ public static void main(String[] args) throws Exception { MDC.put("jobName", jobName); MDC.put("jobId", jobId); - StreamApplication.AppConfig appConfig = new StreamApplication.AppConfig(config); + ApplicationSpec.AppConfig appConfig = new ApplicationSpec.AppConfig(config); + LocalContainerRunner runner = new LocalContainerRunner(jobModel, containerId); + ApplicationSpec appSpec = null; if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { // add configuration-factory and configuration-path to the command line options and invoke the user defined main class // write the complete configuration to a local file in property file format - config.put(StreamApplication.AppConfig.RUNNER_CONFIG, LocalContainerRunner.class.getName()); - File tmpFile = writeConfigToTmpFile(config); - Class cls = Class.forName(appConfig.getAppClass()); - Method mainMethod = cls.getMethod("main", String[].class); - String[] params = new String[] {"--configuration-path", String.format("%s", tmpFile.getAbsoluteFile())}; - mainMethod.invoke(null, (Object) params); + StreamGraphSpec streamGraph = new StreamGraphSpec(runner, config); + StreamApplication userApp = (StreamApplication) Class.forName(appConfig.getAppClass()).newInstance(); + userApp.init(streamGraph, config); + appSpec = new StreamApplicationSpec(streamGraph, config); } else { - LocalContainerRunner localContainerRunner = new LocalContainerRunner(jobModel, containerId); - // run with app = null force to load the task class from configuration - localContainerRunner.run(null); + appSpec = new TaskApplicationSpec((TaskFactory) TaskFactoryUtil.createTaskFactory(config), config); } + + runner.run(appSpec); + runner.waitForFinish(appSpec); } private void startContainerHeartbeatMonitor() { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 4a95c34ac2..778b41ab80 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -20,10 +20,14 @@ package org.apache.samza.runtime; import java.time.Duration; +import java.util.Map; import java.util.UUID; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationInternal; +import org.apache.samza.application.ApplicationRunnable; +import org.apache.samza.application.internal.StreamApplicationRuntime; +import org.apache.samza.application.internal.StreamApplicationSpec; +import org.apache.samza.application.internal.TaskApplicationRuntime; +import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -33,6 +37,8 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.JobRunner; import org.apache.samza.metrics.MetricsRegistryMap; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,179 +57,222 @@ public RemoteApplicationRunner(Config config) { super(config); } - @Deprecated - public void runTask() { - throw new UnsupportedOperationException("Running StreamTask is not implemented for RemoteReplicationRunner"); + @Override + protected ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec) { + return new TaskAppRunnable(appSpec); } - /** - * Run the {@link StreamApplication} on the remote cluster - * @param userApp a StreamApplication - */ @Override - public void run(StreamApplication userApp) { - StreamApplicationInternal app = new StreamApplicationInternal(userApp); - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - // TODO: run.id needs to be set for standalone: SAMZA-1531 - // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision - String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The run id for this run is {}", runId); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(userApp, runId, streamManager); - writePlanJsonFile(plan.getPlanAsJson()); - - // 2. create the necessary streams - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - streamManager.createStreams(plan.getIntermediateStreams()); + protected ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec) { + return new StreamAppRunnable(appSpec); + } + + class TaskAppRunnable implements ApplicationRunnable { + final TaskApplicationRuntime taskApp; + final JobRunner jobRunner; + + TaskAppRunnable(TaskApplicationSpec appSpec) { + this.taskApp = new TaskApplicationRuntime(appSpec); + this.jobRunner = new JobRunner(config); + } + + @Override + public void run() { + jobRunner.run(true); + } + + @Override + public void kill() { + jobRunner.kill(); + } + + @Override + public ApplicationStatus status() { + return jobRunner.status(); + } + + @Override + public void waitForFinish() { + } + + @Override + public boolean waitForFinish(Duration timeout) { + return false; + } - // 3. submit jobs for remote execution - plan.getJobConfigs().forEach(jobConfig -> { + @Override + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + throw new UnsupportedOperationException(); + } + } + + class StreamAppRunnable implements ApplicationRunnable { + final StreamApplicationRuntime streamApp; + + StreamAppRunnable(StreamApplicationSpec appSpec) { + this.streamApp = new StreamApplicationRuntime(appSpec); + } + + @Override + public void run() { + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + // TODO: run.id needs to be set for standalone: SAMZA-1531 + // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); + LOG.info("The run id for this run is {}", runId); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamApp.getStreamGraphSpec(), runId, streamManager); + writePlanJsonFile(plan.getPlanAsJson()); + + // 2. create the necessary streams + if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { + streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); + } + streamManager.createStreams(plan.getIntermediateStreams()); + + // 3. submit jobs for remote execution + plan.getJobConfigs().forEach(jobConfig -> { LOG.info("Starting job {} with config {}", jobConfig.getName(), jobConfig); JobRunner runner = new JobRunner(jobConfig); runner.run(true); }); - } catch (Throwable t) { - throw new SamzaException("Failed to run application", t); - } finally { - if (streamManager != null) { - streamManager.stop(); + } catch (Throwable t) { + throw new SamzaException("Failed to run application", t); + } finally { + if (streamManager != null) { + streamManager.stop(); + } } } - } - @Override - public void kill(StreamApplication app) { - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - ExecutionPlan plan = getExecutionPlan(app, streamManager); + @Override + public void kill() { + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + ExecutionPlan plan = getExecutionPlan(streamApp.getStreamGraphSpec(), streamManager); - plan.getJobConfigs().forEach(jobConfig -> { + plan.getJobConfigs().forEach(jobConfig -> { LOG.info("Killing job {}", jobConfig.getName()); JobRunner runner = new JobRunner(jobConfig); runner.kill(); }); - } catch (Throwable t) { - throw new SamzaException("Failed to kill application", t); - } finally { - if (streamManager != null) { - streamManager.stop(); + } catch (Throwable t) { + throw new SamzaException("Failed to kill application", t); + } finally { + if (streamManager != null) { + streamManager.stop(); + } } } - } - @Override - public ApplicationStatus status(StreamApplication app) { - StreamManager streamManager = null; - try { - boolean hasNewJobs = false; - boolean hasRunningJobs = false; - ApplicationStatus unsuccessfulFinishStatus = null; - - streamManager = buildAndStartStreamManager(); - ExecutionPlan plan = getExecutionPlan(app, streamManager); - for (JobConfig jobConfig : plan.getJobConfigs()) { - ApplicationStatus status = getApplicationStatus(jobConfig); - - switch (status.getStatusCode()) { - case New: - hasNewJobs = true; - break; - case Running: - hasRunningJobs = true; - break; - case UnsuccessfulFinish: - unsuccessfulFinishStatus = status; - break; - case SuccessfulFinish: - break; - default: - // Do nothing + @Override + public ApplicationStatus status() { + StreamManager streamManager = null; + try { + boolean hasNewJobs = false; + boolean hasRunningJobs = false; + ApplicationStatus unsuccessfulFinishStatus = null; + + streamManager = buildAndStartStreamManager(); + ExecutionPlan plan = getExecutionPlan(streamApp.getStreamGraphSpec(), streamManager); + for (JobConfig jobConfig : plan.getJobConfigs()) { + ApplicationStatus status = getApplicationStatus(jobConfig); + + switch (status.getStatusCode()) { + case New: + hasNewJobs = true; + break; + case Running: + hasRunningJobs = true; + break; + case UnsuccessfulFinish: + unsuccessfulFinishStatus = status; + break; + case SuccessfulFinish: + break; + default: + // Do nothing + } } - } - if (hasNewJobs) { - // There are jobs not started, report as New - return New; - } else if (hasRunningJobs) { - // All jobs are started, some are running - return Running; - } else if (unsuccessfulFinishStatus != null) { - // All jobs are finished, some are not successful - return unsuccessfulFinishStatus; - } else { - // All jobs are finished successfully - return SuccessfulFinish; - } - } catch (Throwable t) { - throw new SamzaException("Failed to get status for application", t); - } finally { - if (streamManager != null) { - streamManager.stop(); + if (hasNewJobs) { + // There are jobs not started, report as New + return New; + } else if (hasRunningJobs) { + // All jobs are started, some are running + return Running; + } else if (unsuccessfulFinishStatus != null) { + // All jobs are finished, some are not successful + return unsuccessfulFinishStatus; + } else { + // All jobs are finished successfully + return SuccessfulFinish; + } + } catch (Throwable t) { + throw new SamzaException("Failed to get status for application", t); + } finally { + if (streamManager != null) { + streamManager.stop(); + } } } - } - /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { - JobRunner runner = new JobRunner(jobConfig); - ApplicationStatus status = runner.status(); - LOG.debug("Status is {} for job {}", new Object[]{status, jobConfig.getName()}); - return status; - } + @Override + public void waitForFinish() { + waitForFinish(Duration.ofMillis(0)); + } - /** - * Waits until the application finishes. - */ - public void waitForFinish(StreamApplication app) { - waitForFinish(app, Duration.ofMillis(0)); - } + @Override + public boolean waitForFinish(Duration timeout) { + JobConfig jobConfig = new JobConfig(config); + boolean finished = true; + long timeoutInMs = timeout.toMillis(); + long startTimeInMs = System.currentTimeMillis(); + long timeElapsed = 0L; + + long sleepDurationInMs = timeoutInMs < 1 ? + DEFAULT_SLEEP_DURATION_MS : Math.min(timeoutInMs, DEFAULT_SLEEP_DURATION_MS); + ApplicationStatus status; - /** - * Waits for {@code timeout} duration for the application to finish. - * If timeout < 1, blocks the caller indefinitely. - * - * @param timeout time to wait for the application to finish - * @return true - application finished before timeout - * false - otherwise - */ - public boolean waitForFinish(StreamApplication app, Duration timeout) { - JobConfig jobConfig = new JobConfig(config); - boolean finished = true; - long timeoutInMs = timeout.toMillis(); - long startTimeInMs = System.currentTimeMillis(); - long timeElapsed = 0L; - - long sleepDurationInMs = timeoutInMs < 1 ? - DEFAULT_SLEEP_DURATION_MS : Math.min(timeoutInMs, DEFAULT_SLEEP_DURATION_MS); - ApplicationStatus status; - - try { - while (timeoutInMs < 1 || timeElapsed <= timeoutInMs) { - status = getApplicationStatus(jobConfig); - if (status == SuccessfulFinish || status == UnsuccessfulFinish) { - LOG.info("Application finished with status {}", status); - break; + try { + while (timeoutInMs < 1 || timeElapsed <= timeoutInMs) { + status = getApplicationStatus(jobConfig); + if (status == SuccessfulFinish || status == UnsuccessfulFinish) { + LOG.info("Application finished with status {}", status); + break; + } + + Thread.sleep(sleepDurationInMs); + timeElapsed = System.currentTimeMillis() - startTimeInMs; } - Thread.sleep(sleepDurationInMs); - timeElapsed = System.currentTimeMillis() - startTimeInMs; + if (timeElapsed > timeoutInMs) { + LOG.warn("Timed out waiting for application to finish."); + finished = false; + } + } catch (Exception e) { + LOG.error("Error waiting for application to finish", e); + throw new SamzaException(e); } - if (timeElapsed > timeoutInMs) { - LOG.warn("Timed out waiting for application to finish."); - finished = false; - } - } catch (Exception e) { - LOG.error("Error waiting for application to finish", e); - throw new SamzaException(e); + return finished; } - return finished; + @Override + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + throw new UnsupportedOperationException(); + } + } + + /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { + JobRunner runner = new JobRunner(jobConfig); + ApplicationStatus status = runner.status(); + LOG.debug("Status is {} for job {}", new Object[]{status, jobConfig.getName()}); + return status; } private Config getConfigFromPrevRun() { 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 5d7eff8b9b..db3572591d 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 @@ -38,7 +38,7 @@ import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.Serde; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index abe8969490..bdfcdce2e8 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -31,7 +31,7 @@ import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.LongSerde; import org.apache.samza.serializers.NoOpSerde; diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index c43e242f51..cf1c7f566e 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -29,7 +29,7 @@ import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java index 602b595443..034b867b51 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java @@ -28,7 +28,7 @@ import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TestInMemoryStore; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; import org.apache.samza.system.IncomingMessageEnvelope; diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java index e476abc3c7..b6e621e2e9 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java @@ -31,7 +31,7 @@ import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 6fdcacc35c..589a9fb01b 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -54,7 +54,7 @@ import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec.OpCode; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java index 9741fc4b69..9ae8f361e4 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java @@ -42,7 +42,7 @@ import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java index 00ec1761c8..40e0d630e9 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java @@ -29,7 +29,7 @@ import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.system.StreamSpec; import org.junit.Before; diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 712e8d2ffe..816deb0f4c 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -21,13 +21,10 @@ import java.lang.reflect.Field; import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; -import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.testUtils.TestAsyncStreamTask; import org.apache.samza.testUtils.TestStreamTask; import org.junit.Test; diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java index 044c7cfa90..32f32d20dc 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java @@ -19,20 +19,21 @@ package org.apache.samza.sql.runner; +import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.commons.lang3.Validate; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.ApplicationRunnable; +import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.runtime.AbstractApplicationRunner; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.RemoteApplicationRunner; -import org.apache.samza.sql.interfaces.SqlIOResolver; import org.apache.samza.sql.interfaces.SqlIOConfig; +import org.apache.samza.sql.interfaces.SqlIOResolver; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,26 +46,21 @@ * This runner invokes the SamzaSqlConfig re-writer if it is invoked on a standalone mode (i.e. localRunner == true) * otherwise directly calls the RemoteApplicationRunner which automatically performs the config rewriting . */ -public class SamzaSqlApplicationRunner extends AbstractApplicationRunner { +public class SamzaSqlApplicationRunner implements ApplicationRunnable { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplicationRunner.class); private final Config sqlConfig; - private final ApplicationRunner appRunner; + private final ApplicationRunnable appRunnable; private final Boolean localRunner; public static final String RUNNER_CONFIG = "app.runner.class"; public static final String CFG_FMT_SAMZA_STREAM_SYSTEM = "streams.%s.samza.system"; - public SamzaSqlApplicationRunner(Config config) { - this(false, config); - } - public SamzaSqlApplicationRunner(Boolean localRunner, Config config) { - super(config); this.localRunner = localRunner; sqlConfig = computeSamzaConfigs(localRunner, config); - appRunner = ApplicationRunner.fromConfig(sqlConfig); + appRunnable = StreamApplications.createRunnable(new SamzaSqlApplication(), sqlConfig); } public static Config computeSamzaConfigs(Boolean localRunner, Config config) { @@ -108,29 +104,37 @@ public static Config computeSamzaConfigs(Boolean localRunner, Config config) { public void runAndWaitForFinish() { Validate.isTrue(localRunner, "This method can be called only in standalone mode."); - SamzaSqlApplication app = new SamzaSqlApplication(); - run(app); - appRunner.waitForFinish(); + appRunnable.run(); + appRunnable.waitForFinish(); + } + + @Override + public void run() { + appRunnable.run(); + } + + @Override + public void kill() { + appRunnable.kill(); } @Override - public void runTask() { - appRunner.runTask(); + public ApplicationStatus status() { + return appRunnable.status(); } @Override - public void run(StreamApplication streamApp) { - Validate.isInstanceOf(SamzaSqlApplication.class, streamApp); - appRunner.run(streamApp); + public void waitForFinish() { + appRunnable.waitForFinish(); } @Override - public void kill(StreamApplication streamApp) { - appRunner.kill(streamApp); + public boolean waitForFinish(Duration timeout) { + return appRunnable.waitForFinish(timeout); } @Override - public ApplicationStatus status(StreamApplication streamApp) { - return appRunner.status(streamApp); + public ApplicationRunnable withMetricsReporters(Map metricsReporters) { + return appRunnable.withMetricsReporters(metricsReporters); } } diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index e8be592e03..c1cfbb9769 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -22,14 +22,15 @@ import joptsimple.OptionSet; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; -import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunnerMain; import org.apache.samza.runtime.ApplicationRunnerOperation; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.samza.runtime.ApplicationRunnerMain.STREAM_APPLICATION_CLASS_CONFIG; +import static org.apache.samza.runtime.ApplicationRunnerMain.*; /** * {@link ApplicationRunnerMain} was designed for deploying {@link StreamApplication} in yarn @@ -47,7 +48,7 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationRunner runner = ApplicationRunner.fromConfig(config); + ApplicationRunner runner = ApplicationRunners.fromConfig(config); StreamApplication app = (StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(); ApplicationRunnerOperation op = cmdLine.getOperation(options); 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 69fd816ce6..fca641e608 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 @@ -43,7 +43,6 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; -import static junit.framework.Assert.*; /** * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input. 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 bbb842b896..90e19909dc 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 @@ -28,7 +28,6 @@ import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; From 95577b74c7330f11ee055c363b944c745f034ee0 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Wed, 18 Jul 2018 03:09:52 -0700 Subject: [PATCH 05/38] WIP: trying to figure out the two interface classes for spec: a) spec builder in init(); b) spec reader in all other lifecycle methods --- .../application/ApplicationInitializer.java | 8 + .../application/ApplicationRunnable.java | 78 ---------- .../samza/application/StreamApplication.java | 33 +---- .../StreamApplicationInitializer.java | 134 +++++++++++++++++ .../samza/application/StreamApplications.java | 122 ---------------- .../samza/application/TaskApplication.java | 11 ++ .../TaskApplicationInitializer.java | 19 +++ .../samza/application/UserApplication.java | 17 +++ ...ationSpec.java => ApplicationBuilder.java} | 20 ++- .../internal/StreamApplicationBuilder.java | 99 +++++++++++++ .../internal/StreamApplicationSpec.java | 17 --- .../internal/TaskApplicationBuilder.java | 66 +++++++++ .../internal/TaskApplicationSpec.java | 22 --- .../apache/samza/operators/StreamGraph.java | 131 +---------------- .../samza/runtime/ApplicationRuntime.java | 26 ++++ .../samza/runtime/ApplicationRuntimes.java | 73 ++++++++++ .../runtime/internal/ApplicationRunner.java | 23 +-- .../{ => internal}/ApplicationRunners.java | 5 +- .../runtime/internal/ApplicationSpec.java | 16 ++ .../internal/StreamApplicationSpec.java | 25 ++++ .../runtime/internal/TaskApplicationSpec.java | 20 +++ .../internal/StreamApplicationRuntime.java | 36 ----- .../internal/TaskApplicationRuntime.java | 20 --- .../samza/operators/StreamGraphSpec.java | 57 ++++++-- .../runtime/AbstractApplicationRunner.java | 53 +++++-- .../runtime/ApplicationRunnerCommandLine.java | 6 +- .../samza/runtime/ApplicationRunnerMain.java | 30 ++-- .../samza/runtime/LocalApplicationRunner.java | 105 +++++--------- .../samza/runtime/LocalContainerRunner.java | 82 ++++------- .../runtime/RemoteApplicationRunner.java | 137 ++++++++---------- samza-shell/src/main/bash/run-app.sh | 2 +- .../samza/sql/runner/SamzaSqlApplication.java | 5 +- ...r.java => SamzaSqlApplicationRuntime.java} | 41 +++--- .../samza/sql/e2e/TestSamzaSqlTable.java | 10 +- ...va => TestSamzaSqlApplicationRuntime.java} | 14 +- .../sql/translator/TestQueryTranslator.java | 44 +++--- .../example/AppWithGlobalConfigExample.java | 29 ++-- .../samza/example/BroadcastExample.java | 25 ++-- .../samza/example/KeyValueStoreExample.java | 31 ++-- .../apache/samza/example/MergeExample.java | 25 ++-- .../example/OrderShipmentJoinExample.java | 25 ++-- .../samza/example/PageViewCounterExample.java | 23 +-- .../samza/example/RepartitionExample.java | 24 +-- .../samza/example/TaskApplicationExample.java | 37 +++++ .../apache/samza/example/WindowExample.java | 21 ++- .../samza/system/mock/MockSystemConsumer.java | 6 +- .../LocalApplicationRunnerMain.java | 18 +-- .../processor/TestZkStreamProcessor.java | 2 +- .../TestZkStreamProcessorSession.java | 2 +- .../EndOfStreamIntegrationTest.java | 1 - .../WatermarkIntegrationTest.java | 1 - .../operator/RepartitionJoinWindowApp.java | 1 - .../test/operator/RepartitionWindowApp.java | 1 - .../samza/test/operator/SessionWindowApp.java | 1 - ...reamApplicationIntegrationTestHarness.java | 12 +- .../TestRepartitionJoinWindowApp.java | 8 +- .../operator/TestRepartitionWindowApp.java | 2 +- .../test/operator/TumblingWindowApp.java | 1 - .../test/processor/TestStreamApplication.java | 1 - .../TestZkLocalApplicationRunner.java | 10 +- .../test/samzasql/TestSamzaSqlEndToEnd.java | 30 ++-- .../apache/samza/tools/SamzaSqlConsole.java | 6 +- 62 files changed, 1038 insertions(+), 912 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java delete mode 100644 samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java delete mode 100644 samza-api/src/main/java/org/apache/samza/application/StreamApplications.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/TaskApplication.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/UserApplication.java rename samza-api/src/main/java/org/apache/samza/application/internal/{ApplicationSpec.java => ApplicationBuilder.java} (67%) create mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java delete mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java delete mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java rename samza-api/src/main/java/org/apache/samza/runtime/{ => internal}/ApplicationRunners.java (92%) create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java delete mode 100644 samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java delete mode 100644 samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java rename samza-sql/src/main/java/org/apache/samza/sql/runner/{SamzaSqlApplicationRunner.java => SamzaSqlApplicationRuntime.java} (80%) rename samza-sql/src/test/java/org/apache/samza/sql/runner/{TestSamzaSqlApplicationRunner.java => TestSamzaSqlApplicationRuntime.java} (71%) create mode 100644 samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java new file mode 100644 index 0000000000..ab4709a061 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java @@ -0,0 +1,8 @@ +package org.apache.samza.application; + +/** + * Created by yipan on 7/18/18. + */ +public interface ApplicationInitializer { + +} diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java deleted file mode 100644 index 34f1c60b17..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationRunnable.java +++ /dev/null @@ -1,78 +0,0 @@ -package org.apache.samza.application; - -import java.time.Duration; -import java.util.Map; -import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.functions.InitableFunction; -import org.apache.samza.task.StreamTask; - - -/** - * Describes and initializes the transforms for processing message streams and generating results. - *

    - * The following example removes page views older than 1 hour from the input stream: - *

    {@code
    - * public class PageViewCounter implements StreamApplication {
    - *   public void init(StreamGraph graph, Config config) {
    - *     MessageStream pageViewEvents =
    - *       graph.getInputStream("pageViewEvents", (k, m) -> (PageViewEvent) m);
    - *     OutputStream recentPageViewEvents =
    - *       graph.getOutputStream("recentPageViewEvents", m -> m.memberId, m -> m);
    - *
    - *     pageViewEvents
    - *       .filter(m -> m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis())
    - *       .sendTo(filteredPageViewEvents);
    - *   }
    - * }
    - * }
    - *

    - * 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));
    - *     PageViewCounter app = new PageViewCounter();
    - *     LocalApplicationRunner runner = new LocalApplicationRunner(config);
    - *     runner.run(app);
    - *     runner.waitForFinish();
    - *   }
    - * }
    - * - *

    - * Implementation Notes: Currently StreamApplications are wrapped in a {@link StreamTask} during execution. - * A new StreamApplication instance will be created and initialized with a user-defined {@link StreamGraph} - * when planning the execution. The {@link StreamGraph} and the functions implemented for transforms are required to - * be serializable. The execution planner will generate a serialized DAG which will be deserialized in each {@link StreamTask} - * instance used for processing incoming messages. Execution is synchronous and thread-safe within each {@link StreamTask}. - * - *

    - * 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 StreamTask} instance is closed. - * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. - */ -@InterfaceStability.Evolving -public interface ApplicationRunnable { - - void run(); - - void kill(); - - ApplicationStatus status(); - - void waitForFinish(); - - boolean waitForFinish(Duration timeout); - - /** - * Set {@link MetricsReporter}s for this {@link StreamApplications.ApplicationRuntimeInstance} - * - * @param metricsReporters the map of {@link MetricsReporter}s to be added - * @return this {@link StreamApplications.ApplicationRuntimeInstance} instance - */ - ApplicationRunnable withMetricsReporters(Map metricsReporters); - -} 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 7155db9e41..d788444da9 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 @@ -19,14 +19,11 @@ package org.apache.samza.application; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.config.Config; -import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; +import org.apache.samza.application.internal.StreamApplicationBuilder; import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.InitableFunction; +import org.apache.samza.runtime.internal.StreamApplicationSpec; import org.apache.samza.task.StreamTask; -import org.apache.samza.task.TaskContext; /** * Describes and initializes the transforms for processing message streams and generating results. @@ -47,14 +44,14 @@ * } * } *

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

    {@code
      *   public static void main(String[] args) {
      *     CommandLine cmdLine = new CommandLine();
      *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
      *     PageViewCounter app = new PageViewCounter();
      *     LocalApplicationRunner runner = new LocalApplicationRunner(config);
    - *     runner.run(app);
    + *     runner.start(app);
      *     runner.waitForFinish();
      *   }
      * }
    @@ -73,25 +70,5 @@ * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ @InterfaceStability.Evolving -public interface StreamApplication { - - /** - * Describes and initializes the transforms for processing message streams and generating results. - *

    - * The {@link StreamGraph} provides access to input and output streams. Input {@link MessageStream}s can be - * transformed into other {@link MessageStream}s or sent to an {@link OutputStream} using the {@link MessageStream} - * operators. - *

    - * Most operators accept custom functions for doing the transformations. These functions are {@link InitableFunction}s - * and are provided the {@link Config} and {@link TaskContext} during their own initialization. The config and the - * context can be used, for example, to create custom metrics or access durable state stores. - *

    - * A shared context between {@link InitableFunction}s for different operators within a task instance can be set - * up by providing a {@link ContextManager} using {@link StreamGraph#withContextManager}. - * - * @param graph the {@link StreamGraph} to get input/output streams from - * @param config the configuration for the application - */ - void init(StreamGraph graph, Config config); - +public interface StreamApplication extends UserApplication { } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java new file mode 100644 index 0000000000..c54e3e81d9 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java @@ -0,0 +1,134 @@ +package org.apache.samza.application; + +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; + + +/** + * Created by yipan on 7/18/18. + */ +public interface StreamApplicationInitializer extends ApplicationInitializer { + /** + * Sets the default {@link Serde} to use for (de)serializing messages. + *

    . + * If the default serde is set, it must be set before creating any input or output streams. + *

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

    + * Providing an incompatible message type for the input/output streams that use the default serde will result in + * {@link ClassCastException}s at runtime. + * + * @param serde the default message {@link Serde} to use + */ + void setDefaultSerde(Serde serde); + + /** + * Gets the input {@link MessageStream} corresponding to the {@code streamId}. + *

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

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param serde the {@link Serde} to use for deserializing incoming messages + * @param the type of messages in the input {@link MessageStream} + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + MessageStream getInputStream(String streamId, Serde serde); + + /** + * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via + * {@link #setDefaultSerde(Serde)} for deserializing input messages. + *

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param the type of message in the input {@link MessageStream} + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + MessageStream getInputStream(String streamId); + + /** + * Gets the {@link OutputStream} corresponding to the {@code streamId}. + *

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

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

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param serde the {@link Serde} to use for serializing outgoing messages + * @param the type of messages in the {@link OutputStream} + * @return the output {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + OutputStream getOutputStream(String streamId, Serde serde); + + /** + * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via + * {@link #setDefaultSerde(Serde)} for serializing output messages. + *

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param the type of messages in the {@link OutputStream} + * @return the output {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + OutputStream getOutputStream(String streamId); + + /** + * Gets the {@link Table} corresponding to the {@link TableDescriptor}. + *

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

    + * The provided {@link ContextManager} can be used to setup shared context between the operator functions + * within a task instance + * + * @param contextManager the {@link ContextManager} to use for the {@link StreamGraph} + * @return the {@link StreamGraph} with {@code contextManager} set as its {@link ContextManager} + */ + StreamApplicationInitializer withContextManager(ContextManager contextManager); + +} diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java deleted file mode 100644 index e8916fc8cf..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplications.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application; - -import java.time.Duration; -import java.util.HashMap; -import java.util.Map; -import org.apache.samza.application.internal.ApplicationSpec; -import org.apache.samza.application.internal.StreamApplicationSpec; -import org.apache.samza.application.internal.TaskApplicationSpec; -import org.apache.samza.config.Config; -import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.ApplicationRunners; -import org.apache.samza.task.TaskFactory; - - -/** - * This class defines the methods to create different types of Samza application instances: 1) high-level end-to-end - * {@link StreamApplication}; 2) task-level StreamTaskApplication; 3) task-level AsyncStreamTaskApplication - */ -public final class StreamApplications { - // The static map of all created application instances from the user program - private static final Map USER_APPS = new HashMap<>(); - - private StreamApplications() { - - } - - public static final ApplicationRunnable createRunnable(StreamApplication userApp, Config config) { - StreamGraph graph = StreamGraph.createInstance(); - userApp.init(graph, config); - ApplicationRuntimeInstance appRuntime = new ApplicationRuntimeInstance(graph, config); - USER_APPS.put(appRuntime.getGlobalAppId(), appRuntime); - return appRuntime; - } - - public static final ApplicationRunnable createRunnable(TaskFactory taskFactory, Config config) { - ApplicationRuntimeInstance appRuntime = new ApplicationRuntimeInstance(taskFactory, config); - USER_APPS.put(appRuntime.getGlobalAppId(), appRuntime); - return appRuntime; - } - - static final class ApplicationRuntimeInstance implements ApplicationRunnable { - - /*package private*/ - final ApplicationRunner runner; - final ApplicationSpec appSpec; - - private ApplicationRuntimeInstance(ApplicationSpec appSpec, Config config) { - this.appSpec = appSpec; - this.runner = ApplicationRunners.fromConfig(config); - } - - ApplicationRuntimeInstance(StreamGraph graph, Config config) { - this(new StreamApplicationSpec(graph, config), config); - } - - ApplicationRuntimeInstance(TaskFactory taskFactory, Config config) { - this(new TaskApplicationSpec(taskFactory, config), config); - } - - @Override - public final void run() { - this.runner.run(this.appSpec); - } - - @Override - public final void kill() { - this.runner.kill(this.appSpec); - } - - @Override - public final ApplicationStatus status() { - return this.runner.status(this.appSpec); - } - - @Override - public final void waitForFinish() { - this.runner.waitForFinish(this.appSpec); - } - - @Override - public final boolean waitForFinish(Duration timeout) { - return this.runner.waitForFinish(this.appSpec, timeout); - } - - /** - * Set {@link MetricsReporter}s for this {@link ApplicationRuntimeInstance} - * - * @param metricsReporters the map of {@link MetricsReporter}s to be added - * @return this {@link ApplicationRuntimeInstance} instance - */ - @Override - public final ApplicationRuntimeInstance withMetricsReporters(Map metricsReporters) { - this.runner.addMetricsReporters(metricsReporters); - return this; - } - - String getGlobalAppId() { - return this.appSpec.getGlobalAppId(); - } - } -} 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 new file mode 100644 index 0000000000..d1d2f6926d --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplication.java @@ -0,0 +1,11 @@ +package org.apache.samza.application; + +import org.apache.samza.application.internal.TaskApplicationBuilder; +import org.apache.samza.runtime.internal.TaskApplicationSpec; + + +/** + * Created by yipan on 7/11/18. + */ +public interface TaskApplication extends UserApplication { +} diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java new file mode 100644 index 0000000000..75164c797d --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java @@ -0,0 +1,19 @@ +package org.apache.samza.application; + +import java.util.List; +import org.apache.samza.task.TaskFactory; + + +/** + * Created by yipan on 7/18/18. + */ +public interface TaskApplicationInitializer extends ApplicationInitializer { + void setTaskFactory(TaskFactory factory); + + void addInputStreams(List inputStreams); + + void addOutputStreams(List outputStreams); + + void addTables(List tables); + +} diff --git a/samza-api/src/main/java/org/apache/samza/application/UserApplication.java b/samza-api/src/main/java/org/apache/samza/application/UserApplication.java new file mode 100644 index 0000000000..29e9cebc96 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/UserApplication.java @@ -0,0 +1,17 @@ +package org.apache.samza.application; + +import org.apache.samza.application.internal.ApplicationBuilder; +import org.apache.samza.config.Config; +import org.apache.samza.runtime.internal.ApplicationSpec; + + +/** + * Created by yipan on 7/11/18. + */ +public interface UserApplication, S extends ApplicationSpec> { + void init(B appBuilder, Config config); + default void beforeStart(S appSpec) {} + default void afterStart(S appSpec) {} + default void beforeStop(S appSpec) {} + default void afterStop(S appSpec) {} +} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationBuilder.java similarity index 67% rename from samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java rename to samza-api/src/main/java/org/apache/samza/application/internal/ApplicationBuilder.java index e092f80a0b..602054761e 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationBuilder.java @@ -1,17 +1,22 @@ package org.apache.samza.application.internal; +import org.apache.samza.application.ApplicationInitializer; +import org.apache.samza.application.UserApplication; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; +import org.apache.samza.runtime.internal.ApplicationSpec; /** * Created by yipan on 7/10/18. */ -public abstract class ApplicationSpec { +public abstract class ApplicationBuilder implements ApplicationInitializer, ApplicationSpec { final Config config; + final T userApp; - protected ApplicationSpec(Config config) { + protected ApplicationBuilder(T userApp, Config config) { this.config = config; + this.userApp = userApp; } public static class AppConfig extends MapConfig { @@ -50,8 +55,19 @@ public String getGlobalAppId() { } + @Override public String getGlobalAppId() { return new AppConfig(config).getGlobalAppId(); } + @Override + public Config getConfig() { + return config; + } + + @Override + public T getUserApp() { + return userApp; + } + } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java new file mode 100644 index 0000000000..4d6538224c --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java @@ -0,0 +1,99 @@ +package org.apache.samza.application.internal; + +import java.lang.reflect.Constructor; +import java.util.Collection; +import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationInitializer; +import org.apache.samza.config.Config; +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.runtime.internal.StreamApplicationSpec; +import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; + + +/** + * Created by yipan on 7/10/18. + */ +public class StreamApplicationBuilder extends ApplicationBuilder implements StreamApplicationInitializer, StreamApplicationSpec { + final StreamApplicationBuilder graphBuilder; + + public StreamApplicationBuilder(StreamApplication userApp, Config config) { + super(userApp, config); + this.graphBuilder = createDefault(config); + userApp.init(this, config); + } + + private StreamApplicationBuilder createDefault(Config config) { + try { + Constructor constructor = Class.forName("org.apache.samza.operators.StreamGraphSpec").getConstructor(Config.class); // *sigh* + return (StreamApplicationBuilder) constructor.newInstance(config); + } catch (Exception e) { + throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); + } + } + + @Override + public void setDefaultSerde(Serde serde) { + this.graphBuilder.setDefaultSerde(serde); + } + + @Override + public MessageStream getInputStream(String streamId, Serde serde) { + return this.graphBuilder.getInputStream(streamId, serde); + } + + @Override + public MessageStream getInputStream(String streamId) { + return this.graphBuilder.getInputStream(streamId); + } + + @Override + public OutputStream getOutputStream(String streamId, Serde serde) { + return this.graphBuilder.getOutputStream(streamId, serde); + } + + @Override + public OutputStream getOutputStream(String streamId) { + return this.graphBuilder.getOutputStream(streamId); + } + + @Override + public Table> getTable(TableDescriptor tableDesc) { + return this.graphBuilder.getTable(tableDesc); + } + + @Override + public StreamApplicationInitializer withContextManager(ContextManager contextManager) { + return this.graphBuilder.withContextManager(contextManager); + } + + @Override + public ContextManager getContextManager() { + return this.graphBuilder.getContextManager(); + } + + @Override + public Collection getInputStreams() { + return this.graphBuilder.getInputStreams(); + } + + @Override + public Collection getOutputStreams() { + return this.graphBuilder.getOutputStreams(); + } + + @Override + public Collection getBroadcastStreams() { + return this.graphBuilder.getBroadcastStreams(); + } + + @Override + public Collection getTables() { + return this.graphBuilder.getTables(); + } +} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java deleted file mode 100644 index 4fddd1109f..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationSpec.java +++ /dev/null @@ -1,17 +0,0 @@ -package org.apache.samza.application.internal; - -import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraph; - - -/** - * Created by yipan on 7/10/18. - */ -public class StreamApplicationSpec extends ApplicationSpec { - final StreamGraph graph; - - public StreamApplicationSpec(StreamGraph graph, Config config) { - super(config); - this.graph = graph; - } -} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java new file mode 100644 index 0000000000..b9a433edc9 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java @@ -0,0 +1,66 @@ +package org.apache.samza.application.internal; + +import java.util.ArrayList; +import java.util.List; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.TaskApplicationInitializer; +import org.apache.samza.config.Config; +import org.apache.samza.runtime.internal.TaskApplicationSpec; +import org.apache.samza.task.TaskFactory; + + +/** + * Created by yipan on 7/10/18. + */ +public class TaskApplicationBuilder extends ApplicationBuilder implements TaskApplicationInitializer, TaskApplicationSpec { + + TaskFactory taskFactory; + final List inputStreams = new ArrayList<>(); + final List outputStreams = new ArrayList<>(); + final List tables = new ArrayList<>(); + + public TaskApplicationBuilder(TaskApplication userApp, Config config) { + super(userApp, config); + userApp.init(this, config); + } + + @Override + public void setTaskFactory(TaskFactory factory) { + this.taskFactory = factory; + } + + @Override + public void addInputStreams(List inputStreams) { + this.inputStreams.addAll(inputStreams); + } + + @Override + public void addOutputStreams(List outputStreams) { + this.outputStreams.addAll(outputStreams); + } + + @Override + public void addTables(List tables) { + this.tables.addAll(tables); + } + + @Override + public TaskFactory getTaskFactory() { + return this.taskFactory; + } + + @Override + public List getInputStreams() { + return this.inputStreams; + } + + @Override + public List getOutputStreams() { + return this.outputStreams; + } + + @Override + public List getTables() { + return this.tables; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java deleted file mode 100644 index 5de551c79a..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationSpec.java +++ /dev/null @@ -1,22 +0,0 @@ -package org.apache.samza.application.internal; - -import org.apache.samza.config.Config; -import org.apache.samza.task.TaskFactory; - - -/** - * Created by yipan on 7/10/18. - */ -public class TaskApplicationSpec extends ApplicationSpec { - - final TaskFactory taskFactory; - - public TaskApplicationSpec(TaskFactory taskFactory, Config config) { - super(config); - this.taskFactory = taskFactory; - } - - public TaskFactory getTaskFactory() { - return this.taskFactory; - } -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java index f8e32b98e8..22e13d6f5d 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java @@ -18,140 +18,23 @@ */ package org.apache.samza.operators; -import org.apache.samza.SamzaException; +import java.util.Collection; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.serializers.Serde; -import org.apache.samza.table.Table; /** * Provides access to {@link MessageStream}s and {@link OutputStream}s used to describe application logic. */ -@InterfaceStability.Unstable +@InterfaceStability.Evolving public interface StreamGraph { + ContextManager getContextManager(); - /** - * Sets the default {@link Serde} to use for (de)serializing messages. - *

    . - * If the default serde is set, it must be set before creating any input or output streams. - *

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

    - * Providing an incompatible message type for the input/output streams that use the default serde will result in - * {@link ClassCastException}s at runtime. - * - * @param serde the default message {@link Serde} to use - */ - void setDefaultSerde(Serde serde); + Collection getInputStreams(); - /** - * Gets the input {@link MessageStream} corresponding to the {@code streamId}. - *

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

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param serde the {@link Serde} to use for deserializing incoming messages - * @param the type of messages in the input {@link MessageStream} - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - MessageStream getInputStream(String streamId, Serde serde); + Collection getOutputStreams(); - /** - * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via - * {@link #setDefaultSerde(Serde)} for deserializing input messages. - *

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param the type of message in the input {@link MessageStream} - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - MessageStream getInputStream(String streamId); + Collection getBroadcastStreams(); - /** - * Gets the {@link OutputStream} corresponding to the {@code streamId}. - *

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

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

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param serde the {@link Serde} to use for serializing outgoing messages - * @param the type of messages in the {@link OutputStream} - * @return the output {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - OutputStream getOutputStream(String streamId, Serde serde); + Collection getTables(); - /** - * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via - * {@link #setDefaultSerde(Serde)} for serializing output messages. - *

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param the type of messages in the {@link OutputStream} - * @return the output {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - OutputStream getOutputStream(String streamId); - - /** - * Gets the {@link Table} corresponding to the {@link TableDescriptor}. - *

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

    - * The provided {@link ContextManager} can be used to setup shared context between the operator functions - * within a task instance - * - * @param contextManager the {@link ContextManager} to use for the {@link StreamGraph} - * @return the {@link StreamGraph} with {@code contextManager} set as its {@link ContextManager} - */ - StreamGraph withContextManager(ContextManager contextManager); - - static StreamGraph createInstance() { - try { - return (StreamGraph) Class.forName("org.apache.samza.operators.StreamGraphSpec").newInstance(); - } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) { - throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); - } - } } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java new file mode 100644 index 0000000000..dd62271d73 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java @@ -0,0 +1,26 @@ +package org.apache.samza.runtime; + +import java.time.Duration; +import java.util.Map; +import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; + + +/** + * Created by yipan on 7/11/18. + */ +public interface ApplicationRuntime { + void start(); + void stop(); + ApplicationStatus status(); + void waitForFinish(); + boolean waitForFinish(Duration timeout); + + /** + * Method to add a set of customized {@link MetricsReporter}s in the application + * + * @param metricsReporters the map of customized {@link MetricsReporter}s objects to be used + */ + void addMetricsReporters(Map metricsReporters); + +} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java new file mode 100644 index 0000000000..54f7b21cf3 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java @@ -0,0 +1,73 @@ +package org.apache.samza.runtime; + +import java.time.Duration; +import java.util.Map; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamApplicationBuilder; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.internal.TaskApplicationBuilder; +import org.apache.samza.config.Config; +import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunners; +import org.apache.samza.runtime.internal.ApplicationSpec; + + +/** + * Created by yipan on 7/11/18. + */ +public class ApplicationRuntimes { + private ApplicationRuntimes() { + + } + + public static final ApplicationRuntime createStreamApp(StreamApplication userApp, Config config) { + return new RuntimeAppImpl(new StreamApplicationBuilder(userApp, config)); + } + + public static final ApplicationRuntime createTaskApp(TaskApplication userApp, Config config) { + return new RuntimeAppImpl(new TaskApplicationBuilder(userApp, config)); + } + + + private static class RuntimeAppImpl implements ApplicationRuntime { + private final ApplicationSpec appSpec; + private final ApplicationRunner runner; + + RuntimeAppImpl(ApplicationSpec appSpec) { + this.appSpec = appSpec; + this.runner = ApplicationRunners.fromConfig(appSpec.getConfig()); + } + + @Override + public void start() { + this.runner.run(appSpec); + } + + @Override + public void stop() { + this.runner.kill(appSpec); + } + + @Override + public ApplicationStatus status() { + return this.runner.status(appSpec); + } + + @Override + public void waitForFinish() { + this.runner.waitForFinish(appSpec, Duration.ofSeconds(0)); + } + + @Override + public boolean waitForFinish(Duration timeout) { + return this.runner.waitForFinish(appSpec, timeout); + } + + @Override + public void addMetricsReporters(Map metricsReporters) { + this.runner.addMetricsReporters(metricsReporters); + } + } +} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java index c73143aa9f..1032ff302b 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java @@ -22,10 +22,8 @@ import java.util.Map; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.ApplicationSpec; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.system.StreamSpec; /** @@ -34,13 +32,14 @@ @InterfaceStability.Evolving public interface ApplicationRunner { - void run(ApplicationSpec appRunnable); + void run(ApplicationSpec appSpec); - void kill(ApplicationSpec appRunnable); + void kill(ApplicationSpec appSpec); - ApplicationStatus status(ApplicationSpec appRunnable); + ApplicationStatus status(ApplicationSpec appSpec); - void waitForFinish(ApplicationSpec appRunnable); + @Deprecated + void waitForFinish(ApplicationSpec appSpec); /** * Waits for {@code timeout} duration for the application to finish. @@ -49,7 +48,7 @@ public interface ApplicationRunner { * @return true - application finished before timeout * false - otherwise */ - boolean waitForFinish(ApplicationSpec appRunnable, Duration timeout); + boolean waitForFinish(ApplicationSpec appSpec, Duration timeout); /** * Method to add a set of customized {@link MetricsReporter}s in the application @@ -58,14 +57,4 @@ public interface ApplicationRunner { */ void addMetricsReporters(Map metricsReporters); - /** - * Constructs a {@link StreamSpec} from the configuration for the specified streamId. - * - * The stream configurations are read from the following properties in the config: - * {@code streams.{$streamId}.*} - *
    - * All properties matching this pattern are assumed to be system-specific with two exceptions. The following two - * properties are Samza properties which are used to bind the stream to a system and a physical resource on that system. - */ - StreamSpec getStreamSpec(String streamId); } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java similarity index 92% rename from samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java rename to samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java index d247199fe4..3922a947df 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java @@ -1,9 +1,8 @@ -package org.apache.samza.runtime; +package org.apache.samza.runtime.internal; import java.lang.reflect.Constructor; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; -import org.apache.samza.runtime.internal.ApplicationRunner; /** @@ -35,7 +34,7 @@ static String getAppRunnerCfg() { * Static method to load the {@link ApplicationRunner} * * @param config configuration passed in to initialize the Samza processes - * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications + * @return the configure-driven {@link ApplicationRunner} to start the user-defined stream applications */ public static ApplicationRunner fromConfig(Config config) { AppRunnerConfig appRunnerCfg = new AppRunnerConfig(config); diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java new file mode 100644 index 0000000000..bc265f95a2 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java @@ -0,0 +1,16 @@ +package org.apache.samza.runtime.internal; + +import org.apache.samza.application.UserApplication; +import org.apache.samza.config.Config; + + +/** + * Created by yipan on 7/17/18. + */ +public interface ApplicationSpec { + Config getConfig(); + + T getUserApp(); + + String getGlobalAppId(); +} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java new file mode 100644 index 0000000000..7b5dfa862a --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java @@ -0,0 +1,25 @@ +package org.apache.samza.runtime.internal; + +import java.util.Collection; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.operators.ContextManager; + + +/** + * Created by yipan on 7/17/18. + */ +public interface StreamApplicationSpec extends ApplicationSpec { + + ContextManager getContextManager(); + + Collection getInputStreams(); + + Collection getOutputStreams(); + + Collection getBroadcastStreams(); + + Collection getTables(); + + + +} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java new file mode 100644 index 0000000000..7d47f70b11 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java @@ -0,0 +1,20 @@ +package org.apache.samza.runtime.internal; + +import java.util.List; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.task.TaskFactory; + + +/** + * Created by yipan on 7/17/18. + */ +public interface TaskApplicationSpec extends ApplicationSpec { + TaskFactory getTaskFactory(); + + List getInputStreams(); + + List getOutputStreams(); + + List getTables(); + +} diff --git a/samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java b/samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java deleted file mode 100644 index 382a269b2a..0000000000 --- a/samza-core/src/main/java/org/apache/samza/application/internal/StreamApplicationRuntime.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import org.apache.samza.operators.StreamGraphSpec; - - -public class StreamApplicationRuntime { - - private final StreamApplicationSpec appSpec; - - public StreamApplicationRuntime(StreamApplicationSpec app) { - this.appSpec = app; - } - - public StreamGraphSpec getStreamGraphSpec() { - return (StreamGraphSpec) this.appSpec.graph; - } - -} diff --git a/samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java b/samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java deleted file mode 100644 index da4435030c..0000000000 --- a/samza-core/src/main/java/org/apache/samza/application/internal/TaskApplicationRuntime.java +++ /dev/null @@ -1,20 +0,0 @@ -package org.apache.samza.application.internal; - -import org.apache.samza.task.TaskFactory; - - -/** - * Created by yipan on 7/10/18. - */ -public class TaskApplicationRuntime { - private final TaskApplicationSpec appSpec; - - public TaskApplicationRuntime(TaskApplicationSpec app) { - this.appSpec = app; - } - - public TaskFactory getTaskFactory() { - return (TaskFactory) this.appSpec.taskFactory; - } - -} diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java index a187b94172..964e8a1f97 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java @@ -18,15 +18,19 @@ */ package org.apache.samza.operators; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; - import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationInitializer; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -34,6 +38,7 @@ import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; +import org.apache.samza.runtime.internal.StreamApplicationSpec; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; @@ -42,16 +47,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; - /** * This class defines: * 1) an implementation of {@link StreamGraph} that provides APIs for accessing {@link MessageStream}s to be used to * create the DAG of transforms. * 2) a builder that creates a serializable {@link OperatorSpecGraph} from user-defined DAG */ -public class StreamGraphSpec implements StreamGraph { +public class StreamGraphSpec implements StreamApplicationInitializer, StreamApplicationSpec { private static final Logger LOGGER = LoggerFactory.getLogger(StreamGraphSpec.class); private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); @@ -156,7 +158,7 @@ public Table> getTable(TableDescriptor tableDesc) { } @Override - public StreamGraph withContextManager(ContextManager contextManager) { + public StreamApplicationInitializer withContextManager(ContextManager contextManager) { this.contextManager = contextManager; return this; } @@ -165,6 +167,26 @@ public ContextManager getContextManager() { return this.contextManager; } + @Override + public Collection getInputStreams() { + return Collections.unmodifiableCollection(this.inputOperators.keySet()); + } + + @Override + public Collection getOutputStreams() { + return Collections.unmodifiableCollection(this.outputStreams.keySet()); + } + + @Override + public Collection getBroadcastStreams() { + return Collections.unmodifiableCollection(this.broadcastStreams); + } + + @Override + public Collection getTables() { + return Collections.unmodifiableCollection(this.tables.keySet().stream().collect(HashSet::new, (s1, td) -> s1.add(td.getId()), (s1, s2) -> s1.addAll(s2))); + } + public OperatorSpecGraph getOperatorSpecGraph() { return new OperatorSpecGraph(this); } @@ -255,15 +277,11 @@ Map getInputOperators() { return Collections.unmodifiableMap(inputOperators); } - Map getOutputStreams() { + Map getOutputStreamImpls() { return Collections.unmodifiableMap(outputStreams); } - Set getBroadcastStreams() { - return Collections.unmodifiableSet(broadcastStreams); - } - - Map getTables() { + Map getTableImpls() { return Collections.unmodifiableMap(tables); } @@ -293,4 +311,19 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } + + @Override + public Config getConfig() { + return this.config; + } + + @Override + public StreamApplication getUserApp() { + throw new SamzaException("shouldn't be called here"); + } + + @Override + public String getGlobalAppId() { + throw new SamzaException("shouldn't be called here"); + } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index f7f19b34ae..58fb0463c4 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -26,10 +26,6 @@ import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.ApplicationRunnable; -import org.apache.samza.application.internal.ApplicationSpec; -import org.apache.samza.application.internal.StreamApplicationSpec; -import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ApplicationConfig.ApplicationMode; import org.apache.samza.config.Config; @@ -44,6 +40,9 @@ import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationSpec; +import org.apache.samza.runtime.internal.StreamApplicationSpec; +import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,42 +118,66 @@ StreamManager buildAndStartStreamManager() { return streamManager; } - private ApplicationRunnable getRunnable(ApplicationSpec appSpec) { + private ApplicationLifecycle getLifecycleMethods(ApplicationSpec appSpec) { if (appSpec instanceof StreamApplicationSpec) { - return getStreamAppRunnable((StreamApplicationSpec) appSpec); + return getStreamAppLifecycle((StreamApplicationSpec) appSpec); } if (appSpec instanceof TaskApplicationSpec) { - return getTaskAppRunnable((TaskApplicationSpec) appSpec); + return getTaskAppLifecycle((TaskApplicationSpec) appSpec); } - throw new IllegalArgumentException(String.format("The specified application %s is not valid. Only StreamApplication and Task applications are supported.", appSpec.getClass().getName())); + throw new IllegalArgumentException(String.format("The specified application %s is not valid. " + + "Only StreamApplicationSpec and TaskApplicationSpec are supported.", appSpec.getClass().getName())); } - protected abstract ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec); + protected abstract ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec); - protected abstract ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec); + protected abstract ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec); + + interface ApplicationLifecycle { + + void run(); + + void kill(); + + ApplicationStatus status(); + + /** + * Waits for {@code timeout} duration for the application to finish. + * + * @param timeout time to wait for the application to finish + * @return true - application finished before timeout + * false - otherwise + */ + boolean waitForFinish(Duration timeout); + + } @Override public final void run(ApplicationSpec appSpec) { - getRunnable(appSpec).run(); + appSpec.getUserApp().beforeStart(appSpec); + getLifecycleMethods(appSpec).run(); + appSpec.getUserApp().afterStart(appSpec); } @Override public final ApplicationStatus status(ApplicationSpec appSpec) { - return getRunnable(appSpec).status(); + return getLifecycleMethods(appSpec).status(); } @Override public final void kill(ApplicationSpec appSpec) { - getRunnable(appSpec).kill(); + appSpec.getUserApp().beforeStop(appSpec); + getLifecycleMethods(appSpec).kill(); + appSpec.getUserApp().afterStop(appSpec); } @Override public final void waitForFinish(ApplicationSpec appSpec) { - getRunnable(appSpec).waitForFinish(); + getLifecycleMethods(appSpec).waitForFinish(Duration.ofSeconds(0)); } @Override public final boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { - return getRunnable(appSpec).waitForFinish(timeout); + return getLifecycleMethods(appSpec).waitForFinish(timeout); } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java index 1e672a0376..2e08a8dacf 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java @@ -27,11 +27,11 @@ * The class defines the basic command line arguments for Samza command line scripts. */ public class ApplicationRunnerCommandLine extends CommandLine { - public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; run, status, kill.") + public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; start, status, stop.") .withRequiredArg() .ofType(String.class) - .describedAs("operation=run") - .defaultsTo("run"); + .describedAs("operation=start") + .defaultsTo("start"); public ApplicationRunnerOperation getOperation(OptionSet options) { String rawOp = options.valueOf(operationOpt).toString(); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index cf92f0406c..4b3297677f 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -21,9 +21,7 @@ import joptsimple.OptionSet; import joptsimple.OptionSpec; -import org.apache.samza.application.ApplicationRunnable; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.task.TaskFactory; @@ -33,9 +31,9 @@ /** - * This class contains the main() method used by run-app.sh. - * For a StreamApplication, it creates the {@link ApplicationRunner} based on the config, and then run the application. - * For a Samza job using low level task API, it will create the JobRunner to run it. + * This class contains the main() method used by start-app.sh. + * For a StreamApplication, it creates the {@link ApplicationRunner} based on the config, and then start the application. + * For a Samza job using low level task API, it will create the JobRunner to start it. */ public class ApplicationRunnerMain { // TODO: have the app configs consolidated in one place @@ -43,11 +41,11 @@ public class ApplicationRunnerMain { public static class ApplicationRunnerCommandLine extends CommandLine { public OptionSpec operationOpt = - parser().accepts("operation", "The operation to perform; run, status, kill.") + parser().accepts("operation", "The operation to perform; start, status, stop.") .withRequiredArg() .ofType(String.class) - .describedAs("operation=run") - .defaultsTo("run"); + .describedAs("operation=start") + .defaultsTo("start"); public ApplicationRunnerOperation getOperation(OptionSet options) { String rawOp = options.valueOf(operationOpt).toString(); @@ -62,19 +60,23 @@ public static void main(String[] args) throws Exception { Config config = Util.rewriteConfig(orgConfig); ApplicationRunnerOperation op = cmdLine.getOperation(options); - ApplicationRunnable appRunnable = config.containsKey(STREAM_APPLICATION_CLASS_CONFIG) ? - StreamApplications.createRunnable((StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(), config) : - StreamApplications.createRunnable((TaskFactory) TaskFactoryUtil.createTaskFactory(config), config); + ApplicationRuntime appRuntime = + config.containsKey(STREAM_APPLICATION_CLASS_CONFIG) ? ApplicationRuntimes.createStreamApp( + (StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(), config) : + // TODO: Need to deal with 1) new TaskApplication implemention that populates inputStreams and outputStreams by the user; + // 2) legacy task application that only has input streams specified in config + ApplicationRuntimes.createTaskApp( + (appBuilder, cfg) -> appBuilder.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(cfg)), config); switch (op) { case RUN: - appRunnable.run(); + appRuntime.start(); break; case KILL: - appRunnable.kill(); + appRuntime.stop(); break; case STATUS: - System.out.println(appRunnable.status()); + System.out.println(appRuntime.status()); break; default: throw new IllegalArgumentException("Unrecognized operation: " + op); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 62b351f131..4eb8c56f60 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -33,11 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; -import org.apache.samza.application.ApplicationRunnable; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamApplicationRuntime; -import org.apache.samza.application.internal.StreamApplicationSpec; -import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobCoordinatorConfig; @@ -51,6 +47,8 @@ import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.internal.StreamApplicationSpec; +import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.apache.samza.system.StreamSpec; import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTaskFactory; @@ -135,13 +133,13 @@ public LocalApplicationRunner(Config config) { } @Override - protected ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec) { - return new TaskAppRunnable(appSpec); + protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec) { + return new TaskAppLifecycle(appSpec); } @Override - protected ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec) { - return new StreamAppRunnable(appSpec); + protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec) { + return new StreamAppLifecycle(appSpec); } public LocalApplicationRunner(Config config, Map customMetricsReporters) { @@ -150,22 +148,21 @@ public LocalApplicationRunner(Config config, Map custom this.customMetricsReporters = customMetricsReporters; } - class StreamAppRunnable implements ApplicationRunnable { + class StreamAppLifecycle implements ApplicationLifecycle { final StreamApplicationSpec streamApp; - StreamAppRunnable(StreamApplicationSpec streamApp) { + StreamAppLifecycle(StreamApplicationSpec streamApp) { this.streamApp = streamApp; } @Override public void run() { - StreamApplicationRuntime app = new StreamApplicationRuntime(streamApp); StreamManager streamManager = null; try { streamManager = buildAndStartStreamManager(); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(app.getStreamGraphSpec(), streamManager); + ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), streamManager); String executionPlanJson = plan.getPlanAsJson(); writePlanJsonFile(executionPlanJson); @@ -178,12 +175,12 @@ public void run() { // 3. create the StreamProcessors if (plan.getJobConfigs().isEmpty()) { - throw new SamzaException("No jobs to run."); + throw new SamzaException("No jobs to start."); } plan.getJobConfigs().forEach(jobConfig -> { LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); - StreamProcessor processor = createStreamProcessor(jobConfig, app.getStreamGraphSpec(), listener); + StreamProcessor processor = createStreamProcessor(jobConfig, (StreamGraphSpec) streamApp.getGraph(), listener); listener.setProcessor(processor); processors.add(processor); }); @@ -194,7 +191,7 @@ public void run() { } catch (Throwable throwable) { appStatus = ApplicationStatus.unsuccessfulFinish(throwable); shutdownLatch.countDown(); - throw new SamzaException(String.format("Failed to start application: %s.", app), throwable); + throw new SamzaException(String.format("Failed to start application: %s.", streamApp), throwable); } finally { if (streamManager != null) { streamManager.stop(); @@ -212,51 +209,24 @@ public ApplicationStatus status() { return appStatus; } - @Override - public void waitForFinish() { - waitForFinish(Duration.ofMillis(0)); - } - @Override public boolean waitForFinish(Duration timeout) { - long timeoutInMs = timeout.toMillis(); - boolean finished = true; - - try { - if (timeoutInMs < 1) { - shutdownLatch.await(); - } else { - finished = shutdownLatch.await(timeoutInMs, TimeUnit.MILLISECONDS); - - if (!finished) { - LOG.warn("Timed out waiting for application to finish."); - } - } - } catch (Exception e) { - LOG.error("Error waiting for application to finish", e); - throw new SamzaException(e); - } - - return finished; + return LocalApplicationRunner.this.waitForFinish(timeout); } - @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - throw new UnsupportedOperationException("MetricsReporters should be set to the ApplicationRunner"); - } } - class TaskAppRunnable implements ApplicationRunnable { + class TaskAppLifecycle implements ApplicationLifecycle { final TaskApplicationSpec appSpec; StreamProcessor sp; - TaskAppRunnable(TaskApplicationSpec appSpec) { + TaskAppLifecycle(TaskApplicationSpec appSpec) { this.appSpec = appSpec; } @Override public void run() { - LOG.info("LocalApplicationRunner will run task " + appSpec.getGlobalAppId()); + LOG.info("LocalApplicationRunner will start task " + appSpec.getGlobalAppId()); LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); sp = getStreamProcessorInstance(config, appSpec.getTaskFactory(), listener); @@ -277,37 +247,32 @@ public ApplicationStatus status() { } @Override - public void waitForFinish() { - waitForFinish(Duration.ofMillis(0)); + public boolean waitForFinish(Duration timeout) { + return LocalApplicationRunner.this.waitForFinish(timeout); } - @Override - public boolean waitForFinish(Duration timeout) { - long timeoutInMs = timeout.toMillis(); - boolean finished = true; + } - try { - if (timeoutInMs < 1) { - shutdownLatch.await(); - } else { - finished = shutdownLatch.await(timeoutInMs, TimeUnit.MILLISECONDS); - - if (!finished) { - LOG.warn("Timed out waiting for application to finish."); - } + private boolean waitForFinish(Duration timeout) { + long timeoutInMs = timeout.toMillis(); + boolean finished = true; + + try { + if (timeoutInMs < 1) { + shutdownLatch.await(); + } else { + finished = shutdownLatch.await(timeoutInMs, TimeUnit.MILLISECONDS); + + if (!finished) { + LOG.warn("Timed out waiting for application to finish."); } - } catch (Exception e) { - LOG.error("Error waiting for application to finish", e); - throw new SamzaException(e); } - - return finished; + } catch (Exception e) { + LOG.error("Error waiting for application to finish", e); + throw new SamzaException(e); } - @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - throw new UnsupportedOperationException("MetricsReporters should be set to the ApplicationRunner"); - } + return finished; } /** diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index d5fabd5b82..fd6938d755 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -19,22 +19,15 @@ package org.apache.samza.runtime; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; import java.time.Duration; import java.util.HashMap; -import java.util.Map; import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; -import org.apache.samza.application.ApplicationRunnable; +import org.apache.samza.application.internal.ApplicationBuilder; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.ApplicationSpec; -import org.apache.samza.application.internal.StreamApplicationRuntime; -import org.apache.samza.application.internal.StreamApplicationSpec; -import org.apache.samza.application.internal.TaskApplicationRuntime; -import org.apache.samza.application.internal.TaskApplicationSpec; +import org.apache.samza.application.internal.StreamApplicationBuilder; +import org.apache.samza.application.internal.TaskApplicationBuilder; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -45,8 +38,9 @@ import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.model.JobModel; -import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.StreamGraphSpec; +import org.apache.samza.runtime.internal.StreamApplicationSpec; +import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.SamzaUncaughtExceptionHandler; @@ -80,20 +74,20 @@ public LocalContainerRunner(JobModel jobModel, String containerId) { } @Override - protected ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec) { - return new TaskAppRunnable(appSpec); + protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec) { + return new TaskAppLifecycle(appSpec); } @Override - protected ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec) { - return new StreamAppRunnable(appSpec); + protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec) { + return new StreamAppLifecycle(appSpec); } - class TaskAppRunnable implements ApplicationRunnable { - final TaskApplicationRuntime taskApp; + class TaskAppLifecycle implements ApplicationLifecycle { + final TaskApplicationSpec taskApp; - TaskAppRunnable(TaskApplicationSpec taskApp) { - this.taskApp = new TaskApplicationRuntime(taskApp); + TaskAppLifecycle(TaskApplicationSpec taskApp) { + this.taskApp = taskApp; } @Override @@ -148,34 +142,24 @@ public ApplicationStatus status() { throw new UnsupportedOperationException(); } - @Override - public void waitForFinish() { - - } - @Override public boolean waitForFinish(Duration timeout) { return false; } - @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted - throw new UnsupportedOperationException(); - } } - class StreamAppRunnable implements ApplicationRunnable { - final StreamApplicationRuntime streamApp; + class StreamAppLifecycle implements ApplicationLifecycle { + final StreamApplicationSpec streamApp; - StreamAppRunnable(StreamApplicationSpec streamApp) { - this.streamApp = new StreamApplicationRuntime(streamApp); + StreamAppLifecycle(StreamApplicationSpec streamApp) { + this.streamApp = streamApp; } @Override public void run() { - Object taskFactory = TaskFactoryUtil.createTaskFactory(streamApp.getStreamGraphSpec().getOperatorSpecGraph(), - streamApp.getStreamGraphSpec().getContextManager()); + Object taskFactory = TaskFactoryUtil.createTaskFactory(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), + ((StreamGraphSpec) streamApp.getGraph()).getContextManager()); container = SamzaContainer$.MODULE$.apply( containerId, @@ -225,20 +209,11 @@ public ApplicationStatus status() { throw new UnsupportedOperationException(); } - @Override - public void waitForFinish() { - - } - @Override public boolean waitForFinish(Duration timeout) { return false; } - @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - throw new UnsupportedOperationException(); - } } // only invoked by legacy applications w/o user-defined main @@ -268,20 +243,15 @@ public static void main(String[] args) throws Exception { MDC.put("jobName", jobName); MDC.put("jobId", jobId); - ApplicationSpec.AppConfig appConfig = new ApplicationSpec.AppConfig(config); + ApplicationBuilder.AppConfig appConfig = new ApplicationBuilder.AppConfig(config); LocalContainerRunner runner = new LocalContainerRunner(jobModel, containerId); - ApplicationSpec appSpec = null; - if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { - // add configuration-factory and configuration-path to the command line options and invoke the user defined main class - // write the complete configuration to a local file in property file format - StreamGraphSpec streamGraph = new StreamGraphSpec(runner, config); - StreamApplication userApp = (StreamApplication) Class.forName(appConfig.getAppClass()).newInstance(); - userApp.init(streamGraph, config); - appSpec = new StreamApplicationSpec(streamGraph, config); - } else { - appSpec = new TaskApplicationSpec((TaskFactory) TaskFactoryUtil.createTaskFactory(config), config); - } + + ApplicationBuilder appSpec = appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty() ? + new StreamApplicationBuilder((StreamApplication) Class.forName(appConfig.getAppClass()).newInstance(), config) : + // TODO: Need to deal with 1) new TaskApplication implemention that populates inputStreams and outputStreams by the user; + // 2) legacy task application that only has input streams specified in config + new TaskApplicationBuilder((appBuilder, cfg) -> appBuilder.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(cfg)), config); runner.run(appSpec); runner.waitForFinish(appSpec); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 778b41ab80..b48092beea 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -20,14 +20,8 @@ package org.apache.samza.runtime; import java.time.Duration; -import java.util.Map; import java.util.UUID; import org.apache.samza.SamzaException; -import org.apache.samza.application.ApplicationRunnable; -import org.apache.samza.application.internal.StreamApplicationRuntime; -import org.apache.samza.application.internal.StreamApplicationSpec; -import org.apache.samza.application.internal.TaskApplicationRuntime; -import org.apache.samza.application.internal.TaskApplicationSpec; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -37,8 +31,10 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.JobRunner; import org.apache.samza.metrics.MetricsRegistryMap; -import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.internal.StreamApplicationSpec; +import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,21 +54,21 @@ public RemoteApplicationRunner(Config config) { } @Override - protected ApplicationRunnable getTaskAppRunnable(TaskApplicationSpec appSpec) { - return new TaskAppRunnable(appSpec); + protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec) { + return new TaskAppLifecycle(appSpec); } @Override - protected ApplicationRunnable getStreamAppRunnable(StreamApplicationSpec appSpec) { - return new StreamAppRunnable(appSpec); + protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec) { + return new StreamAppLifecycle(appSpec); } - class TaskAppRunnable implements ApplicationRunnable { - final TaskApplicationRuntime taskApp; + class TaskAppLifecycle implements ApplicationLifecycle { + final TaskApplicationSpec taskApp; final JobRunner jobRunner; - TaskAppRunnable(TaskApplicationSpec appSpec) { - this.taskApp = new TaskApplicationRuntime(appSpec); + TaskAppLifecycle(TaskApplicationSpec appSpec) { + this.taskApp = appSpec; this.jobRunner = new JobRunner(config); } @@ -91,26 +87,18 @@ public ApplicationStatus status() { return jobRunner.status(); } - @Override - public void waitForFinish() { - } - @Override public boolean waitForFinish(Duration timeout) { - return false; + return RemoteApplicationRunner.this.waitForFinish(timeout); } - @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - throw new UnsupportedOperationException(); - } } - class StreamAppRunnable implements ApplicationRunnable { - final StreamApplicationRuntime streamApp; + class StreamAppLifecycle implements ApplicationLifecycle { + final StreamApplicationSpec streamApp; - StreamAppRunnable(StreamApplicationSpec appSpec) { - this.streamApp = new StreamApplicationRuntime(appSpec); + StreamAppLifecycle(StreamApplicationSpec appSpec) { + this.streamApp = appSpec; } @Override @@ -118,13 +106,13 @@ public void run() { StreamManager streamManager = null; try { streamManager = buildAndStartStreamManager(); - // TODO: run.id needs to be set for standalone: SAMZA-1531 - // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + // TODO: start.id needs to be set for standalone: SAMZA-1531 + // start.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The run id for this run is {}", runId); + LOG.info("The start id for this start is {}", runId); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamApp.getStreamGraphSpec(), runId, streamManager); + ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), runId, streamManager); writePlanJsonFile(plan.getPlanAsJson()); // 2. create the necessary streams @@ -140,7 +128,7 @@ public void run() { runner.run(true); }); } catch (Throwable t) { - throw new SamzaException("Failed to run application", t); + throw new SamzaException("Failed to start application", t); } finally { if (streamManager != null) { streamManager.stop(); @@ -153,7 +141,7 @@ public void kill() { StreamManager streamManager = null; try { streamManager = buildAndStartStreamManager(); - ExecutionPlan plan = getExecutionPlan(streamApp.getStreamGraphSpec(), streamManager); + ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), streamManager); plan.getJobConfigs().forEach(jobConfig -> { LOG.info("Killing job {}", jobConfig.getName()); @@ -161,7 +149,7 @@ public void kill() { runner.kill(); }); } catch (Throwable t) { - throw new SamzaException("Failed to kill application", t); + throw new SamzaException("Failed to stop application", t); } finally { if (streamManager != null) { streamManager.stop(); @@ -178,7 +166,7 @@ public ApplicationStatus status() { ApplicationStatus unsuccessfulFinishStatus = null; streamManager = buildAndStartStreamManager(); - ExecutionPlan plan = getExecutionPlan(streamApp.getStreamGraphSpec(), streamManager); + ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), streamManager); for (JobConfig jobConfig : plan.getJobConfigs()) { ApplicationStatus status = getApplicationStatus(jobConfig); @@ -221,51 +209,11 @@ public ApplicationStatus status() { } } - @Override - public void waitForFinish() { - waitForFinish(Duration.ofMillis(0)); - } - @Override public boolean waitForFinish(Duration timeout) { - JobConfig jobConfig = new JobConfig(config); - boolean finished = true; - long timeoutInMs = timeout.toMillis(); - long startTimeInMs = System.currentTimeMillis(); - long timeElapsed = 0L; - - long sleepDurationInMs = timeoutInMs < 1 ? - DEFAULT_SLEEP_DURATION_MS : Math.min(timeoutInMs, DEFAULT_SLEEP_DURATION_MS); - ApplicationStatus status; - - try { - while (timeoutInMs < 1 || timeElapsed <= timeoutInMs) { - status = getApplicationStatus(jobConfig); - if (status == SuccessfulFinish || status == UnsuccessfulFinish) { - LOG.info("Application finished with status {}", status); - break; - } - - Thread.sleep(sleepDurationInMs); - timeElapsed = System.currentTimeMillis() - startTimeInMs; - } - - if (timeElapsed > timeoutInMs) { - LOG.warn("Timed out waiting for application to finish."); - finished = false; - } - } catch (Exception e) { - LOG.error("Error waiting for application to finish", e); - throw new SamzaException(e); - } - - return finished; + return RemoteApplicationRunner.this.waitForFinish(timeout); } - @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - throw new UnsupportedOperationException(); - } } /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { @@ -275,6 +223,41 @@ public ApplicationRunnable withMetricsReporters(Map met return status; } + private boolean waitForFinish(Duration timeout) { + JobConfig jobConfig = new JobConfig(config); + boolean finished = true; + long timeoutInMs = timeout.toMillis(); + long startTimeInMs = System.currentTimeMillis(); + long timeElapsed = 0L; + + long sleepDurationInMs = timeoutInMs < 1 ? + DEFAULT_SLEEP_DURATION_MS : Math.min(timeoutInMs, DEFAULT_SLEEP_DURATION_MS); + ApplicationStatus status; + + try { + while (timeoutInMs < 1 || timeElapsed <= timeoutInMs) { + status = getApplicationStatus(jobConfig); + if (status == SuccessfulFinish || status == UnsuccessfulFinish) { + LOG.info("Application finished with status {}", status); + break; + } + + Thread.sleep(sleepDurationInMs); + timeElapsed = System.currentTimeMillis() - startTimeInMs; + } + + if (timeElapsed > timeoutInMs) { + LOG.warn("Timed out waiting for application to finish."); + finished = false; + } + } catch (Exception e) { + LOG.error("Error waiting for application to finish", e); + throw new SamzaException(e); + } + + return finished; + } + private Config getConfigFromPrevRun() { CoordinatorStreamSystemConsumer consumer = new CoordinatorStreamSystemConsumer(config, new MetricsRegistryMap()); consumer.register(); diff --git a/samza-shell/src/main/bash/run-app.sh b/samza-shell/src/main/bash/run-app.sh index f8849c0fd2..3880e3c04e 100755 --- a/samza-shell/src/main/bash/run-app.sh +++ b/samza-shell/src/main/bash/run-app.sh @@ -27,4 +27,4 @@ mkdir -p $EXECUTION_PLAN_DIR [[ $JAVA_OPTS != *-Dlog4j.configuration* ]] && export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$(dirname $0)/log4j-console.xml" -exec $(dirname $0)/run-class.sh "$@" +exec $(dirname $0)/run-class.sh org.apache.samza.runtime.ApplicationRunnerMain "$@" 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 181971a222..f3c144caee 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 @@ -23,6 +23,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationInitializer; import org.apache.samza.config.Config; import org.apache.samza.operators.StreamGraph; import org.apache.samza.sql.translator.QueryTranslator; @@ -39,14 +40,14 @@ public class SamzaSqlApplication implements StreamApplication { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplication.class); @Override - public void init(StreamGraph streamGraph, Config config) { + public void init(StreamApplicationInitializer appBuilder, Config config) { try { SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(config); QueryTranslator queryTranslator = new QueryTranslator(sqlConfig); List queries = sqlConfig.getQueryInfo(); for (SamzaSqlQueryParser.QueryInfo query : queries) { LOG.info("Translating the query {} to samza stream graph", query.getSelectQuery()); - queryTranslator.translate(query, streamGraph); + queryTranslator.translate(query, appBuilder); } } catch (RuntimeException e) { LOG.error("SamzaSqlApplication threw exception.", e); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java similarity index 80% rename from samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java rename to samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java index 32f32d20dc..a0ab40a5f1 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java @@ -24,14 +24,17 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.Validate; -import org.apache.samza.application.ApplicationRunnable; -import org.apache.samza.application.StreamApplications; +import org.apache.samza.application.internal.ApplicationBuilder; +import org.apache.samza.application.internal.StreamApplicationBuilder; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.RemoteApplicationRunner; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.internal.ApplicationRunners; import org.apache.samza.sql.interfaces.SqlIOConfig; import org.apache.samza.sql.interfaces.SqlIOResolver; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; @@ -46,21 +49,23 @@ * This runner invokes the SamzaSqlConfig re-writer if it is invoked on a standalone mode (i.e. localRunner == true) * otherwise directly calls the RemoteApplicationRunner which automatically performs the config rewriting . */ -public class SamzaSqlApplicationRunner implements ApplicationRunnable { +public class SamzaSqlApplicationRuntime implements ApplicationRuntime { - private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplicationRunner.class); + private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplicationRuntime.class); private final Config sqlConfig; - private final ApplicationRunnable appRunnable; + private final ApplicationBuilder appSpec; + private final ApplicationRunner runner; private final Boolean localRunner; public static final String RUNNER_CONFIG = "app.runner.class"; public static final String CFG_FMT_SAMZA_STREAM_SYSTEM = "streams.%s.samza.system"; - public SamzaSqlApplicationRunner(Boolean localRunner, Config config) { + public SamzaSqlApplicationRuntime(Boolean localRunner, Config config) { this.localRunner = localRunner; sqlConfig = computeSamzaConfigs(localRunner, config); - appRunnable = StreamApplications.createRunnable(new SamzaSqlApplication(), sqlConfig); + appSpec = new StreamApplicationBuilder(new SamzaSqlApplication(), sqlConfig); + runner = ApplicationRunners.fromConfig(sqlConfig); } public static Config computeSamzaConfigs(Boolean localRunner, Config config) { @@ -104,37 +109,37 @@ public static Config computeSamzaConfigs(Boolean localRunner, Config config) { public void runAndWaitForFinish() { Validate.isTrue(localRunner, "This method can be called only in standalone mode."); - appRunnable.run(); - appRunnable.waitForFinish(); + start(); + waitForFinish(); } @Override - public void run() { - appRunnable.run(); + public void start() { + runner.run(appSpec); } @Override - public void kill() { - appRunnable.kill(); + public void stop() { + runner.kill(appSpec); } @Override public ApplicationStatus status() { - return appRunnable.status(); + return runner.status(appSpec); } @Override public void waitForFinish() { - appRunnable.waitForFinish(); + runner.waitForFinish(appSpec); } @Override public boolean waitForFinish(Duration timeout) { - return appRunnable.waitForFinish(timeout); + return runner.waitForFinish(appSpec, timeout); } @Override - public ApplicationRunnable withMetricsReporters(Map metricsReporters) { - return appRunnable.withMetricsReporters(metricsReporters); + public void addMetricsReporters(Map metricsReporters) { + runner.addMetricsReporters(metricsReporters); } } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java index c4cacbdb1d..8f4322c33e 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java @@ -25,7 +25,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; +import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; import org.apache.samza.sql.testutil.JsonUtil; import org.apache.samza.sql.testutil.SamzaSqlTestConfig; import org.apache.samza.sql.testutil.TestIOResolverFactory; @@ -45,8 +45,8 @@ public void testEndToEnd() throws Exception { String sql1 = "Insert into testDb.testTable.`$table` select id, name from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); - runner.runAndWaitForFinish(); + SamzaSqlApplicationRuntime appRunnable = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + appRunnable.runAndWaitForFinish(); Assert.assertEquals(numMessages, TestIOResolverFactory.TestTable.records.size()); } @@ -61,8 +61,8 @@ public void testEndToEndWithKey() throws Exception { String sql1 = "Insert into testDb.testTable.`$table` select id __key__, name from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); - runner.runAndWaitForFinish(); + SamzaSqlApplicationRuntime appRunnable = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + appRunnable.runAndWaitForFinish(); Assert.assertEquals(numMessages, TestIOResolverFactory.TestTable.records.size()); } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java similarity index 71% rename from samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java rename to samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java index b6dcac5f6b..32674ed8de 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java @@ -25,30 +25,28 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.RemoteApplicationRunner; -import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; import org.apache.samza.sql.testutil.SamzaSqlTestConfig; import org.junit.Assert; -import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; import org.junit.Test; -public class TestSamzaSqlApplicationRunner { +public class TestSamzaSqlApplicationRuntime { @Test public void testComputeSamzaConfigs() { Map configs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1"; configs.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql1); - configs.put(SamzaSqlApplicationRunner.RUNNER_CONFIG, SamzaSqlApplicationRunner.class.getName()); + configs.put(SamzaSqlApplicationRuntime.RUNNER_CONFIG, SamzaSqlApplicationRuntime.class.getName()); MapConfig samzaConfig = new MapConfig(configs); - Config newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(true, samzaConfig); - Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), LocalApplicationRunner.class.getName()); + Config newConfigs = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, samzaConfig); + Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRuntime.RUNNER_CONFIG), LocalApplicationRunner.class.getName()); // Check whether three new configs added. Assert.assertEquals(newConfigs.size(), configs.size() + 3); - newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(false, samzaConfig); - Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), RemoteApplicationRunner.class.getName()); + newConfigs = SamzaSqlApplicationRuntime.computeSamzaConfigs(false, samzaConfig); + Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRuntime.RUNNER_CONFIG), RemoteApplicationRunner.class.getName()); // Check whether three new configs added. Assert.assertEquals(newConfigs.size(), configs.size() + 3); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index ede79951a3..38ed0d6cfa 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -35,7 +35,7 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; +import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; import org.apache.samza.sql.testutil.SamzaSqlTestConfig; import org.junit.Assert; @@ -81,7 +81,7 @@ public void testTranslate() { Map config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.outputTopic select MyTest(id) from testavro.level1.level2.SIMPLE1 as s where s.id = 10"); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -133,7 +133,7 @@ public void testTranslateComplex() { // config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, // "Insert into testavro.foo2 select string_value, SUM(id) from testavro.COMPLEX1 " // + "GROUP BY TumbleWindow(CURRENT_TIME, INTERVAL '1' HOUR), string_value"); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -165,7 +165,7 @@ public void testTranslateSubQuery() { Map config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.outputTopic select Flatten(a), id from (select id, array_values a, string_value s from testavro.COMPLEX1)"); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -201,7 +201,7 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p" + " where p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -220,7 +220,7 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { + " full join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -239,7 +239,7 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { + " join testavro.PROFILE.`$table` as p2" + " on p1.id = p2.id"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -258,7 +258,7 @@ public void testTranslateStreamTableJoinWithThetaCondition() { + " join testavro.PROFILE.`$table` as p" + " on p.id <> pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -275,7 +275,7 @@ public void testTranslateStreamTableCrossJoin() { + " select p.name as profileName, pv.pageKey" + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -294,7 +294,7 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { + " join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId and p.name = 'John'"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -314,7 +314,7 @@ public void testTranslateStreamTableJoinWithSubQuery() { + " (select p.id from testavro.PROFILE.`$table` as p" + " where p.id = pv.profileId)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -333,7 +333,7 @@ public void testTranslateTableTableJoin() { + " join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -352,7 +352,7 @@ public void testTranslateStreamStreamJoin() { + " join testavro.PROFILE as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -371,7 +371,7 @@ public void testTranslateJoinWithIncorrectLeftJoin() { + " left join testavro.PROFILE as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -390,7 +390,7 @@ public void testTranslateJoinWithIncorrectRightJoin() { + " right join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -413,7 +413,7 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { + " join testavro.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -432,7 +432,7 @@ public void testTranslateStreamTableInnerJoinWithUdf() { + " join testavro.PROFILE.`$table` as p" + " on MyTest(p.id) = MyTest(pv.profileId)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -451,7 +451,7 @@ public void testTranslateStreamTableInnerJoin() { + " join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -504,7 +504,7 @@ public void testTranslateStreamTableLeftJoin() { + " left join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -558,7 +558,7 @@ public void testTranslateStreamTableRightJoin() { + " right join testavro.PAGEVIEW as pv" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -612,7 +612,7 @@ public void testTranslateGroupBy() { + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'" + " group by (pv.pageKey)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); @@ -636,7 +636,7 @@ public void testTranslateGroupByWithSumAggregator() { + " from testavro.PAGEVIEW as pv" + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'" + " group by (pv.pageKey)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); 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 4973761c54..73c7928d5c 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 @@ -19,17 +19,19 @@ package org.apache.samza.example; import java.time.Duration; +import java.util.HashMap; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamApplicationBuilder; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; -import java.util.HashMap; -import org.apache.samza.application.StreamApplications; -import org.apache.samza.serializers.JsonSerdeV2; -import org.apache.samza.serializers.KVSerde; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.serializers.JsonSerdeV2; +import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; @@ -37,24 +39,29 @@ /** * Example code to implement window-based counter */ -public class AppWithGlobalConfigExample { +public class AppWithGlobalConfigExample implements StreamApplication { // local execution mode public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new AppWithGlobalConfigExample(), config); + app.addMetricsReporters(new HashMap<>()); - StreamApplication app = StreamApplications.createStreamApp(config).withMetricsReporters(new HashMap<>()); - app.openInput("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) + app.start(); + app.waitForFinish(); + } + + @Override + public void init(StreamApplicationBuilder appBuilder, Config config) { + + appBuilder.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) .map(KV::getValue) .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(app.openOutput("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); - - app.run(); - app.waitForFinish(); + .sendTo(appBuilder.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); } 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 d6c238dece..295edb9ed3 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 @@ -19,13 +19,15 @@ package org.apache.samza.example; -import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamApplicationBuilder; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; -import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; @@ -33,23 +35,26 @@ /** * Example implementation of a task that splits its input into multiple output streams. */ -public class BroadcastExample { +public class BroadcastExample implements StreamApplication { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new BroadcastExample(), config); + app.start(); + app.waitForFinish(); + } + @Override + public void init(StreamApplicationBuilder appBuilder, Config config) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); - StreamApplication app = StreamApplications.createStreamApp(config); - MessageStream> inputStream = app.openInput("pageViewEventStream", pgeMsgSerde); + MessageStream> inputStream = appBuilder.getInputStream("pageViewEventStream", pgeMsgSerde); - inputStream.filter(m -> m.key.equals("key1")).sendTo(app.openOutput("outStream1", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key2")).sendTo(app.openOutput("outStream2", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key3")).sendTo(app.openOutput("outStream3", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key1")).sendTo(appBuilder.getOutputStream("outStream1", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key2")).sendTo(appBuilder.getOutputStream("outStream2", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key3")).sendTo(appBuilder.getOutputStream("outStream3", pgeMsgSerde)); - app.run(); - app.waitForFinish(); } 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 e47b3e6519..5409492dd0 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 @@ -18,14 +18,19 @@ */ package org.apache.samza.example; - -import org.apache.samza.application.StreamApplications; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FlatMapFunction; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -33,27 +38,28 @@ import org.apache.samza.task.TaskContext; import org.apache.samza.util.CommandLine; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.TimeUnit; - /** * Example code using {@link KeyValueStore} to implement event-time window */ -public class KeyValueStoreExample { +public class KeyValueStoreExample implements StreamApplication { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new KeyValueStoreExample(), config); + app.start(); + app.waitForFinish(); + } + + @Override + public void init(StreamGraph graph, Config config) { MessageStream pageViewEvents = - app.openInput("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = - app.openOutput("pageViewEventPerMember", + graph.getOutputStream("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class))); pageViewEvents @@ -63,9 +69,6 @@ public static void main(String[] args) throws Exception { .flatMap(new MyStatsCounter()) .map(stats -> KV.of(stats.memberId, stats)) .sendTo(pageViewEventPerMember); - - app.run(); - app.waitForFinish(); } static class MyStatsCounter implements FlatMapFunction { 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 150fe82e7d..d7e195778d 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 @@ -20,32 +20,37 @@ package org.apache.samza.example; import com.google.common.collect.ImmutableList; -import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.StreamGraph; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; -import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; -public class MergeExample { +public class MergeExample implements StreamApplication { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new MergeExample(), config); + app.start(); + app.waitForFinish(); + } + + @Override + public void init(StreamGraph graph, Config config) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); - MessageStream.mergeAll(ImmutableList.of(app.openInput("viewStream1", pgeMsgSerde), - app.openInput("viewStream2", pgeMsgSerde), app.openInput("viewStream3", pgeMsgSerde))) - .sendTo(app.openOutput("mergedStream", pgeMsgSerde)); - - app.run(); - app.waitForFinish(); + MessageStream.mergeAll(ImmutableList.of(graph.getInputStream("viewStream1", pgeMsgSerde), + graph.getInputStream("viewStream2", pgeMsgSerde), graph.getInputStream("viewStream3", pgeMsgSerde))) + .sendTo(graph.getOutputStream("mergedStream", pgeMsgSerde)); } 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 ac951c5542..f9b2b3a2cb 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 @@ -18,38 +18,44 @@ */ package org.apache.samza.example; +import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; -import java.time.Duration; - /** * Simple 2-way stream-to-stream join example */ -public class OrderShipmentJoinExample { +public class OrderShipmentJoinExample implements StreamApplication { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new OrderShipmentJoinExample(), config); + app.start(); + app.waitForFinish(); + } + @Override + public void init(StreamGraph graph, Config config) { MessageStream orders = - app.openInput("orders", new JsonSerdeV2<>(OrderRecord.class)); + graph.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class)); MessageStream shipments = - app.openInput("shipments", new JsonSerdeV2<>(ShipmentRecord.class)); + graph.getInputStream("shipments", new JsonSerdeV2<>(ShipmentRecord.class)); OutputStream> fulfilledOrders = - app.openOutput("fulfilledOrders", + graph.getOutputStream("fulfilledOrders", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(FulfilledOrderRecord.class))); orders @@ -58,9 +64,6 @@ public static void main(String[] args) throws Exception { Duration.ofMinutes(1), "join") .map(fulFilledOrder -> KV.of(fulFilledOrder.orderId, fulFilledOrder)) .sendTo(fulfilledOrders); - - app.run(); - app.waitForFinish(); } static class MyJoinFunction implements JoinFunction { 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 c511de5ab8..55d261ec39 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 @@ -20,17 +20,19 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FoldLeftFunction; import org.apache.samza.operators.functions.SupplierFunction; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -39,31 +41,34 @@ /** * Example code to implement window-based counter */ -public class PageViewCounterExample { +public class PageViewCounterExample implements StreamApplication { // local execution mode public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication appRuntime = StreamApplications.createStreamApp(config); + ApplicationRuntime appRuntime = ApplicationRuntimes.createStreamApp(new PageViewCounterExample(), config); + appRuntime.start(); + appRuntime.waitForFinish(); + } + @Override + public void init(StreamGraph graph, Config config) { MessageStream pageViewEvents = null; - pageViewEvents = appRuntime.openInput("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMemberStream = - appRuntime.openOutput("pageViewEventPerMemberStream", + graph.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); SupplierFunction initialValue = () -> 0; FoldLeftFunction foldLeftFn = (m, c) -> c + 1; pageViewEvents - .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, null, null) + .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, + null, null) .setEarlyTrigger(Triggers.repeat(Triggers.count(5))) .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow") .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane))) .sendTo(pageViewEventPerMemberStream); - - appRuntime.run(); - appRuntime.waitForFinish(); } class PageViewEvent { 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 8761aab39a..5f4bccbc24 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 @@ -18,37 +18,44 @@ */ package org.apache.samza.example; +import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; -import java.time.Duration; - /** * Example {@link StreamApplication} code to test the API methods with re-partition operator */ -public class RepartitionExample { +public class RepartitionExample implements StreamApplication { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new RepartitionExample(), config); + + app.start(); + app.waitForFinish(); + } + @Override + public void init(StreamGraph graph, Config config) { MessageStream pageViewEvents = - app.openInput("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); + graph.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = - app.openOutput("pageViewEventPerMember", + graph.getOutputStream("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class))); pageViewEvents @@ -58,9 +65,6 @@ public static void main(String[] args) throws Exception { "window") .map(windowPane -> KV.of(windowPane.getKey().getKey(), new MyStreamOutput(windowPane))) .sendTo(pageViewEventPerMember); - - app.run(); - app.waitForFinish(); } class PageViewEvent { 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 new file mode 100644 index 0000000000..65a56b120a --- /dev/null +++ b/samza-test/src/main/java/org/apache/samza/example/TaskApplicationExample.java @@ -0,0 +1,37 @@ +package org.apache.samza.example; + +import java.util.Collections; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.internal.TaskApplicationBuilder; +import org.apache.samza.config.Config; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.task.TaskFactory; +import org.apache.samza.task.TaskFactoryUtil; +import org.apache.samza.util.CommandLine; + + +/** + * Created by yipan on 7/16/18. + */ +public class TaskApplicationExample implements TaskApplication { + + public static void main(String[] args) { + CommandLine cmdLine = new CommandLine(); + Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); + ApplicationRuntime appRuntime = ApplicationRuntimes.createTaskApp(new TaskApplicationExample(), config); + appRuntime.start(); + appRuntime.waitForFinish(); + } + + @Override + public void init(TaskApplicationBuilder appBuilder, Config config) { + // add input and output streams + appBuilder.addInputStreams(Collections.singletonList("myinput")); + appBuilder.addOutputStreams(Collections.singletonList("myoutput")); + appBuilder.addTables(Collections.singletonList("mytable")); + // create the task factory based on configuration + appBuilder.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(config)); + } + +} 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 f1f2375f58..f25ac76ea7 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 @@ -20,15 +20,17 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FoldLeftFunction; import org.apache.samza.operators.functions.SupplierFunction; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.Windows; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.util.CommandLine; @@ -40,18 +42,24 @@ * Example implementation of a simple user-defined task w/ a window operator. * */ -public class WindowExample { +public class WindowExample implements StreamApplication { // local execution mode public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new WindowExample(), config); + app.start(); + app.waitForFinish(); + } + + @Override + public void init(StreamGraph graph, Config config) { SupplierFunction initialValue = () -> 0; FoldLeftFunction counter = (m, c) -> c == null ? 1 : c + 1; - MessageStream inputStream = app.openInput("inputStream", new JsonSerdeV2()); - OutputStream outputStream = app.openOutput("outputStream", new IntegerSerde()); + MessageStream inputStream = graph.getInputStream("inputStream", new JsonSerdeV2()); + OutputStream outputStream = graph.getOutputStream("outputStream", new IntegerSerde()); // 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 @@ -62,9 +70,6 @@ public static void main(String[] args) throws Exception { Triggers.timeSinceLastMessage(Duration.ofMinutes(1)))), "window") .map(WindowPane::getMessage) .sendTo(outputStream); - - app.run(); - app.waitForFinish(); } class PageViewEvent { diff --git a/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java b/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java index a282dbb297..beda1e2f1b 100644 --- a/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java +++ b/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java @@ -34,7 +34,7 @@ * MockSystemConsumer is a class that simulates a multi-threaded consumer that * uses BlockingEnvelopeMap. The primary use for this class is to do performance * testing. - * + * * This class works by starting up (threadCount) threads. Each thread adds * (messagesPerBatch) to the BlockingEnvelopeMap, then sleeps for * (brokerSleepMs). The sleep is important to simulate network latency when @@ -57,12 +57,12 @@ public class MockSystemConsumer extends BlockingEnvelopeMap { private List threads; /** - * + * * @param messagesPerBatch * The number of messages to add to the BlockingEnvelopeMap before * sleeping. * @param threadCount - * How many threads to run. + * How many threads to start. * @param brokerSleepMs * How long each thread should sleep between batch writes. */ diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index c1cfbb9769..e401e8d384 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -22,10 +22,9 @@ import joptsimple.OptionSet; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; -import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunnerMain; -import org.apache.samza.runtime.ApplicationRunnerOperation; -import org.apache.samza.runtime.ApplicationRunners; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,17 +47,14 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationRunner runner = ApplicationRunners.fromConfig(config); - StreamApplication app = (StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(); - - ApplicationRunnerOperation op = cmdLine.getOperation(options); + ApplicationRuntime appRuntime = ApplicationRuntimes.createStreamApp((StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(), config); try { - LOGGER.info("Launching stream application: {} to run.", app); - runner.run(app); - runner.waitForFinish(); + LOGGER.info("Launching stream application: {} to start.", appRuntime); + appRuntime.start(); + appRuntime.waitForFinish(); } catch (Exception e) { - LOGGER.error("Exception occurred when invoking: {} on application: {}.", op, app, e); + LOGGER.error("Exception occurred when running application: {}.", appRuntime, e); } } } diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java index 7253b29157..83fea32665 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java @@ -66,7 +66,7 @@ private void testStreamProcessor(String[] processorIds) { } - // run the processors in separate threads + // start the processors in separate threads Thread[] threads = new Thread[processorIds.length]; CountDownLatch[] stopLatches = new CountDownLatch[processorIds.length]; for (int i = 0; i < processorIds.length; i++) { diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java index 40eeaf07c3..22c2fc2e9e 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java @@ -79,7 +79,7 @@ private void testStreamProcessorWithSessionRestart(String[] processorIds) { jobCoordinators[i] = (ZkJobCoordinator) streamProcessors[i].getCurrentJobCoordinator(); } - // run the processors in separate threads + // start the processors in separate threads for (int i = 0; i < processorIds.length; i++) { threadStopLatches[i] = new CountDownLatch(1); // is used int stopProcessor threads[i] = runInThread(streamProcessors[i], threadStopLatches[i]); 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 fca641e608..9ed5aff0aa 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 @@ -25,7 +25,6 @@ import java.util.Map; import java.util.Random; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; 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 c6628c7b19..5c52f242c2 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 @@ -26,7 +26,6 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; -import org.apache.samza.application.StreamApplications; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; 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 44f05ea20c..4ffa212c40 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 @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; 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 724969e687..402cfca513 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 @@ -24,7 +24,6 @@ import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.application.StreamApplications; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; 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 f38db675db..7ec9c7d7cc 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 @@ -21,7 +21,6 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java index 177d2cb68a..16fcebdba3 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java @@ -74,8 +74,8 @@ * State persistence: {@link #tearDown()} clears all associated state (including topics and metadata) in Kafka and * Zookeeper. Hence, the state is not durable across invocations of {@link #tearDown()}
    * - * Execution model: {@link StreamApplication}s are run as their own {@link org.apache.samza.job.local.ThreadJob}s. - * Similarly, embedded Kafka servers and Zookeeper servers are run as their own threads. + * Execution model: {@link StreamApplication}s are start as their own {@link org.apache.samza.job.local.ThreadJob}s. + * Similarly, embedded Kafka servers and Zookeeper servers are start as their own threads. * {@link #produceMessage(String, int, String, String)} and {@link #consumeMessages(Collection, int)} are blocking calls. * *

    Usage Example

    @@ -242,14 +242,14 @@ protected RunApplicationContext runApplication(String userAppClass, configMap.put("task.checkpoint.factory", TestStreamManager.MockCheckpointManagerFactory.class.getName()); // This is to prevent tests from taking a long time to stop after they're done. The issue is that - // tearDown currently doesn't call runner.kill(app), and shuts down the Kafka and ZK servers immediately. + // tearDown currently doesn't call runner.stop(app), and shuts down the Kafka and ZK servers immediately. // The test process then exits, triggering the SamzaContainer shutdown hook, which in turn tries to flush any // store changelogs, which then get stuck trying to produce to the stopped Kafka server. - // Calling runner.kill doesn't work since RemoteApplicationRunner creates a new ThreadJob instance when - // kill is called. We can't use LocalApplicationRunner since ZkJobCoordinator doesn't currently create + // Calling runner.stop doesn't work since RemoteApplicationRunner creates a new ThreadJob instance when + // stop is called. We can't use LocalApplicationRunner since ZkJobCoordinator doesn't currently create // changelog streams. Hence we just force an unclean shutdown here to. This _should be_ OK // since the test method has already executed by the time the shutdown hook is called. The side effect is - // that buffered state (e.g. changelog contents) might not be flushed correctly after the test run. + // that buffered state (e.g. changelog contents) might not be flushed correctly after the test start. configMap.put("task.shutdown.ms", "1"); if (overriddenConfigs != null) { diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java index c89a5e21cb..b681d08ec1 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java @@ -72,7 +72,7 @@ public void testRepartitionJoinWindowAppWithoutDeletionOnCommit() throws Excepti initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - // run the application + // start the application RepartitionJoinWindowApp app = new RepartitionJoinWindowApp(); String appName = "UserPageAdClickCounter"; Map configs = new HashMap<>(); @@ -81,7 +81,7 @@ public void testRepartitionJoinWindowAppWithoutDeletionOnCommit() throws Excepti configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_2_PROP, inputTopicName2); configs.put(RepartitionJoinWindowApp.OUTPUT_TOPIC_NAME_PROP, outputTopicName); - // run the application + // start the application Thread runThread = runApplication(app.getClass().getName(), appName, new MapConfig(configs)).getRunThread(); // consume and validate result @@ -103,7 +103,7 @@ public void testRepartitionJoinWindowAppAndDeleteMessagesOnCommit() throws Excep initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - // run the application + // start the application RepartitionJoinWindowApp app = new RepartitionJoinWindowApp(); final String appName = "UserPageAdClickCounter2"; Map configs = new HashMap<>(); @@ -160,7 +160,7 @@ public void testBroadcastApp() throws InterruptedException { configs.put(BroadcastAssertApp.INPUT_TOPIC_NAME_PROP, inputTopicName1); initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - // run the application + // start the application Thread runThread = runApplication(BroadcastAssertApp.class.getName(), "BroadcastTest", new MapConfig(configs)).getRunThread(); runThread.interrupt(); diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java index e7c4aa07de..f3f7d8cffb 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java @@ -66,7 +66,7 @@ public void testRepartitionedSessionWindowCounter() throws Exception { configs.put(String.format("streams.%s.samza.msg.serde", INPUT_TOPIC), "string"); configs.put(String.format("streams.%s.samza.key.serde", INPUT_TOPIC), "string"); - // run the application + // start the application Thread runThread = runApplication(RepartitionWindowApp.class.getName(), APP_NAME, new MapConfig(configs)).getRunThread(); // consume and validate result 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 90e19909dc..97ca50148a 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 @@ -22,7 +22,6 @@ import java.time.Duration; import java.io.IOException; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; 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 e82e7062dc..8b483a5939 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 @@ -24,7 +24,6 @@ import java.io.Serializable; import java.util.concurrent.CountDownLatch; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplications; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.MessageStream; diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 405a549969..460615467d 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -347,7 +347,7 @@ public void shouldUpdateJobModelWhenNewProcessorJoiningGroupUsingAllSspToSingleT } }; - // This is the latch for the messages received by streamApp1. Since streamApp1 is run first, it gets one event + // This is the latch for the messages received by streamApp1. Since streamApp1 is start first, it gets one event // redelivered due to re-balancing done by Zk after the streamApp2 joins (See the callback above). CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2 + 1); @@ -390,9 +390,9 @@ public void shouldUpdateJobModelWhenNewProcessorJoiningGroupUsingAllSspToSingleT assertEquals(ApplicationStatus.Running, localApplicationRunner2.status(streamApp2)); - // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665 - // localApplicationRunner1.kill(streamApp1); - // localApplicationRunner2.kill(streamApp2); + // TODO: re-enable the following stop and waitForFinish calls after fixing SAMZA-1665 + // localApplicationRunner1.stop(streamApp1); + // localApplicationRunner2.stop(streamApp2); // localApplicationRunner1.waitForFinish(); // localApplicationRunner2.waitForFinish(); @@ -485,7 +485,7 @@ public void shouldFailWhenNewProcessorJoinsWithSameIdAsExistingProcessor() throw processedMessagesLatch1.await(); processedMessagesLatch2.await(); - // Create a stream app with same processor id as SP2 and run it. It should fail. + // Create a stream app with same processor id as SP2 and start it. It should fail. publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[2]); kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); StreamApplication diff --git a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java index d511a39b22..b758a2859a 100644 --- a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java +++ b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java @@ -34,7 +34,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.serializers.JsonSerdeV2Factory; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; +import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; import org.apache.samza.sql.system.TestAvroSystemFactory; import org.apache.samza.sql.testutil.JsonUtil; import org.apache.samza.sql.testutil.MyTestUdf; @@ -78,7 +78,7 @@ public void testEndToEnd() throws Exception { String sql1 = "Insert into testavro.outputTopic select id, CURRENT_TIME as long_value from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -99,7 +99,7 @@ public void testEndToEndFlatten() throws Exception { "Insert into testavro.outputTopic select Flatten(array_values) as string_value, id from testavro.COMPLEX1"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = new ArrayList<>(TestAvroSystemFactory.messages); @@ -121,7 +121,7 @@ public void testEndToEndSubQuery() throws Exception { "Insert into testavro.outputTopic select Flatten(a) as id from (select MyTestArray(id) a from testavro.SIMPLE1)"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = new ArrayList<>(TestAvroSystemFactory.messages); @@ -142,7 +142,7 @@ public void testEndToEndUdf() throws Exception { String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); LOG.info("output Messages " + TestAvroSystemFactory.messages); @@ -166,7 +166,7 @@ public void testRegexMatchUdfInWhereClause() throws Exception { String sql1 = "Insert into testavro.outputTopic select id from testavro.SIMPLE1 where RegexMatch('.*4', Name)"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); LOG.info("output Messages " + TestAvroSystemFactory.messages); @@ -190,7 +190,7 @@ public void testEndToEndStreamTableInnerJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -219,7 +219,7 @@ public void testEndToEndStreamTableInnerJoinWithNestedRecord() throws Exception List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -254,7 +254,7 @@ public void testEndToEndStreamTableInnerJoinWithFilter() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -286,7 +286,7 @@ public void testEndToEndStreamTableInnerJoinWithNullForeignKeys() throws Excepti List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -315,7 +315,7 @@ public void testEndToEndStreamTableLeftJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -344,7 +344,7 @@ public void testEndToEndStreamTableRightJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -375,7 +375,7 @@ public void testEndToEndStreamTableTableJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -405,7 +405,7 @@ public void testEndToEndStreamTableTableJoinWithCompositeKey() throws Exception List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -439,7 +439,7 @@ public void testEndToEndGroupBy() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); // Let's capture the list of windows/counts per key. diff --git a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java index bfd217c33f..9064e9fb08 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java @@ -42,7 +42,7 @@ import org.apache.samza.sql.impl.ConfigBasedUdfResolver; import org.apache.samza.sql.interfaces.SqlIOConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; +import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; import org.apache.samza.sql.testutil.JsonUtil; import org.apache.samza.sql.testutil.SqlFileParser; import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; @@ -107,8 +107,8 @@ public static void main(String[] args) { public static void executeSql(List sqlStmts) { Map staticConfigs = fetchSamzaSqlConfig(); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); - runner.runAndWaitForFinish(); + SamzaSqlApplicationRuntime appRunnable = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + appRunnable.runAndWaitForFinish(); } public static Map fetchSamzaSqlConfig() { From 30a4e5f0ae6ff1b99fc8a6d08e742d8bff06e0cb Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Mon, 23 Jul 2018 02:02:06 -0700 Subject: [PATCH 06/38] WIP: application runner refactor - proto-type for SEP-13 --- .../versioned/hello-samza-high-level-code.md | 16 +-- .../application/ApplicationInitializer.java | 8 -- .../samza/application/ApplicationSpec.java | 34 +++++ .../LifecycleAwareApplication.java | 12 ++ .../samza/application/StreamApplication.java | 71 +--------- .../StreamApplicationInitializer.java | 134 ------------------ .../application/StreamApplicationSpec.java | 32 +++++ .../samza/application/TaskApplication.java | 6 +- ...tializer.java => TaskApplicationSpec.java} | 8 +- .../samza/application/UserApplication.java | 17 --- ...plicationBuilder.java => AppSpecImpl.java} | 20 ++- .../internal/StreamAppSpecImpl.java | 79 +++++++++++ .../internal/StreamApplicationBuilder.java | 99 ------------- ...ationBuilder.java => TaskAppSpecImpl.java} | 35 ++--- .../apache/samza/operators/MessageStream.java | 7 +- .../apache/samza/operators/StreamGraph.java | 131 +++++++++++++---- .../operators/functions/ClosableFunction.java | 4 +- .../operators/functions/InitableFunction.java | 3 +- .../samza/runtime/ApplicationRuntime.java | 24 ++++ .../samza/runtime/ApplicationRuntimes.java | 28 ++-- .../runtime/internal/ApplicationRunner.java | 4 +- .../runtime/internal/ApplicationSpec.java | 16 --- .../internal/StreamApplicationSpec.java | 25 ---- .../runtime/internal/TaskApplicationSpec.java | 20 --- .../MockEventHubClientManagerFactory.java | 4 +- .../application/ApplicationClassUtils.java | 30 ++++ .../samza/operators/StreamGraphSpec.java | 68 +++------ .../samza/operators/spec/OperatorSpec.java | 3 +- .../samza/processor/StreamProcessor.java | 71 +++++----- .../runtime/AbstractApplicationRunner.java | 132 +++++++++-------- .../samza/runtime/ApplicationRunnerMain.java | 14 +- .../samza/runtime/LocalApplicationRunner.java | 60 ++++---- .../samza/runtime/LocalContainerRunner.java | 98 +++++++++---- .../runtime/RemoteApplicationRunner.java | 22 +-- .../apache/samza/task/StreamOperatorTask.java | 3 +- .../apache/samza/task/TaskFactoryUtil.java | 10 +- .../operators/impl/TestOperatorImplGraph.java | 6 +- .../runtime/TestApplicationRunnerMain.java | 57 +++++--- .../runtime/TestLocalApplicationRunner.java | 1 - .../samza/sql/runner/SamzaSqlApplication.java | 14 +- .../runner/SamzaSqlApplicationRuntime.java | 10 +- .../samza/sql/translator/QueryTranslator.java | 23 ++- .../samza/sql/translator/ScanTranslator.java | 2 +- .../sql/translator/TranslatorContext.java | 4 +- .../sql/translator/TestFilterTranslator.java | 2 +- .../sql/translator/TestProjectTranslator.java | 4 +- .../sql/translator/TestQueryTranslator.java | 25 ++-- .../example/AppWithGlobalConfigExample.java | 10 +- .../samza/example/BroadcastExample.java | 6 +- .../samza/example/KeyValueStoreExample.java | 6 +- .../apache/samza/example/MergeExample.java | 6 +- .../example/OrderShipmentJoinExample.java | 6 +- .../samza/example/PageViewCounterExample.java | 36 ++--- .../samza/example/RepartitionExample.java | 6 +- .../samza/example/TaskApplicationExample.java | 14 +- .../apache/samza/example/WindowExample.java | 11 +- .../LocalApplicationRunnerMain.java | 6 +- .../TestStandaloneIntegrationApplication.java | 7 +- .../EndOfStreamIntegrationTest.java | 34 +++-- .../WatermarkIntegrationTest.java | 30 ++-- .../StreamApplicationIntegrationTest.java | 1 - .../test/operator/BroadcastAssertApp.java | 12 +- .../operator/RepartitionJoinWindowApp.java | 30 ++-- .../test/operator/RepartitionWindowApp.java | 1 - .../samza/test/operator/SessionWindowApp.java | 1 - ...reamApplicationIntegrationTestHarness.java | 1 - .../test/operator/TumblingWindowApp.java | 1 - .../test/processor/TestStreamApplication.java | 1 - .../TestZkLocalApplicationRunner.java | 1 - .../samza/test/table/TestLocalTable.java | 1 - .../samza/test/table/TestRemoteTable.java | 5 +- .../apache/samza/test/timer/TestTimerApp.java | 7 +- .../SystemConsumerWithSamzaBench.java | 1 - 73 files changed, 841 insertions(+), 896 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java delete mode 100644 samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java create mode 100644 samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java rename samza-api/src/main/java/org/apache/samza/application/{TaskApplicationInitializer.java => TaskApplicationSpec.java} (55%) delete mode 100644 samza-api/src/main/java/org/apache/samza/application/UserApplication.java rename samza-api/src/main/java/org/apache/samza/application/internal/{ApplicationBuilder.java => AppSpecImpl.java} (70%) create mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java delete mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java rename samza-api/src/main/java/org/apache/samza/application/internal/{TaskApplicationBuilder.java => TaskAppSpecImpl.java} (54%) delete mode 100644 samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java delete mode 100644 samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java delete mode 100644 samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java create mode 100644 samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md index 2f6a4a6094..4fb2d8a6c1 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md @@ -176,7 +176,7 @@ package samza.examples.wikipedia.application; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraph; +import org.apache.samza.application.StreamApplicationSpec; public class MyWikipediaApplication implements StreamApplication{ @Override @@ -188,12 +188,12 @@ public class MyWikipediaApplication implements StreamApplication{ Be sure to include the Apache header. The project will not compile without it. -The [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method is where the application logic is defined. The [Config](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/config/Config.html) argument is the runtime configuration loaded from the properties file we defined earlier. The [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) argument provides methods to declare input streams. You can then invoke a number of flexible operations on those streams. The result of each operation is another stream, so you can keep chaining more operations or direct the result to an output stream. +The [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method is where the application logic is defined. The [Config](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/config/Config.html) argument is the runtime configuration loaded from the properties file we defined earlier. The [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) argument provides methods to declare input streams. You can then invoke a number of flexible operations on those streams. The result of each operation is another stream, so you can keep chaining more operations or direct the result to an output stream. Next, we will declare the input streams for the Wikipedia application. #### Inputs -The Wikipedia application consumes events from three channels. Let's declare each of those channels as an input streams via the [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) in the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method. +The Wikipedia application consumes events from three channels. Let's declare each of those channels as an input streams via the [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) in the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method. {% highlight java %} MessageStream wikipediaEvents = streamGraph.getInputStream("en-wikipedia", new NoOpSerde<>()); MessageStream wiktionaryEvents = streamGraph.getInputStream("en-wiktionary", new NoOpSerde<>()); @@ -208,7 +208,7 @@ Note the streams are all MessageStreams of type WikipediaFeedEvent. [MessageStre #### Merge We'd like to use the same processing logic for all three input streams, so we will use the [mergeAll](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#mergeAll-java.util.Collection-) operator to merge them together. Note: this is not the same as a [join](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#join-org.apache.samza.operators.MessageStream-org.apache.samza.operators.functions.JoinFunction-java.time.Duration-) because we are not associating events by key. We are simply combining three streams into one, like a union. -Add the following snippet to the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method. It merges all the input streams into a new one called _allWikipediaEvents_ +Add the following snippet to the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method. It merges all the input streams into a new one called _allWikipediaEvents_ {% highlight java %} MessageStream allWikipediaEvents = MessageStream.mergeAll(ImmutableList.of(wikipediaEvents, wiktionaryEvents, wikiNewsEvents)); {% endhighlight %} @@ -218,7 +218,7 @@ Note there is a [merge](/learn/documentation/{{site.version}}/api/javadocs/org/a #### Parse The next step is to parse the events and extract some information. We will use the pre-existing `WikipediaParser.parseEvent()' method to do this. The parser extracts some flags we want to monitor as well as some metadata about the event. Inspect the method signature. The input is a WikipediaFeedEvents and the output is a Map. These types will be reflected in the types of the streams before and after the operation. -In the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method, invoke the [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation on `allWikipediaEvents`, passing the `WikipediaParser::parseEvent` method reference as follows: +In the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method, invoke the [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation on `allWikipediaEvents`, passing the `WikipediaParser::parseEvent` method reference as follows: {% highlight java %} allWikipediaEvents.map(WikipediaParser::parseEvent); @@ -227,7 +227,7 @@ allWikipediaEvents.map(WikipediaParser::parseEvent); #### Window Now that we have the relevant information extracted, let's perform some aggregations over a 10-second [window](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/windows/Window.html). -First, we need a container class for statistics we want to track. Add the following static class after the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method. +First, we need a container class for statistics we want to track. Add the following static class after the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method. {% highlight java %} private static class WikipediaStats { int edits = 0; @@ -262,7 +262,7 @@ private class WikipediaStatsAggregator implements FoldLeftFunction(WikipediaStats.class))); @@ -301,7 +301,7 @@ Paste the following after the aggregator class: } {% endhighlight %} -Now, we can invoke the method by adding another [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation to the chain in [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-). The operator chain should now look like this: +Now, we can invoke the method by adding another [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation to the chain in [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-). The operator chain should now look like this: {% highlight java %} allWikipediaEvents.map(WikipediaParser::parseEvent) .window(Windows.tumblingWindow(Duration.ofSeconds(10), WikipediaStats::new, new WikipediaStatsAggregator())) diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java deleted file mode 100644 index ab4709a061..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationInitializer.java +++ /dev/null @@ -1,8 +0,0 @@ -package org.apache.samza.application; - -/** - * Created by yipan on 7/18/18. - */ -public interface ApplicationInitializer { - -} diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java new file mode 100644 index 0000000000..0131caf3fd --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java @@ -0,0 +1,34 @@ +package org.apache.samza.application; + +import org.apache.samza.config.Config; +import org.apache.samza.operators.ContextManager; + + +/** + * Created by yipan on 7/19/18. + */ +public interface ApplicationSpec { + /** + * Get the global unique application ID in the runtime process + * @return globally unique application ID + */ + String getGlobalAppId(); + + /** + * Get the user defined {@link Config} + * @return config object + */ + Config getConfig(); + + /** + * Sets the {@link ContextManager} for this {@link StreamApplicationSpec}. + *

    + * The provided {@link ContextManager} can be used to setup shared context between the operator functions + * within a task instance + * + * @param contextManager the {@link ContextManager} to use for the {@link StreamApplicationSpec} + * @return the {@link StreamApplicationSpec} with {@code contextManager} set as its {@link ContextManager} + */ + ApplicationSpec withContextManager(ContextManager contextManager); + +} diff --git a/samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java b/samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java new file mode 100644 index 0000000000..2a8ac40acb --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java @@ -0,0 +1,12 @@ +package org.apache.samza.application; + +/** + * Created by yipan on 7/11/18. + */ +public interface LifecycleAwareApplication { + void describe(S appSpec); + default void beforeStart() {} + default void afterStart() {} + default void beforeStop() {} + default void afterStop() {} +} 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 d788444da9..b1f63d0849 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 @@ -1,74 +1,7 @@ -/* - * 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.application; -import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.internal.StreamApplicationBuilder; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.functions.InitableFunction; -import org.apache.samza.runtime.internal.StreamApplicationSpec; -import org.apache.samza.task.StreamTask; - /** - * Describes and initializes the transforms for processing message streams and generating results. - *

    - * The following example removes page views older than 1 hour from the input stream: - *

    {@code
    - * public class PageViewCounter implements StreamApplication {
    - *   public void init(StreamGraph graph, Config config) {
    - *     MessageStream pageViewEvents =
    - *       graph.getInputStream("pageViewEvents", (k, m) -> (PageViewEvent) m);
    - *     OutputStream recentPageViewEvents =
    - *       graph.getOutputStream("recentPageViewEvents", m -> m.memberId, m -> m);
    - *
    - *     pageViewEvents
    - *       .filter(m -> m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis())
    - *       .sendTo(filteredPageViewEvents);
    - *   }
    - * }
    - * }
    - *

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

    {@code
    - *   public static void main(String[] args) {
    - *     CommandLine cmdLine = new CommandLine();
    - *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
    - *     PageViewCounter app = new PageViewCounter();
    - *     LocalApplicationRunner runner = new LocalApplicationRunner(config);
    - *     runner.start(app);
    - *     runner.waitForFinish();
    - *   }
    - * }
    - * - *

    - * Implementation Notes: Currently StreamApplications are wrapped in a {@link StreamTask} during execution. - * A new StreamApplication instance will be created and initialized with a user-defined {@link StreamGraph} - * when planning the execution. The {@link StreamGraph} and the functions implemented for transforms are required to - * be serializable. The execution planner will generate a serialized DAG which will be deserialized in each {@link StreamTask} - * instance used for processing incoming messages. Execution is synchronous and thread-safe within each {@link StreamTask}. - * - *

    - * 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 StreamTask} instance is closed. - * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. + * Created by yipan on 7/20/18. */ -@InterfaceStability.Evolving -public interface StreamApplication extends UserApplication { +public interface StreamApplication extends LifecycleAwareApplication { } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java deleted file mode 100644 index c54e3e81d9..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationInitializer.java +++ /dev/null @@ -1,134 +0,0 @@ -package org.apache.samza.application; - -import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.serializers.Serde; -import org.apache.samza.table.Table; - - -/** - * Created by yipan on 7/18/18. - */ -public interface StreamApplicationInitializer extends ApplicationInitializer { - /** - * Sets the default {@link Serde} to use for (de)serializing messages. - *

    . - * If the default serde is set, it must be set before creating any input or output streams. - *

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

    - * Providing an incompatible message type for the input/output streams that use the default serde will result in - * {@link ClassCastException}s at runtime. - * - * @param serde the default message {@link Serde} to use - */ - void setDefaultSerde(Serde serde); - - /** - * Gets the input {@link MessageStream} corresponding to the {@code streamId}. - *

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

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param serde the {@link Serde} to use for deserializing incoming messages - * @param the type of messages in the input {@link MessageStream} - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - MessageStream getInputStream(String streamId, Serde serde); - - /** - * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via - * {@link #setDefaultSerde(Serde)} for deserializing input messages. - *

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param the type of message in the input {@link MessageStream} - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - MessageStream getInputStream(String streamId); - - /** - * Gets the {@link OutputStream} corresponding to the {@code streamId}. - *

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

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

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param serde the {@link Serde} to use for serializing outgoing messages - * @param the type of messages in the {@link OutputStream} - * @return the output {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - OutputStream getOutputStream(String streamId, Serde serde); - - /** - * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via - * {@link #setDefaultSerde(Serde)} for serializing output messages. - *

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param the type of messages in the {@link OutputStream} - * @return the output {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - OutputStream getOutputStream(String streamId); - - /** - * Gets the {@link Table} corresponding to the {@link TableDescriptor}. - *

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

    - * The provided {@link ContextManager} can be used to setup shared context between the operator functions - * within a task instance - * - * @param contextManager the {@link ContextManager} to use for the {@link StreamGraph} - * @return the {@link StreamGraph} with {@code contextManager} set as its {@link ContextManager} - */ - StreamApplicationInitializer withContextManager(ContextManager contextManager); - -} diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java new file mode 100644 index 0000000000..2f5a5c1127 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.application; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; + + +/** + * Provides access to {@link MessageStream}s and {@link OutputStream}s used to describe application logic. + */ +@InterfaceStability.Evolving +public interface StreamApplicationSpec extends ApplicationSpec, StreamGraph { +} 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 d1d2f6926d..fd6b79c207 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 @@ -1,11 +1,7 @@ package org.apache.samza.application; -import org.apache.samza.application.internal.TaskApplicationBuilder; -import org.apache.samza.runtime.internal.TaskApplicationSpec; - - /** * Created by yipan on 7/11/18. */ -public interface TaskApplication extends UserApplication { +public interface TaskApplication extends LifecycleAwareApplication { } diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java similarity index 55% rename from samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java rename to samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java index 75164c797d..d7dba831e6 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationInitializer.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java @@ -1,19 +1,21 @@ package org.apache.samza.application; import java.util.List; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.task.TaskFactory; /** - * Created by yipan on 7/18/18. + * Created by yipan on 7/19/18. */ -public interface TaskApplicationInitializer extends ApplicationInitializer { +public interface TaskApplicationSpec extends ApplicationSpec { + void setTaskFactory(TaskFactory factory); void addInputStreams(List inputStreams); void addOutputStreams(List outputStreams); - void addTables(List tables); + void addTables(List tables); } diff --git a/samza-api/src/main/java/org/apache/samza/application/UserApplication.java b/samza-api/src/main/java/org/apache/samza/application/UserApplication.java deleted file mode 100644 index 29e9cebc96..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/UserApplication.java +++ /dev/null @@ -1,17 +0,0 @@ -package org.apache.samza.application; - -import org.apache.samza.application.internal.ApplicationBuilder; -import org.apache.samza.config.Config; -import org.apache.samza.runtime.internal.ApplicationSpec; - - -/** - * Created by yipan on 7/11/18. - */ -public interface UserApplication, S extends ApplicationSpec> { - void init(B appBuilder, Config config); - default void beforeStart(S appSpec) {} - default void afterStart(S appSpec) {} - default void beforeStop(S appSpec) {} - default void afterStop(S appSpec) {} -} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationBuilder.java b/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java similarity index 70% rename from samza-api/src/main/java/org/apache/samza/application/internal/ApplicationBuilder.java rename to samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java index 602054761e..b1ebc7c59a 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/ApplicationBuilder.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java @@ -1,20 +1,21 @@ package org.apache.samza.application.internal; -import org.apache.samza.application.ApplicationInitializer; -import org.apache.samza.application.UserApplication; +import org.apache.samza.application.ApplicationSpec; +import org.apache.samza.application.LifecycleAwareApplication; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; -import org.apache.samza.runtime.internal.ApplicationSpec; +import org.apache.samza.operators.ContextManager; /** * Created by yipan on 7/10/18. */ -public abstract class ApplicationBuilder implements ApplicationInitializer, ApplicationSpec { +abstract class AppSpecImpl implements ApplicationSpec { final Config config; final T userApp; + ContextManager contextManager; - protected ApplicationBuilder(T userApp, Config config) { + protected AppSpecImpl(T userApp, Config config) { this.config = config; this.userApp = userApp; } @@ -65,9 +66,16 @@ public Config getConfig() { return config; } - @Override public T getUserApp() { return userApp; } + public ContextManager getContextManager() { + return contextManager; + } + + protected void setContextManager(ContextManager contextManager) { + this.contextManager = contextManager; + } + } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java new file mode 100644 index 0000000000..e41e5a5271 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java @@ -0,0 +1,79 @@ +package org.apache.samza.application.internal; + +import java.lang.reflect.Constructor; +import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.config.Config; +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; + + +/** + * Created by yipan on 7/10/18. + */ +public class StreamAppSpecImpl extends AppSpecImpl implements StreamApplicationSpec { + final StreamGraph graph; + + public StreamAppSpecImpl(StreamApplication userApp, Config config) { + super(userApp, config); + this.graph = createDefaultGraph(config); + userApp.describe(this); + } + + private StreamApplicationSpec createDefaultGraph(Config config) { + try { + Constructor constructor = Class.forName("org.apache.samza.operators.StreamGraphSpec").getConstructor(Config.class); // *sigh* + return (StreamApplicationSpec) constructor.newInstance(config); + } catch (Exception e) { + throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); + } + } + + @Override + public void setDefaultSerde(Serde serde) { + this.graph.setDefaultSerde(serde); + } + + @Override + public MessageStream getInputStream(String streamId, Serde serde) { + return this.graph.getInputStream(streamId, serde); + } + + @Override + public MessageStream getInputStream(String streamId) { + return this.graph.getInputStream(streamId); + } + + @Override + public OutputStream getOutputStream(String streamId, Serde serde) { + return this.graph.getOutputStream(streamId, serde); + } + + @Override + public OutputStream getOutputStream(String streamId) { + return this.graph.getOutputStream(streamId); + } + + @Override + public Table> getTable(TableDescriptor tableDesc) { + return this.graph.getTable(tableDesc); + } + + @Override + public StreamApplicationSpec withContextManager(ContextManager contextManager) { + super.setContextManager(contextManager); + return this; + } + + public StreamGraph getGraph() { + return graph; + } + +} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java deleted file mode 100644 index 4d6538224c..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/internal/StreamApplicationBuilder.java +++ /dev/null @@ -1,99 +0,0 @@ -package org.apache.samza.application.internal; - -import java.lang.reflect.Constructor; -import java.util.Collection; -import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationInitializer; -import org.apache.samza.config.Config; -import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.runtime.internal.StreamApplicationSpec; -import org.apache.samza.serializers.Serde; -import org.apache.samza.table.Table; - - -/** - * Created by yipan on 7/10/18. - */ -public class StreamApplicationBuilder extends ApplicationBuilder implements StreamApplicationInitializer, StreamApplicationSpec { - final StreamApplicationBuilder graphBuilder; - - public StreamApplicationBuilder(StreamApplication userApp, Config config) { - super(userApp, config); - this.graphBuilder = createDefault(config); - userApp.init(this, config); - } - - private StreamApplicationBuilder createDefault(Config config) { - try { - Constructor constructor = Class.forName("org.apache.samza.operators.StreamGraphSpec").getConstructor(Config.class); // *sigh* - return (StreamApplicationBuilder) constructor.newInstance(config); - } catch (Exception e) { - throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); - } - } - - @Override - public void setDefaultSerde(Serde serde) { - this.graphBuilder.setDefaultSerde(serde); - } - - @Override - public MessageStream getInputStream(String streamId, Serde serde) { - return this.graphBuilder.getInputStream(streamId, serde); - } - - @Override - public MessageStream getInputStream(String streamId) { - return this.graphBuilder.getInputStream(streamId); - } - - @Override - public OutputStream getOutputStream(String streamId, Serde serde) { - return this.graphBuilder.getOutputStream(streamId, serde); - } - - @Override - public OutputStream getOutputStream(String streamId) { - return this.graphBuilder.getOutputStream(streamId); - } - - @Override - public Table> getTable(TableDescriptor tableDesc) { - return this.graphBuilder.getTable(tableDesc); - } - - @Override - public StreamApplicationInitializer withContextManager(ContextManager contextManager) { - return this.graphBuilder.withContextManager(contextManager); - } - - @Override - public ContextManager getContextManager() { - return this.graphBuilder.getContextManager(); - } - - @Override - public Collection getInputStreams() { - return this.graphBuilder.getInputStreams(); - } - - @Override - public Collection getOutputStreams() { - return this.graphBuilder.getOutputStreams(); - } - - @Override - public Collection getBroadcastStreams() { - return this.graphBuilder.getBroadcastStreams(); - } - - @Override - public Collection getTables() { - return this.graphBuilder.getTables(); - } -} diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java similarity index 54% rename from samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java rename to samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java index b9a433edc9..0e5c0cd892 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/TaskApplicationBuilder.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java @@ -3,25 +3,26 @@ import java.util.ArrayList; import java.util.List; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TaskApplicationInitializer; +import org.apache.samza.application.TaskApplicationSpec; import org.apache.samza.config.Config; -import org.apache.samza.runtime.internal.TaskApplicationSpec; +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.task.TaskFactory; /** * Created by yipan on 7/10/18. */ -public class TaskApplicationBuilder extends ApplicationBuilder implements TaskApplicationInitializer, TaskApplicationSpec { +public class TaskAppSpecImpl extends AppSpecImpl implements TaskApplicationSpec { TaskFactory taskFactory; final List inputStreams = new ArrayList<>(); final List outputStreams = new ArrayList<>(); - final List tables = new ArrayList<>(); + final List tables = new ArrayList<>(); - public TaskApplicationBuilder(TaskApplication userApp, Config config) { + public TaskAppSpecImpl(TaskApplication userApp, Config config) { super(userApp, config); - userApp.init(this, config); + userApp.describe(this); } @Override @@ -40,27 +41,17 @@ public void addOutputStreams(List outputStreams) { } @Override - public void addTables(List tables) { + public void addTables(List tables) { this.tables.addAll(tables); } @Override - public TaskFactory getTaskFactory() { - return this.taskFactory; + public TaskApplicationSpec withContextManager(ContextManager contextManager) { + super.setContextManager(contextManager); + return this; } - @Override - public List getInputStreams() { - return this.inputStreams; - } - - @Override - public List getOutputStreams() { - return this.outputStreams; - } - - @Override - public List getTables() { - return this.tables; + public TaskFactory getTaskFactory() { + return taskFactory; } } diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index 7797f9a255..bd25005a44 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -23,6 +23,7 @@ import java.util.Collection; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; @@ -39,7 +40,7 @@ /** * A stream of messages that can be transformed into another {@link MessageStream}. *

    - * A {@link MessageStream} corresponding to an input stream can be obtained using {@link StreamGraph#getInputStream}. + * A {@link MessageStream} corresponding to an input stream can be obtained using {@link StreamApplicationSpec#getInputStream}. * * @param the type of messages in this stream */ @@ -217,7 +218,7 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, - * uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default + * uses the default serde provided via {@link StreamApplicationSpec#setDefaultSerde}, which must be a KVSerde. If the default * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided * before calling this method, a {@code KVSerde} is used. *

    @@ -251,7 +252,7 @@ MessageStream> partitionBy(MapFunction k /** * Same as calling {@link #partitionBy(MapFunction, MapFunction, KVSerde, String)} with a null KVSerde. *

    - * Uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default + * Uses the default serde provided via {@link StreamApplicationSpec#setDefaultSerde}, which must be a KVSerde. If the default * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided * before calling this method, a {@code KVSerde} is used. * diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java index 22e13d6f5d..3bd5dbd424 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java @@ -1,40 +1,117 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ package org.apache.samza.operators; -import java.util.Collection; -import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; /** - * Provides access to {@link MessageStream}s and {@link OutputStream}s used to describe application logic. + * Created by yipan on 7/20/18. */ -@InterfaceStability.Evolving public interface StreamGraph { - ContextManager getContextManager(); - Collection getInputStreams(); + /** + * Sets the default {@link Serde} to use for (de)serializing messages. + *

    . + * If the default serde is set, it must be set before creating any input or output streams. + *

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

    + * Providing an incompatible message type for the input/output streams that use the default serde will result in + * {@link ClassCastException}s at runtime. + * + * @param serde the default message {@link Serde} to use + */ + void setDefaultSerde(Serde serde); + + /** + * Gets the input {@link MessageStream} corresponding to the {@code streamId}. + *

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

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param serde the {@link Serde} to use for deserializing incoming messages + * @param the type of messages in the input {@link MessageStream} + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + MessageStream getInputStream(String streamId, Serde serde); - Collection getOutputStreams(); + /** + * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via + * {@link #setDefaultSerde(Serde)} for deserializing input messages. + *

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param the type of message in the input {@link MessageStream} + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + MessageStream getInputStream(String streamId); - Collection getBroadcastStreams(); + /** + * Gets the {@link OutputStream} corresponding to the {@code streamId}. + *

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

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

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param serde the {@link Serde} to use for serializing outgoing messages + * @param the type of messages in the {@link OutputStream} + * @return the output {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + OutputStream getOutputStream(String streamId, Serde serde); - Collection getTables(); + /** + * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via + * {@link #setDefaultSerde(Serde)} for serializing output messages. + *

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param the type of messages in the {@link OutputStream} + * @return the output {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + OutputStream getOutputStream(String streamId); + /** + * Gets the {@link Table} corresponding to the {@link TableDescriptor}. + *

    + * Multiple invocations of this method with the same {@link TableDescriptor} will throw an + * {@link IllegalStateException}. + * + * @param tableDesc the {@link TableDescriptor} + * @param the type of the key + * @param the type of the value + * @return the {@link Table} corresponding to the {@code tableDesc} + * @throws IllegalStateException when invoked multiple times with the same {@link TableDescriptor} + */ + Table> getTable(TableDescriptor tableDesc); } diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java index faf9fc5a35..7de53f8431 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java @@ -20,6 +20,8 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationSpec; + /** * A function that can be closed after its execution. @@ -27,7 +29,7 @@ *

    Implement {@link #close()} to free resources used during the execution of the function, clean up state etc. * *

    Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the - * {@link org.apache.samza.operators.StreamGraph}. For any two operators A and B in the graph, if operator B consumes results + * {@link StreamApplicationSpec}. For any two operators A and B in the graph, if operator B consumes results * from operator A, then operator B is guaranteed to be closed before operator A. * */ diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index 6651819780..c56ecd44cc 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -20,6 +20,7 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.task.TaskContext; @@ -27,7 +28,7 @@ * A function that can be initialized before execution. * *

    Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the - * {@link org.apache.samza.operators.StreamGraph}. For any two operators A and B in the graph, if operator B consumes results + * {@link StreamApplicationSpec}. For any two operators A and B in the graph, if operator B consumes results * from operator A, then operator A is guaranteed to be initialized before operator B. * */ diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java index dd62271d73..e0483ec09a 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java @@ -10,10 +10,34 @@ * Created by yipan on 7/11/18. */ public interface ApplicationRuntime { + /** + * Start an application + */ void start(); + + /** + * Stop an application + */ void stop(); + + /** + * Get the {@link ApplicationStatus} of an application + * @return the runtime status of the application + */ ApplicationStatus status(); + + /** + * Wait the application to complete. + * This method will block until the application completes. + */ void waitForFinish(); + + /** + * Wait the application to complete with a {@code timeout} + * + * @param timeout the time to block to wait for the application to complete + * @return true if the application completes within timeout; false otherwise + */ boolean waitForFinish(Duration timeout); /** diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java index 54f7b21cf3..cb02ec0535 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java @@ -2,40 +2,44 @@ import java.time.Duration; import java.util.Map; +import org.apache.samza.application.ApplicationSpec; +import org.apache.samza.application.LifecycleAwareApplication; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamApplicationBuilder; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.TaskApplicationBuilder; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.runtime.internal.ApplicationRunners; -import org.apache.samza.runtime.internal.ApplicationSpec; /** * Created by yipan on 7/11/18. */ public class ApplicationRuntimes { - private ApplicationRuntimes() { - } + private ApplicationRuntimes() { - public static final ApplicationRuntime createStreamApp(StreamApplication userApp, Config config) { - return new RuntimeAppImpl(new StreamApplicationBuilder(userApp, config)); } - public static final ApplicationRuntime createTaskApp(TaskApplication userApp, Config config) { - return new RuntimeAppImpl(new TaskApplicationBuilder(userApp, config)); + public static final ApplicationRuntime getApplicationRuntime(LifecycleAwareApplication userApp, Config config) { + if (userApp instanceof StreamApplication) { + return new AppRuntimeImpl(new StreamAppSpecImpl((StreamApplication) userApp, config)); + } + if (userApp instanceof TaskApplication) { + return new AppRuntimeImpl(new TaskAppSpecImpl((TaskApplication) userApp, config)); + } + throw new IllegalArgumentException(String.format("User application instance has to be either StreamApplicationFactory or TaskApplicationFactory. " + + "Invalid userApp class %s.", userApp.getClass().getName())); } - - private static class RuntimeAppImpl implements ApplicationRuntime { + private static class AppRuntimeImpl implements ApplicationRuntime { private final ApplicationSpec appSpec; private final ApplicationRunner runner; - RuntimeAppImpl(ApplicationSpec appSpec) { + AppRuntimeImpl(ApplicationSpec appSpec) { this.appSpec = appSpec; this.runner = ApplicationRunners.fromConfig(appSpec.getConfig()); } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java index 1032ff302b..5e290bda73 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java @@ -21,13 +21,13 @@ import java.time.Duration; import java.util.Map; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.ApplicationSpec; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; /** - * The primary means of managing execution of the {@link StreamApplication} at runtime. + * The primary means of managing execution of user applications defined in {@link ApplicationSpec} at runtime. */ @InterfaceStability.Evolving public interface ApplicationRunner { diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java deleted file mode 100644 index bc265f95a2..0000000000 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationSpec.java +++ /dev/null @@ -1,16 +0,0 @@ -package org.apache.samza.runtime.internal; - -import org.apache.samza.application.UserApplication; -import org.apache.samza.config.Config; - - -/** - * Created by yipan on 7/17/18. - */ -public interface ApplicationSpec { - Config getConfig(); - - T getUserApp(); - - String getGlobalAppId(); -} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java deleted file mode 100644 index 7b5dfa862a..0000000000 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/StreamApplicationSpec.java +++ /dev/null @@ -1,25 +0,0 @@ -package org.apache.samza.runtime.internal; - -import java.util.Collection; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.operators.ContextManager; - - -/** - * Created by yipan on 7/17/18. - */ -public interface StreamApplicationSpec extends ApplicationSpec { - - ContextManager getContextManager(); - - Collection getInputStreams(); - - Collection getOutputStreams(); - - Collection getBroadcastStreams(); - - Collection getTables(); - - - -} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java deleted file mode 100644 index 7d47f70b11..0000000000 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/TaskApplicationSpec.java +++ /dev/null @@ -1,20 +0,0 @@ -package org.apache.samza.runtime.internal; - -import java.util.List; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.task.TaskFactory; - - -/** - * Created by yipan on 7/17/18. - */ -public interface TaskApplicationSpec extends ApplicationSpec { - TaskFactory getTaskFactory(); - - List getInputStreams(); - - List getOutputStreams(); - - List getTables(); - -} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java index 6ee9bcfd15..c5056ab063 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java @@ -169,7 +169,7 @@ public void init() { @Override public EventHubClient getEventHubClient() { if (!initiated) { - Assert.fail("Should have called init() on EventHubClient before getEventHubClient()"); + Assert.fail("Should have called bootstrap() on EventHubClient before getEventHubClient()"); } return mockEventHubClient; } @@ -177,7 +177,7 @@ public EventHubClient getEventHubClient() { @Override public void close(long timeoutMS) { if (!initiated) { - Assert.fail("Should have called init() on EventHubClient before close()"); + Assert.fail("Should have called bootstrap() on EventHubClient before close()"); } initiated = false; } diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java new file mode 100644 index 0000000000..a41dd53b73 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java @@ -0,0 +1,30 @@ +package org.apache.samza.application; + +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.task.TaskFactory; +import org.apache.samza.task.TaskFactoryUtil; + + +/** + * Created by yipan on 7/22/18. + */ +public class ApplicationClassUtils { + public static LifecycleAwareApplication fromConfig(Config config) { + ApplicationConfig appConfig = new ApplicationConfig(config); + if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { + try { + Class appClass = (Class) Class.forName(appConfig.getAppClass()); + if (StreamApplication.class.isAssignableFrom(appClass) || TaskApplication.class.isAssignableFrom(appClass)) { + return appClass.newInstance(); + } + } catch (IllegalAccessException | InstantiationException | ClassNotFoundException e) { + throw new ConfigException(String.format("Loading app.class %s failed. The user application has to implement " + + "StreamApplication or TaskApplication.", appConfig.getAppClass()), e); + } + } + // no app.class defined. It has to be a legacy application with task.class configuration + return (TaskApplication) (appSpec) -> appSpec.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(config)); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java index 964e8a1f97..318db364c7 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java @@ -20,7 +20,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; @@ -29,8 +28,6 @@ import java.util.regex.Pattern; import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationInitializer; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -38,7 +35,6 @@ import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; -import org.apache.samza.runtime.internal.StreamApplicationSpec; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; @@ -53,7 +49,7 @@ * create the DAG of transforms. * 2) a builder that creates a serializable {@link OperatorSpecGraph} from user-defined DAG */ -public class StreamGraphSpec implements StreamApplicationInitializer, StreamApplicationSpec { +public class StreamGraphSpec implements StreamGraph { private static final Logger LOGGER = LoggerFactory.getLogger(StreamGraphSpec.class); private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); @@ -72,7 +68,7 @@ public class StreamGraphSpec implements StreamApplicationInitializer, StreamAppl private int nextOpNum = 0; private final Set operatorIds = new HashSet<>(); private Serde defaultSerde = new KVSerde(new NoOpSerde(), new NoOpSerde()); - private ContextManager contextManager = null; +// private ContextManager contextManager = null; public StreamGraphSpec(Config config) { this.config = config; @@ -157,35 +153,15 @@ public Table> getTable(TableDescriptor tableDesc) { return tables.get(tableSpec); } - @Override - public StreamApplicationInitializer withContextManager(ContextManager contextManager) { - this.contextManager = contextManager; - return this; - } - - public ContextManager getContextManager() { - return this.contextManager; - } - - @Override - public Collection getInputStreams() { - return Collections.unmodifiableCollection(this.inputOperators.keySet()); - } - - @Override - public Collection getOutputStreams() { - return Collections.unmodifiableCollection(this.outputStreams.keySet()); - } - - @Override - public Collection getBroadcastStreams() { - return Collections.unmodifiableCollection(this.broadcastStreams); - } - - @Override - public Collection getTables() { - return Collections.unmodifiableCollection(this.tables.keySet().stream().collect(HashSet::new, (s1, td) -> s1.add(td.getId()), (s1, s2) -> s1.addAll(s2))); - } +// @Override +// public StreamApplicationSpec withContextManager(ContextManager contextManager) { +// this.contextManager = contextManager; +// return this; +// } +// +// public ContextManager getContextManager() { +// return this.contextManager; +// } public OperatorSpecGraph getOperatorSpecGraph() { return new OperatorSpecGraph(this); @@ -277,11 +253,15 @@ Map getInputOperators() { return Collections.unmodifiableMap(inputOperators); } - Map getOutputStreamImpls() { + Map getOutputStreams() { return Collections.unmodifiableMap(outputStreams); } - Map getTableImpls() { + Set getBroadcastStreams() { + return Collections.unmodifiableSet(broadcastStreams); + } + + Map getTables() { return Collections.unmodifiableMap(tables); } @@ -312,18 +292,4 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } - @Override - public Config getConfig() { - return this.config; - } - - @Override - public StreamApplication getUserApp() { - throw new SamzaException("shouldn't be called here"); - } - - @Override - public String getGlobalAppId() { - throw new SamzaException("shouldn't be called here"); - } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index e1e1c5555e..1fe740c6f1 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -23,6 +23,7 @@ import java.util.LinkedHashSet; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.functions.TimerFunction; @@ -103,7 +104,7 @@ public final OpCode getOpCode() { } /** - * Get the unique ID of this operator in the {@link org.apache.samza.operators.StreamGraph}. + * Get the unique ID of this operator in the {@link StreamApplicationSpec}. * @return the unique operator ID */ public final String getOpId() { 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 73f32e77fc..e465880d18 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 @@ -39,8 +39,7 @@ import org.apache.samza.coordinator.JobCoordinatorListener; import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.task.AsyncStreamTaskFactory; -import org.apache.samza.task.StreamTaskFactory; +import org.apache.samza.task.TaskFactory; import org.apache.samza.util.ScalaJavaUtil; import org.apache.samza.util.Util; import org.slf4j.Logger; @@ -61,7 +60,7 @@ public class StreamProcessor { private final JobCoordinator jobCoordinator; private final StreamProcessorLifecycleListener processorListener; - private final Object taskFactory; + private final TaskFactory taskFactory; private final Map customMetricsReporter; private final Config config; private final long taskShutdownMs; @@ -80,38 +79,38 @@ public class StreamProcessor { @VisibleForTesting JobCoordinatorListener jobCoordinatorListener = null; - /** - * Create an instance of StreamProcessor that encapsulates a JobCoordinator and Samza Container - *

    - * JobCoordinator controls how the various StreamProcessor instances belonging to a job coordinate. It is also - * responsible generating and updating JobModel. - * When StreamProcessor starts, it starts the JobCoordinator and brings up a SamzaContainer based on the JobModel. - * SamzaContainer is executed using an ExecutorService. - *

    - * Note: Lifecycle of the ExecutorService is fully managed by the StreamProcessor, and NOT exposed to the user - * - * @param config Instance of config object - contains all configuration required for processing - * @param customMetricsReporters Map of custom MetricReporter instances that are to be injected in the Samza job - * @param asyncStreamTaskFactory The {@link AsyncStreamTaskFactory} to be used for creating task instances. - * @param processorListener listener to the StreamProcessor life cycle - */ - public StreamProcessor(Config config, Map customMetricsReporters, - AsyncStreamTaskFactory asyncStreamTaskFactory, StreamProcessorLifecycleListener processorListener) { - this(config, customMetricsReporters, asyncStreamTaskFactory, processorListener, null); - } - - /** - *Same as {@link #StreamProcessor(Config, Map, AsyncStreamTaskFactory, StreamProcessorLifecycleListener)}, except task - * instances are created using the provided {@link StreamTaskFactory}. - * @param config - config - * @param customMetricsReporters metric Reporter - * @param streamTaskFactory task factory to instantiate the Task - * @param processorListener listener to the StreamProcessor life cycle - */ - public StreamProcessor(Config config, Map customMetricsReporters, - StreamTaskFactory streamTaskFactory, StreamProcessorLifecycleListener processorListener) { - this(config, customMetricsReporters, streamTaskFactory, processorListener, null); - } +// /** +// * Create an instance of StreamProcessor that encapsulates a JobCoordinator and Samza Container +// *

    +// * JobCoordinator controls how the various StreamProcessor instances belonging to a job coordinate. It is also +// * responsible generating and updating JobModel. +// * When StreamProcessor starts, it starts the JobCoordinator and brings up a SamzaContainer based on the JobModel. +// * SamzaContainer is executed using an ExecutorService. +// *

    +// * Note: Lifecycle of the ExecutorService is fully managed by the StreamProcessor, and NOT exposed to the user +// * +// * @param config Instance of config object - contains all configuration required for processing +// * @param customMetricsReporters Map of custom MetricReporter instances that are to be injected in the Samza job +// * @param asyncStreamTaskFactory The {@link AsyncStreamTaskFactory} to be used for creating task instances. +// * @param processorListener listener to the StreamProcessor life cycle +// */ +// private StreamProcessor(Config config, Map customMetricsReporters, +// AsyncStreamTaskFactory asyncStreamTaskFactory, StreamProcessorLifecycleListener processorListener) { +// this(config, customMetricsReporters, asyncStreamTaskFactory, processorListener, null); +// } +// +// /** +// *Same as {@link #StreamProcessor(Config, Map, AsyncStreamTaskFactory, StreamProcessorLifecycleListener)}, except task +// * instances are created using the provided {@link StreamTaskFactory}. +// * @param config - config +// * @param customMetricsReporters metric Reporter +// * @param streamTaskFactory task factory to instantiate the Task +// * @param processorListener listener to the StreamProcessor life cycle +// */ +// private StreamProcessor(Config config, Map customMetricsReporters, +// StreamTaskFactory streamTaskFactory, StreamProcessorLifecycleListener processorListener) { +// this(config, customMetricsReporters, streamTaskFactory, processorListener, null); +// } /* package private */ JobCoordinator getJobCoordinator() { @@ -124,7 +123,7 @@ JobCoordinator getCurrentJobCoordinator() { return jobCoordinator; } - StreamProcessor(Config config, Map customMetricsReporters, Object taskFactory, + public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, StreamProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator) { this.taskFactory = taskFactory; this.config = config; diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 58fb0463c4..916db3a6cd 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -26,6 +26,10 @@ import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; +import org.apache.samza.application.ApplicationSpec; +import org.apache.samza.application.LifecycleAwareApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ApplicationConfig.ApplicationMode; import org.apache.samza.config.Config; @@ -38,11 +42,7 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.internal.ApplicationSpec; -import org.apache.samza.runtime.internal.StreamApplicationSpec; -import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,17 +56,58 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { protected final Config config; protected final Map metricsReporters = new HashMap<>(); - public AbstractApplicationRunner(Config config) { + AbstractApplicationRunner(Config config) { this.config = config; } - public ExecutionPlan getExecutionPlan(StreamGraphSpec graphSpec, StreamManager streamManager) throws Exception { + @Override + public final void addMetricsReporters(Map metricsReporters) { + this.metricsReporters.putAll(metricsReporters); + } + + @Override + public final void run(ApplicationSpec appSpec) { + LifecycleAwareApplication userApp = getUserApp(appSpec); + userApp.beforeStart(); + getLifecycleMethods(appSpec).run(); + userApp.afterStart(); + } + + @Override + public final ApplicationStatus status(ApplicationSpec appSpec) { + return getLifecycleMethods(appSpec).status(); + } + + @Override + public final void kill(ApplicationSpec appSpec) { + LifecycleAwareApplication userApp = getUserApp(appSpec); + userApp.beforeStop(); + getLifecycleMethods(appSpec).kill(); + userApp.afterStop(); + } + + @Override + public final void waitForFinish(ApplicationSpec appSpec) { + getLifecycleMethods(appSpec).waitForFinish(Duration.ofSeconds(0)); + } + + @Override + public final boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { + return getLifecycleMethods(appSpec).waitForFinish(timeout); + } + + final StreamManager buildAndStartStreamManager() { + StreamManager streamManager = new StreamManager(this.config); + streamManager.start(); + return streamManager; + } + + final ExecutionPlan getExecutionPlan(OperatorSpecGraph graphSpec, StreamManager streamManager) throws Exception { return getExecutionPlan(graphSpec, null, streamManager); } /* package private */ - ExecutionPlan getExecutionPlan(StreamGraphSpec graphSpec, String runId, StreamManager streamManager) throws Exception { - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + final ExecutionPlan getExecutionPlan(OperatorSpecGraph graphSpec, String runId, StreamManager streamManager) throws Exception { // update application configs Map cfg = new HashMap<>(config); @@ -75,15 +116,15 @@ ExecutionPlan getExecutionPlan(StreamGraphSpec graphSpec, String runId, StreamMa } StreamConfig streamConfig = new StreamConfig(config); - Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); - inputStreams.removeAll(specGraph.getOutputStreams().keySet()); + Set inputStreams = new HashSet<>(graphSpec.getInputOperators().keySet()); + inputStreams.removeAll(graphSpec.getOutputStreams().keySet()); ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) ? ApplicationMode.BATCH : ApplicationMode.STREAM; cfg.put(ApplicationConfig.APP_MODE, mode.name()); // create the physical execution plan ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager); - return planner.plan(specGraph); + return planner.plan(graphSpec); } /** @@ -107,33 +148,11 @@ final void writePlanJsonFile(String planJson) { } } - @Override - public final void addMetricsReporters(Map metricsReporters) { - this.metricsReporters.putAll(metricsReporters); - } + protected abstract ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec); - StreamManager buildAndStartStreamManager() { - StreamManager streamManager = new StreamManager(this.config); - streamManager.start(); - return streamManager; - } - - private ApplicationLifecycle getLifecycleMethods(ApplicationSpec appSpec) { - if (appSpec instanceof StreamApplicationSpec) { - return getStreamAppLifecycle((StreamApplicationSpec) appSpec); - } - if (appSpec instanceof TaskApplicationSpec) { - return getTaskAppLifecycle((TaskApplicationSpec) appSpec); - } - throw new IllegalArgumentException(String.format("The specified application %s is not valid. " - + "Only StreamApplicationSpec and TaskApplicationSpec are supported.", appSpec.getClass().getName())); - } + protected abstract ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec); - protected abstract ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec); - - protected abstract ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec); - - interface ApplicationLifecycle { + protected interface ApplicationLifecycle { void run(); @@ -152,32 +171,25 @@ interface ApplicationLifecycle { } - @Override - public final void run(ApplicationSpec appSpec) { - appSpec.getUserApp().beforeStart(appSpec); - getLifecycleMethods(appSpec).run(); - appSpec.getUserApp().afterStart(appSpec); - } - - @Override - public final ApplicationStatus status(ApplicationSpec appSpec) { - return getLifecycleMethods(appSpec).status(); - } - - @Override - public final void kill(ApplicationSpec appSpec) { - appSpec.getUserApp().beforeStop(appSpec); - getLifecycleMethods(appSpec).kill(); - appSpec.getUserApp().afterStop(appSpec); + private ApplicationLifecycle getLifecycleMethods(ApplicationSpec appSpec) { + if (appSpec instanceof StreamAppSpecImpl) { + return getStreamAppLifecycle((StreamAppSpecImpl) appSpec); + } + if (appSpec instanceof TaskAppSpecImpl) { + return getTaskAppLifecycle((TaskAppSpecImpl) appSpec); + } + throw new IllegalArgumentException(String.format("The specified application %s is not valid. " + + "Only StreamApplicationSpec and TaskApplicationSpec are supported.", appSpec.getClass().getName())); } - @Override - public final void waitForFinish(ApplicationSpec appSpec) { - getLifecycleMethods(appSpec).waitForFinish(Duration.ofSeconds(0)); + private LifecycleAwareApplication getUserApp(ApplicationSpec appSpec) { + if (appSpec instanceof StreamAppSpecImpl) { + return ((StreamAppSpecImpl) appSpec).getUserApp(); + } + if (appSpec instanceof TaskAppSpecImpl) { + return ((TaskAppSpecImpl) appSpec).getUserApp(); + } + throw new IllegalArgumentException(); } - @Override - public final boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { - return getLifecycleMethods(appSpec).waitForFinish(timeout); - } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 4b3297677f..2a6111d202 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -21,11 +21,9 @@ import joptsimple.OptionSet; import joptsimple.OptionSpec; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.config.Config; import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.task.TaskFactory; -import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.CommandLine; import org.apache.samza.util.Util; @@ -36,8 +34,6 @@ * For a Samza job using low level task API, it will create the JobRunner to start it. */ public class ApplicationRunnerMain { - // TODO: have the app configs consolidated in one place - public static final String STREAM_APPLICATION_CLASS_CONFIG = "app.class"; public static class ApplicationRunnerCommandLine extends CommandLine { public OptionSpec operationOpt = @@ -60,13 +56,7 @@ public static void main(String[] args) throws Exception { Config config = Util.rewriteConfig(orgConfig); ApplicationRunnerOperation op = cmdLine.getOperation(options); - ApplicationRuntime appRuntime = - config.containsKey(STREAM_APPLICATION_CLASS_CONFIG) ? ApplicationRuntimes.createStreamApp( - (StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(), config) : - // TODO: Need to deal with 1) new TaskApplication implemention that populates inputStreams and outputStreams by the user; - // 2) legacy task application that only has input streams specified in config - ApplicationRuntimes.createTaskApp( - (appBuilder, cfg) -> appBuilder.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(cfg)), config); + ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); switch (op) { case RUN: diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 4eb8c56f60..214dcecadc 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -33,7 +33,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobCoordinatorConfig; @@ -43,15 +44,14 @@ import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.internal.StreamApplicationSpec; -import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.apache.samza.system.StreamSpec; -import org.apache.samza.task.AsyncStreamTaskFactory; -import org.apache.samza.task.StreamTaskFactory; +import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,12 +133,12 @@ public LocalApplicationRunner(Config config) { } @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec) { + protected ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec) { return new TaskAppLifecycle(appSpec); } @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec) { + protected ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec) { return new StreamAppLifecycle(appSpec); } @@ -149,9 +149,9 @@ public LocalApplicationRunner(Config config, Map custom } class StreamAppLifecycle implements ApplicationLifecycle { - final StreamApplicationSpec streamApp; + final StreamAppSpecImpl streamApp; - StreamAppLifecycle(StreamApplicationSpec streamApp) { + StreamAppLifecycle(StreamAppSpecImpl streamApp) { this.streamApp = streamApp; } @@ -162,7 +162,7 @@ public void run() { streamManager = buildAndStartStreamManager(); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), streamManager); + ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec)streamApp.getGraph()).getOperatorSpecGraph(), streamManager); String executionPlanJson = plan.getPlanAsJson(); writePlanJsonFile(executionPlanJson); @@ -180,7 +180,8 @@ public void run() { plan.getJobConfigs().forEach(jobConfig -> { LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); - StreamProcessor processor = createStreamProcessor(jobConfig, (StreamGraphSpec) streamApp.getGraph(), listener); + StreamProcessor processor = createStreamProcessor(jobConfig, ((StreamGraphSpec)streamApp.getGraph()).getOperatorSpecGraph(), + streamApp.getContextManager(), listener); listener.setProcessor(processor); processors.add(processor); }); @@ -217,10 +218,10 @@ public boolean waitForFinish(Duration timeout) { } class TaskAppLifecycle implements ApplicationLifecycle { - final TaskApplicationSpec appSpec; + final TaskAppSpecImpl appSpec; StreamProcessor sp; - TaskAppLifecycle(TaskApplicationSpec appSpec) { + TaskAppLifecycle(TaskAppSpecImpl appSpec) { this.appSpec = appSpec; } @@ -229,7 +230,7 @@ public void run() { LOG.info("LocalApplicationRunner will start task " + appSpec.getGlobalAppId()); LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); - sp = getStreamProcessorInstance(config, appSpec.getTaskFactory(), listener); + sp = createStreamProcessor(config, appSpec.getTaskFactory(), listener); numProcessorsToStart.set(1); listener.setProcessor(sp); @@ -325,44 +326,33 @@ private void createStreams(String planId, } /** - * Create {@link StreamProcessor} based on {@link StreamApplication} and the config + * Create {@link StreamProcessor} based on the config * @param config config * @return {@link StreamProcessor]} */ /* package private */ StreamProcessor createStreamProcessor( Config config, + TaskFactory taskFactory, StreamProcessorLifecycleListener listener) { - Object taskFactory = TaskFactoryUtil.createTaskFactory(config); - return getStreamProcessorInstance(config, taskFactory, listener); + return new StreamProcessor(config, customMetricsReporters, taskFactory, listener, null); } /** - * Create {@link StreamProcessor} based on {@link StreamApplication} and the config + * Create {@link StreamProcessor} based on {@link OperatorSpecGraph}, {@link ContextManager} and the config * @param config config - * @param graphBuilder {@link StreamGraphSpec} + * @param graph {@link OperatorSpecGraph} + * @param contextManager {@link ContextManager} * @return {@link StreamProcessor]} */ /* package private */ StreamProcessor createStreamProcessor( Config config, - StreamGraphSpec graphBuilder, + OperatorSpecGraph graph, + ContextManager contextManager, StreamProcessorLifecycleListener listener) { - Object taskFactory = TaskFactoryUtil.createTaskFactory(graphBuilder.getOperatorSpecGraph(), graphBuilder.getContextManager()); - return getStreamProcessorInstance(config, taskFactory, listener); - } - - private StreamProcessor getStreamProcessorInstance(Config config, Object taskFactory, StreamProcessorLifecycleListener listener) { - if (taskFactory instanceof StreamTaskFactory) { - return new StreamProcessor( - config, customMetricsReporters, (StreamTaskFactory) taskFactory, listener); - } else if (taskFactory instanceof AsyncStreamTaskFactory) { - return new StreamProcessor( - config, customMetricsReporters, (AsyncStreamTaskFactory) taskFactory, listener); - } else { - throw new SamzaException(String.format("%s is not a valid task factory", - taskFactory.getClass().getCanonicalName())); - } + TaskFactory taskFactory = TaskFactoryUtil.createTaskFactory(graph, contextManager); + return new StreamProcessor(config, customMetricsReporters, taskFactory, listener, null); } /* package private for testing */ diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index fd6938d755..d7aab4ff67 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -21,13 +21,19 @@ import java.time.Duration; import java.util.HashMap; +import java.util.Map; import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; -import org.apache.samza.application.internal.ApplicationBuilder; +import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamApplicationBuilder; -import org.apache.samza.application.internal.TaskApplicationBuilder; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.ApplicationSpec; +import org.apache.samza.application.LifecycleAwareApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.StreamApplicationSpecRuntime; +import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.internal.TaskApplicationSpecRuntime; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -38,10 +44,8 @@ import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.model.JobModel; +import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.runtime.internal.StreamApplicationSpec; -import org.apache.samza.runtime.internal.TaskApplicationSpec; -import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.SamzaUncaughtExceptionHandler; import org.apache.samza.util.ScalaJavaUtil; @@ -74,19 +78,19 @@ public LocalContainerRunner(JobModel jobModel, String containerId) { } @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec) { + protected ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec) { return new TaskAppLifecycle(appSpec); } @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec) { + protected ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec) { return new StreamAppLifecycle(appSpec); } class TaskAppLifecycle implements ApplicationLifecycle { - final TaskApplicationSpec taskApp; + final TaskAppSpecImpl taskApp; - TaskAppLifecycle(TaskApplicationSpec taskApp) { + TaskAppLifecycle(TaskAppSpecImpl taskApp) { this.taskApp = taskApp; } @@ -150,16 +154,16 @@ public boolean waitForFinish(Duration timeout) { } class StreamAppLifecycle implements ApplicationLifecycle { - final StreamApplicationSpec streamApp; + final StreamAppSpecImpl streamApp; - StreamAppLifecycle(StreamApplicationSpec streamApp) { + StreamAppLifecycle(StreamAppSpecImpl streamApp) { this.streamApp = streamApp; } @Override public void run() { Object taskFactory = TaskFactoryUtil.createTaskFactory(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), - ((StreamGraphSpec) streamApp.getGraph()).getContextManager()); + streamApp.getContextManager()); container = SamzaContainer$.MODULE$.apply( containerId, @@ -243,18 +247,10 @@ public static void main(String[] args) throws Exception { MDC.put("jobName", jobName); MDC.put("jobId", jobId); - ApplicationBuilder.AppConfig appConfig = new ApplicationBuilder.AppConfig(config); - - LocalContainerRunner runner = new LocalContainerRunner(jobModel, containerId); - - ApplicationBuilder appSpec = appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty() ? - new StreamApplicationBuilder((StreamApplication) Class.forName(appConfig.getAppClass()).newInstance(), config) : - // TODO: Need to deal with 1) new TaskApplication implemention that populates inputStreams and outputStreams by the user; - // 2) legacy task application that only has input streams specified in config - new TaskApplicationBuilder((appBuilder, cfg) -> appBuilder.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(cfg)), config); - - runner.run(appSpec); - runner.waitForFinish(appSpec); + ApplicationRuntime appSpec = createAppRuntime(ApplicationClassUtils.fromConfig(config), + new LocalContainerRunner(jobModel, containerId), config); + appSpec.start(); + appSpec.waitForFinish(); } private void startContainerHeartbeatMonitor() { @@ -284,4 +280,56 @@ private void stopContainerHeartbeatMonitor() { } } + private static ApplicationRuntime createAppRuntime(LifecycleAwareApplication userApp, LocalContainerRunner runner, Config config) { + if (userApp instanceof StreamApplication) { + return new ContainerAppRuntimeImpl(new StreamAppSpecImpl((StreamApplication) userApp, config), runner); + } + if (userApp instanceof TaskApplication) { + return new ContainerAppRuntimeImpl(new TaskAppSpecImpl((TaskApplication) userApp, config), runner); + } + + throw new IllegalArgumentException(String.format("Invalid user application class %s. Only StreamApplication and TaskApplication " + + "are supported", userApp.getClass().getName())); + } + + private static class ContainerAppRuntimeImpl implements ApplicationRuntime { + private final ApplicationSpec appSpec; + private final LocalContainerRunner runner; + + public ContainerAppRuntimeImpl(ApplicationSpec appSpec, LocalContainerRunner runner) { + this.appSpec = appSpec; + this.runner = runner; + } + + @Override + public void start() { + this.runner.run(appSpec); + } + + @Override + public void stop() { + this.runner.kill(appSpec); + } + + @Override + public ApplicationStatus status() { + return this.runner.status(appSpec); + } + + @Override + public void waitForFinish() { + this.runner.waitForFinish(appSpec, Duration.ofSeconds(0)); + } + + @Override + public boolean waitForFinish(Duration timeout) { + return this.runner.waitForFinish(appSpec, timeout); + } + + @Override + public void addMetricsReporters(Map metricsReporters) { + this.runner.addMetricsReporters(metricsReporters); + } + } + } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index b48092beea..602e06e686 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -22,6 +22,8 @@ import java.time.Duration; import java.util.UUID; import org.apache.samza.SamzaException; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -33,8 +35,6 @@ import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.internal.StreamApplicationSpec; -import org.apache.samza.runtime.internal.TaskApplicationSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,20 +54,20 @@ public RemoteApplicationRunner(Config config) { } @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpec appSpec) { + protected ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec) { return new TaskAppLifecycle(appSpec); } @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpec appSpec) { + protected ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec) { return new StreamAppLifecycle(appSpec); } class TaskAppLifecycle implements ApplicationLifecycle { - final TaskApplicationSpec taskApp; + final TaskAppSpecImpl taskApp; final JobRunner jobRunner; - TaskAppLifecycle(TaskApplicationSpec appSpec) { + TaskAppLifecycle(TaskAppSpecImpl appSpec) { this.taskApp = appSpec; this.jobRunner = new JobRunner(config); } @@ -95,9 +95,9 @@ public boolean waitForFinish(Duration timeout) { } class StreamAppLifecycle implements ApplicationLifecycle { - final StreamApplicationSpec streamApp; + final StreamAppSpecImpl streamApp; - StreamAppLifecycle(StreamApplicationSpec appSpec) { + StreamAppLifecycle(StreamAppSpecImpl appSpec) { this.streamApp = appSpec; } @@ -112,7 +112,7 @@ public void run() { LOG.info("The start id for this start is {}", runId); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), runId, streamManager); + ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), runId, streamManager); writePlanJsonFile(plan.getPlanAsJson()); // 2. create the necessary streams @@ -141,7 +141,7 @@ public void kill() { StreamManager streamManager = null; try { streamManager = buildAndStartStreamManager(); - ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), streamManager); + ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), streamManager); plan.getJobConfigs().forEach(jobConfig -> { LOG.info("Killing job {}", jobConfig.getName()); @@ -166,7 +166,7 @@ public ApplicationStatus status() { ApplicationStatus unsuccessfulFinishStatus = null; streamManager = buildAndStartStreamManager(); - ExecutionPlan plan = getExecutionPlan((StreamGraphSpec) streamApp.getGraph(), streamManager); + ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), streamManager); for (JobConfig jobConfig : plan.getJobConfigs()) { ApplicationStatus status = getApplicationStatus(jobConfig); diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java index fdd134fe90..6a4076cb7c 100644 --- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java +++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java @@ -18,6 +18,7 @@ */ package org.apache.samza.task; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.system.EndOfStreamMessage; @@ -71,7 +72,7 @@ public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextMan *

    * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}. * The {@link org.apache.samza.operators.StreamGraphSpec} sets the input and output streams and the task-wide - * context manager using the {@link org.apache.samza.operators.StreamGraph} APIs, + * context manager using the {@link StreamApplicationSpec} APIs, * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. After the * {@link org.apache.samza.operators.StreamGraphSpec} is initialized once by the application, it then creates * an immutable {@link OperatorSpecGraph} accordingly, which is passed in to this class to create the {@link OperatorImplGraph} diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index fb6438ec3a..c7b35f39e2 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -45,7 +45,7 @@ public class TaskFactoryUtil { * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph} * @return a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} */ - public static Object createTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) { + public static TaskFactory createTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) { return createStreamOperatorTaskFactory(specGraph, contextManager); } @@ -55,7 +55,7 @@ public static Object createTaskFactory(OperatorSpecGraph specGraph, ContextManag * @param config the {@link Config} for this job * @return a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} */ - public static Object createTaskFactory(Config config) { + public static TaskFactory createTaskFactory(Config config) { return fromTaskClassConfig(config); } @@ -68,7 +68,7 @@ private static StreamTaskFactory createStreamOperatorTaskFactory(OperatorSpecGra * @param config the {@link Config} * @return task factory instance */ - private static Object fromTaskClassConfig(Config config) { + private static TaskFactory fromTaskClassConfig(Config config) { // if there is configuration to set the job w/ a specific type of task, instantiate the corresponding task factory String taskClassName = new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( () -> { @@ -120,7 +120,7 @@ public StreamTask createInstance() { * @param taskThreadPool the thread pool to run the {@link AsyncStreamTaskAdapter} tasks * @return the finalized task factory object */ - public static Object finalizeTaskFactory(Object factory, boolean singleThreadMode, ExecutorService taskThreadPool) { + public static TaskFactory finalizeTaskFactory(TaskFactory factory, boolean singleThreadMode, ExecutorService taskThreadPool) { validateFactory(factory); @@ -146,7 +146,7 @@ public AsyncStreamTask createInstance() { return factory; } - private static void validateFactory(Object factory) { + private static void validateFactory(TaskFactory factory) { if (factory == null) { throw new SamzaException("Either the task class name or the task factory instance is required."); } diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 4e77faebe2..77bdaadaea 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -175,10 +175,10 @@ static public List getCloseListByTaskName(TaskName taskName) { @Override public void close() { if (this.taskName == null) { - throw new IllegalStateException("Close called before init"); + throw new IllegalStateException("Close called before bootstrap"); } if (perTaskFunctionMap.get(this.taskName) == null || !perTaskFunctionMap.get(this.taskName).containsKey(opId)) { - throw new IllegalStateException("Close called before init"); + throw new IllegalStateException("Close called before bootstrap"); } if (perTaskCloseList.get(this.taskName) == null) { @@ -196,7 +196,7 @@ public void init(Config config, TaskContext context) { perTaskFunctionMap.put(context.getTaskName(), new HashMap() { { this.put(opId, BaseTestFunction.this); } }); } else { if (perTaskFunctionMap.get(context.getTaskName()).containsKey(opId)) { - throw new IllegalStateException(String.format("Multiple init called for op %s in the same task instance %s", opId, this.taskName.getTaskName())); + throw new IllegalStateException(String.format("Multiple bootstrap called for op %s in the same task instance %s", opId, this.taskName.getTaskName())); } perTaskFunctionMap.get(context.getTaskName()).put(opId, this); } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index 7e6433cdea..c6440b7a79 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -18,10 +18,11 @@ */ package org.apache.samza.runtime; -import org.apache.samza.application.StreamApplication; +import java.time.Duration; +import org.apache.samza.application.internal.StreamApplicationSpecRuntime; +import org.apache.samza.application.internal.TaskApplicationSpecRuntime; import org.apache.samza.config.Config; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.operators.StreamGraph; import org.junit.Test; import static org.junit.Assert.*; @@ -37,7 +38,7 @@ public void TestRunOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationRunnerMain.STREAM_APPLICATION_CLASS_CONFIG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", + "-config", ApplicationRunnerMain.APP_CLASS_CFG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts" }); @@ -52,7 +53,7 @@ public void TestKillOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationRunnerMain.STREAM_APPLICATION_CLASS_CONFIG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", + "-config", ApplicationRunnerMain.APP_CLASS_CFG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts", "--operation=kill" }); @@ -68,7 +69,7 @@ public void TestStatusOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationRunnerMain.STREAM_APPLICATION_CLASS_CONFIG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", + "-config", ApplicationRunnerMain.APP_CLASS_CFG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts", "--operation=status" }); @@ -85,33 +86,51 @@ public TestApplicationRunnerInvocationCounts(Config config) { super(config); } - @Override - public void runTask() { - throw new UnsupportedOperationException("runTask() not supported in this test"); - } - - @Override - public void run(StreamApplication streamApp) { + private void run() { runCount++; } - @Override - public void kill(StreamApplication streamApp) { + private void kill() { killCount++; } - @Override - public ApplicationStatus status(StreamApplication streamApp) { + private ApplicationStatus status() { statusCount++; return ApplicationStatus.Running; } - } - public static class TestStreamApplicationDummy implements StreamApplication { + class TestAppLifecycle implements ApplicationLifecycle { + + @Override + public void run() { + run(); + } + + @Override + public void kill() { + kill(); + } + + @Override + public ApplicationStatus status() { + return status(); + } + + @Override + public boolean waitForFinish(Duration timeout) { + return false; + } + } @Override - public void init(StreamGraph graph, Config config) { + protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpecRuntime appSpec) { + return new TestAppLifecycle(); + } + @Override + protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpecRuntime appSpec) { + return new TestAppLifecycle(); } } + } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 5eb139b9a0..5eb70307ee 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; 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 f3c144caee..ddcc2735d9 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 @@ -20,14 +20,10 @@ package org.apache.samza.sql.runner; import java.util.List; - -import org.apache.samza.SamzaException; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationInitializer; -import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.sql.translator.QueryTranslator; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; +import org.apache.samza.sql.translator.QueryTranslator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,14 +36,14 @@ public class SamzaSqlApplication implements StreamApplication { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplication.class); @Override - public void init(StreamApplicationInitializer appBuilder, Config config) { + public void setup(StreamApplicationSpec appSpec) { try { - SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(config); + SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(appSpec.getConfig()); QueryTranslator queryTranslator = new QueryTranslator(sqlConfig); List queries = sqlConfig.getQueryInfo(); for (SamzaSqlQueryParser.QueryInfo query : queries) { LOG.info("Translating the query {} to samza stream graph", query.getSelectQuery()); - queryTranslator.translate(query, appBuilder); + queryTranslator.translate(query, appSpec); } } catch (RuntimeException e) { LOG.error("SamzaSqlApplication threw exception.", e); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java index a0ab40a5f1..dae4548fa0 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java @@ -24,15 +24,15 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.Validate; -import org.apache.samza.application.internal.ApplicationBuilder; -import org.apache.samza.application.internal.StreamApplicationBuilder; +import org.apache.samza.application.ApplicationSpec; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.RemoteApplicationRunner; -import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.runtime.internal.ApplicationRunners; import org.apache.samza.sql.interfaces.SqlIOConfig; @@ -54,7 +54,7 @@ public class SamzaSqlApplicationRuntime implements ApplicationRuntime { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplicationRuntime.class); private final Config sqlConfig; - private final ApplicationBuilder appSpec; + private final ApplicationSpec appSpec; private final ApplicationRunner runner; private final Boolean localRunner; @@ -64,7 +64,7 @@ public class SamzaSqlApplicationRuntime implements ApplicationRuntime { public SamzaSqlApplicationRuntime(Boolean localRunner, Config config) { this.localRunner = localRunner; sqlConfig = computeSamzaConfigs(localRunner, config); - appSpec = new StreamApplicationBuilder(new SamzaSqlApplication(), sqlConfig); + appSpec = new StreamAppSpecImpl(new SamzaSqlApplication(), sqlConfig); runner = ApplicationRunners.fromConfig(sqlConfig); } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index 1db300021e..db63330564 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -21,7 +21,6 @@ import java.util.Map; import java.util.Optional; - import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.RelShuttleImpl; @@ -30,25 +29,25 @@ import org.apache.calcite.rel.logical.LogicalFilter; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; -import org.apache.samza.SamzaException; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.sql.data.SamzaSqlRelMessage; import org.apache.samza.sql.interfaces.SamzaRelConverter; -import org.apache.samza.sql.interfaces.SqlIOResolver; import org.apache.samza.sql.interfaces.SqlIOConfig; +import org.apache.samza.sql.interfaces.SqlIOResolver; import org.apache.samza.sql.planner.QueryPlanner; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; -import org.apache.samza.task.TaskContext; import org.apache.samza.table.Table; +import org.apache.samza.task.TaskContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +55,7 @@ /** * This class is used to populate the StreamGraph using the SQL queries. * This class contains the core of the SamzaSQL control code that converts the SQL statements to calcite relational graph. - * It then walks the relational graph and then populates the Samza's {@link StreamGraph} accordingly. + * It then walks the relational graph and then populates the Samza's {@link StreamApplicationSpec} accordingly. */ public class QueryTranslator { private static final Logger LOG = LoggerFactory.getLogger(QueryTranslator.class); @@ -92,13 +91,13 @@ public QueryTranslator(SamzaSqlApplicationConfig sqlConfig) { this.converters = sqlConfig.getSamzaRelConverters(); } - public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamGraph streamGraph) { + public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplicationSpec appSpec) { QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getInputSystemStreamConfigBySource(), sqlConfig.getUdfMetadata()); final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig); final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery()); - final TranslatorContext context = new TranslatorContext(streamGraph, relRoot, executionContext, this.converters); + final TranslatorContext context = new TranslatorContext(appSpec, relRoot, executionContext, this.converters); final RelNode node = relRoot.project(); final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver(); @@ -151,9 +150,9 @@ public RelNode visit(LogicalAggregate aggregate) { Optional tableDescriptor = sinkConfig.getTableDescriptor(); if (!tableDescriptor.isPresent()) { - outputStream.sendTo(streamGraph.getOutputStream(sinkConfig.getStreamName())); + outputStream.sendTo(appSpec.getOutputStream(sinkConfig.getStreamName())); } else { - Table outputTable = streamGraph.getTable(tableDescriptor.get()); + Table outputTable = appSpec.getTable(tableDescriptor.get()); if (outputTable == null) { String msg = "Failed to obtain table descriptor of " + sinkConfig.getSource(); LOG.error(msg); @@ -162,7 +161,7 @@ public RelNode visit(LogicalAggregate aggregate) { outputStream.sendTo(outputTable); } - streamGraph.withContextManager(new ContextManager() { + appSpec.withContextManager(new ContextManager() { @Override public void init(Config config, TaskContext taskContext) { taskContext.setUserContext(context.clone()); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java index fa3d9d33b7..cfa4185060 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java @@ -30,8 +30,8 @@ import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.sql.data.SamzaSqlRelMessage; import org.apache.samza.sql.interfaces.SamzaRelConverter; -import org.apache.samza.task.TaskContext; import org.apache.samza.sql.interfaces.SqlIOConfig; +import org.apache.samza.task.TaskContext; /** diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java index 7a25efb6b0..ec758ac3ec 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java @@ -21,7 +21,6 @@ import java.util.HashMap; import java.util.Map; - import org.apache.calcite.DataContext; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.linq4j.QueryProvider; @@ -32,6 +31,7 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.SchemaPlus; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.StreamGraph; import org.apache.samza.sql.data.RexToJavaCompiler; @@ -40,7 +40,7 @@ /** - * State that is maintained while translating the Calcite relational graph to Samza {@link StreamGraph}. + * State that is maintained while translating the Calcite relational graph to Samza {@link StreamApplicationSpec}. */ public class TranslatorContext implements Cloneable { /** diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java index 88ce443c78..ea67100d03 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java @@ -95,7 +95,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { assertNotNull(filterSpec); assertEquals(filterSpec.getOpCode(), OperatorSpec.OpCode.FILTER); - // Verify that the init() method will establish the context for the filter function + // Verify that the bootstrap() method will establish the context for the filter function Config mockConfig = mock(Config.class); TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null, new HashSet<>(), null, null, null, null, null, null); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java index f84dd3fdb1..eb23b46c0c 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java @@ -113,7 +113,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { assertNotNull(projectSpec); assertEquals(projectSpec.getOpCode(), OperatorSpec.OpCode.MAP); - // Verify that the init() method will establish the context for the map function + // Verify that the bootstrap() method will establish the context for the map function Config mockConfig = mock(Config.class); TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null, new HashSet<>(), null, null, null, null, null, null); @@ -248,7 +248,7 @@ public TimerFunction getTimerFn() { assertNotNull(projectSpec); assertEquals(projectSpec.getOpCode(), OperatorSpec.OpCode.MAP); - // Verify that the init() method will establish the context for the map function + // Verify that the bootstrap() method will establish the context for the map function Config mockConfig = mock(Config.class); TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null, new HashSet<>(), null, null, null, null, null, null); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index 38ed0d6cfa..c2c1e7a558 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -19,11 +19,13 @@ package org.apache.samza.sql.translator; -import java.util.HashSet; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; @@ -31,9 +33,10 @@ import org.apache.samza.container.TaskName; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory; +import org.apache.samza.sql.runner.SamzaSqlApplication; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; @@ -85,10 +88,12 @@ public void testTranslate() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); + // FIXME: construction of QueryTranslator and other variables within SamzaSqlApplication is now coupled within + // the constructor of StreamApplicationSpec. + StreamAppSpecImpl + graphSpec = new StreamAppSpecImpl(new SamzaSqlApplication(), new MapConfig(config)); translator.translate(queryInfo, graphSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = ((StreamGraphSpec) graphSpec.getGraph()).getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -97,28 +102,28 @@ public void testTranslate() { String outputStreamId = specGraph.getOutputStreams().keySet().stream().findFirst().get(); String outputSystem = streamConfig.getSystem(outputStreamId); String outputPhysicalName = streamConfig.getPhysicalName(outputStreamId); - + Assert.assertEquals(1, specGraph.getOutputStreams().size()); Assert.assertEquals("testavro", outputSystem); Assert.assertEquals("outputTopic", outputPhysicalName); Assert.assertEquals(1, specGraph.getInputOperators().size()); - + Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("SIMPLE1", inputPhysicalName); validatePerTaskContextInit(graphSpec, samzaConfig); } - private void validatePerTaskContextInit(StreamGraphSpec graphSpec, Config samzaConfig) { + private void validatePerTaskContextInit(StreamAppSpecImpl graphSpec, Config samzaConfig) { // make sure that each task context would have a separate instance of cloned TranslatorContext TaskContextImpl testContext = new TaskContextImpl(new TaskName("Partition 1"), null, null, new HashSet<>(), null, null, null, null, null, null); - // call ContextManager.init() to instantiate the per-task TranslatorContext + // call ContextManager.bootstrap() to instantiate the per-task TranslatorContext graphSpec.getContextManager().init(samzaConfig, testContext); Assert.assertNotNull(testContext.getUserContext()); Assert.assertTrue(testContext.getUserContext() instanceof TranslatorContext); TranslatorContext contextPerTaskOne = (TranslatorContext) testContext.getUserContext(); - // call ContextManager.init() second time to instantiate another clone of TranslatorContext + // call ContextManager.bootstrap() second time to instantiate another clone of TranslatorContext graphSpec.getContextManager().init(samzaConfig, testContext); Assert.assertTrue(testContext.getUserContext() instanceof TranslatorContext); // validate the two copies of TranslatorContext are clones of each other 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 73c7928d5c..5cbc8a4068 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 @@ -21,7 +21,7 @@ import java.time.Duration; import java.util.HashMap; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamApplicationBuilder; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.triggers.Triggers; @@ -45,7 +45,7 @@ public class AppWithGlobalConfigExample implements StreamApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new AppWithGlobalConfigExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new AppWithGlobalConfigExample(), config); app.addMetricsReporters(new HashMap<>()); app.start(); @@ -53,15 +53,15 @@ public static void main(String[] args) { } @Override - public void init(StreamApplicationBuilder appBuilder, Config config) { + public void describe(StreamApplicationSpec appSpec) { - appBuilder.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) + appSpec.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) .map(KV::getValue) .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(appBuilder.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); + .sendTo(appSpec.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); } 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 295edb9ed3..6ffc126941 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 @@ -20,7 +20,7 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamApplicationBuilder; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -41,13 +41,13 @@ public class BroadcastExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new BroadcastExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new BroadcastExample(), config); app.start(); app.waitForFinish(); } @Override - public void init(StreamApplicationBuilder appBuilder, Config config) { + public void describe(StreamApplicationSpec appBuilder) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); MessageStream> inputStream = appBuilder.getInputStream("pageViewEventStream", pgeMsgSerde); 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 5409492dd0..be02559faf 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 @@ -23,11 +23,11 @@ import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.ApplicationRuntimes; @@ -48,14 +48,14 @@ public class KeyValueStoreExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new KeyValueStoreExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new KeyValueStoreExample(), config); app.start(); app.waitForFinish(); } @Override - public void init(StreamGraph graph, Config config) { + public void describe(StreamApplicationSpec graph) { MessageStream pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = 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 d7e195778d..155996c242 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 @@ -21,9 +21,9 @@ import com.google.common.collect.ImmutableList; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.serializers.JsonSerdeV2; @@ -37,14 +37,14 @@ public class MergeExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new MergeExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new MergeExample(), config); app.start(); app.waitForFinish(); } @Override - public void init(StreamGraph graph, Config config) { + public void describe(StreamApplicationSpec graph) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); 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 f9b2b3a2cb..5400850fa3 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 @@ -20,11 +20,11 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.ApplicationRuntimes; @@ -43,13 +43,13 @@ public class OrderShipmentJoinExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new OrderShipmentJoinExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new OrderShipmentJoinExample(), config); app.start(); app.waitForFinish(); } @Override - public void init(StreamGraph graph, Config config) { + public void describe(StreamApplicationSpec graph) { MessageStream orders = graph.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class)); MessageStream shipments = 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 55d261ec39..08daeec691 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 @@ -19,12 +19,13 @@ package org.apache.samza.example; import java.time.Duration; +import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FoldLeftFunction; import org.apache.samza.operators.functions.SupplierFunction; import org.apache.samza.operators.triggers.Triggers; @@ -38,6 +39,7 @@ import org.apache.samza.serializers.StringSerde; import org.apache.samza.util.CommandLine; + /** * Example code to implement window-based counter */ @@ -47,28 +49,28 @@ public class PageViewCounterExample implements StreamApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime appRuntime = ApplicationRuntimes.createStreamApp(new PageViewCounterExample(), config); + ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); appRuntime.start(); appRuntime.waitForFinish(); } @Override - public void init(StreamGraph graph, Config config) { - MessageStream pageViewEvents = null; - pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); - OutputStream> pageViewEventPerMemberStream = - graph.getOutputStream("pageViewEventPerMemberStream", - KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); + public void describe(StreamApplicationSpec graph) { + MessageStream pageViewEvents = null; + pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + OutputStream> pageViewEventPerMemberStream = + graph.getOutputStream("pageViewEventPerMemberStream", + KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); - SupplierFunction initialValue = () -> 0; - FoldLeftFunction foldLeftFn = (m, c) -> c + 1; - pageViewEvents - .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, - null, null) - .setEarlyTrigger(Triggers.repeat(Triggers.count(5))) - .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow") - .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane))) - .sendTo(pageViewEventPerMemberStream); + SupplierFunction initialValue = () -> 0; + FoldLeftFunction foldLeftFn = (m, c) -> c + 1; + pageViewEvents + .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, + null, null) + .setEarlyTrigger(Triggers.repeat(Triggers.count(5))) + .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow") + .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane))) + .sendTo(pageViewEventPerMemberStream); } class PageViewEvent { 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 5f4bccbc24..d6efa5374e 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 @@ -20,11 +20,11 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; import org.apache.samza.runtime.ApplicationRuntime; @@ -44,14 +44,14 @@ public class RepartitionExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new RepartitionExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new RepartitionExample(), config); app.start(); app.waitForFinish(); } @Override - public void init(StreamGraph graph, Config config) { + public void describe(StreamApplicationSpec graph) { MessageStream pageViewEvents = graph.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = 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 65a56b120a..8b79b8b64b 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 @@ -2,11 +2,12 @@ import java.util.Collections; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.TaskApplicationBuilder; +import org.apache.samza.application.TaskApplicationSpec; import org.apache.samza.config.Config; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.ApplicationRuntimes; -import org.apache.samza.task.TaskFactory; +import org.apache.samza.storage.kv.RocksDbTableDescriptor; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.CommandLine; @@ -19,19 +20,20 @@ public class TaskApplicationExample implements TaskApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime appRuntime = ApplicationRuntimes.createTaskApp(new TaskApplicationExample(), config); + ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(new TaskApplicationExample(), config); appRuntime.start(); appRuntime.waitForFinish(); } @Override - public void init(TaskApplicationBuilder appBuilder, Config config) { + public void describe(TaskApplicationSpec appBuilder) { // add input and output streams appBuilder.addInputStreams(Collections.singletonList("myinput")); appBuilder.addOutputStreams(Collections.singletonList("myoutput")); - appBuilder.addTables(Collections.singletonList("mytable")); + TableDescriptor td = new RocksDbTableDescriptor("mytable"); + appBuilder.addTables(Collections.singletonList(td)); // create the task factory based on configuration - appBuilder.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(config)); + appBuilder.setTaskFactory(TaskFactoryUtil.createTaskFactory(appBuilder.getConfig())); } } 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 f25ac76ea7..d57bd63aff 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 @@ -19,15 +19,16 @@ package org.apache.samza.example; +import java.time.Duration; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.FoldLeftFunction; import org.apache.samza.operators.functions.SupplierFunction; -import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.triggers.Triggers; +import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; import org.apache.samza.runtime.ApplicationRuntime; import org.apache.samza.runtime.ApplicationRuntimes; @@ -35,8 +36,6 @@ import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.util.CommandLine; -import java.time.Duration; - /** * Example implementation of a simple user-defined task w/ a window operator. @@ -48,14 +47,14 @@ public class WindowExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new WindowExample(), config); + ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new WindowExample(), config); app.start(); app.waitForFinish(); } @Override - public void init(StreamGraph graph, Config config) { + public void describe(StreamApplicationSpec graph) { SupplierFunction initialValue = () -> 0; FoldLeftFunction counter = (m, c) -> c == null ? 1 : c + 1; MessageStream inputStream = graph.getInputStream("inputStream", new JsonSerdeV2()); diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index e401e8d384..918b4186cb 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -21,6 +21,7 @@ import joptsimple.OptionSet; import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; import org.apache.samza.runtime.ApplicationRuntime; @@ -29,8 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.samza.runtime.ApplicationRunnerMain.*; - /** * {@link ApplicationRunnerMain} was designed for deploying {@link StreamApplication} in yarn * and doesn't work for in standalone. @@ -47,7 +46,8 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationRuntime appRuntime = ApplicationRuntimes.createStreamApp((StreamApplication) Class.forName(config.get(STREAM_APPLICATION_CLASS_CONFIG)).newInstance(), config); + ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime( + (StreamApplication) Class.forName(new ApplicationConfig(config).getAppClass()).newInstance(), config); try { LOGGER.info("Launching stream application: {} to start.", appRuntime); 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 f6e3d5fe6e..eb7c8d5e8a 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 @@ -19,8 +19,7 @@ package org.apache.samza.test.integration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraph; +import org.apache.samza.application.StreamApplicationSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,8 +32,8 @@ public class TestStandaloneIntegrationApplication implements StreamApplication { private static final Logger LOGGER = LoggerFactory.getLogger(TestStandaloneIntegrationApplication.class); @Override - public void init(StreamGraph graph, Config config) { - String inputStream = config.get("input.stream.name"); + public void setup(StreamApplicationSpec graph) { + String inputStream = graph.getConfig().get("input.stream.name"); String outputStreamName = "standaloneIntegrationTestKafkaOutputTopic"; LOGGER.info("Publishing message to: {}.", outputStreamName); graph.getInputStream(inputStream).sendTo(graph.getOutputStream(outputStreamName)); 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 9ed5aff0aa..b7e4ffae5e 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 @@ -24,7 +24,8 @@ import java.util.List; import java.util.Map; import java.util.Random; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; @@ -32,6 +33,8 @@ import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory; import org.apache.samza.operators.KV; import org.apache.samza.operators.functions.MapFunction; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory; import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; import org.apache.samza.test.controlmessages.TestData.PageView; @@ -39,9 +42,9 @@ import org.apache.samza.test.harness.AbstractIntegrationTestHarness; import org.apache.samza.test.util.ArraySystemFactory; import org.apache.samza.test.util.Base64Serializer; - import org.junit.Test; -import static org.junit.Assert.assertEquals; + +import static org.junit.Assert.*; /** * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input. @@ -89,20 +92,29 @@ public void testPipeline() throws Exception { configs.put("serializers.registry.int.class", "org.apache.samza.serializers.IntegerSerdeFactory"); configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName()); - final StreamApplication app = StreamApplications.createStreamApp(new MapConfig(configs)); + class PipelineApplication { + + @Override + public void init(StreamAppSpecImpl appSpec, Config config) { + appSpec.>getInputStream("PageView") + .map(Values.create()) + .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") + .sink((m, collector, coordinator) -> { + received.add(m.getValue()); + }); + } + } - app.>openInput("PageView") - .map(Values.create()) - .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") - .sink((m, collector, coordinator) -> { - received.add(m.getValue()); - }); - app.run(); + final ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new PipelineApplication(), new MapConfig(configs)); + + app.start(); app.waitForFinish(); assertEquals(received.size(), count * partitionCount); } + + public static final class Values { public static > MapFunction create() { return (M m) -> m.getValue(); 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 5c52f242c2..fe900f6641 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 @@ -26,7 +26,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -45,6 +45,9 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.TestStreamProcessorUtil; +import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRuntimes.AppRuntimeImpl; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.TestLocalApplicationRunner; import org.apache.samza.serializers.IntegerSerdeFactory; @@ -68,7 +71,7 @@ import org.junit.Test; import scala.collection.JavaConverters; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.*; public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness { @@ -142,14 +145,21 @@ public void testWatermark() throws Exception { configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName()); List received = new ArrayList<>(); - final StreamApplication app = StreamApplications.createStreamApp(new MapConfig(configs)); - app.>openInput("PageView") - .map(EndOfStreamIntegrationTest.Values.create()) - .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") - .sink((m, collector, coordinator) -> { + class TestStreamApp { + + @Override + public void init(StreamAppSpecImpl appSpec, Config config) { + appSpec.>getInputStream("PageView") + .map(EndOfStreamIntegrationTest.Values.create()) + .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") + .sink((m, collector, coordinator) -> { received.add(m.getValue()); }); - app.run(); + } + } + + final ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new TestStreamApp(), new MapConfig(configs)); + app.start(); Map tasks = getTaskOperationGraphs(app); app.waitForFinish(); @@ -173,8 +183,8 @@ public void testWatermark() throws Exception { assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3); } - Map getTaskOperationGraphs(StreamApplication app) throws Exception { - Field appRunnerField = StreamApplication.class.getDeclaredField("runner"); + Map getTaskOperationGraphs(ApplicationRuntime app) throws Exception { + Field appRunnerField = AppRuntimeImpl.class.getDeclaredField("runner"); appRunnerField.setAccessible(true); LocalApplicationRunner runner = (LocalApplicationRunner) appRunnerField.get(app); StreamProcessor processor = TestLocalApplicationRunner.getProcessors(runner).iterator().next(); 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 307c1b50bb..84a06695f8 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 @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; -import org.apache.samza.application.StreamApplication; import org.apache.samza.operators.KV; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.system.OutgoingMessageEnvelope; diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java index 88e27653f9..0671230409 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java @@ -19,23 +19,21 @@ package org.apache.samza.test.operator; -import org.apache.samza.application.StreamApplication; +import java.util.Arrays; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.serializers.JsonSerdeV2; -import org.apache.samza.test.operator.data.PageView; import org.apache.samza.test.framework.StreamAssert; +import org.apache.samza.test.operator.data.PageView; -import java.util.Arrays; - -public class BroadcastAssertApp implements StreamApplication { +public class BroadcastAssertApp { public static final String INPUT_TOPIC_NAME_PROP = "inputTopicName"; @Override - public void init(StreamGraph graph, Config config) { + public void init(StreamAppSpecImpl graph, Config config) { String inputTopic = config.get(INPUT_TOPIC_NAME_PROP); final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); 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 4ffa212c40..ad4938e071 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 @@ -21,7 +21,7 @@ import java.util.ArrayList; import java.util.List; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -57,14 +57,22 @@ public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); + app.run(); + app.waitForFinish(); + } + + List getIntermediateStreamIds() { + return intermediateStreamIds; + } + + @Override + public void init(StreamAppSpecImpl appSpec, Config config) { String inputTopicName1 = config.get(INPUT_TOPIC_NAME_1_PROP); String inputTopicName2 = config.get(INPUT_TOPIC_NAME_2_PROP); String outputTopic = config.get(OUTPUT_TOPIC_NAME_PROP); - StreamApplication app = StreamApplications.createStreamApp(config); - - MessageStream pageViews = app.openInput(inputTopicName1, new JsonSerdeV2<>(PageView.class)); - MessageStream adClicks = app.openInput(inputTopicName2, new JsonSerdeV2<>(AdClick.class)); + MessageStream pageViews = appSpec.getInputStream(inputTopicName1, new JsonSerdeV2<>(PageView.class)); + MessageStream adClicks = appSpec.getInputStream(inputTopicName2, new JsonSerdeV2<>(AdClick.class)); MessageStream> pageViewsRepartitionedByViewId = pageViews .partitionBy(PageView::getViewId, pv -> pv, @@ -91,21 +99,15 @@ public static void main(String[] args) throws Exception { new StringSerde(), new JsonSerdeV2<>(UserPageAdClick.class)), "userAdClickWindow") .map(windowPane -> KV.of(windowPane.getKey().getKey(), String.valueOf(windowPane.getMessage().size()))) .sink((message, messageCollector, taskCoordinator) -> { - taskCoordinator.commit(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER); - messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue())); - }); + taskCoordinator.commit(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER); + messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue())); + }); intermediateStreamIds.clear(); intermediateStreamIds.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamId()); intermediateStreamIds.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamId()); intermediateStreamIds.add(((IntermediateMessageStreamImpl) userPageAdClicksByUserId).getStreamId()); - app.run(); - app.waitForFinish(); - } - - List getIntermediateStreamIds() { - return intermediateStreamIds; } private static class UserPageViewAdClicksJoiner implements JoinFunction { 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 402cfca513..6918aad37f 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 @@ -20,7 +20,6 @@ package org.apache.samza.test.operator; import java.time.Duration; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.windows.Windows; 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 7ec9c7d7cc..3462689d4a 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 @@ -20,7 +20,6 @@ package org.apache.samza.test.operator; import java.time.Duration; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java index 16fcebdba3..00a5320211 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java @@ -36,7 +36,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.KafkaConfig; import org.apache.samza.config.MapConfig; 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 97ca50148a..977236449a 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 @@ -21,7 +21,6 @@ import java.time.Duration; import java.io.IOException; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; 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 8b483a5939..6035cc33b8 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 @@ -23,7 +23,6 @@ import java.io.ObjectInputStream; import java.io.Serializable; import java.util.concurrent.CountDownLatch; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.MessageStream; diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 460615467d..1acd679f52 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -37,7 +37,6 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; 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 d7f0570e33..12b85a6352 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 @@ -26,7 +26,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java index 8d07570d1d..8990b3f054 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java @@ -33,14 +33,13 @@ import java.util.stream.Collectors; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.MapConfig; import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; import org.apache.samza.operators.KV; -import org.apache.samza.operators.StreamGraph; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.table.Table; @@ -127,7 +126,7 @@ public void deleteAll(Collection keys) { } } - private Table> getCachingTable(Table> actualTable, boolean defaultCache, String id, StreamGraph streamGraph) { + private Table> getCachingTable(Table> actualTable, boolean defaultCache, String id, StreamApplicationSpec streamGraph) { CachingTableDescriptor cachingDesc = new CachingTableDescriptor<>("caching-table-" + id); if (defaultCache) { cachingDesc.withReadTtl(Duration.ofMinutes(5)); diff --git a/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java b/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java index 94c1eca526..1c4b233195 100644 --- a/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java @@ -19,10 +19,9 @@ package org.apache.samza.test.timer; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.TimerFunction; @@ -36,11 +35,11 @@ import java.util.Collections; import java.util.List; -public class TestTimerApp implements StreamApplication { +public class TestTimerApp { public static final String PAGE_VIEWS = "page-views"; @Override - public void init(StreamGraph graph, Config config) { + public void init(StreamApplicationSpec graph, Config config) { final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); final MessageStream pageViews = graph.getInputStream(PAGE_VIEWS, serde); final MessageStream output = pageViews.flatMap(new FlatmapTimerFn()); diff --git a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java index 5456db680c..82b8f37c19 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java @@ -29,7 +29,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.commons.cli.ParseException; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.TaskConfig; From 973eb5261432da50a3540c96a101136f9c854f1d Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 3 Aug 2018 12:08:30 -0700 Subject: [PATCH 07/38] WIP: compiles, still working on LocalContainerRunner refactor --- .../samza/application/ApplicationBase.java | 8 + .../samza/application/ApplicationSpec.java | 15 +- ...n.java => ProcessorLifecycleListener.java} | 8 +- .../samza/application/StreamApplication.java | 2 +- .../samza/application/TaskApplication.java | 2 +- .../application/internal/AppSpecImpl.java | 43 +- .../internal/StreamAppSpecImpl.java | 11 +- .../application/internal/TaskAppSpecImpl.java | 11 +- .../java/org/apache/samza/config/Config.java | 3 +- .../samza/runtime/ApplicationRuntime.java | 4 +- .../samza/runtime/ApplicationRuntimes.java | 8 +- .../runtime/internal/ApplicationRunners.java | 2 +- .../org/apache/samza/task/TaskFactory.java | 3 +- .../internal/TestStreamAppSpecImpl.java | 7 + .../internal/TestTaskAppSpecImpl.java | 7 + .../runtime/TestApplicationRuntimes.java | 7 + .../internal/TestApplicationRunners.java | 7 + .../application/ApplicationClassUtils.java | 4 +- .../samza/operators/StreamGraphSpec.java | 11 - .../runtime/AbstractApplicationRunner.java | 80 ++-- .../runtime/ApplicationRunnerCommandLine.java | 2 +- .../samza/runtime/ApplicationRunnerMain.java | 6 +- .../samza/runtime/LocalApplicationRunner.java | 74 ++-- .../samza/runtime/LocalContainerRunner.java | 137 ++++--- .../runtime/RemoteApplicationRunner.java | 30 +- .../apache/samza/task/TaskFactoryUtil.java | 19 +- .../samza/container/SamzaContainer.scala | 2 +- .../samza/job/local/ThreadJobFactory.scala | 23 +- .../samza/application/TestAppClassUtils.java | 7 + .../runtime/TestApplicationRunnerMain.java | 22 +- .../runtime/TestLocalApplicationRunner.java | 61 +-- .../runtime/TestRemoteApplicationRunner.java | 7 +- .../samza/task/TestTaskFactoryUtil.java | 14 +- .../samza/sql/runner/SamzaSqlApplication.java | 2 +- .../runner/SamzaSqlApplicationRuntime.java | 6 +- .../samza/sql/translator/QueryTranslator.java | 3 +- .../sql/translator/TestQueryTranslator.java | 374 ++++++++++++++++-- .../example/AppWithGlobalConfigExample.java | 2 +- .../samza/example/BroadcastExample.java | 2 +- .../samza/example/KeyValueStoreExample.java | 2 +- .../apache/samza/example/MergeExample.java | 2 +- .../example/OrderShipmentJoinExample.java | 2 +- .../samza/example/PageViewCounterExample.java | 2 +- .../samza/example/RepartitionExample.java | 2 +- .../samza/example/TaskApplicationExample.java | 2 +- .../apache/samza/example/WindowExample.java | 2 +- .../samza/test/framework/TestRunner.java | 22 +- .../LocalApplicationRunnerMain.java | 7 +- .../TestStandaloneIntegrationApplication.java | 2 +- .../processor/TestZkStreamProcessor.java | 6 +- .../TestZkStreamProcessorSession.java | 2 +- .../EndOfStreamIntegrationTest.java | 4 +- .../WatermarkIntegrationTest.java | 2 +- ...reamApplicationIntegrationTestHarness.java | 12 +- .../TestRepartitionJoinWindowApp.java | 8 +- .../operator/TestRepartitionWindowApp.java | 2 +- .../TestZkLocalApplicationRunner.java | 36 +- .../test/integration/StreamTaskTestUtil.scala | 2 +- .../SystemConsumerWithSamzaBench.java | 12 +- 59 files changed, 772 insertions(+), 395 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java rename samza-api/src/main/java/org/apache/samza/application/{LifecycleAwareApplication.java => ProcessorLifecycleListener.java} (56%) create mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java create mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java create mode 100644 samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java create mode 100644 samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java create mode 100644 samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java new file mode 100644 index 0000000000..85938131e6 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java @@ -0,0 +1,8 @@ +package org.apache.samza.application; + +/** + * Created by yipan on 7/11/18. + */ +public interface ApplicationBase { + void describe(S appSpec); +} diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java index 0131caf3fd..55eabeebf0 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java @@ -7,7 +7,7 @@ /** * Created by yipan on 7/19/18. */ -public interface ApplicationSpec { +public interface ApplicationSpec { /** * Get the global unique application ID in the runtime process * @return globally unique application ID @@ -21,14 +21,23 @@ public interface ApplicationSpec { Config getConfig(); /** - * Sets the {@link ContextManager} for this {@link StreamApplicationSpec}. + * Sets the {@link ContextManager} for this {@link ApplicationSpec}. *

    * The provided {@link ContextManager} can be used to setup shared context between the operator functions * within a task instance * * @param contextManager the {@link ContextManager} to use for the {@link StreamApplicationSpec} - * @return the {@link StreamApplicationSpec} with {@code contextManager} set as its {@link ContextManager} + * @return the {@link ApplicationSpec} with {@code contextManager} set as its {@link ContextManager} */ ApplicationSpec withContextManager(ContextManager contextManager); + /** + * Sets the {@link ProcessorLifecycleListener} for this {@link ApplicationSpec}. + * + * @param listener the user implemented {@link ProcessorLifecycleListener} with lifecycle aware methods to be invoked + * before and after the start/stop of the processing logic defined in this {@link ApplicationSpec} + * @return the {@link ApplicationSpec} with {@code listener} set as its {@link ProcessorLifecycleListener} + */ + ApplicationSpec withProcessorLifecycleListener(ProcessorLifecycleListener listener); + } diff --git a/samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java b/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java similarity index 56% rename from samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java rename to samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java index 2a8ac40acb..38b73ad415 100644 --- a/samza-api/src/main/java/org/apache/samza/application/LifecycleAwareApplication.java +++ b/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java @@ -1,10 +1,12 @@ package org.apache.samza.application; +import java.io.Serializable; + + /** - * Created by yipan on 7/11/18. + * Created by yipan on 8/1/18. */ -public interface LifecycleAwareApplication { - void describe(S appSpec); +public interface ProcessorLifecycleListener extends Serializable { default void beforeStart() {} default void afterStart() {} default void beforeStop() {} 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 b1f63d0849..23e1839e67 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 @@ -3,5 +3,5 @@ /** * Created by yipan on 7/20/18. */ -public interface StreamApplication extends LifecycleAwareApplication { +public interface StreamApplication extends ApplicationBase { } 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 fd6b79c207..c3f5e93e08 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 @@ -3,5 +3,5 @@ /** * Created by yipan on 7/11/18. */ -public interface TaskApplication extends LifecycleAwareApplication { +public interface TaskApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java index b1ebc7c59a..44b1ebe0d4 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java @@ -1,23 +1,38 @@ package org.apache.samza.application.internal; import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.LifecycleAwareApplication; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ProcessorLifecycleListener; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.operators.ContextManager; +import org.apache.samza.task.TaskContext; /** * Created by yipan on 7/10/18. */ -abstract class AppSpecImpl implements ApplicationSpec { +public abstract class AppSpecImpl> implements ApplicationSpec { + final Config config; - final T userApp; - ContextManager contextManager; - protected AppSpecImpl(T userApp, Config config) { + // Default to no-op functions in ContextManager + ContextManager contextManager = new ContextManager() { + @Override + public void init(Config config, TaskContext context) { + } + + @Override + public void close() { + } + }; + + // Default to no-op functions in ProcessorLifecycleListener + ProcessorLifecycleListener listener = new ProcessorLifecycleListener() { + }; + + protected AppSpecImpl(Config config) { this.config = config; - this.userApp = userApp; } public static class AppConfig extends MapConfig { @@ -66,16 +81,24 @@ public Config getConfig() { return config; } - public T getUserApp() { - return userApp; + @Override + public S withContextManager(ContextManager contextManager) { + this.contextManager = contextManager; + return (S) this; + } + + @Override + public S withProcessorLifecycleListener(ProcessorLifecycleListener listener) { + this.listener = listener; + return (S) this; } public ContextManager getContextManager() { return contextManager; } - protected void setContextManager(ContextManager contextManager) { - this.contextManager = contextManager; + public ProcessorLifecycleListener getProcessorLifecycleListner() { + return listener; } } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java index e41e5a5271..cbed9f595c 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java @@ -5,7 +5,6 @@ import org.apache.samza.application.StreamApplication; import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; -import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; @@ -18,11 +17,11 @@ /** * Created by yipan on 7/10/18. */ -public class StreamAppSpecImpl extends AppSpecImpl implements StreamApplicationSpec { +public class StreamAppSpecImpl extends AppSpecImpl implements StreamApplicationSpec { final StreamGraph graph; public StreamAppSpecImpl(StreamApplication userApp, Config config) { - super(userApp, config); + super(config); this.graph = createDefaultGraph(config); userApp.describe(this); } @@ -66,12 +65,6 @@ public Table> getTable(TableDescriptor tableDesc) { return this.graph.getTable(tableDesc); } - @Override - public StreamApplicationSpec withContextManager(ContextManager contextManager) { - super.setContextManager(contextManager); - return this; - } - public StreamGraph getGraph() { return graph; } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java index 0e5c0cd892..59e93f1f24 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java @@ -5,7 +5,6 @@ import org.apache.samza.application.TaskApplication; import org.apache.samza.application.TaskApplicationSpec; import org.apache.samza.config.Config; -import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.task.TaskFactory; @@ -13,7 +12,7 @@ /** * Created by yipan on 7/10/18. */ -public class TaskAppSpecImpl extends AppSpecImpl implements TaskApplicationSpec { +public class TaskAppSpecImpl extends AppSpecImpl implements TaskApplicationSpec { TaskFactory taskFactory; final List inputStreams = new ArrayList<>(); @@ -21,7 +20,7 @@ public class TaskAppSpecImpl extends AppSpecImpl implements Tas final List tables = new ArrayList<>(); public TaskAppSpecImpl(TaskApplication userApp, Config config) { - super(userApp, config); + super(config); userApp.describe(this); } @@ -45,12 +44,6 @@ public void addTables(List tables) { this.tables.addAll(tables); } - @Override - public TaskApplicationSpec withContextManager(ContextManager contextManager) { - super.setContextManager(contextManager); - return this; - } - public TaskFactory getTaskFactory() { return taskFactory; } diff --git a/samza-api/src/main/java/org/apache/samza/config/Config.java b/samza-api/src/main/java/org/apache/samza/config/Config.java index 7abe55781a..ac5e3a66a5 100644 --- a/samza-api/src/main/java/org/apache/samza/config/Config.java +++ b/samza-api/src/main/java/org/apache/samza/config/Config.java @@ -19,6 +19,7 @@ package org.apache.samza.config; +import java.io.Serializable; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Arrays; @@ -32,7 +33,7 @@ /** * Store and retrieve named, typed values as configuration for classes implementing this interface. */ -public abstract class Config implements Map { +public abstract class Config implements Map, Serializable { public static final String SENSITIVE_PREFIX = "sensitive."; public static final String SENSITIVE_MASK = "********"; diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java index e0483ec09a..e5bdcf4d12 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java @@ -13,12 +13,12 @@ public interface ApplicationRuntime { /** * Start an application */ - void start(); + void run(); /** * Stop an application */ - void stop(); + void kill(); /** * Get the {@link ApplicationStatus} of an application diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java index cb02ec0535..0cf11f5383 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java @@ -3,7 +3,7 @@ import java.time.Duration; import java.util.Map; import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.LifecycleAwareApplication; +import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; import org.apache.samza.application.internal.StreamAppSpecImpl; @@ -24,7 +24,7 @@ private ApplicationRuntimes() { } - public static final ApplicationRuntime getApplicationRuntime(LifecycleAwareApplication userApp, Config config) { + public static final ApplicationRuntime getApplicationRuntime(ApplicationBase userApp, Config config) { if (userApp instanceof StreamApplication) { return new AppRuntimeImpl(new StreamAppSpecImpl((StreamApplication) userApp, config)); } @@ -45,12 +45,12 @@ private static class AppRuntimeImpl implements ApplicationRuntime { } @Override - public void start() { + public void run() { this.runner.run(appSpec); } @Override - public void stop() { + public void kill() { this.runner.kill(appSpec); } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java index 3922a947df..7089a8babf 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java @@ -34,7 +34,7 @@ static String getAppRunnerCfg() { * Static method to load the {@link ApplicationRunner} * * @param config configuration passed in to initialize the Samza processes - * @return the configure-driven {@link ApplicationRunner} to start the user-defined stream applications + * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications */ public static ApplicationRunner fromConfig(Config config) { AppRunnerConfig appRunnerCfg = new AppRunnerConfig(config); 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 65d03dca36..360a89bd6d 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 @@ -1,5 +1,6 @@ package org.apache.samza.task; +import java.io.Serializable; import org.apache.samza.annotation.InterfaceStability; @@ -7,6 +8,6 @@ * Created by yipan on 7/10/18. */ @InterfaceStability.Stable -public interface TaskFactory { +public interface TaskFactory extends Serializable { T createInstance(); } diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java new file mode 100644 index 0000000000..33056dcb21 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java @@ -0,0 +1,7 @@ +package org.apache.samza.application.internal; + +/** + * Created by yipan on 8/3/18. + */ +public class TestStreamAppSpecImpl { +} diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java new file mode 100644 index 0000000000..388b6138ea --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java @@ -0,0 +1,7 @@ +package org.apache.samza.application.internal; + +/** + * Created by yipan on 8/3/18. + */ +public class TestTaskAppSpecImpl { +} diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java new file mode 100644 index 0000000000..c08d9fe49f --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java @@ -0,0 +1,7 @@ +package org.apache.samza.runtime; + +/** + * Created by yipan on 8/3/18. + */ +public class TestApplicationRuntimes { +} diff --git a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java new file mode 100644 index 0000000000..bd51fc7287 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java @@ -0,0 +1,7 @@ +package org.apache.samza.runtime.internal; + +/** + * Created by yipan on 8/3/18. + */ +public class TestApplicationRunners { +} diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java index a41dd53b73..9cf2d3c017 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java @@ -11,11 +11,11 @@ * Created by yipan on 7/22/18. */ public class ApplicationClassUtils { - public static LifecycleAwareApplication fromConfig(Config config) { + public static ApplicationBase fromConfig(Config config) { ApplicationConfig appConfig = new ApplicationConfig(config); if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { try { - Class appClass = (Class) Class.forName(appConfig.getAppClass()); + Class appClass = (Class) Class.forName(appConfig.getAppClass()); if (StreamApplication.class.isAssignableFrom(appClass) || TaskApplication.class.isAssignableFrom(appClass)) { return appClass.newInstance(); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java index 318db364c7..1ef94ab876 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java @@ -68,7 +68,6 @@ public class StreamGraphSpec implements StreamGraph { private int nextOpNum = 0; private final Set operatorIds = new HashSet<>(); private Serde defaultSerde = new KVSerde(new NoOpSerde(), new NoOpSerde()); -// private ContextManager contextManager = null; public StreamGraphSpec(Config config) { this.config = config; @@ -153,16 +152,6 @@ public Table> getTable(TableDescriptor tableDesc) { return tables.get(tableSpec); } -// @Override -// public StreamApplicationSpec withContextManager(ContextManager contextManager) { -// this.contextManager = contextManager; -// return this; -// } -// -// public ContextManager getContextManager() { -// return this.contextManager; -// } - public OperatorSpecGraph getOperatorSpecGraph() { return new OperatorSpecGraph(this); } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 94451c0218..6ae0e84fe4 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -27,7 +27,6 @@ import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.LifecycleAwareApplication; import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.ApplicationConfig; @@ -67,35 +66,53 @@ public final void addMetricsReporters(Map metricsReport @Override public final void run(ApplicationSpec appSpec) { - LifecycleAwareApplication userApp = getUserApp(appSpec); - userApp.beforeStart(); - getLifecycleMethods(appSpec).run(); - userApp.afterStart(); + getAppRuntimeExecutable(appSpec).run(); } @Override public final ApplicationStatus status(ApplicationSpec appSpec) { - return getLifecycleMethods(appSpec).status(); + return getAppRuntimeExecutable(appSpec).status(); } @Override public final void kill(ApplicationSpec appSpec) { - LifecycleAwareApplication userApp = getUserApp(appSpec); - userApp.beforeStop(); - getLifecycleMethods(appSpec).kill(); - userApp.afterStop(); + getAppRuntimeExecutable(appSpec).kill(); } + @Deprecated @Override public final void waitForFinish(ApplicationSpec appSpec) { - getLifecycleMethods(appSpec).waitForFinish(Duration.ofSeconds(0)); + getAppRuntimeExecutable(appSpec).waitForFinish(Duration.ofSeconds(0)); } @Override public final boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { - return getLifecycleMethods(appSpec).waitForFinish(timeout); + return getAppRuntimeExecutable(appSpec).waitForFinish(timeout); } + interface AppRuntimeExecutable { + + void run(); + + void kill(); + + ApplicationStatus status(); + + /** + * Waits for {@code timeout} duration for the application to finish. + * + * @param timeout time to wait for the application to finish + * @return true - application finished before timeout + * false - otherwise + */ + boolean waitForFinish(Duration timeout); + + } + + abstract AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec); + + abstract AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec); + final StreamManager buildAndStartStreamManager() { StreamManager streamManager = new StreamManager(this.config); streamManager.start(); @@ -148,48 +165,15 @@ final void writePlanJsonFile(String planJson) { } } - protected abstract ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec); - - protected abstract ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec); - - protected interface ApplicationLifecycle { - - void run(); - - void kill(); - - ApplicationStatus status(); - - /** - * Waits for {@code timeout} duration for the application to finish. - * - * @param timeout time to wait for the application to finish - * @return true - application finished before timeout - * false - otherwise - */ - boolean waitForFinish(Duration timeout); - - } - - private ApplicationLifecycle getLifecycleMethods(ApplicationSpec appSpec) { + private AppRuntimeExecutable getAppRuntimeExecutable(ApplicationSpec appSpec) { if (appSpec instanceof StreamAppSpecImpl) { - return getStreamAppLifecycle((StreamAppSpecImpl) appSpec); + return getStreamAppRuntimeExecutable((StreamAppSpecImpl) appSpec); } if (appSpec instanceof TaskAppSpecImpl) { - return getTaskAppLifecycle((TaskAppSpecImpl) appSpec); + return getTaskAppRuntimeExecutable((TaskAppSpecImpl) appSpec); } throw new IllegalArgumentException(String.format("The specified application %s is not valid. " + "Only StreamApplicationSpec and TaskApplicationSpec are supported.", appSpec.getClass().getName())); } - private LifecycleAwareApplication getUserApp(ApplicationSpec appSpec) { - if (appSpec instanceof StreamAppSpecImpl) { - return ((StreamAppSpecImpl) appSpec).getUserApp(); - } - if (appSpec instanceof TaskAppSpecImpl) { - return ((TaskAppSpecImpl) appSpec).getUserApp(); - } - throw new IllegalArgumentException(); - } - } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java index 2e08a8dacf..059d2a5b65 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java @@ -27,7 +27,7 @@ * The class defines the basic command line arguments for Samza command line scripts. */ public class ApplicationRunnerCommandLine extends CommandLine { - public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; start, status, stop.") + public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; start, status, kill.") .withRequiredArg() .ofType(String.class) .describedAs("operation=start") diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 2a6111d202..9c43733a35 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -37,7 +37,7 @@ public class ApplicationRunnerMain { public static class ApplicationRunnerCommandLine extends CommandLine { public OptionSpec operationOpt = - parser().accepts("operation", "The operation to perform; start, status, stop.") + parser().accepts("operation", "The operation to perform; start, status, kill.") .withRequiredArg() .ofType(String.class) .describedAs("operation=start") @@ -60,10 +60,10 @@ public static void main(String[] args) throws Exception { switch (op) { case RUN: - appRuntime.start(); + appRuntime.run(); break; case KILL: - appRuntime.stop(); + appRuntime.kill(); break; case STATUS: System.out.println(appRuntime.status()); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 214dcecadc..780d841641 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -21,9 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import java.time.Duration; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -43,7 +41,6 @@ import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; @@ -51,8 +48,8 @@ import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.system.StreamSpec; +import org.apache.samza.task.StreamOperatorTask; import org.apache.samza.task.TaskFactory; -import org.apache.samza.task.TaskFactoryUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,12 +66,16 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private final CountDownLatch shutdownLatch = new CountDownLatch(1); private final AtomicInteger numProcessorsToStart = new AtomicInteger(); private final AtomicReference failure = new AtomicReference<>(); - private final Map customMetricsReporters; private ApplicationStatus appStatus = ApplicationStatus.New; - final class LocalStreamProcessorLifeCycleListener implements StreamProcessorLifecycleListener { - StreamProcessor processor; + public LocalApplicationRunner(Config config) { + super(config); + this.uid = UUID.randomUUID().toString(); + } + + private final class LocalStreamProcessorLifeCycleListener implements StreamProcessorLifecycleListener { + private StreamProcessor processor; void setProcessor(StreamProcessor processor) { this.processor = processor; @@ -128,30 +129,10 @@ private void shutdownAndNotify() { } } - public LocalApplicationRunner(Config config) { - this(config, new HashMap<>()); - } - - @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec) { - return new TaskAppLifecycle(appSpec); - } - - @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec) { - return new StreamAppLifecycle(appSpec); - } + private class StreamAppExecutable implements AppRuntimeExecutable { + private final StreamAppSpecImpl streamApp; - public LocalApplicationRunner(Config config, Map customMetricsReporters) { - super(config); - this.uid = UUID.randomUUID().toString(); - this.customMetricsReporters = customMetricsReporters; - } - - class StreamAppLifecycle implements ApplicationLifecycle { - final StreamAppSpecImpl streamApp; - - StreamAppLifecycle(StreamAppSpecImpl streamApp) { + private StreamAppExecutable(StreamAppSpecImpl streamApp) { this.streamApp = streamApp; } @@ -188,7 +169,9 @@ public void run() { numProcessorsToStart.set(processors.size()); // 4. start the StreamProcessors + streamApp.getProcessorLifecycleListner().beforeStart(); processors.forEach(StreamProcessor::start); + streamApp.getProcessorLifecycleListner().afterStart(); } catch (Throwable throwable) { appStatus = ApplicationStatus.unsuccessfulFinish(throwable); shutdownLatch.countDown(); @@ -202,7 +185,9 @@ public void run() { @Override public void kill() { + streamApp.getProcessorLifecycleListner().beforeStop(); processors.forEach(StreamProcessor::stop); + streamApp.getProcessorLifecycleListner().afterStop(); } @Override @@ -217,11 +202,11 @@ public boolean waitForFinish(Duration timeout) { } - class TaskAppLifecycle implements ApplicationLifecycle { - final TaskAppSpecImpl appSpec; - StreamProcessor sp; + private class TaskAppExecutable implements AppRuntimeExecutable { + private final TaskAppSpecImpl appSpec; + private StreamProcessor sp; - TaskAppLifecycle(TaskAppSpecImpl appSpec) { + private TaskAppExecutable(TaskAppSpecImpl appSpec) { this.appSpec = appSpec; } @@ -234,12 +219,16 @@ public void run() { numProcessorsToStart.set(1); listener.setProcessor(sp); + appSpec.getProcessorLifecycleListner().beforeStart(); sp.start(); + appSpec.getProcessorLifecycleListner().afterStart(); } @Override public void kill() { + appSpec.getProcessorLifecycleListner().beforeStop(); sp.stop(); + appSpec.getProcessorLifecycleListner().afterStop(); } @Override @@ -335,7 +324,7 @@ StreamProcessor createStreamProcessor( Config config, TaskFactory taskFactory, StreamProcessorLifecycleListener listener) { - return new StreamProcessor(config, customMetricsReporters, taskFactory, listener, null); + return new StreamProcessor(config, this.metricsReporters, taskFactory, listener, null); } /** @@ -351,8 +340,8 @@ StreamProcessor createStreamProcessor( OperatorSpecGraph graph, ContextManager contextManager, StreamProcessorLifecycleListener listener) { - TaskFactory taskFactory = TaskFactoryUtil.createTaskFactory(graph, contextManager); - return new StreamProcessor(config, customMetricsReporters, taskFactory, listener, null); + TaskFactory taskFactory = () -> new StreamOperatorTask(graph, contextManager); + return new StreamProcessor(config, this.metricsReporters, taskFactory, listener, null); } /* package private for testing */ @@ -364,4 +353,15 @@ Set getProcessors() { CountDownLatch getShutdownLatch() { return shutdownLatch; } + + @Override + protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { + return new TaskAppExecutable(appSpec); + } + + @Override + protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { + return new StreamAppExecutable(appSpec); + } + } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 8be0b28cc1..b01d6ea2be 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -25,11 +25,11 @@ import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.ApplicationClassUtils; +import org.apache.samza.application.ApplicationSpec; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.LifecycleAwareApplication; import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.Config; @@ -44,7 +44,8 @@ import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.task.TaskFactoryUtil; +import org.apache.samza.task.StreamOperatorTask; +import org.apache.samza.task.TaskFactory; import org.apache.samza.util.SamzaUncaughtExceptionHandler; import org.apache.samza.util.ScalaJavaUtil; import org.slf4j.Logger; @@ -69,32 +70,22 @@ public class LocalContainerRunner extends AbstractApplicationRunner { private ContainerHeartbeatMonitor containerHeartbeatMonitor; private SamzaContainer container; - public LocalContainerRunner(JobModel jobModel, String containerId) { + LocalContainerRunner(JobModel jobModel, String containerId) { super(jobModel.getConfig()); this.jobModel = jobModel; this.containerId = containerId; } - @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec) { - return new TaskAppLifecycle(appSpec); - } - - @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec) { - return new StreamAppLifecycle(appSpec); - } - - class TaskAppLifecycle implements ApplicationLifecycle { - final TaskAppSpecImpl taskApp; + private class TaskAppExecutable implements AppRuntimeExecutable { + private final TaskAppSpecImpl taskApp; - TaskAppLifecycle(TaskAppSpecImpl taskApp) { + private TaskAppExecutable(TaskAppSpecImpl taskApp) { this.taskApp = taskApp; } @Override public void run() { - Object taskFactory = this.taskApp.getTaskFactory(); + TaskFactory taskFactory = this.taskApp.getTaskFactory(); container = SamzaContainer$.MODULE$.apply( containerId, @@ -122,9 +113,13 @@ public void onContainerFailed(Throwable t) { } }); + taskApp.getProcessorLifecycleListner().beforeStart(); + taskApp.getProcessorLifecycleListner().afterStart(); startContainerHeartbeatMonitor(); container.run(); stopContainerHeartbeatMonitor(); + taskApp.getProcessorLifecycleListner().beforeStop(); + taskApp.getProcessorLifecycleListner().afterStop(); if (containerRunnerException != null) { log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); @@ -151,16 +146,16 @@ public boolean waitForFinish(Duration timeout) { } - class StreamAppLifecycle implements ApplicationLifecycle { - final StreamAppSpecImpl streamApp; + private class StreamAppExecutable implements AppRuntimeExecutable { + private final StreamAppSpecImpl streamApp; - StreamAppLifecycle(StreamAppSpecImpl streamApp) { + private StreamAppExecutable(StreamAppSpecImpl streamApp) { this.streamApp = streamApp; } @Override public void run() { - Object taskFactory = TaskFactoryUtil.createTaskFactory(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), + TaskFactory taskFactory = () -> new StreamOperatorTask(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), streamApp.getContextManager()); container = SamzaContainer$.MODULE$.apply( @@ -189,9 +184,13 @@ public void onContainerFailed(Throwable t) { } }); + streamApp.getProcessorLifecycleListner().beforeStart(); + streamApp.getProcessorLifecycleListner().afterStart(); startContainerHeartbeatMonitor(); container.run(); stopContainerHeartbeatMonitor(); + streamApp.getProcessorLifecycleListner().beforeStop(); + streamApp.getProcessorLifecycleListner().afterStop(); if (containerRunnerException != null) { log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); @@ -218,6 +217,46 @@ public boolean waitForFinish(Duration timeout) { } + private static class ContainerAppRuntimeImpl implements ApplicationRuntime { + private final ApplicationSpec appSpec; + private final LocalContainerRunner runner; + + public ContainerAppRuntimeImpl(ApplicationSpec appSpec, LocalContainerRunner runner) { + this.appSpec = appSpec; + this.runner = runner; + } + + @Override + public void run() { + this.runner.run(appSpec); + } + + @Override + public void kill() { + this.runner.kill(appSpec); + } + + @Override + public ApplicationStatus status() { + return this.runner.status(appSpec); + } + + @Override + public void waitForFinish() { + this.runner.waitForFinish(appSpec, Duration.ofSeconds(0)); + } + + @Override + public boolean waitForFinish(Duration timeout) { + return this.runner.waitForFinish(appSpec, timeout); + } + + @Override + public void addMetricsReporters(Map metricsReporters) { + this.runner.addMetricsReporters(metricsReporters); + } + } + // only invoked by legacy applications w/o user-defined main public static void main(String[] args) throws Exception { Thread.setDefaultUncaughtExceptionHandler( @@ -247,10 +286,20 @@ public static void main(String[] args) throws Exception { ApplicationRuntime appSpec = createAppRuntime(ApplicationClassUtils.fromConfig(config), new LocalContainerRunner(jobModel, containerId), config); - appSpec.start(); + appSpec.run(); appSpec.waitForFinish(); } + @Override + protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { + return new TaskAppExecutable(appSpec); + } + + @Override + protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { + return new StreamAppExecutable(appSpec); + } + private void startContainerHeartbeatMonitor() { String coordinatorUrl = System.getenv(ShellCommandConfig.ENV_COORDINATOR_URL()); String executionEnvContainerId = System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID()); @@ -278,7 +327,7 @@ private void stopContainerHeartbeatMonitor() { } } - private static ApplicationRuntime createAppRuntime(LifecycleAwareApplication userApp, LocalContainerRunner runner, Config config) { + private static ApplicationRuntime createAppRuntime(ApplicationBase userApp, LocalContainerRunner runner, Config config) { if (userApp instanceof StreamApplication) { return new ContainerAppRuntimeImpl(new StreamAppSpecImpl((StreamApplication) userApp, config), runner); } @@ -290,44 +339,4 @@ private static ApplicationRuntime createAppRuntime(LifecycleAwareApplication use + "are supported", userApp.getClass().getName())); } - private static class ContainerAppRuntimeImpl implements ApplicationRuntime { - private final ApplicationSpec appSpec; - private final LocalContainerRunner runner; - - public ContainerAppRuntimeImpl(ApplicationSpec appSpec, LocalContainerRunner runner) { - this.appSpec = appSpec; - this.runner = runner; - } - - @Override - public void start() { - this.runner.run(appSpec); - } - - @Override - public void stop() { - this.runner.kill(appSpec); - } - - @Override - public ApplicationStatus status() { - return this.runner.status(appSpec); - } - - @Override - public void waitForFinish() { - this.runner.waitForFinish(appSpec, Duration.ofSeconds(0)); - } - - @Override - public boolean waitForFinish(Duration timeout) { - return this.runner.waitForFinish(appSpec, timeout); - } - - @Override - public void addMetricsReporters(Map metricsReporters) { - this.runner.addMetricsReporters(metricsReporters); - } - } - } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 602e06e686..d8152af90b 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -53,21 +53,11 @@ public RemoteApplicationRunner(Config config) { super(config); } - @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskAppSpecImpl appSpec) { - return new TaskAppLifecycle(appSpec); - } - - @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamAppSpecImpl appSpec) { - return new StreamAppLifecycle(appSpec); - } - - class TaskAppLifecycle implements ApplicationLifecycle { + class TaskAppExecutable implements AppRuntimeExecutable { final TaskAppSpecImpl taskApp; final JobRunner jobRunner; - TaskAppLifecycle(TaskAppSpecImpl appSpec) { + TaskAppExecutable(TaskAppSpecImpl appSpec) { this.taskApp = appSpec; this.jobRunner = new JobRunner(config); } @@ -94,10 +84,10 @@ public boolean waitForFinish(Duration timeout) { } - class StreamAppLifecycle implements ApplicationLifecycle { + class StreamAppExecutable implements AppRuntimeExecutable { final StreamAppSpecImpl streamApp; - StreamAppLifecycle(StreamAppSpecImpl appSpec) { + StreamAppExecutable(StreamAppSpecImpl appSpec) { this.streamApp = appSpec; } @@ -149,7 +139,7 @@ public void kill() { runner.kill(); }); } catch (Throwable t) { - throw new SamzaException("Failed to stop application", t); + throw new SamzaException("Failed to kill application", t); } finally { if (streamManager != null) { streamManager.stop(); @@ -216,6 +206,16 @@ public boolean waitForFinish(Duration timeout) { } + @Override + protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { + return new TaskAppExecutable(appSpec); + } + + @Override + protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { + return new StreamAppExecutable(appSpec); + } + /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { JobRunner runner = new JobRunner(jobConfig); ApplicationStatus status = runner.status(); diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index c7b35f39e2..3094d44ad5 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -32,23 +32,12 @@ import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; /** - * This class provides utility functions to load task factory classes based on config, and to wrap {@link StreamTaskFactory} in {@link AsyncStreamTaskFactory} - * when running {@link StreamTask}s in multi-thread mode + * This class provides utility functions to load task factory classes based on config, and to wrap {@link StreamTaskFactory} + * in {@link AsyncStreamTaskFactory} when running {@link StreamTask}s in multi-thread mode */ public class TaskFactoryUtil { private static final Logger log = LoggerFactory.getLogger(TaskFactoryUtil.class); - /** - * This method creates a task factory class based on the {@link OperatorSpecGraph} - * - * @param specGraph the {@link OperatorSpecGraph} - * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph} - * @return a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} - */ - public static TaskFactory createTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) { - return createStreamOperatorTaskFactory(specGraph, contextManager); - } - /** * This method creates a task factory class based on the configuration * @@ -59,10 +48,6 @@ public static TaskFactory createTaskFactory(Config config) { return fromTaskClassConfig(config); } - private static StreamTaskFactory createStreamOperatorTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) { - return () -> new StreamOperatorTask(specGraph, contextManager); - } - /** * Create {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} based on the configured task.class. * @param config the {@link Config} diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 47b73c1c32..71e62fd0e1 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -125,7 +125,7 @@ object SamzaContainer extends Logging { jobModel: JobModel, config: Config, customReporters: Map[String, MetricsReporter] = Map[String, MetricsReporter](), - taskFactory: Object) = { + taskFactory: TaskFactory[_]) = { val containerModel = jobModel.getContainers.get(containerId) val containerName = "samza-container-%s" format containerId val maxChangeLogStreamPartitions = jobModel.maxChangeLogStreamPartitions diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index 6365d3a025..eb570c020b 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,6 +19,8 @@ package org.apache.samza.job.local +import org.apache.samza.application.internal.{StreamAppSpecImpl, TaskAppSpecImpl} +import org.apache.samza.application.{ApplicationClassUtils, StreamApplication, TaskApplication} import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ @@ -30,7 +32,7 @@ import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} import org.apache.samza.operators.StreamGraphSpec import org.apache.samza.storage.ChangelogStreamManager -import org.apache.samza.task.TaskFactoryUtil +import org.apache.samza.task._ import org.apache.samza.util.Logging import scala.collection.JavaConversions._ @@ -72,13 +74,24 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val containerId = "0" val jmxServer = new JmxServer - // TODO: ThreadJobFactory does not support launch StreamApplication. Launching user-defined StreamApplication is via new - // user program w/ main(). - val taskFactory = TaskFactoryUtil.createTaskFactory(config) + val taskFactory : TaskFactory[_] = ApplicationClassUtils.fromConfig(config) match { + case app if (app.isInstanceOf[TaskApplication]) => { + val appSpec = new TaskAppSpecImpl(app.asInstanceOf[TaskApplication], config) + appSpec.getTaskFactory + } + case app if (app.isInstanceOf[StreamApplication]) => { + val appSpec = new StreamAppSpecImpl(app.asInstanceOf[StreamApplication], config) + new StreamTaskFactory { + override def createInstance(): StreamTask = + new StreamOperatorTask(appSpec.getGraph.asInstanceOf[StreamGraphSpec].getOperatorSpecGraph, appSpec.getContextManager) + } + } + } // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job. config.getTaskOpts match { - case Some(taskOpts) => warn("%s was specified in config, but is not being used because job is being executed with ThreadJob. You probably want to run %s=%s." format (TASK_JVM_OPTS, STREAM_JOB_FACTORY_CLASS, classOf[ProcessJobFactory].getName)) + case Some(taskOpts) => warn("%s was specified in config, but is not being used because job is being executed with ThreadJob. " + + "You probably want to run %s=%s." format (TASK_JVM_OPTS, STREAM_JOB_FACTORY_CLASS, classOf[ProcessJobFactory].getName)) case _ => None } diff --git a/samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java b/samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java new file mode 100644 index 0000000000..7d40ebd9e2 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java @@ -0,0 +1,7 @@ +package org.apache.samza.application; + +/** + * Created by yipan on 8/3/18. + */ +public class TestAppClassUtils { +} diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index c6440b7a79..0a502bb011 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -19,8 +19,9 @@ package org.apache.samza.runtime; import java.time.Duration; -import org.apache.samza.application.internal.StreamApplicationSpecRuntime; -import org.apache.samza.application.internal.TaskApplicationSpecRuntime; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.job.ApplicationStatus; import org.junit.Test; @@ -38,7 +39,7 @@ public void TestRunOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationRunnerMain.APP_CLASS_CFG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", + "-config", ApplicationConfig.APP_CLASS + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts" }); @@ -53,7 +54,7 @@ public void TestKillOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationRunnerMain.APP_CLASS_CFG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", + "-config", ApplicationConfig.APP_CLASS + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts", "--operation=kill" }); @@ -69,7 +70,7 @@ public void TestStatusOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationRunnerMain.APP_CLASS_CFG + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", + "-config", ApplicationConfig.APP_CLASS + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts", "--operation=status" }); @@ -99,7 +100,7 @@ private ApplicationStatus status() { return ApplicationStatus.Running; } - class TestAppLifecycle implements ApplicationLifecycle { + class TestAppExecutable implements AppRuntimeExecutable { @Override public void run() { @@ -123,14 +124,15 @@ public boolean waitForFinish(Duration timeout) { } @Override - protected ApplicationLifecycle getTaskAppLifecycle(TaskApplicationSpecRuntime appSpec) { - return new TestAppLifecycle(); + AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { + return new TestAppExecutable(); } @Override - protected ApplicationLifecycle getStreamAppLifecycle(StreamApplicationSpecRuntime appSpec) { - return new TestAppLifecycle(); + AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { + return new TestAppExecutable(); } + } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 5eb70307ee..4481130ac5 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -27,6 +27,11 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationClassUtils; +import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.TaskApplicationSpec; +import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -42,6 +47,7 @@ import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.system.StreamSpec; +import org.apache.samza.task.TaskFactory; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; @@ -49,13 +55,17 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.*; +import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.doReturn; @@ -75,7 +85,7 @@ public void testStreamCreation() throws Exception { Map config = new HashMap<>(); LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config))); - StreamApplication app = mock(StreamApplication.class); + StreamApplicationSpec appSpec = mock(StreamApplicationSpec.class); StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(runner).buildAndStartStreamManager(); @@ -91,8 +101,8 @@ public void testStreamCreation() PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig); try { - runner.run(app); - runner.waitForFinish(); + runner.run(appSpec); + runner.waitForFinish(appSpec); } catch (Throwable t) { assertNotNull(t); //no jobs exception } @@ -112,7 +122,7 @@ public void testStreamCreationWithCoordination() LocalApplicationRunner localRunner = new LocalApplicationRunner(new MapConfig(config)); LocalApplicationRunner runner = spy(localRunner); - StreamApplication app = mock(StreamApplication.class); + StreamApplicationSpec appSpec = mock(StreamApplicationSpec.class); StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(runner).buildAndStartStreamManager(); @@ -135,8 +145,8 @@ public void testStreamCreationWithCoordination() .thenReturn(coordinationUtils); try { - runner.run(app); - runner.waitForFinish(); + runner.run(appSpec); + runner.waitForFinish(appSpec); } catch (Throwable t) { assertNotNull(t); //no jobs exception } @@ -157,26 +167,31 @@ public void testRunStreamTask() config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); - LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config)); + Config samzaConfig = new MapConfig(config); + LocalApplicationRunner runner = new LocalApplicationRunner(samzaConfig); + TaskApplicationSpec appSpec = new TaskAppSpecImpl((TaskApplication) ApplicationClassUtils.fromConfig(samzaConfig), samzaConfig); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = + ArgumentCaptor captor1 = + ArgumentCaptor.forClass(TaskFactory.class); + ArgumentCaptor captor2 = ArgumentCaptor.forClass(StreamProcessorLifecycleListener.class); doAnswer(i -> { - StreamProcessorLifecycleListener listener = captor.getValue(); + StreamProcessorLifecycleListener listener = captor2.getValue(); listener.onStart(); listener.onShutdown(); return null; }).when(sp).start(); LocalApplicationRunner spy = spy(runner); - doReturn(sp).when(spy).createStreamProcessor(any(Config.class), captor.capture()); + doReturn(sp).when(spy).createStreamProcessor(any(Config.class), captor1.capture(), captor2.capture()); + - spy.runTask(); + spy.run(appSpec); - assertEquals(ApplicationStatus.SuccessfulFinish, spy.status(null)); + assertEquals(ApplicationStatus.SuccessfulFinish, spy.status(appSpec)); } @Test @@ -185,7 +200,7 @@ public void testRunComplete() final Map config = new HashMap<>(); config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config))); - StreamApplication app = mock(StreamApplication.class); + StreamApplicationSpec app = mock(StreamApplicationSpec.class); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -211,7 +226,7 @@ public void testRunComplete() doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); runner.run(app); - runner.waitForFinish(); + runner.waitForFinish(app); assertEquals(runner.status(app), ApplicationStatus.SuccessfulFinish); verify(streamManager).stop(); @@ -223,7 +238,7 @@ public void testRunFailure() final Map config = new HashMap<>(); config.put(ApplicationConfig.PROCESSOR_ID, "0"); LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config))); - StreamApplication app = mock(StreamApplication.class); + StreamApplicationSpec app = mock(StreamApplicationSpec.class); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -247,7 +262,7 @@ public void testRunFailure() try { runner.run(app); - runner.waitForFinish(); + runner.waitForFinish(app); } catch (Throwable th) { assertNotNull(th); } @@ -319,18 +334,20 @@ public void testGeneratePlanIdWithDifferentStreamSpecs() { @Test public void testWaitForFinishReturnsBeforeTimeout() { LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig()); + StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); long timeoutInMs = 1000; runner.getShutdownLatch().countDown(); - boolean finished = runner.waitForFinish(Duration.ofMillis(timeoutInMs)); + boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(timeoutInMs)); assertTrue("Application did not finish before the timeout.", finished); } @Test public void testWaitForFinishTimesout() { LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig()); + StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); long timeoutInMs = 100; - boolean finished = runner.waitForFinish(Duration.ofMillis(timeoutInMs)); + boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(timeoutInMs)); assertFalse("Application finished before the timeout.", finished); } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java index 2ef2b33d7d..1c8aeeead2 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java @@ -20,6 +20,7 @@ package org.apache.samza.runtime; import java.time.Duration; +import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.job.ApplicationStatus; @@ -36,18 +37,20 @@ public class TestRemoteApplicationRunner { @Test public void testWaitForFinishReturnsBeforeTimeout() { RemoteApplicationRunner runner = spy(new RemoteApplicationRunner(new MapConfig())); + StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); doReturn(ApplicationStatus.SuccessfulFinish).when(runner).getApplicationStatus(any(JobConfig.class)); - boolean finished = runner.waitForFinish(Duration.ofMillis(5000)); + boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(5000)); assertTrue("Application did not finish before the timeout.", finished); } @Test public void testWaitForFinishTimesout() { RemoteApplicationRunner runner = spy(new RemoteApplicationRunner(new MapConfig())); + StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); doReturn(ApplicationStatus.Running).when(runner).getApplicationStatus(any(JobConfig.class)); - boolean finished = runner.waitForFinish(Duration.ofMillis(1000)); + boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(1000)); assertFalse("Application finished before the timeout.", finished); } } diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 70aada6712..c17cc6ffac 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -19,24 +19,18 @@ package org.apache.samza.task; import java.lang.reflect.Field; +import java.util.HashMap; import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.testUtils.TestAsyncStreamTask; import org.apache.samza.testUtils.TestStreamTask; import org.junit.Test; -import java.util.HashMap; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; /** * Test methods to create {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} based on task class configuration @@ -93,7 +87,7 @@ public void testAsyncStreamTask() { @Test public void testFinalizeTaskFactory() throws NoSuchFieldException, IllegalAccessException { - Object mockFactory = mock(Object.class); + TaskFactory mockFactory = mock(TaskFactory.class); try { TaskFactoryUtil.finalizeTaskFactory(mockFactory, true, null); fail("Should have failed with validation"); 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 ddcc2735d9..668a5157b7 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 @@ -36,7 +36,7 @@ public class SamzaSqlApplication implements StreamApplication { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplication.class); @Override - public void setup(StreamApplicationSpec appSpec) { + public void describe(StreamApplicationSpec appSpec) { try { SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(appSpec.getConfig()); QueryTranslator queryTranslator = new QueryTranslator(sqlConfig); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java index dae4548fa0..4106a6f8d8 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java @@ -109,17 +109,17 @@ public static Config computeSamzaConfigs(Boolean localRunner, Config config) { public void runAndWaitForFinish() { Validate.isTrue(localRunner, "This method can be called only in standalone mode."); - start(); + run(); waitForFinish(); } @Override - public void start() { + public void run() { runner.run(appSpec); } @Override - public void stop() { + public void kill() { runner.kill(appSpec); } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index db63330564..870fc13ac0 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -31,6 +31,7 @@ import org.apache.calcite.rel.logical.LogicalProject; import org.apache.samza.SamzaException; import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; @@ -97,7 +98,7 @@ public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplication sqlConfig.getUdfMetadata()); final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig); final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery()); - final TranslatorContext context = new TranslatorContext(appSpec, relRoot, executionContext, this.converters); + final TranslatorContext context = new TranslatorContext(((StreamAppSpecImpl)appSpec).getGraph(), relRoot, executionContext, this.converters); final RelNode node = relRoot.project(); final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver(); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index 8def60b725..12193421e5 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -24,20 +24,19 @@ import java.util.HashSet; import java.util.Map; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; +import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.sql.data.SamzaSqlExecutionContext; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory; -import org.apache.samza.sql.runner.SamzaSqlApplication; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; @@ -45,8 +44,12 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentCaptor; import org.mockito.internal.util.reflection.Whitebox; +import static org.mockito.Mockito.*; + + public class TestQueryTranslator { // Helper functions to validate the cloned copies of TranslatorContext and SamzaSqlExecutionContext @@ -89,12 +92,24 @@ public void testTranslate() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - // FIXME: construction of QueryTranslator and other variables within SamzaSqlApplication is now coupled within - // the constructor of StreamApplicationSpec. - StreamAppSpecImpl - graphSpec = new StreamAppSpecImpl(new SamzaSqlApplication(), new MapConfig(config)); - translator.translate(queryInfo, graphSpec); - OperatorSpecGraph specGraph = ((StreamGraphSpec) graphSpec.getGraph()).getOperatorSpecGraph(); + StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); + OperatorSpecGraph specGraph = ((StreamGraphSpec) mockSpec.getGraph()).getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -112,7 +127,7 @@ public void testTranslate() { Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("SIMPLE1", inputPhysicalName); - validatePerTaskContextInit(graphSpec, samzaConfig); + validatePerTaskContextInit(mockSpec, samzaConfig); } private void validatePerTaskContextInit(StreamAppSpecImpl graphSpec, Config samzaConfig) { @@ -145,7 +160,22 @@ public void testTranslateComplex() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); @@ -163,7 +193,7 @@ public void testTranslateComplex() { Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("COMPLEX1", inputPhysicalName); - validatePerTaskContextInit(graphSpec, samzaConfig); + validatePerTaskContextInit(mockSpec, samzaConfig); } @Test @@ -177,7 +207,22 @@ public void testTranslateSubQuery() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); @@ -195,7 +240,7 @@ public void testTranslateSubQuery() { Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("COMPLEX1", inputPhysicalName); - validatePerTaskContextInit(graphSpec, samzaConfig); + validatePerTaskContextInit(mockSpec, samzaConfig); } @Test (expected = SamzaException.class) @@ -213,7 +258,22 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -232,7 +292,22 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); } @Test (expected = IllegalStateException.class) @@ -251,7 +326,22 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -270,7 +360,21 @@ public void testTranslateStreamTableJoinWithThetaCondition() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -287,7 +391,21 @@ public void testTranslateStreamTableCrossJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -306,7 +424,21 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -326,7 +458,21 @@ public void testTranslateStreamTableJoinWithSubQuery() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -345,7 +491,21 @@ public void testTranslateTableTableJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -364,7 +524,21 @@ public void testTranslateStreamStreamJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -383,7 +557,21 @@ public void testTranslateJoinWithIncorrectLeftJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -402,7 +590,21 @@ public void testTranslateJoinWithIncorrectRightJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -425,7 +627,21 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test (expected = SamzaException.class) @@ -444,7 +660,21 @@ public void testTranslateStreamTableInnerJoinWithUdf() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } @Test @@ -463,7 +693,22 @@ public void testTranslateStreamTableInnerJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); @@ -497,7 +742,7 @@ public void testTranslateStreamTableInnerJoin() { Assert.assertEquals("kafka", input3System); Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); - validatePerTaskContextInit(graphSpec, samzaConfig); + validatePerTaskContextInit(mockSpec, samzaConfig); } @Test @@ -516,7 +761,22 @@ public void testTranslateStreamTableLeftJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); @@ -551,7 +811,7 @@ public void testTranslateStreamTableLeftJoin() { Assert.assertEquals("kafka", input3System); Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); - validatePerTaskContextInit(graphSpec, samzaConfig); + validatePerTaskContextInit(mockSpec, samzaConfig); } @Test @@ -570,7 +830,22 @@ public void testTranslateStreamTableRightJoin() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); @@ -605,7 +880,7 @@ public void testTranslateStreamTableRightJoin() { Assert.assertEquals("kafka", input3System); Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); - validatePerTaskContextInit(graphSpec, samzaConfig); + validatePerTaskContextInit(mockSpec, samzaConfig); } @Test @@ -624,7 +899,22 @@ public void testTranslateGroupBy() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + + translator.translate(queryInfo, mockSpec); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); Assert.assertEquals(1, specGraph.getInputOperators().size()); @@ -648,6 +938,20 @@ public void testTranslateGroupByWithSumAggregator() { SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - translator.translate(queryInfo, graphSpec); + StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + // The following steps are used to simulate the actual logic in the constructor + when(mockSpec.getConfig()).thenReturn(samzaConfig); + when(mockSpec.getGraph()).thenReturn(graphSpec); + ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); + when(mockSpec.getOutputStream(streamCaptor.capture())).then( + invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); + ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); + when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); + ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); + when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { + when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); + return null; + }); + translator.translate(queryInfo, mockSpec); } } 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 5cbc8a4068..bb6035f627 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 @@ -48,7 +48,7 @@ public static void main(String[] args) { ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new AppWithGlobalConfigExample(), config); app.addMetricsReporters(new HashMap<>()); - app.start(); + app.run(); app.waitForFinish(); } 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 6ffc126941..68535dc537 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 @@ -42,7 +42,7 @@ public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new BroadcastExample(), config); - app.start(); + app.run(); app.waitForFinish(); } 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 be02559faf..4f508ecb75 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 @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new KeyValueStoreExample(), config); - app.start(); + app.run(); app.waitForFinish(); } 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 155996c242..db85833a93 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 @@ -39,7 +39,7 @@ public static void main(String[] args) throws Exception { Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new MergeExample(), config); - app.start(); + app.run(); app.waitForFinish(); } 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 5400850fa3..5f90584bfa 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 @@ -44,7 +44,7 @@ public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new OrderShipmentJoinExample(), config); - app.start(); + app.run(); app.waitForFinish(); } 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 08daeec691..3c24a2b6cb 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 @@ -50,7 +50,7 @@ public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); - appRuntime.start(); + appRuntime.run(); appRuntime.waitForFinish(); } 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 d6efa5374e..bab7e5b552 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 @@ -46,7 +46,7 @@ public static void main(String[] args) throws Exception { Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new RepartitionExample(), config); - app.start(); + app.run(); app.waitForFinish(); } 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 8b79b8b64b..0b507fcb56 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 @@ -21,7 +21,7 @@ public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(new TaskApplicationExample(), config); - appRuntime.start(); + appRuntime.run(); appRuntime.waitForFinish(); } 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 d57bd63aff..6fbd9ba948 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 @@ -49,7 +49,7 @@ public static void main(String[] args) throws Exception { Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new WindowExample(), config); - app.start(); + app.run(); app.waitForFinish(); } diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index 6e647d9705..6ee5d79a04 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -31,8 +31,12 @@ import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationSpec; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; import org.apache.samza.config.InMemorySystemConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -287,14 +291,18 @@ public void run(Duration timeout) { Preconditions.checkState(!timeout.isZero() || !timeout.isNegative(), "Timeouts should be positive"); final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - if (app == null) { - runner.runTask(); - } else { - runner.run(app); - } - boolean timedOut = !runner.waitForFinish(timeout); + ApplicationSpec appSpec = app == null ? new TaskAppSpecImpl(spec -> spec.setTaskFactory(() -> { + try { + return taskClass.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new ConfigException(e); + } + }), new MapConfig(configs)) : + new StreamAppSpecImpl(app, new MapConfig(configs)); + runner.run(appSpec); + boolean timedOut = !runner.waitForFinish(appSpec, timeout); Assert.assertFalse("Timed out waiting for application to finish", timedOut); - ApplicationStatus status = runner.status(app); + ApplicationStatus status = runner.status(appSpec); if (status.getStatusCode() == ApplicationStatus.StatusCode.UnsuccessfulFinish) { throw new SamzaException(ExceptionUtils.getStackTrace(status.getThrowable())); } diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index 918b4186cb..31692d4c1c 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -20,8 +20,8 @@ package org.apache.samza.test.integration; import joptsimple.OptionSet; +import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; import org.apache.samza.runtime.ApplicationRuntime; @@ -46,12 +46,11 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime( - (StreamApplication) Class.forName(new ApplicationConfig(config).getAppClass()).newInstance(), config); + ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); try { LOGGER.info("Launching stream application: {} to start.", appRuntime); - appRuntime.start(); + appRuntime.run(); appRuntime.waitForFinish(); } catch (Exception e) { LOGGER.error("Exception occurred when running application: {}.", appRuntime, e); 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 eb7c8d5e8a..f1a235ead8 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 @@ -32,7 +32,7 @@ public class TestStandaloneIntegrationApplication implements StreamApplication { private static final Logger LOGGER = LoggerFactory.getLogger(TestStandaloneIntegrationApplication.class); @Override - public void setup(StreamApplicationSpec graph) { + public void describe(StreamApplicationSpec graph) { String inputStream = graph.getConfig().get("input.stream.name"); String outputStreamName = "standaloneIntegrationTestKafkaOutputTopic"; LOGGER.info("Publishing message to: {}.", outputStreamName); diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java index 9e14d6dd1b..72ebfcff40 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java @@ -213,14 +213,14 @@ private void testStreamProcessor(String[] processorIds) { waitUntilMessagesLeftN(totalEventsToGenerate - messageCount); CountDownLatch containerStopped2 = sp2.containerShutdownLatch; - // stop the first processor + // kill the first processor stopProcessor(stopLatch1); // wait until it's really down waitForProcessorToStartStop(waitStop1); - // processor2 will stop it container and start again. - // We wait for the container's stop to make sure we can count EXACTLY how many messages it reads. + // processor2 will kill it container and start again. + // We wait for the container's kill to make sure we can count EXACTLY how many messages it reads. LOG.info("containerStopped latch = " + containerStopped2); waitForProcessorToStartStop(containerStopped2); diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java index 619149ab99..1533fc32b2 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java @@ -97,7 +97,7 @@ private void testStreamProcessorWithSessionRestart(String[] processorIds) { // make sure it consumes all the messages from the first batch waitUntilMessagesLeftN(totalEventsToGenerate - messageCount); - // Get the container stop latch to be able to check when a container is stopped. + // Get the container kill latch to be able to check when a container is stopped. // New containerShutdownLatch is created after each onNewJobModel, // so we need to get the current one, before it changed.. for (int i = 0; i < processorIds.length; i++) { 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 b7e4ffae5e..a16423bbd9 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 @@ -48,7 +48,7 @@ /** * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input. - * It verifies the pipeline will stop and the number of output messages should equal to the input. + * It verifies the pipeline will kill and the number of output messages should equal to the input. */ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness { @@ -107,7 +107,7 @@ public void init(StreamAppSpecImpl appSpec, Config config) { final ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new PipelineApplication(), new MapConfig(configs)); - app.start(); + app.run(); app.waitForFinish(); assertEquals(received.size(), count * partitionCount); 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 fe900f6641..eb1ed2254d 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 @@ -159,7 +159,7 @@ public void init(StreamAppSpecImpl appSpec, Config config) { } final ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new TestStreamApp(), new MapConfig(configs)); - app.start(); + app.run(); Map tasks = getTaskOperationGraphs(app); app.waitForFinish(); diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java index 00a5320211..a2d325b862 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java @@ -73,7 +73,7 @@ * State persistence: {@link #tearDown()} clears all associated state (including topics and metadata) in Kafka and * Zookeeper. Hence, the state is not durable across invocations of {@link #tearDown()}
    * - * Execution model: {@link StreamApplication}s are start as their own {@link org.apache.samza.job.local.ThreadJob}s. + * Execution model: {@link StreamApplication}s are run as their own {@link org.apache.samza.job.local.ThreadJob}s. * Similarly, embedded Kafka servers and Zookeeper servers are start as their own threads. * {@link #produceMessage(String, int, String, String)} and {@link #consumeMessages(Collection, int)} are blocking calls. * @@ -240,15 +240,15 @@ protected RunApplicationContext runApplication(String userAppClass, configMap.put("task.window.ms", "1000"); configMap.put("task.checkpoint.factory", TestStreamManager.MockCheckpointManagerFactory.class.getName()); - // This is to prevent tests from taking a long time to stop after they're done. The issue is that - // tearDown currently doesn't call runner.stop(app), and shuts down the Kafka and ZK servers immediately. + // This is to prevent tests from taking a long time to kill after they're done. The issue is that + // tearDown currently doesn't call runner.kill(app), and shuts down the Kafka and ZK servers immediately. // The test process then exits, triggering the SamzaContainer shutdown hook, which in turn tries to flush any // store changelogs, which then get stuck trying to produce to the stopped Kafka server. - // Calling runner.stop doesn't work since RemoteApplicationRunner creates a new ThreadJob instance when - // stop is called. We can't use LocalApplicationRunner since ZkJobCoordinator doesn't currently create + // Calling runner.kill doesn't work since RemoteApplicationRunner creates a new ThreadJob instance when + // kill is called. We can't use LocalApplicationRunner since ZkJobCoordinator doesn't currently create // changelog streams. Hence we just force an unclean shutdown here to. This _should be_ OK // since the test method has already executed by the time the shutdown hook is called. The side effect is - // that buffered state (e.g. changelog contents) might not be flushed correctly after the test start. + // that buffered state (e.g. changelog contents) might not be flushed correctly after the test run. configMap.put("task.shutdown.ms", "1"); if (overriddenConfigs != null) { diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java index b681d08ec1..c89a5e21cb 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java @@ -72,7 +72,7 @@ public void testRepartitionJoinWindowAppWithoutDeletionOnCommit() throws Excepti initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - // start the application + // run the application RepartitionJoinWindowApp app = new RepartitionJoinWindowApp(); String appName = "UserPageAdClickCounter"; Map configs = new HashMap<>(); @@ -81,7 +81,7 @@ public void testRepartitionJoinWindowAppWithoutDeletionOnCommit() throws Excepti configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_2_PROP, inputTopicName2); configs.put(RepartitionJoinWindowApp.OUTPUT_TOPIC_NAME_PROP, outputTopicName); - // start the application + // run the application Thread runThread = runApplication(app.getClass().getName(), appName, new MapConfig(configs)).getRunThread(); // consume and validate result @@ -103,7 +103,7 @@ public void testRepartitionJoinWindowAppAndDeleteMessagesOnCommit() throws Excep initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - // start the application + // run the application RepartitionJoinWindowApp app = new RepartitionJoinWindowApp(); final String appName = "UserPageAdClickCounter2"; Map configs = new HashMap<>(); @@ -160,7 +160,7 @@ public void testBroadcastApp() throws InterruptedException { configs.put(BroadcastAssertApp.INPUT_TOPIC_NAME_PROP, inputTopicName1); initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); - // start the application + // run the application Thread runThread = runApplication(BroadcastAssertApp.class.getName(), "BroadcastTest", new MapConfig(configs)).getRunThread(); runThread.interrupt(); diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java index f3f7d8cffb..e7c4aa07de 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java @@ -66,7 +66,7 @@ public void testRepartitionedSessionWindowCounter() throws Exception { configs.put(String.format("streams.%s.samza.msg.serde", INPUT_TOPIC), "string"); configs.put(String.format("streams.%s.samza.key.serde", INPUT_TOPIC), "string"); - // start the application + // run the application Thread runThread = runApplication(RepartitionWindowApp.class.getName(), APP_NAME, new MapConfig(configs)).getRunThread(); // consume and validate result diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 7a0f488224..cb0a207a16 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -233,7 +233,7 @@ private Map buildStreamApplicationConfigMap(String systemName, S // CountDownLatch secondProcessorRegistered = new CountDownLatch(1); // // zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { -// // When streamApp2 with id: PROCESSOR_IDS[1] is registered, start processing message in streamApp1. +// // When streamApp2 with id: PROCESSOR_IDS[1] is registered, run processing message in streamApp1. // if (currentChilds.contains(PROCESSOR_IDS[1])) { // secondProcessorRegistered.countDown(); // } @@ -279,9 +279,9 @@ private Map buildStreamApplicationConfigMap(String systemName, S // assertEquals(previousJobModel[0], updatedJobModel); // assertEquals(new MapConfig(), updatedJobModel.getConfig()); // assertEquals(NUM_KAFKA_EVENTS, processedMessagesLatch.getCount()); -// streamApp1.stop(); +// streamApp1.kill(); // streamApp1.waitForFinish(); -// streamApp2.stop(); +// streamApp2.kill(); // streamApp2.waitForFinish(); // assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); // assertEquals(streamApp2.status(), ApplicationStatus.UnsuccessfulFinish); @@ -315,7 +315,7 @@ private Map buildStreamApplicationConfigMap(String systemName, S // final CountDownLatch secondProcessorRegistered = new CountDownLatch(1); // // zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { -// // When streamApp2 with id: PROCESSOR_IDS[1] is registered, start processing message in streamApp1. +// // When streamApp2 with id: PROCESSOR_IDS[1] is registered, run processing message in streamApp1. // if (currentChilds.contains(PROCESSOR_IDS[1])) { // secondProcessorRegistered.countDown(); // } @@ -341,7 +341,7 @@ private Map buildStreamApplicationConfigMap(String systemName, S // } // }; // -// // This is the latch for the messages received by streamApp1. Since streamApp1 is start first, it gets one event +// // This is the latch for the messages received by streamApp1. Since streamApp1 is run first, it gets one event // // redelivered due to re-balancing done by Zk after the streamApp2 joins (See the callback above). // CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2 + 1); // @@ -383,9 +383,9 @@ private Map buildStreamApplicationConfigMap(String systemName, S // processedMessagesLatch.await(); // // assertEquals(ApplicationStatus.Running, localApplicationRunner2.status(streamApp2)); -// streamApp1.stop(); +// streamApp1.kill(); // streamApp1.waitForFinish(); -// streamApp2.stop(); +// streamApp2.kill(); // streamApp2.waitForFinish(); // assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); // } @@ -429,7 +429,7 @@ private Map buildStreamApplicationConfigMap(String systemName, S // assertEquals(PROCESSOR_IDS[0], processorIdsFromZK.get(0)); // // // Kill the leader. Since streamApp1 is the first to join the cluster, it's the leader. -// streamApp1.stop(); +// streamApp1.kill(); // streamApp1.waitForFinish(); // // assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); @@ -449,10 +449,10 @@ private Map buildStreamApplicationConfigMap(String systemName, S // assertEquals(Sets.newHashSet("0000000001", "0000000002"), jobModel.getContainers().keySet()); // assertEquals(2, jobModel.getContainers().size()); // -// streamApp2.stop(); +// streamApp2.kill(); // streamApp2.waitForFinish(); // assertEquals(streamApp2.status(), ApplicationStatus.SuccessfulFinish); -// streamApp3.stop(); +// streamApp3.kill(); // streamApp3.waitForFinish(); // assertEquals(streamApp3.status(), ApplicationStatus.SuccessfulFinish); // } @@ -476,11 +476,11 @@ private Map buildStreamApplicationConfigMap(String systemName, S // streamApp1.run(); // streamApp2.run(); // -// // Wait for message processing to start in both the processors. +// // Wait for message processing to run in both the processors. // processedMessagesLatch1.await(); // processedMessagesLatch2.await(); // -// // Create a stream app with same processor id as SP2 and start it. It should fail. +// // Create a stream app with same processor id as SP2 and run it. It should fail. // publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[2]); // kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); // StreamApplication @@ -488,10 +488,10 @@ private Map buildStreamApplicationConfigMap(String systemName, S // // Fail when the duplicate processor joins. // expectedException.expect(SamzaException.class); // try { -// streamApp3.start(); +// streamApp3.run(); // } finally { -// streamApp1.stop(); -// streamApp2.stop(); +// streamApp1.kill(); +// streamApp2.kill(); // // streamApp1.waitForFinish(); // streamApp2.waitForFinish(); @@ -548,7 +548,7 @@ private Map buildStreamApplicationConfigMap(String systemName, S // processedMessagesLatch1 = new CountDownLatch(1); // publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); // streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, streamApplicationCallback, kafkaEventsConsumedLatch, applicationConfig1); -// streamApp1.start(); +// streamApp1.run(); // // processedMessagesLatch1.await(); // @@ -559,10 +559,10 @@ private Map buildStreamApplicationConfigMap(String systemName, S // assertEquals(Integer.parseInt(jobModelVersion) + 1, Integer.parseInt(newJobModelVersion)); // assertEquals(jobModel.getContainers(), newJobModel.getContainers()); // -// streamApp2.stop(); +// streamApp2.kill(); // streamApp2.waitForFinish(); // assertEquals(streamApp2.status(), ApplicationStatus.SuccessfulFinish); -// streamApp1.stop(); +// streamApp1.kill(); // streamApp1.waitForFinish(); // assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); // } diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala index 864d2e57ff..7c02f37648 100644 --- a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala +++ b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala @@ -194,7 +194,7 @@ object StreamTaskTestUtil { } /* This class implement the base utility to implement an integration test for StreamTask - * It implements helper functions to start/stop the job, send messages to a task, and read all messages from a topic + * It implements helper functions to start/kill the job, send messages to a task, and read all messages from a topic */ class StreamTaskTestUtil { import StreamTaskTestUtil._ diff --git a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java index 82b8f37c19..ad2155c3a6 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java @@ -29,8 +29,11 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.commons.cli.ParseException; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.functions.MapFunction; @@ -67,12 +70,13 @@ public void start() throws IOException, InterruptedException { LocalApplicationRunner runner = new LocalApplicationRunner(config); super.start(); MessageConsumer consumeFn = new MessageConsumer(); - StreamApplication app = (graph, config) -> { - MessageStream stream = graph.getInputStream(streamId); + StreamApplication app = spec -> { + MessageStream stream = spec.getInputStream(streamId); stream.map(consumeFn); }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(app, new MapConfig()); - runner.run(app); + runner.run(appSpec); while (consumeFn.getEventsConsumed() < totalEvents) { Thread.sleep(10); @@ -80,7 +84,7 @@ public void start() throws IOException, InterruptedException { Instant endTime = Instant.now(); - runner.kill(app); + runner.kill(appSpec); System.out.println("\n*******************"); System.out.println(String.format("Started at %s Ending at %s ", consumeFn.startTime, endTime)); From f20cdcda68ebf96924ece4ad16fbe25a9b370323 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Mon, 6 Aug 2018 01:32:03 -0700 Subject: [PATCH 08/38] WIP: adding unit tests. Pending update on StreamProcessorLifecycleListener, LocalContainerRunner, and SamzaContainerListener --- .../samza/application/ApplicationBase.java | 23 ++++- .../samza/application/ApplicationSpec.java | 22 ++++- .../ProcessorLifecycleListener.java | 42 +++++++- .../samza/application/StreamApplication.java | 20 +++- .../application/StreamApplicationSpec.java | 4 +- .../samza/application/TaskApplication.java | 20 +++- .../application/TaskApplicationSpec.java | 41 +++++++- .../application/internal/AppSpecImpl.java | 57 ++++++++--- .../internal/StreamAppSpecImpl.java | 31 +++++- .../application/internal/TaskAppSpecImpl.java | 55 ++++++++++- .../apache/samza/operators/StreamGraph.java | 22 ++++- .../samza/runtime/ApplicationRuntime.java | 32 ++++-- .../samza/runtime/ApplicationRuntimes.java | 20 +++- .../runtime/internal/ApplicationRunner.java | 18 +++- .../runtime/internal/ApplicationRunners.java | 28 +++++- .../internal/AppSpecImplTestBase.java | 56 +++++++++++ .../internal/TestStreamAppSpecImpl.java | 98 ++++++++++++++++++- .../internal/TestTaskAppSpecImpl.java | 80 ++++++++++++++- .../samza/operators/StreamGraphSpec.java | 85 ++++++++++++++++ .../runtime/TestApplicationRuntimes.java | 48 ++++++++- .../internal/TestApplicationRunner.java | 67 +++++++++++++ .../internal/TestApplicationRunners.java | 38 ++++++- .../container/SamzaContainerListener.java | 4 + .../samza/processor/StreamProcessor.java | 14 +++ .../StreamProcessorLifecycleListener.java | 4 + .../samza/runtime/LocalApplicationRunner.java | 43 +++++--- .../samza/runtime/LocalContainerRunner.java | 33 +++++-- .../samza/container/SamzaContainer.scala | 7 ++ 28 files changed, 939 insertions(+), 73 deletions(-) create mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java create mode 100644 samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java create mode 100644 samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java index 85938131e6..77717b6bb0 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java @@ -1,7 +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.application; /** - * Created by yipan on 7/11/18. + * The base interface for all user-implemented applications in Samza. The main processing logic of the user application + * should be implemented in {@link ApplicationBase#describe(ApplicationSpec)} method. Sub-classes {@link StreamApplication} + * and {@link TaskApplication} are specific interfaces for applications written in high-level DAG and low-level task APIs, + * respectively. */ public interface ApplicationBase { void describe(S appSpec); diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java index 55eabeebf0..79f3fd7949 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application; import org.apache.samza.config.Config; @@ -5,7 +23,9 @@ /** - * Created by yipan on 7/19/18. + * The base interface class to create the specification of a user application in Samza. Sub-classes {@link StreamApplicationSpec} + * and {@link TaskApplicationSpec} are specific interfaces for applications written in high-level DAG and low-level task APIs, + * respectively. */ public interface ApplicationSpec { /** diff --git a/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java b/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java index 38b73ad415..cbe56e5a0e 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java +++ b/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java @@ -1,14 +1,46 @@ +/* + * 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.application; -import java.io.Serializable; - - /** - * Created by yipan on 8/1/18. + * This interface defines methods that are invoked in various different stages of an application runtime instance + * where it actually runs the processing logic in local process (i.e. as a standalone process, or a container process + * in YARN NodeManager). User can implement this interface to instantiate/release shared objects in the local process. */ -public interface ProcessorLifecycleListener extends Serializable { +public interface ProcessorLifecycleListener { + /** + * User defined initialization before any processor in an application runtime instance is started + */ default void beforeStart() {} + + /** + * User defined callback after all processors in an application runtime instance are started + */ default void afterStart() {} + + /** + * User defined callback before any processor in an application runtime instance is stopped + */ default void beforeStop() {} + + /** + * User defined callback after all processors in an application runtime instance are stopped + */ default void afterStop() {} } 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 23e1839e67..da3b81aa45 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 @@ -1,7 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application; /** - * Created by yipan on 7/20/18. + * The interface to implement user applications defining a high-level DAG as the main stream processing logic. */ public interface StreamApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java index 2f5a5c1127..87f45ce506 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java @@ -19,13 +19,11 @@ package org.apache.samza.application; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.StreamGraph; /** - * Provides access to {@link MessageStream}s and {@link OutputStream}s used to describe application logic. + * The interface class to create the specification of a user application as high-level DAG in Samza. */ @InterfaceStability.Evolving public interface StreamApplicationSpec extends ApplicationSpec, StreamGraph { 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 c3f5e93e08..48be7d8887 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 @@ -1,7 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application; /** - * Created by yipan on 7/11/18. + * The interface to implement user applications defining a low-level task as the main stream processing logic. */ public interface TaskApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java index d7dba831e6..e3f3b91c8c 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application; import java.util.List; @@ -6,16 +24,37 @@ /** - * Created by yipan on 7/19/18. + * The interface class to create the specification of a user application as low-level task in Samza. */ public interface TaskApplicationSpec extends ApplicationSpec { + /** + * Sets the {@link TaskFactory} for the user application. The {@link TaskFactory#createInstance()} creates task instance + * that implements the processing logic of the user application. + * + * @param factory the user implemented {@link TaskFactory} including the low-level task processing logic + */ void setTaskFactory(TaskFactory factory); + /** + * Adds the input streams to the user application. + * + * @param inputStreams the list of streamIds for the input streams + */ void addInputStreams(List inputStreams); + /** + * Adds the output streams to the user application. + * + * @param outputStreams the list of streamIds for the output streams + */ void addOutputStreams(List outputStreams); + /** + * Adds the {@link TableDescriptor}s used in the application + * + * @param tables the list of {@link TableDescriptor}s + */ void addTables(List tables); } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java index 44b1ebe0d4..c18f32e342 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application.internal; import org.apache.samza.application.ApplicationSpec; @@ -10,7 +28,9 @@ /** - * Created by yipan on 7/10/18. + * This is the base class that implements interface {@link ApplicationSpec}. This base class contains the common objects + * that are used by both high-level and low-level API applications, such as {@link Config}, {@link ContextManager}, and + * {@link ProcessorLifecycleListener}. */ public abstract class AppSpecImpl> implements ApplicationSpec { @@ -31,41 +51,36 @@ public void close() { ProcessorLifecycleListener listener = new ProcessorLifecycleListener() { }; - protected AppSpecImpl(Config config) { + AppSpecImpl(Config config) { this.config = config; } - public static class AppConfig extends MapConfig { + static class AppConfig extends MapConfig { - public static final String APP_NAME = "app.name"; - public static final String APP_ID = "app.id"; - public static final String APP_CLASS = "app.class"; + static final String APP_NAME = "app.name"; + static final String APP_ID = "app.id"; - public static final String JOB_NAME = "job.name"; - public static final String JOB_ID = "job.id"; + static final String JOB_NAME = "job.name"; + static final String JOB_ID = "job.id"; - public AppConfig(Config config) { + AppConfig(Config config) { super(config); } - public String getAppName() { + String getAppName() { return get(APP_NAME, get(JOB_NAME)); } - public String getAppId() { + String getAppId() { return get(APP_ID, get(JOB_ID, "1")); } - public String getAppClass() { - return get(APP_CLASS, null); - } - /** * Returns full application id * * @return full app id */ - public String getGlobalAppId() { + String getGlobalAppId() { return String.format("app-%s-%s", getAppName(), getAppId()); } @@ -93,10 +108,20 @@ public S withProcessorLifecycleListener(ProcessorLifecycleListener listener) { return (S) this; } + /** + * Get the user-implemented {@link ContextManager} object associated with this application + * + * @return the {@link ContextManager} object + */ public ContextManager getContextManager() { return contextManager; } + /** + * Get the user-implemented {@link ProcessorLifecycleListener} object associated with this application + * + * @return the {@link ProcessorLifecycleListener} object + */ public ProcessorLifecycleListener getProcessorLifecycleListner() { return listener; } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java index cbed9f595c..53269e6962 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application.internal; import java.lang.reflect.Constructor; @@ -15,7 +33,9 @@ /** - * Created by yipan on 7/10/18. + * This class implements interface {@link StreamApplicationSpec}. In addition to the common objects for an application + * defined in {@link AppSpecImpl}, this class also includes the high-level DAG {@link StreamGraph} object that user will + * use to create the processing logic in DAG. */ public class StreamAppSpecImpl extends AppSpecImpl implements StreamApplicationSpec { final StreamGraph graph; @@ -26,10 +46,10 @@ public StreamAppSpecImpl(StreamApplication userApp, Config config) { userApp.describe(this); } - private StreamApplicationSpec createDefaultGraph(Config config) { + private StreamGraph createDefaultGraph(Config config) { try { Constructor constructor = Class.forName("org.apache.samza.operators.StreamGraphSpec").getConstructor(Config.class); // *sigh* - return (StreamApplicationSpec) constructor.newInstance(config); + return (StreamGraph) constructor.newInstance(config); } catch (Exception e) { throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); } @@ -65,6 +85,11 @@ public Table> getTable(TableDescriptor tableDesc) { return this.graph.getTable(tableDesc); } + /** + * Get the user-defined high-level DAG {@link StreamGraph} object + * + * @return the {@link StreamGraph} object defined by the user application + */ public StreamGraph getGraph() { return graph; } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java index 59e93f1f24..8be3ea83e8 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java @@ -1,6 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application.internal; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.samza.application.TaskApplication; import org.apache.samza.application.TaskApplicationSpec; @@ -10,7 +29,10 @@ /** - * Created by yipan on 7/10/18. + * This class implements interface {@link TaskApplicationSpec}. In addition to the common objects for an application + * defined in {@link AppSpecImpl}, this class also includes the low-level {@link TaskFactory} object that creates + * user-defined task instances, the lists of input/output streams, and the list of {@link TableDescriptor}s used in + * the application. */ public class TaskAppSpecImpl extends AppSpecImpl implements TaskApplicationSpec { @@ -44,7 +66,38 @@ public void addTables(List tables) { this.tables.addAll(tables); } + /** + * Get the user-defined {@link TaskFactory} + * @return the {@link TaskFactory} object + */ public TaskFactory getTaskFactory() { return taskFactory; } + + /** + * Get the input streams to this application + * + * @return the list of input streamIds + */ + public List getInputStreams() { + return Collections.unmodifiableList(this.inputStreams); + } + + /** + * Get the output streams to this application + * + * @return the list of output streamIds + */ + public List getOutputStreams() { + return Collections.unmodifiableList(this.outputStreams); + } + + /** + * Get the {@link TableDescriptor}s used in this application + * + * @return the list of {@link TableDescriptor}s + */ + public List getTables() { + return Collections.unmodifiableList(this.tables); + } } diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java index 3bd5dbd424..f19ef07b82 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java @@ -1,12 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.operators; +import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.serializers.Serde; import org.apache.samza.table.Table; /** - * Created by yipan on 7/20/18. + * Provides access to {@link MessageStream}s, {@link OutputStream}s, and {@link Table}s used to describe application logic. */ +@InterfaceStability.Evolving public interface StreamGraph { /** diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java index e5bdcf4d12..1b7580d4e9 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.runtime; import java.time.Duration; @@ -7,33 +25,33 @@ /** - * Created by yipan on 7/11/18. + * The primary execution methods of a runtime instance of the user application. */ public interface ApplicationRuntime { /** - * Start an application + * Start a runtime instance of the application */ void run(); /** - * Stop an application + * Stop a runtime instance of the application */ void kill(); /** - * Get the {@link ApplicationStatus} of an application + * Get the {@link ApplicationStatus} of a runtime instance of the application * @return the runtime status of the application */ ApplicationStatus status(); /** - * Wait the application to complete. + * Wait the runtime instance of the application to complete. * This method will block until the application completes. */ void waitForFinish(); /** - * Wait the application to complete with a {@code timeout} + * Wait the runtime instance of the application to complete with a {@code timeout} * * @param timeout the time to block to wait for the application to complete * @return true if the application completes within timeout; false otherwise @@ -41,7 +59,7 @@ public interface ApplicationRuntime { boolean waitForFinish(Duration timeout); /** - * Method to add a set of customized {@link MetricsReporter}s in the application + * Method to add a set of customized {@link MetricsReporter}s in the application runtime instance * * @param metricsReporters the map of customized {@link MetricsReporter}s objects to be used */ diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java index 0cf11f5383..1218bd193b 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.runtime; import java.time.Duration; @@ -16,7 +34,7 @@ /** - * Created by yipan on 7/11/18. + * Creates {@link ApplicationRuntime} instances based on configuration and user-implemented {@link ApplicationBase} */ public class ApplicationRuntimes { diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java index 5e290bda73..9c86fe3478 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java @@ -27,15 +27,31 @@ /** - * The primary means of managing execution of user applications defined in {@link ApplicationSpec} at runtime. + * The primary means of managing execution of user applications deployed in various runtime environments. */ @InterfaceStability.Evolving public interface ApplicationRunner { + /** + * Launch the application defined in {@link ApplicationSpec} + * + * @param appSpec the user defined {@link ApplicationSpec} + */ void run(ApplicationSpec appSpec); + /** + * Stop the application already deployed in a runtime environment + * + * @param appSpec the user defined {@link ApplicationSpec} + */ void kill(ApplicationSpec appSpec); + /** + * Query the status of the application deployed in a runtime environment + * + * @param appSpec the user defined {@link ApplicationSpec} + * @return the current status of a deployed application + */ ApplicationStatus status(ApplicationSpec appSpec); @Deprecated diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java index 7089a8babf..f92a2cd616 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.runtime.internal; import java.lang.reflect.Constructor; @@ -6,10 +24,14 @@ /** - * Created by yipan on 7/10/18. + * Creates {@link ApplicationRunner} instances based on configuration. */ public class ApplicationRunners { + private ApplicationRunners() { + + } + static class AppRunnerConfig { private static final String APP_RUNNER_CFG = "app.runner.class"; private static final String DEFAULT_APP_RUNNER = "org.apache.samza.runtime.RemoteApplicationRunner"; @@ -24,10 +46,6 @@ String getAppRunnerClass() { return this.config.getOrDefault(APP_RUNNER_CFG, DEFAULT_APP_RUNNER); } - static String getAppRunnerCfg() { - return APP_RUNNER_CFG; - } - } /** diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java b/samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java new file mode 100644 index 0000000000..6be532aa8c --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.application.internal; + +import org.apache.samza.application.ProcessorLifecycleListener; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.Config; +import org.apache.samza.operators.ContextManager; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + + +/** + * Base class for unit tests for {@link AppSpecImpl} + */ +public class AppSpecImplTestBase { + @Test + public void testContextManager() { + ContextManager cntxMan = mock(ContextManager.class); + Config mockConf = mock(Config.class); + StreamApplication testApp = appSpec -> { + appSpec.withContextManager(cntxMan); + }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); + assertEquals(appSpec.getContextManager(), cntxMan); + } + + @Test + public void testProcessorLifecycleListener() { + ProcessorLifecycleListener listener = mock(ProcessorLifecycleListener.class); + Config mockConf = mock(Config.class); + StreamApplication testApp = appSpec -> { + appSpec.withProcessorLifecycleListener(listener); + }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); + assertEquals(appSpec.getProcessorLifecycleListner(), listener); + } +} diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java index 33056dcb21..738a24f874 100644 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java @@ -1,7 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application.internal; +import java.util.ArrayList; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.Config; +import org.apache.samza.operators.StreamGraphSpec; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.StringSerde; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + + /** - * Created by yipan on 8/3/18. + * Unit test for {@link StreamAppSpecImpl} */ -public class TestStreamAppSpecImpl { +public class TestStreamAppSpecImpl extends AppSpecImplTestBase { + @Test + public void testConstructor() { + StreamApplication mockApp = mock(StreamApplication.class); + Config mockConf = mock(Config.class); + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(mockApp, mockConf); + verify(mockApp, times(1)).describe(appSpec); + assertEquals(mockConf, appSpec.config); + assertTrue(appSpec.graph instanceof StreamGraphSpec); + } + + @Test + public void testGetInputStream() { + Serde testSerde = new StringSerde(); + Config mockConf = mock(Config.class); + StreamApplication testApp = appSpec -> { + appSpec.getInputStream("myinput1"); + appSpec.getInputStream("myinput2", testSerde); + }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); + assertEquals(((StreamGraphSpec) appSpec.graph).inputStreams, + new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }); + assertEquals(((StreamGraphSpec) appSpec.graph).inputSerdes.get("myinput2"), testSerde); + } + + @Test + public void testGetOutputStream() { + Serde testSerde = new StringSerde(); + Config mockConf = mock(Config.class); + StreamApplication testApp = appSpec -> { + appSpec.getOutputStream("myoutput1"); + appSpec.getOutputStream("myoutput2", testSerde); + }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); + assertEquals(((StreamGraphSpec) appSpec.graph).outputStreams, + new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }); + assertEquals(((StreamGraphSpec) appSpec.graph).outputSerdes.get("myoutput2"), testSerde); + } + + @Test + public void testGetTable() { + TableDescriptor mockTd = mock(TableDescriptor.class); + Config mockConf = mock(Config.class); + StreamApplication testApp = appSpec -> { + appSpec.getTable(mockTd); + }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); + assertEquals(((StreamGraphSpec) appSpec.graph).tables, + new ArrayList() { { this.add(mockTd); } }); + } + + @Test + public void testSetDefaultSerde() { + Serde testSerde = new StringSerde(); + Config mockConf = mock(Config.class); + StreamApplication testApp = appSpec -> { + appSpec.setDefaultSerde(testSerde); + }; + StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); + assertEquals(((StreamGraphSpec) appSpec.graph).defaultSerde, testSerde); + } } diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java index 388b6138ea..6665591453 100644 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java @@ -1,7 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application.internal; +import java.util.ArrayList; +import java.util.List; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.config.Config; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.task.TaskFactory; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + + /** - * Created by yipan on 8/3/18. + * Unit test for {@link TaskAppSpecImpl} */ -public class TestTaskAppSpecImpl { +public class TestTaskAppSpecImpl extends AppSpecImplTestBase { + @Test + public void testConstructor() { + TaskApplication mockApp = mock(TaskApplication.class); + Config mockConf = mock(Config.class); + TaskAppSpecImpl appSpec = new TaskAppSpecImpl(mockApp, mockConf); + verify(mockApp, times(1)).describe(appSpec); + assertEquals(mockConf, appSpec.config); + } + + @Test + public void testAddInputStreams() { + List testInputs = new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }; + TaskApplication testApp = appSpec -> appSpec.addInputStreams(testInputs); + Config mockConf = mock(Config.class); + TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); + assertEquals(appSpec.getInputStreams(), testInputs); + } + + @Test + public void testAddOutputStreams() { + List testOutputs = new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }; + TaskApplication testApp = appSpec -> appSpec.addOutputStreams(testOutputs); + Config mockConf = mock(Config.class); + TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); + assertEquals(appSpec.getOutputStreams(), testOutputs); + } + + @Test + public void testAddTables() { + List testTables = new ArrayList() { { this.add(mock(TableDescriptor.class)); } }; + TaskApplication testApp = appSpec -> appSpec.addTables(testTables); + Config mockConf = mock(Config.class); + TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); + assertEquals(appSpec.getTables(), testTables); + } + + @Test + public void testSetTaskFactory() { + TaskFactory mockTf = mock(TaskFactory.class); + TaskApplication testApp = appSpec -> appSpec.setTaskFactory(mockTf); + Config mockConf = mock(Config.class); + TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); + assertEquals(appSpec.getTaskFactory(), mockTf); + } } diff --git a/samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java new file mode 100644 index 0000000000..654aab3810 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.samza.config.Config; +import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; + +import static org.mockito.Mockito.mock; + + +/** + * Test class only to enable loading a fake impl of {@link StreamGraphSpec} for unit test in samza-api module. The real + * implementation of {@link StreamGraphSpec} is in samza-core module. + */ +public class StreamGraphSpec implements StreamGraph { + public final Config config; + public Serde defaultSerde; + public final List inputStreams = new ArrayList<>(); + public final Map inputSerdes = new HashMap<>(); + public final List outputStreams = new ArrayList<>(); + public final Map outputSerdes = new HashMap<>(); + public final List tables = new ArrayList<>(); + + public StreamGraphSpec(Config config) { + this.config = config; + } + + @Override + public void setDefaultSerde(Serde serde) { + this.defaultSerde = serde; + } + + @Override + public MessageStream getInputStream(String streamId, Serde serde) { + inputStreams.add(streamId); + inputSerdes.put(streamId, serde); + return mock(MessageStream.class); + } + + @Override + public MessageStream getInputStream(String streamId) { + inputStreams.add(streamId); + return mock(MessageStream.class); + } + + @Override + public OutputStream getOutputStream(String streamId, Serde serde) { + outputStreams.add(streamId); + outputSerdes.put(streamId, serde); + return mock(OutputStream.class); + } + + @Override + public OutputStream getOutputStream(String streamId) { + outputStreams.add(streamId); + return mock(OutputStream.class); + } + + @Override + public Table> getTable(TableDescriptor tableDesc) { + tables.add(tableDesc); + return mock(Table.class); + } +} diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java index c08d9fe49f..b232de46d8 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java @@ -1,7 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.runtime; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.runtime.internal.ApplicationRunner; +import org.apache.samza.runtime.internal.TestApplicationRunner; +import org.junit.Test; +import org.mockito.internal.util.reflection.Whitebox; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + + /** - * Created by yipan on 8/3/18. + * Unit test for {@link ApplicationRuntimes} */ public class TestApplicationRuntimes { + @Test + public void testGetApplicationRuntime() { + StreamApplication mockApp = mock(StreamApplication.class); + Map configMap = new HashMap<>(); + configMap.put("app.runner.class", TestApplicationRunner.class.getName()); + Config config = new MapConfig(configMap); + ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(mockApp, config); + StreamApplicationSpec appSpec = (StreamApplicationSpec) Whitebox.getInternalState(appRuntime, "appSpec"); + ApplicationRunner appRunner = (ApplicationRunner) Whitebox.getInternalState(appRuntime, "runner"); + assertEquals(appSpec.getConfig(), config); + assertTrue(appRunner instanceof TestApplicationRunner); + } } diff --git a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java b/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java new file mode 100644 index 0000000000..3e41f70234 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.runtime.internal; + +import java.time.Duration; +import java.util.Map; +import org.apache.samza.application.ApplicationSpec; +import org.apache.samza.config.Config; +import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; + + +/** + * Test class for {@link ApplicationRunners} unit test + */ +public class TestApplicationRunner implements ApplicationRunner { + + public TestApplicationRunner(Config config) { + + } + + @Override + public void run(ApplicationSpec appSpec) { + + } + + @Override + public void kill(ApplicationSpec appSpec) { + + } + + @Override + public ApplicationStatus status(ApplicationSpec appSpec) { + return null; + } + + @Override + public void waitForFinish(ApplicationSpec appSpec) { + + } + + @Override + public boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { + return false; + } + + @Override + public void addMetricsReporters(Map metricsReporters) { + + } +} diff --git a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java index bd51fc7287..e857c89f64 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java @@ -1,7 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.runtime.internal; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + + /** - * Created by yipan on 8/3/18. + * Unit test for {@link ApplicationRunners} */ public class TestApplicationRunners { + + @Test + public void testFromConfig() { + Map configMap = new HashMap<>(); + configMap.put("app.runner.class", TestApplicationRunner.class.getName()); + Config config = new MapConfig(configMap); + ApplicationRunner appRunner = ApplicationRunners.fromConfig(config); + assertTrue(appRunner instanceof TestApplicationRunner); + } } diff --git a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java index fe8bc66e28..ca31040fa7 100644 --- a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java +++ b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java @@ -49,4 +49,8 @@ public interface SamzaContainerListener { * @param t Throwable that caused the container failure. */ void onContainerFailed(Throwable t); + + void beforeStop(); + + void beforeStart(); } 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 4e8c65941e..225c7629b7 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 @@ -213,6 +213,7 @@ public StreamProcessor(Config config, Map customMetrics *

    */ public void start() { + processorListener.beforeStart(); synchronized (lock) { if (state == State.NEW) { state = State.STARTED; @@ -249,6 +250,7 @@ public void start() { * */ public void stop() { + processorListener.beforeStop(); synchronized (lock) { if (state != State.STOPPING && state != State.STOPPED) { state = State.STOPPING; @@ -407,5 +409,17 @@ public void onContainerFailed(Throwable t) { jobCoordinator.stop(); } } + + @Override + public void beforeStop() { + // There is no need to invoked user defined logic before shutting down container here, since the container lifecycle + // inside a StreamProcessor is managed internally. User defined beforeStop() method is called in StreamProcessor.stop() + } + + @Override + public void beforeStart() { + // There is no need to invoked user defined logic before starting container here, since the container lifecycle + // inside a StreamProcessor is managed internally. User defined beforeStart() method is called in StreamProcessor.start() + } } } diff --git a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java index 7a4da7d8dd..0ac4d827a7 100644 --- a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java +++ b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java @@ -46,4 +46,8 @@ public interface StreamProcessorLifecycleListener { */ void onFailure(Throwable t); + void beforeStop(); + + void beforeStart(); + } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 780d841641..639e3ee263 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; +import org.apache.samza.application.ProcessorLifecycleListener; import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.application.internal.TaskAppSpecImpl; import org.apache.samza.config.ApplicationConfig; @@ -65,6 +66,7 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private final Set processors = ConcurrentHashMap.newKeySet(); private final CountDownLatch shutdownLatch = new CountDownLatch(1); private final AtomicInteger numProcessorsToStart = new AtomicInteger(); + private final AtomicInteger numProcessorsStopped = new AtomicInteger(); private final AtomicReference failure = new AtomicReference<>(); private ApplicationStatus appStatus = ApplicationStatus.New; @@ -76,6 +78,11 @@ public LocalApplicationRunner(Config config) { private final class LocalStreamProcessorLifeCycleListener implements StreamProcessorLifecycleListener { private StreamProcessor processor; + private final ProcessorLifecycleListener processorLifecycleListener; + + private LocalStreamProcessorLifeCycleListener(ProcessorLifecycleListener processorLifecycleListener) { + this.processorLifecycleListener = processorLifecycleListener; + } void setProcessor(StreamProcessor processor) { this.processor = processor; @@ -83,8 +90,9 @@ void setProcessor(StreamProcessor processor) { @Override public void onStart() { - if (numProcessorsToStart.decrementAndGet() == 0) { + if (numProcessorsToStart.get() == 0) { appStatus = ApplicationStatus.Running; + processorLifecycleListener.afterStart(); } } @@ -94,6 +102,7 @@ public void onShutdown() { processor = null; if (processors.isEmpty()) { + processorLifecycleListener.afterStop(); shutdownAndNotify(); } } @@ -109,10 +118,30 @@ public void onFailure(Throwable t) { } if (processors.isEmpty()) { + // TODO: shutdown due to failure may not have the processorLifecycleListener.beforeStop() invoked. + // Hence, we don't have a corresponding processorLifecycleListener.afterStop() here either. shutdownAndNotify(); } } + @Override + public void beforeStop() { + // This is to record the number of processors that are stopped via normal shutdown sequence (i.e. calling sp.stop()) + // If this is the first call to stop in all processors in the application, we also call beforeStop() as well. + if (numProcessorsStopped.getAndIncrement() == 0) { + processorLifecycleListener.beforeStop(); + } + } + + @Override + public void beforeStart() { + // This is to record the number of processors that are to be started (i.e. calling sp.start()) + // If this is the first call to start in all processors in the application, we also call beforeStart() method as well. + if (numProcessorsToStart.getAndDecrement() == processors.size()) { + processorLifecycleListener.beforeStart(); + } + } + private void shutdownAndNotify() { if (failure.get() != null) { appStatus = ApplicationStatus.unsuccessfulFinish(failure.get()); @@ -160,7 +189,7 @@ public void run() { } plan.getJobConfigs().forEach(jobConfig -> { LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); - LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); + LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(streamApp.getProcessorLifecycleListner()); StreamProcessor processor = createStreamProcessor(jobConfig, ((StreamGraphSpec)streamApp.getGraph()).getOperatorSpecGraph(), streamApp.getContextManager(), listener); listener.setProcessor(processor); @@ -169,9 +198,7 @@ public void run() { numProcessorsToStart.set(processors.size()); // 4. start the StreamProcessors - streamApp.getProcessorLifecycleListner().beforeStart(); processors.forEach(StreamProcessor::start); - streamApp.getProcessorLifecycleListner().afterStart(); } catch (Throwable throwable) { appStatus = ApplicationStatus.unsuccessfulFinish(throwable); shutdownLatch.countDown(); @@ -185,9 +212,7 @@ public void run() { @Override public void kill() { - streamApp.getProcessorLifecycleListner().beforeStop(); processors.forEach(StreamProcessor::stop); - streamApp.getProcessorLifecycleListner().afterStop(); } @Override @@ -213,22 +238,18 @@ private TaskAppExecutable(TaskAppSpecImpl appSpec) { @Override public void run() { LOG.info("LocalApplicationRunner will start task " + appSpec.getGlobalAppId()); - LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); + LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(appSpec.getProcessorLifecycleListner()); sp = createStreamProcessor(config, appSpec.getTaskFactory(), listener); numProcessorsToStart.set(1); listener.setProcessor(sp); - appSpec.getProcessorLifecycleListner().beforeStart(); sp.start(); - appSpec.getProcessorLifecycleListner().afterStart(); } @Override public void kill() { - appSpec.getProcessorLifecycleListner().beforeStop(); sp.stop(); - appSpec.getProcessorLifecycleListner().afterStop(); } @Override diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index b01d6ea2be..9119db1de2 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -99,27 +99,36 @@ public void run() { @Override public void onContainerStart() { log.info("Container Started"); + taskApp.getProcessorLifecycleListner().afterStart(); } @Override public void onContainerStop() { log.info("Container Stopped"); + taskApp.getProcessorLifecycleListner().afterStop(); } @Override public void onContainerFailed(Throwable t) { log.info("Container Failed"); containerRunnerException = t; + // TODO: No defined behavior on {@link ProcessorLifecycleListener} methods for failure scenario yet. + } + + @Override + public void beforeStop() { + taskApp.getProcessorLifecycleListner().beforeStop(); + } + + @Override + public void beforeStart() { + taskApp.getProcessorLifecycleListner().beforeStart(); } }); - taskApp.getProcessorLifecycleListner().beforeStart(); - taskApp.getProcessorLifecycleListner().afterStart(); startContainerHeartbeatMonitor(); container.run(); stopContainerHeartbeatMonitor(); - taskApp.getProcessorLifecycleListner().beforeStop(); - taskApp.getProcessorLifecycleListner().afterStop(); if (containerRunnerException != null) { log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); @@ -170,11 +179,13 @@ public void run() { @Override public void onContainerStart() { log.info("Container Started"); + streamApp.getProcessorLifecycleListner().afterStart(); } @Override public void onContainerStop() { log.info("Container Stopped"); + streamApp.getProcessorLifecycleListner().afterStop(); } @Override @@ -182,15 +193,21 @@ public void onContainerFailed(Throwable t) { log.info("Container Failed"); containerRunnerException = t; } + + @Override + public void beforeStop() { + streamApp.getProcessorLifecycleListner().beforeStop(); + } + + @Override + public void beforeStart() { + streamApp.getProcessorLifecycleListner().beforeStart(); + } }); - streamApp.getProcessorLifecycleListner().beforeStart(); - streamApp.getProcessorLifecycleListner().afterStart(); startContainerHeartbeatMonitor(); container.run(); stopContainerHeartbeatMonitor(); - streamApp.getProcessorLifecycleListner().beforeStop(); - streamApp.getProcessorLifecycleListner().afterStop(); if (containerRunnerException != null) { log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 71e62fd0e1..7f506f072f 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -797,6 +797,9 @@ class SamzaContainer( jmxServer = new JmxServer() + if (containerListener != null) { + containerListener.beforeStart() + } startMetrics startDiagnostics startAdmins @@ -834,6 +837,10 @@ class SamzaContainer( info("Shutting down.") removeShutdownHook + if (containerListener != null) { + containerListener.beforeStop() + } + jmxServer.stop shutdownConsumers From a82708bb0f509affba19a81a0b2a10b2d8191df8 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Sun, 12 Aug 2018 23:52:45 -0700 Subject: [PATCH 09/38] SAMZA-1789: unify ApplicationDescriptor and ApplicationRunner for high- and low-level APIs in YARN and standalone environment --- .../versioned/hello-samza-high-level-code.md | 16 +- .../samza/application/ApplicationBase.java | 12 +- ...onSpec.java => ApplicationDescriptor.java} | 25 +- ...tionSpec.java => StreamAppDescriptor.java} | 4 +- .../samza/application/StreamApplication.java | 2 +- ...cationSpec.java => TaskAppDescriptor.java} | 6 +- .../samza/application/TaskApplication.java | 2 +- ...ppSpecImpl.java => AppDescriptorImpl.java} | 26 +- ...Impl.java => StreamAppDescriptorImpl.java} | 45 +- ...ecImpl.java => TaskAppDescriptorImpl.java} | 12 +- .../apache/samza/operators/MessageStream.java | 7 +- .../operators/functions/ClosableFunction.java | 4 +- .../operators/functions/InitableFunction.java | 3 +- .../{internal => }/ApplicationRunner.java | 38 +- .../{internal => }/ApplicationRunners.java | 41 +- .../samza/runtime/ApplicationRuntime.java | 68 -- .../samza/runtime/ApplicationRuntimes.java | 95 -- .../samza/runtime/ProcessorContext.java | 6 +- .../ProcessorLifecycleListener.java | 22 +- .../ProcessorLifecycleListenerFactory.java | 37 + .../org/apache/samza/task/TaskFactory.java | 25 +- .../internal/AppSpecImplTestBase.java | 56 -- .../internal/TestStreamAppDescriptorImpl.java | 122 +++ .../internal/TestStreamAppSpecImpl.java | 101 --- .../internal/TestStreamGraph.java} | 44 +- .../internal/TestTaskAppDescriptorImpl.java | 102 +++ .../internal/TestTaskAppSpecImpl.java | 83 -- .../{internal => }/TestApplicationRunner.java | 19 +- .../TestApplicationRunners.java | 13 +- .../runtime/TestApplicationRuntimes.java | 53 -- .../application/ApplicationClassUtils.java | 39 +- .../container/SamzaContainerListener.java | 21 +- .../samza/operators/StreamGraphSpec.java | 7 +- .../samza/operators/spec/OperatorSpec.java | 3 +- .../samza/processor/StreamProcessor.java | 40 +- .../StreamProcessorLifecycleListener.java | 53 -- .../runtime/AbstractApplicationRunner.java | 59 +- .../runtime/ApplicationRunnerCommandLine.java | 6 +- .../samza/runtime/ApplicationRunnerMain.java | 19 +- .../samza/runtime/LocalApplicationRunner.java | 139 ++- .../samza/runtime/LocalContainerRunner.java | 108 ++- .../runtime/RemoteApplicationRunner.java | 46 +- .../apache/samza/task/StreamOperatorTask.java | 9 +- .../apache/samza/task/TaskFactoryUtil.java | 53 +- .../samza/container/SamzaContainer.scala | 4 +- .../samza/job/local/ThreadJobFactory.scala | 14 +- .../samza/application/TestAppClassUtils.java | 7 - .../TestApplicationClassUtils.java | 80 ++ .../application/TestStreamApplication.java | 29 + .../application/TestTaskApplication.java | 29 + .../samza/execution/TestExecutionPlanner.java | 9 +- .../apache/samza/execution/TestJobNode.java | 19 +- .../samza/operators/TestJoinOperator.java | 29 +- .../operators/impl/TestOperatorImplGraph.java | 10 +- .../operators/impl/TestWindowOperator.java | 27 +- .../samza/processor/TestStreamProcessor.java | 120 ++- .../runtime/TestApplicationRunnerMain.java | 36 +- .../runtime/TestLocalApplicationRunner.java | 139 +-- .../runtime/TestRemoteApplicationRunner.java | 44 +- .../TestAsyncStreamTask.java | 6 +- .../{testUtils => task}/TestStreamTask.java | 5 +- .../samza/task/TestTaskFactoryUtil.java | 6 +- .../samza/container/TestSamzaContainer.scala | 62 ++ .../samza/sql/runner/SamzaSqlApplication.java | 8 +- ...me.java => SamzaSqlApplicationRunner.java} | 34 +- .../samza/sql/translator/QueryTranslator.java | 16 +- .../sql/translator/TranslatorContext.java | 4 +- .../samza/sql/e2e/TestSamzaSqlTable.java | 6 +- .../TestSamzaSqlApplicationRuntime.java | 10 +- .../sql/translator/TestQueryTranslator.java | 90 +- .../example/AppWithGlobalConfigExample.java | 20 +- .../samza/example/BroadcastExample.java | 22 +- .../samza/example/KeyValueStoreExample.java | 18 +- .../apache/samza/example/MergeExample.java | 20 +- .../example/OrderShipmentJoinExample.java | 20 +- .../samza/example/PageViewCounterExample.java | 45 +- .../samza/example/RepartitionExample.java | 18 +- .../samza/example/TaskApplicationExample.java | 42 +- .../apache/samza/example/WindowExample.java | 18 +- .../samza/test/framework/TestRunner.java | 54 +- .../LocalApplicationRunnerMain.java | 10 +- .../TestStandaloneIntegrationApplication.java | 8 +- .../processor/TestZkStreamProcessorBase.java | 30 +- .../EndOfStreamIntegrationTest.java | 24 +- .../WatermarkIntegrationTest.java | 31 +- .../test/framework/BroadcastAssertApp.java | 8 +- .../StreamApplicationIntegrationTest.java | 11 +- ...reamApplicationIntegrationTestHarness.java | 16 +- .../samza/test/framework/TestTimerApp.java | 6 +- .../samza/test/framework/TimerTest.java | 13 +- .../operator/RepartitionJoinWindowApp.java | 20 +- .../test/operator/RepartitionWindowApp.java | 10 +- .../samza/test/operator/SessionWindowApp.java | 19 +- .../TestRepartitionJoinWindowApp.java | 10 + .../operator/TestRepartitionWindowApp.java | 2 + .../test/operator/TumblingWindowApp.java | 22 +- .../test/processor/TestStreamApplication.java | 76 +- .../test/processor/TestStreamProcessor.java | 21 +- .../TestZkLocalApplicationRunner.java | 832 +++++++++--------- .../test/samzasql/TestSamzaSqlEndToEnd.java | 30 +- .../samza/test/table/TestLocalTable.java | 44 +- .../table/TestLocalTableWithSideInputs.java | 23 +- .../samza/test/table/TestRemoteTable.java | 36 +- .../apache/samza/tools/SamzaSqlConsole.java | 4 +- .../SystemConsumerWithSamzaBench.java | 10 +- 105 files changed, 2171 insertions(+), 1956 deletions(-) rename samza-api/src/main/java/org/apache/samza/application/{ApplicationSpec.java => ApplicationDescriptor.java} (54%) rename samza-api/src/main/java/org/apache/samza/application/{StreamApplicationSpec.java => StreamAppDescriptor.java} (83%) rename samza-api/src/main/java/org/apache/samza/application/{TaskApplicationSpec.java => TaskAppDescriptor.java} (84%) rename samza-api/src/main/java/org/apache/samza/application/internal/{AppSpecImpl.java => AppDescriptorImpl.java} (76%) rename samza-api/src/main/java/org/apache/samza/application/internal/{StreamAppSpecImpl.java => StreamAppDescriptorImpl.java} (64%) rename samza-api/src/main/java/org/apache/samza/application/internal/{TaskAppSpecImpl.java => TaskAppDescriptorImpl.java} (82%) rename samza-api/src/main/java/org/apache/samza/runtime/{internal => }/ApplicationRunner.java (60%) rename samza-api/src/main/java/org/apache/samza/runtime/{internal => }/ApplicationRunners.java (56%) delete mode 100644 samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java delete mode 100644 samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java rename samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java => samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java (83%) rename samza-api/src/main/java/org/apache/samza/{application => runtime}/ProcessorLifecycleListener.java (55%) create mode 100644 samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java delete mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java create mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java delete mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java rename samza-api/src/test/java/org/apache/samza/{operators/StreamGraphSpec.java => application/internal/TestStreamGraph.java} (62%) create mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java delete mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java rename samza-api/src/test/java/org/apache/samza/runtime/{internal => }/TestApplicationRunner.java (70%) rename samza-api/src/test/java/org/apache/samza/runtime/{internal => }/TestApplicationRunners.java (73%) delete mode 100644 samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java delete mode 100644 samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java delete mode 100644 samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java create mode 100644 samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java create mode 100644 samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java create mode 100644 samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java rename samza-core/src/test/java/org/apache/samza/{testUtils => task}/TestAsyncStreamTask.java (84%) rename samza-core/src/test/java/org/apache/samza/{testUtils => task}/TestStreamTask.java (86%) rename samza-sql/src/main/java/org/apache/samza/sql/runner/{SamzaSqlApplicationRuntime.java => SamzaSqlApplicationRunner.java} (81%) diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md index 4fb2d8a6c1..a1a98396c2 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md @@ -176,7 +176,7 @@ package samza.examples.wikipedia.application; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamGraph; public class MyWikipediaApplication implements StreamApplication{ @Override @@ -188,12 +188,12 @@ public class MyWikipediaApplication implements StreamApplication{ Be sure to include the Apache header. The project will not compile without it. -The [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method is where the application logic is defined. The [Config](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/config/Config.html) argument is the runtime configuration loaded from the properties file we defined earlier. The [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) argument provides methods to declare input streams. You can then invoke a number of flexible operations on those streams. The result of each operation is another stream, so you can keep chaining more operations or direct the result to an output stream. +The [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method is where the application logic is defined. The [Config](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/config/Config.html) argument is the runtime configuration loaded from the properties file we defined earlier. The [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) argument provides methods to declare input streams. You can then invoke a number of flexible operations on those streams. The result of each operation is another stream, so you can keep chaining more operations or direct the result to an output stream. Next, we will declare the input streams for the Wikipedia application. #### Inputs -The Wikipedia application consumes events from three channels. Let's declare each of those channels as an input streams via the [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) in the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method. +The Wikipedia application consumes events from three channels. Let's declare each of those channels as an input streams via the [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) in the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method. {% highlight java %} MessageStream wikipediaEvents = streamGraph.getInputStream("en-wikipedia", new NoOpSerde<>()); MessageStream wiktionaryEvents = streamGraph.getInputStream("en-wiktionary", new NoOpSerde<>()); @@ -208,7 +208,7 @@ Note the streams are all MessageStreams of type WikipediaFeedEvent. [MessageStre #### Merge We'd like to use the same processing logic for all three input streams, so we will use the [mergeAll](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#mergeAll-java.util.Collection-) operator to merge them together. Note: this is not the same as a [join](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#join-org.apache.samza.operators.MessageStream-org.apache.samza.operators.functions.JoinFunction-java.time.Duration-) because we are not associating events by key. We are simply combining three streams into one, like a union. -Add the following snippet to the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method. It merges all the input streams into a new one called _allWikipediaEvents_ +Add the following snippet to the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method. It merges all the input streams into a new one called _allWikipediaEvents_ {% highlight java %} MessageStream allWikipediaEvents = MessageStream.mergeAll(ImmutableList.of(wikipediaEvents, wiktionaryEvents, wikiNewsEvents)); {% endhighlight %} @@ -218,7 +218,7 @@ Note there is a [merge](/learn/documentation/{{site.version}}/api/javadocs/org/a #### Parse The next step is to parse the events and extract some information. We will use the pre-existing `WikipediaParser.parseEvent()' method to do this. The parser extracts some flags we want to monitor as well as some metadata about the event. Inspect the method signature. The input is a WikipediaFeedEvents and the output is a Map. These types will be reflected in the types of the streams before and after the operation. -In the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method, invoke the [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation on `allWikipediaEvents`, passing the `WikipediaParser::parseEvent` method reference as follows: +In the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method, invoke the [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation on `allWikipediaEvents`, passing the `WikipediaParser::parseEvent` method reference as follows: {% highlight java %} allWikipediaEvents.map(WikipediaParser::parseEvent); @@ -227,7 +227,7 @@ allWikipediaEvents.map(WikipediaParser::parseEvent); #### Window Now that we have the relevant information extracted, let's perform some aggregations over a 10-second [window](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/windows/Window.html). -First, we need a container class for statistics we want to track. Add the following static class after the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-) method. +First, we need a container class for statistics we want to track. Add the following static class after the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method. {% highlight java %} private static class WikipediaStats { int edits = 0; @@ -262,7 +262,7 @@ private class WikipediaStatsAggregator implements FoldLeftFunction(WikipediaStats.class))); @@ -301,7 +301,7 @@ Paste the following after the aggregator class: } {% endhighlight %} -Now, we can invoke the method by adding another [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation to the chain in [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamApplicationSpec-org.apache.samza.config.Config-). The operator chain should now look like this: +Now, we can invoke the method by adding another [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation to the chain in [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-). The operator chain should now look like this: {% highlight java %} allWikipediaEvents.map(WikipediaParser::parseEvent) .window(Windows.tumblingWindow(Duration.ofSeconds(10), WikipediaStats::new, new WikipediaStatsAggregator())) diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java index 77717b6bb0..eaf982c641 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java @@ -20,10 +20,16 @@ /** * The base interface for all user-implemented applications in Samza. The main processing logic of the user application - * should be implemented in {@link ApplicationBase#describe(ApplicationSpec)} method. Sub-classes {@link StreamApplication} + * should be implemented in {@link ApplicationBase#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. */ -public interface ApplicationBase { - void describe(S appSpec); +public interface ApplicationBase { + + /** + * Describes the user processing logic via {@link ApplicationDescriptor} + * + * @param appDesc the {@link ApplicationDescriptor} object to describe user application logic + */ + void describe(S appDesc); } diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java similarity index 54% rename from samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java rename to samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index 79f3fd7949..aafe3ff5ab 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -20,14 +20,15 @@ import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; +import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; /** - * The base interface class to create the specification of a user application in Samza. Sub-classes {@link StreamApplicationSpec} - * and {@link TaskApplicationSpec} are specific interfaces for applications written in high-level DAG and low-level task APIs, + * The base interface class to describe a user application in Samza. Sub-classes {@link StreamAppDescriptor} + * and {@link TaskAppDescriptor} are specific interfaces for applications written in high-level DAG and low-level task APIs, * respectively. */ -public interface ApplicationSpec { +public interface ApplicationDescriptor { /** * Get the global unique application ID in the runtime process * @return globally unique application ID @@ -41,23 +42,23 @@ public interface ApplicationSpec { Config getConfig(); /** - * Sets the {@link ContextManager} for this {@link ApplicationSpec}. + * Sets the {@link ContextManager} for this application. *

    * The provided {@link ContextManager} can be used to setup shared context between the operator functions * within a task instance * - * @param contextManager the {@link ContextManager} to use for the {@link StreamApplicationSpec} - * @return the {@link ApplicationSpec} with {@code contextManager} set as its {@link ContextManager} + * @param contextManager the {@link ContextManager} to use for the application + * @return the {@link ApplicationDescriptor} with {@code contextManager} set as its {@link ContextManager} */ - ApplicationSpec withContextManager(ContextManager contextManager); + ApplicationDescriptor withContextManager(ContextManager contextManager); /** - * Sets the {@link ProcessorLifecycleListener} for this {@link ApplicationSpec}. + * Sets the {@link ProcessorLifecycleListenerFactory} for this application. * - * @param listener the user implemented {@link ProcessorLifecycleListener} with lifecycle aware methods to be invoked - * before and after the start/stop of the processing logic defined in this {@link ApplicationSpec} - * @return the {@link ApplicationSpec} with {@code listener} set as its {@link ProcessorLifecycleListener} + * @param listenerFactory the user implemented {@link ProcessorLifecycleListenerFactory} that creates lifecycle aware + * methods to be invoked before and after the start/stop of the StreamProcessor(s) in the application + * @return the {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory} */ - ApplicationSpec withProcessorLifecycleListener(ProcessorLifecycleListener listener); + ApplicationDescriptor withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java similarity index 83% rename from samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java rename to samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java index 87f45ce506..4cfc7cd851 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java @@ -23,8 +23,8 @@ /** - * The interface class to create the specification of a user application as high-level DAG in Samza. + * The interface class to describe a user applicationin high-level DAG in Samza. */ @InterfaceStability.Evolving -public interface StreamApplicationSpec extends ApplicationSpec, StreamGraph { +public interface StreamAppDescriptor extends ApplicationDescriptor, StreamGraph { } 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 da3b81aa45..848c26ac1b 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,5 +21,5 @@ /** * The interface to implement user applications defining a high-level DAG as the main stream processing logic. */ -public interface StreamApplication extends ApplicationBase { +public interface StreamApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java similarity index 84% rename from samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java rename to samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java index e3f3b91c8c..aad0f75806 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationSpec.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java @@ -24,9 +24,9 @@ /** - * The interface class to create the specification of a user application as low-level task in Samza. + * The interface class to describe a user application as low-level task in Samza. */ -public interface TaskApplicationSpec extends ApplicationSpec { +public interface TaskAppDescriptor extends ApplicationDescriptor { /** * Sets the {@link TaskFactory} for the user application. The {@link TaskFactory#createInstance()} creates task instance @@ -41,6 +41,7 @@ public interface TaskApplicationSpec extends ApplicationSpec { * * @param inputStreams the list of streamIds for the input streams */ + // TODO: needs to be replaced by InputStreamDescriptor after SAMZA-1804 is implemented void addInputStreams(List inputStreams); /** @@ -48,6 +49,7 @@ public interface TaskApplicationSpec extends ApplicationSpec { * * @param outputStreams the list of streamIds for the output streams */ + // TODO: needs to be replaced by OutputStreamDescriptor after SAMZA-1804 is implemented void addOutputStreams(List outputStreams); /** 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 48be7d8887..ef580b2759 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 @@ -21,5 +21,5 @@ /** * The interface to implement user applications defining a low-level task as the main stream processing logic. */ -public interface TaskApplication extends ApplicationBase { +public interface TaskApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/AppDescriptorImpl.java similarity index 76% rename from samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java rename to samza-api/src/main/java/org/apache/samza/application/internal/AppDescriptorImpl.java index c18f32e342..cb7a1252d9 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/AppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/AppDescriptorImpl.java @@ -18,25 +18,28 @@ */ package org.apache.samza.application.internal; -import org.apache.samza.application.ApplicationSpec; import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.ProcessorLifecycleListener; +import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.operators.ContextManager; +import org.apache.samza.runtime.ProcessorLifecycleListener; +import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; import org.apache.samza.task.TaskContext; /** - * This is the base class that implements interface {@link ApplicationSpec}. This base class contains the common objects + * This is the base class that implements interface {@link ApplicationDescriptor}. This base class contains the common objects * that are used by both high-level and low-level API applications, such as {@link Config}, {@link ContextManager}, and * {@link ProcessorLifecycleListener}. */ -public abstract class AppSpecImpl> implements ApplicationSpec { +public abstract class AppDescriptorImpl> + implements ApplicationDescriptor { final Config config; // Default to no-op functions in ContextManager + // TODO: this should be replaced by shared context factory defined in SAMZA-1714 ContextManager contextManager = new ContextManager() { @Override public void init(Config config, TaskContext context) { @@ -47,11 +50,10 @@ public void close() { } }; - // Default to no-op functions in ProcessorLifecycleListener - ProcessorLifecycleListener listener = new ProcessorLifecycleListener() { - }; + // Default to no-op ProcessorLifecycleListenerFactory + ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { }; - AppSpecImpl(Config config) { + AppDescriptorImpl(Config config) { this.config = config; } @@ -103,8 +105,8 @@ public S withContextManager(ContextManager contextManager) { } @Override - public S withProcessorLifecycleListener(ProcessorLifecycleListener listener) { - this.listener = listener; + public S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory) { + this.listenerFactory = listenerFactory; return (S) this; } @@ -122,8 +124,8 @@ public ContextManager getContextManager() { * * @return the {@link ProcessorLifecycleListener} object */ - public ProcessorLifecycleListener getProcessorLifecycleListner() { - return listener; + public ProcessorLifecycleListenerFactory getProcessorLifecycleListenerFactory() { + return listenerFactory; } } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java similarity index 64% rename from samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java rename to samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java index 53269e6962..a62eb66a0e 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java @@ -20,9 +20,10 @@ import java.lang.reflect.Constructor; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; @@ -33,28 +34,24 @@ /** - * This class implements interface {@link StreamApplicationSpec}. In addition to the common objects for an application - * defined in {@link AppSpecImpl}, this class also includes the high-level DAG {@link StreamGraph} object that user will + * This class implements interface {@link StreamAppDescriptor}. In addition to the common objects for an application + * defined in {@link AppDescriptorImpl}, this class also includes the high-level DAG {@link StreamGraph} object that user will * use to create the processing logic in DAG. */ -public class StreamAppSpecImpl extends AppSpecImpl implements StreamApplicationSpec { +public class StreamAppDescriptorImpl extends AppDescriptorImpl + implements StreamAppDescriptor { final StreamGraph graph; - public StreamAppSpecImpl(StreamApplication userApp, Config config) { + // this config variable is for unit test in samza-api only. *MUST NOT* be set by the user + private static final String TEST_GRAPH_CLASS_CFG = "app.test.graph.class"; + private static final String DEFAULT_GRAPH_CLASS = "org.apache.samza.operators.StreamGraphSpec"; + + public StreamAppDescriptorImpl(StreamApplication userApp, Config config) { super(config); this.graph = createDefaultGraph(config); userApp.describe(this); } - private StreamGraph createDefaultGraph(Config config) { - try { - Constructor constructor = Class.forName("org.apache.samza.operators.StreamGraphSpec").getConstructor(Config.class); // *sigh* - return (StreamGraph) constructor.newInstance(config); - } catch (Exception e) { - throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); - } - } - @Override public void setDefaultSerde(Serde serde) { this.graph.setDefaultSerde(serde); @@ -94,4 +91,24 @@ public StreamGraph getGraph() { return graph; } + /** + * Helper method to load the implementation class of {@link StreamGraph} interface + * + * @param config the configuration of the application + * @return an object implements {@link StreamGraph} interface + */ + private StreamGraph createDefaultGraph(Config config) { + String graphClass = config.getOrDefault(TEST_GRAPH_CLASS_CFG, DEFAULT_GRAPH_CLASS); + try { + if (StreamGraph.class.isAssignableFrom(Class.forName(graphClass))) { + Constructor constructor = Class.forName(graphClass).getConstructor(Config.class); // *sigh* + return (StreamGraph) constructor.newInstance(config); + } else { + throw new ConfigException(String.format("Incompatible class %s is invalid. Must implement StreamGraph.", graphClass)); + } + } catch (Exception e) { + throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); + } + } + } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppDescriptorImpl.java similarity index 82% rename from samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java rename to samza-api/src/main/java/org/apache/samza/application/internal/TaskAppDescriptorImpl.java index 8be3ea83e8..441cc6d9ce 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppSpecImpl.java +++ b/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppDescriptorImpl.java @@ -22,26 +22,28 @@ import java.util.Collections; import java.util.List; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TaskApplicationSpec; +import org.apache.samza.application.TaskAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.task.TaskFactory; /** - * This class implements interface {@link TaskApplicationSpec}. In addition to the common objects for an application - * defined in {@link AppSpecImpl}, this class also includes the low-level {@link TaskFactory} object that creates + * This class implements interface {@link TaskAppDescriptor}. In addition to the common objects for an application + * defined in {@link AppDescriptorImpl}, this class also includes the low-level {@link TaskFactory} object that creates * user-defined task instances, the lists of input/output streams, and the list of {@link TableDescriptor}s used in * the application. */ -public class TaskAppSpecImpl extends AppSpecImpl implements TaskApplicationSpec { +public class TaskAppDescriptorImpl extends AppDescriptorImpl + implements TaskAppDescriptor { TaskFactory taskFactory; + //TODO: need to replace with InputStreamDescriptor and OutputStreamDescriptor when SAMZA-1804 is implemented final List inputStreams = new ArrayList<>(); final List outputStreams = new ArrayList<>(); final List tables = new ArrayList<>(); - public TaskAppSpecImpl(TaskApplication userApp, Config config) { + public TaskAppDescriptorImpl(TaskApplication userApp, Config config) { super(config); userApp.describe(this); } diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index bd25005a44..7797f9a255 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -23,7 +23,6 @@ import java.util.Collection; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; @@ -40,7 +39,7 @@ /** * A stream of messages that can be transformed into another {@link MessageStream}. *

    - * A {@link MessageStream} corresponding to an input stream can be obtained using {@link StreamApplicationSpec#getInputStream}. + * A {@link MessageStream} corresponding to an input stream can be obtained using {@link StreamGraph#getInputStream}. * * @param the type of messages in this stream */ @@ -218,7 +217,7 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, - * uses the default serde provided via {@link StreamApplicationSpec#setDefaultSerde}, which must be a KVSerde. If the default + * uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided * before calling this method, a {@code KVSerde} is used. *

    @@ -252,7 +251,7 @@ MessageStream> partitionBy(MapFunction k /** * Same as calling {@link #partitionBy(MapFunction, MapFunction, KVSerde, String)} with a null KVSerde. *

    - * Uses the default serde provided via {@link StreamApplicationSpec#setDefaultSerde}, which must be a KVSerde. If the default + * Uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided * before calling this method, a {@code KVSerde} is used. * diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java index 7de53f8431..faf9fc5a35 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java @@ -20,8 +20,6 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationSpec; - /** * A function that can be closed after its execution. @@ -29,7 +27,7 @@ *

    Implement {@link #close()} to free resources used during the execution of the function, clean up state etc. * *

    Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the - * {@link StreamApplicationSpec}. For any two operators A and B in the graph, if operator B consumes results + * {@link org.apache.samza.operators.StreamGraph}. For any two operators A and B in the graph, if operator B consumes results * from operator A, then operator B is guaranteed to be closed before operator A. * */ diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index c56ecd44cc..6651819780 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -20,7 +20,6 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.task.TaskContext; @@ -28,7 +27,7 @@ * A function that can be initialized before execution. * *

    Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the - * {@link StreamApplicationSpec}. For any two operators A and B in the graph, if operator B consumes results + * {@link org.apache.samza.operators.StreamGraph}. For any two operators A and B in the graph, if operator B consumes results * from operator A, then operator A is guaranteed to be initialized before operator B. * */ diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java similarity index 60% rename from samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java rename to samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index 9c86fe3478..c52cfe6e8c 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -16,46 +16,44 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime.internal; +package org.apache.samza.runtime; import java.time.Duration; import java.util.Map; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.ApplicationSpec; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; /** - * The primary means of managing execution of user applications deployed in various runtime environments. + * The primary means of managing execution of the {@link org.apache.samza.application.ApplicationBase} at runtime. */ @InterfaceStability.Evolving public interface ApplicationRunner { - /** - * Launch the application defined in {@link ApplicationSpec} - * - * @param appSpec the user defined {@link ApplicationSpec} + * Deploy and run the Samza jobs to execute {@link org.apache.samza.application.ApplicationBase}. + * It is non-blocking so it doesn't wait for the application running. */ - void run(ApplicationSpec appSpec); + void run(); /** - * Stop the application already deployed in a runtime environment - * - * @param appSpec the user defined {@link ApplicationSpec} + * Kill the Samza jobs represented by {@link org.apache.samza.application.ApplicationBase} + * It is non-blocking so it doesn't wait for the application stopping. */ - void kill(ApplicationSpec appSpec); + void kill(); /** - * Query the status of the application deployed in a runtime environment + * Get the collective status of the Samza jobs represented by {@link org.apache.samza.application.ApplicationBase}. + * Returns {@link ApplicationStatus} object. * - * @param appSpec the user defined {@link ApplicationSpec} - * @return the current status of a deployed application + * @return the current status of an instance of {@link org.apache.samza.application.ApplicationBase} */ - ApplicationStatus status(ApplicationSpec appSpec); + ApplicationStatus status(); - @Deprecated - void waitForFinish(ApplicationSpec appSpec); + /** + * Waits until the application finishes. + */ + void waitForFinish(); /** * Waits for {@code timeout} duration for the application to finish. @@ -64,10 +62,10 @@ public interface ApplicationRunner { * @return true - application finished before timeout * false - otherwise */ - boolean waitForFinish(ApplicationSpec appSpec, Duration timeout); + boolean waitForFinish(Duration timeout); /** - * Method to add a set of customized {@link MetricsReporter}s in the application + * Add a set of customized {@link MetricsReporter}s in the application * * @param metricsReporters the map of customized {@link MetricsReporter}s objects to be used */ diff --git a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java similarity index 56% rename from samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java rename to samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java index f92a2cd616..9c589d8733 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/internal/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java @@ -16,15 +16,21 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime.internal; +package org.apache.samza.runtime; import java.lang.reflect.Constructor; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; /** - * Creates {@link ApplicationRunner} instances based on configuration. + * Creates {@link ApplicationRunner} instances based on configuration and user-implemented {@link ApplicationBase} */ public class ApplicationRunners { @@ -32,6 +38,24 @@ private ApplicationRunners() { } + /** + * Get the {@link ApplicationRunner} that runs the {@code userApp} + * + * @param userApp the user application object + * @param config the configuration for this application + * @return the {@link ApplicationRunner} object that will run the {@code userApp} + */ + public static final ApplicationRunner getApplicationRunner(ApplicationBase userApp, Config config) { + if (userApp instanceof StreamApplication) { + return getRunner(new StreamAppDescriptorImpl((StreamApplication) userApp, config)); + } + if (userApp instanceof TaskApplication) { + return getRunner(new TaskAppDescriptorImpl((TaskApplication) userApp, config)); + } + throw new IllegalArgumentException(String.format("User application instance has to be either StreamApplicationFactory or TaskApplicationFactory. " + + "Invalid userApp class %s.", userApp.getClass().getName())); + } + static class AppRunnerConfig { private static final String APP_RUNNER_CFG = "app.runner.class"; private static final String DEFAULT_APP_RUNNER = "org.apache.samza.runtime.RemoteApplicationRunner"; @@ -49,18 +73,18 @@ String getAppRunnerClass() { } /** - * Static method to load the {@link ApplicationRunner} + * Static method to get the {@link ApplicationRunner} * - * @param config configuration passed in to initialize the Samza processes + * @param appSpec configuration passed in to initialize the Samza processes * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications */ - public static ApplicationRunner fromConfig(Config config) { - AppRunnerConfig appRunnerCfg = new AppRunnerConfig(config); + static ApplicationRunner getRunner(AppDescriptorImpl appSpec) { + AppRunnerConfig appRunnerCfg = new AppRunnerConfig(appSpec.getConfig()); try { Class runnerClass = Class.forName(appRunnerCfg.getAppRunnerClass()); if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { - Constructor constructor = runnerClass.getConstructor(Config.class); // *sigh* - return (ApplicationRunner) constructor.newInstance(config); + Constructor constructor = runnerClass.getConstructor(AppDescriptorImpl.class); // *sigh* + return (ApplicationRunner) constructor.newInstance(appSpec); } } catch (Exception e) { throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", @@ -70,5 +94,4 @@ public static ApplicationRunner fromConfig(Config config) { "Class %s does not extend ApplicationRunner properly", appRunnerCfg.getAppRunnerClass())); } - } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java deleted file mode 100644 index 1b7580d4e9..0000000000 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntime.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.runtime; - -import java.time.Duration; -import java.util.Map; -import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; - - -/** - * The primary execution methods of a runtime instance of the user application. - */ -public interface ApplicationRuntime { - /** - * Start a runtime instance of the application - */ - void run(); - - /** - * Stop a runtime instance of the application - */ - void kill(); - - /** - * Get the {@link ApplicationStatus} of a runtime instance of the application - * @return the runtime status of the application - */ - ApplicationStatus status(); - - /** - * Wait the runtime instance of the application to complete. - * This method will block until the application completes. - */ - void waitForFinish(); - - /** - * Wait the runtime instance of the application to complete with a {@code timeout} - * - * @param timeout the time to block to wait for the application to complete - * @return true if the application completes within timeout; false otherwise - */ - boolean waitForFinish(Duration timeout); - - /** - * Method to add a set of customized {@link MetricsReporter}s in the application runtime instance - * - * @param metricsReporters the map of customized {@link MetricsReporter}s objects to be used - */ - void addMetricsReporters(Map metricsReporters); - -} diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java deleted file mode 100644 index 1218bd193b..0000000000 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRuntimes.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.runtime; - -import java.time.Duration; -import java.util.Map; -import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; -import org.apache.samza.config.Config; -import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.internal.ApplicationRunners; - - -/** - * Creates {@link ApplicationRuntime} instances based on configuration and user-implemented {@link ApplicationBase} - */ -public class ApplicationRuntimes { - - private ApplicationRuntimes() { - - } - - public static final ApplicationRuntime getApplicationRuntime(ApplicationBase userApp, Config config) { - if (userApp instanceof StreamApplication) { - return new AppRuntimeImpl(new StreamAppSpecImpl((StreamApplication) userApp, config)); - } - if (userApp instanceof TaskApplication) { - return new AppRuntimeImpl(new TaskAppSpecImpl((TaskApplication) userApp, config)); - } - throw new IllegalArgumentException(String.format("User application instance has to be either StreamApplicationFactory or TaskApplicationFactory. " - + "Invalid userApp class %s.", userApp.getClass().getName())); - } - - private static class AppRuntimeImpl implements ApplicationRuntime { - private final ApplicationSpec appSpec; - private final ApplicationRunner runner; - - AppRuntimeImpl(ApplicationSpec appSpec) { - this.appSpec = appSpec; - this.runner = ApplicationRunners.fromConfig(appSpec.getConfig()); - } - - @Override - public void run() { - this.runner.run(appSpec); - } - - @Override - public void kill() { - this.runner.kill(appSpec); - } - - @Override - public ApplicationStatus status() { - return this.runner.status(appSpec); - } - - @Override - public void waitForFinish() { - this.runner.waitForFinish(appSpec, Duration.ofSeconds(0)); - } - - @Override - public boolean waitForFinish(Duration timeout) { - return this.runner.waitForFinish(appSpec, timeout); - } - - @Override - public void addMetricsReporters(Map metricsReporters) { - this.runner.addMetricsReporters(metricsReporters); - } - } -} diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java similarity index 83% rename from samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java rename to samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java index 45bd6cebcb..a255858a5c 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalContainerRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java @@ -19,7 +19,9 @@ package org.apache.samza.runtime; /** - * TODO: adding tests to unit test {@link LocalContainerRunner} + * The context for a StreamProcessor. Currently, only defines a method to report the processorId + * */ -public class TestLocalContainerRunner { +public interface ProcessorContext { + String getProcessorId(); } diff --git a/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java similarity index 55% rename from samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java rename to samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java index cbe56e5a0e..5ce1157859 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ProcessorLifecycleListener.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java @@ -16,31 +16,35 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application; +package org.apache.samza.runtime; /** - * This interface defines methods that are invoked in various different stages of an application runtime instance - * where it actually runs the processing logic in local process (i.e. as a standalone process, or a container process - * in YARN NodeManager). User can implement this interface to instantiate/release shared objects in the local process. + * This interface defines methods that are invoked in different stages of StreamProcessor's lifecycle in local + * process (i.e. as a standalone process, or a container process in YARN NodeManager). User can implement this interface + * to instantiate/release shared objects in the local process. */ public interface ProcessorLifecycleListener { /** - * User defined initialization before any processor in an application runtime instance is started + * User defined initialization before a StreamProcessor is started */ default void beforeStart() {} /** - * User defined callback after all processors in an application runtime instance are started + * User defined callback after a StreamProcessor is started + * */ default void afterStart() {} /** - * User defined callback before any processor in an application runtime instance is stopped + * User defined callback before a StreamProcessor is stopped + * */ default void beforeStop() {} /** - * User defined callback after all processors in an application runtime instance are stopped + * User defined callback after a StreamProcessor is stopped + * + * @param t the error causing the stop of the StreamProcessor. null value of this parameter indicates a successful completion. */ - default void afterStop() {} + default void afterStop(Throwable t) {} } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java new file mode 100644 index 0000000000..cbc49fee08 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.runtime; + +import java.io.Serializable; +import org.apache.samza.config.Config; + + +/** + * This interface class defines the factory method to create an instance of {@link ProcessorLifecycleListener}. + */ +public interface ProcessorLifecycleListenerFactory extends Serializable { + /** + * Create an instance of {@link ProcessorLifecycleListener} for the StreamProcessor + * + * @param pContext the context of the corresponding StreamProcessor + * @param config the configuration of the corresponding StreamProcessor + * @return the {@link ProcessorLifecycleListener} callback object for the StreamProcessor + */ + ProcessorLifecycleListener createInstance(ProcessorContext pContext, Config config); +} 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 360a89bd6d..49277928dd 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 @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.task; import java.io.Serializable; @@ -5,9 +23,14 @@ /** - * Created by yipan on 7/10/18. + * The base interface class for all task factories (i.e. {@link StreamTaskFactory} and {@link AsyncStreamTaskFactory} */ @InterfaceStability.Stable public interface TaskFactory extends Serializable { + /** + * Create instance of task + * + * @return task of type T + */ T createInstance(); } diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java b/samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java deleted file mode 100644 index 6be532aa8c..0000000000 --- a/samza-api/src/test/java/org/apache/samza/application/internal/AppSpecImplTestBase.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import org.apache.samza.application.ProcessorLifecycleListener; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.Config; -import org.apache.samza.operators.ContextManager; -import org.junit.Test; - -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - - -/** - * Base class for unit tests for {@link AppSpecImpl} - */ -public class AppSpecImplTestBase { - @Test - public void testContextManager() { - ContextManager cntxMan = mock(ContextManager.class); - Config mockConf = mock(Config.class); - StreamApplication testApp = appSpec -> { - appSpec.withContextManager(cntxMan); - }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); - assertEquals(appSpec.getContextManager(), cntxMan); - } - - @Test - public void testProcessorLifecycleListener() { - ProcessorLifecycleListener listener = mock(ProcessorLifecycleListener.class); - Config mockConf = mock(Config.class); - StreamApplication testApp = appSpec -> { - appSpec.withProcessorLifecycleListener(listener); - }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); - assertEquals(appSpec.getProcessorLifecycleListner(), listener); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java new file mode 100644 index 0000000000..ef90a98556 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java @@ -0,0 +1,122 @@ +/* + * 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.application.internal; + +import java.util.ArrayList; +import java.util.HashMap; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.StringSerde; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + + +/** + * Unit test for {@link StreamAppDescriptorImpl} + */ +public class TestStreamAppDescriptorImpl { + private Config config = new MapConfig(new HashMap() { + { + this.put("app.test.graph.class", TestStreamGraph.class.getName()); + } + }); + + @Test + public void testConstructor() { + StreamApplication mockApp = mock(StreamApplication.class); + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(mockApp, config); + verify(mockApp, times(1)).describe(appDesc); + assertEquals(config, appDesc.config); + assertTrue(appDesc.graph instanceof TestStreamGraph); + } + + @Test + public void testGetInputStream() { + Serde testSerde = new StringSerde(); + StreamApplication testApp = appDesc -> { + appDesc.getInputStream("myinput1"); + appDesc.getInputStream("myinput2", testSerde); + }; + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); + assertEquals(((TestStreamGraph) appDesc.graph).inputStreams, + new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }); + assertEquals(((TestStreamGraph) appDesc.graph).inputSerdes.get("myinput2"), testSerde); + } + + @Test + public void testGetOutputStream() { + Serde testSerde = new StringSerde(); + StreamApplication testApp = appDesc -> { + appDesc.getOutputStream("myoutput1"); + appDesc.getOutputStream("myoutput2", testSerde); + }; + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); + assertEquals(((TestStreamGraph) appDesc.graph).outputStreams, + new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }); + assertEquals(((TestStreamGraph) appDesc.graph).outputSerdes.get("myoutput2"), testSerde); + } + + @Test + public void testGetTable() { + TableDescriptor mockTd = mock(TableDescriptor.class); + StreamApplication testApp = appDesc -> { + appDesc.getTable(mockTd); + }; + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); + assertEquals(((TestStreamGraph) appDesc.graph).tables, + new ArrayList() { { this.add(mockTd); } }); + } + + @Test + public void testSetDefaultSerde() { + Serde testSerde = new StringSerde(); + StreamApplication testApp = appDesc -> { + appDesc.setDefaultSerde(testSerde); + }; + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); + assertEquals(((TestStreamGraph) appDesc.graph).defaultSerde, testSerde); + } + + @Test + public void testContextManager() { + ContextManager cntxMan = mock(ContextManager.class); + StreamApplication testApp = appDesc -> { + appDesc.withContextManager(cntxMan); + }; + StreamAppDescriptorImpl appSpec = new StreamAppDescriptorImpl(testApp, config); + assertEquals(appSpec.getContextManager(), cntxMan); + } + + @Test + public void testProcessorLifecycleListenerFactory() { + ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class); + StreamApplication testApp = appSpec -> { + appSpec.withProcessorLifecycleListenerFactory(mockFactory); + }; + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); + } +} diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java deleted file mode 100644 index 738a24f874..0000000000 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppSpecImpl.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import java.util.ArrayList; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.Config; -import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.serializers.Serde; -import org.apache.samza.serializers.StringSerde; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - - -/** - * Unit test for {@link StreamAppSpecImpl} - */ -public class TestStreamAppSpecImpl extends AppSpecImplTestBase { - @Test - public void testConstructor() { - StreamApplication mockApp = mock(StreamApplication.class); - Config mockConf = mock(Config.class); - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(mockApp, mockConf); - verify(mockApp, times(1)).describe(appSpec); - assertEquals(mockConf, appSpec.config); - assertTrue(appSpec.graph instanceof StreamGraphSpec); - } - - @Test - public void testGetInputStream() { - Serde testSerde = new StringSerde(); - Config mockConf = mock(Config.class); - StreamApplication testApp = appSpec -> { - appSpec.getInputStream("myinput1"); - appSpec.getInputStream("myinput2", testSerde); - }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); - assertEquals(((StreamGraphSpec) appSpec.graph).inputStreams, - new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }); - assertEquals(((StreamGraphSpec) appSpec.graph).inputSerdes.get("myinput2"), testSerde); - } - - @Test - public void testGetOutputStream() { - Serde testSerde = new StringSerde(); - Config mockConf = mock(Config.class); - StreamApplication testApp = appSpec -> { - appSpec.getOutputStream("myoutput1"); - appSpec.getOutputStream("myoutput2", testSerde); - }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); - assertEquals(((StreamGraphSpec) appSpec.graph).outputStreams, - new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }); - assertEquals(((StreamGraphSpec) appSpec.graph).outputSerdes.get("myoutput2"), testSerde); - } - - @Test - public void testGetTable() { - TableDescriptor mockTd = mock(TableDescriptor.class); - Config mockConf = mock(Config.class); - StreamApplication testApp = appSpec -> { - appSpec.getTable(mockTd); - }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); - assertEquals(((StreamGraphSpec) appSpec.graph).tables, - new ArrayList() { { this.add(mockTd); } }); - } - - @Test - public void testSetDefaultSerde() { - Serde testSerde = new StringSerde(); - Config mockConf = mock(Config.class); - StreamApplication testApp = appSpec -> { - appSpec.setDefaultSerde(testSerde); - }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(testApp, mockConf); - assertEquals(((StreamGraphSpec) appSpec.graph).defaultSerde, testSerde); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java similarity index 62% rename from samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java rename to samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java index 654aab3810..1925062812 100644 --- a/samza-api/src/test/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java @@ -16,33 +16,37 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.operators; +package org.apache.samza.application.internal; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.samza.config.Config; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraph; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.serializers.Serde; import org.apache.samza.table.Table; -import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.*; /** - * Test class only to enable loading a fake impl of {@link StreamGraphSpec} for unit test in samza-api module. The real - * implementation of {@link StreamGraphSpec} is in samza-core module. + * Test implementation of {@link StreamGraph} used only in unit test */ -public class StreamGraphSpec implements StreamGraph { - public final Config config; - public Serde defaultSerde; - public final List inputStreams = new ArrayList<>(); - public final Map inputSerdes = new HashMap<>(); - public final List outputStreams = new ArrayList<>(); - public final Map outputSerdes = new HashMap<>(); - public final List tables = new ArrayList<>(); +public class TestStreamGraph implements StreamGraph { + final Config config; + final List inputStreams = new ArrayList<>(); + final List outputStreams = new ArrayList<>(); + final List tables = new ArrayList<>(); + final Map inputSerdes = new HashMap<>(); + final Map outputSerdes = new HashMap<>(); + Serde defaultSerde; - public StreamGraphSpec(Config config) { + public TestStreamGraph(Config config) { this.config = config; } @@ -53,33 +57,33 @@ public void setDefaultSerde(Serde serde) { @Override public MessageStream getInputStream(String streamId, Serde serde) { - inputStreams.add(streamId); - inputSerdes.put(streamId, serde); + this.inputStreams.add(streamId); + this.inputSerdes.put(streamId, serde); return mock(MessageStream.class); } @Override public MessageStream getInputStream(String streamId) { - inputStreams.add(streamId); + this.inputStreams.add(streamId); return mock(MessageStream.class); } @Override public OutputStream getOutputStream(String streamId, Serde serde) { - outputStreams.add(streamId); - outputSerdes.put(streamId, serde); + this.outputStreams.add(streamId); + this.outputSerdes.put(streamId, serde); return mock(OutputStream.class); } @Override public OutputStream getOutputStream(String streamId) { - outputStreams.add(streamId); + this.outputStreams.add(streamId); return mock(OutputStream.class); } @Override public Table> getTable(TableDescriptor tableDesc) { - tables.add(tableDesc); + this.tables.add(tableDesc); return mock(Table.class); } } diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java new file mode 100644 index 0000000000..48145b056a --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java @@ -0,0 +1,102 @@ +/* + * 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.application.internal; + +import java.util.ArrayList; +import java.util.List; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.config.Config; +import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; +import org.apache.samza.task.TaskFactory; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + + +/** + * Unit test for {@link TaskAppDescriptorImpl} + */ +public class TestTaskAppDescriptorImpl { + + private Config config = mock(Config.class); + + @Test + public void testConstructor() { + TaskApplication mockApp = mock(TaskApplication.class); + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(mockApp, config); + verify(mockApp, times(1)).describe(appDesc); + assertEquals(config, appDesc.config); + } + + @Test + public void testAddInputStreams() { + List testInputs = new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }; + TaskApplication testApp = appDesc -> appDesc.addInputStreams(testInputs); + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getInputStreams(), testInputs); + } + + @Test + public void testAddOutputStreams() { + List testOutputs = new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }; + TaskApplication testApp = appDesc -> appDesc.addOutputStreams(testOutputs); + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getOutputStreams(), testOutputs); + } + + @Test + public void testAddTables() { + List testTables = new ArrayList() { { this.add(mock(TableDescriptor.class)); } }; + TaskApplication testApp = appDesc -> appDesc.addTables(testTables); + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getTables(), testTables); + } + + @Test + public void testSetTaskFactory() { + TaskFactory mockTf = mock(TaskFactory.class); + TaskApplication testApp = appDesc -> appDesc.setTaskFactory(mockTf); + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getTaskFactory(), mockTf); + } + + @Test + public void testContextManager() { + ContextManager cntxMan = mock(ContextManager.class); + TaskApplication testApp = appDesc -> { + appDesc.withContextManager(cntxMan); + }; + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getContextManager(), cntxMan); + } + + @Test + public void testProcessorLifecycleListener() { + ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class); + TaskApplication testApp = appDesc -> { + appDesc.withProcessorLifecycleListenerFactory(mockFactory); + }; + TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); + } + +} diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java deleted file mode 100644 index 6665591453..0000000000 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppSpecImpl.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import java.util.ArrayList; -import java.util.List; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.config.Config; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.task.TaskFactory; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - - -/** - * Unit test for {@link TaskAppSpecImpl} - */ -public class TestTaskAppSpecImpl extends AppSpecImplTestBase { - @Test - public void testConstructor() { - TaskApplication mockApp = mock(TaskApplication.class); - Config mockConf = mock(Config.class); - TaskAppSpecImpl appSpec = new TaskAppSpecImpl(mockApp, mockConf); - verify(mockApp, times(1)).describe(appSpec); - assertEquals(mockConf, appSpec.config); - } - - @Test - public void testAddInputStreams() { - List testInputs = new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }; - TaskApplication testApp = appSpec -> appSpec.addInputStreams(testInputs); - Config mockConf = mock(Config.class); - TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); - assertEquals(appSpec.getInputStreams(), testInputs); - } - - @Test - public void testAddOutputStreams() { - List testOutputs = new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }; - TaskApplication testApp = appSpec -> appSpec.addOutputStreams(testOutputs); - Config mockConf = mock(Config.class); - TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); - assertEquals(appSpec.getOutputStreams(), testOutputs); - } - - @Test - public void testAddTables() { - List testTables = new ArrayList() { { this.add(mock(TableDescriptor.class)); } }; - TaskApplication testApp = appSpec -> appSpec.addTables(testTables); - Config mockConf = mock(Config.class); - TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); - assertEquals(appSpec.getTables(), testTables); - } - - @Test - public void testSetTaskFactory() { - TaskFactory mockTf = mock(TaskFactory.class); - TaskApplication testApp = appSpec -> appSpec.setTaskFactory(mockTf); - Config mockConf = mock(Config.class); - TaskAppSpecImpl appSpec = new TaskAppSpecImpl(testApp, mockConf); - assertEquals(appSpec.getTaskFactory(), mockTf); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java similarity index 70% rename from samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java rename to samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java index 3e41f70234..999e51fa33 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunner.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java @@ -16,47 +16,46 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime.internal; +package org.apache.samza.runtime; import java.time.Duration; import java.util.Map; -import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.config.Config; +import org.apache.samza.application.internal.AppDescriptorImpl; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; /** - * Test class for {@link ApplicationRunners} unit test + * Test class for {@link org.apache.samza.runtime.ApplicationRunners} unit test */ public class TestApplicationRunner implements ApplicationRunner { - public TestApplicationRunner(Config config) { + public TestApplicationRunner(AppDescriptorImpl appDesc) { } @Override - public void run(ApplicationSpec appSpec) { + public void run() { } @Override - public void kill(ApplicationSpec appSpec) { + public void kill() { } @Override - public ApplicationStatus status(ApplicationSpec appSpec) { + public ApplicationStatus status() { return null; } @Override - public void waitForFinish(ApplicationSpec appSpec) { + public void waitForFinish() { } @Override - public boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { + public boolean waitForFinish(Duration timeout) { return false; } diff --git a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java similarity index 73% rename from samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java rename to samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java index e857c89f64..772a5b23f5 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/internal/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java @@ -16,15 +16,18 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime.internal; +package org.apache.samza.runtime; import java.util.HashMap; import java.util.Map; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.TestStreamGraph; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.junit.Test; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; /** @@ -33,11 +36,13 @@ public class TestApplicationRunners { @Test - public void testFromConfig() { + public void testGetAppRunner() { Map configMap = new HashMap<>(); configMap.put("app.runner.class", TestApplicationRunner.class.getName()); + configMap.put("app.test.graph.class", TestStreamGraph.class.getName()); Config config = new MapConfig(configMap); - ApplicationRunner appRunner = ApplicationRunners.fromConfig(config); + StreamApplication app = mock(StreamApplication.class); + ApplicationRunner appRunner = ApplicationRunners.getApplicationRunner(app, config); assertTrue(appRunner instanceof TestApplicationRunner); } } diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java deleted file mode 100644 index b232de46d8..0000000000 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRuntimes.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.runtime; - -import java.util.HashMap; -import java.util.Map; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; -import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; -import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.internal.TestApplicationRunner; -import org.junit.Test; -import org.mockito.internal.util.reflection.Whitebox; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; - - -/** - * Unit test for {@link ApplicationRuntimes} - */ -public class TestApplicationRuntimes { - @Test - public void testGetApplicationRuntime() { - StreamApplication mockApp = mock(StreamApplication.class); - Map configMap = new HashMap<>(); - configMap.put("app.runner.class", TestApplicationRunner.class.getName()); - Config config = new MapConfig(configMap); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(mockApp, config); - StreamApplicationSpec appSpec = (StreamApplicationSpec) Whitebox.getInternalState(appRuntime, "appSpec"); - ApplicationRunner appRunner = (ApplicationRunner) Whitebox.getInternalState(appRuntime, "runner"); - assertEquals(appSpec.getConfig(), config); - assertTrue(appRunner instanceof TestApplicationRunner); - } -} diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java index 9cf2d3c017..63b473d0ae 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java @@ -1,19 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.application; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; -import org.apache.samza.task.TaskFactory; +import org.apache.samza.config.TaskConfig; import org.apache.samza.task.TaskFactoryUtil; +import static org.apache.samza.util.ScalaJavaUtil.*; + /** - * Created by yipan on 7/22/18. + * Util class to create {@link ApplicationBase} from the configuration. */ public class ApplicationClassUtils { + + /** + * Creates the {@link ApplicationBase} object from the {@code config} + * + * @param config the configuration of the application + * @return the {@link ApplicationBase} object + */ public static ApplicationBase fromConfig(Config config) { ApplicationConfig appConfig = new ApplicationConfig(config); if (appConfig.getAppClass() != null && !appConfig.getAppClass().isEmpty()) { + // app.class is configured try { Class appClass = (Class) Class.forName(appConfig.getAppClass()); if (StreamApplication.class.isAssignableFrom(appClass) || TaskApplication.class.isAssignableFrom(appClass)) { @@ -25,6 +53,11 @@ public static ApplicationBase fromConfig(Config config) { } } // no app.class defined. It has to be a legacy application with task.class configuration - return (TaskApplication) (appSpec) -> appSpec.setTaskFactory((TaskFactory) TaskFactoryUtil.createTaskFactory(config)); + new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( + () -> { + throw new ConfigException("No task class defined in the configuration."); + })); + return (TaskApplication) (appSpec) -> appSpec.setTaskFactory(TaskFactoryUtil.createTaskFactory(config)); } + } diff --git a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java index ca31040fa7..3ef6dbe680 100644 --- a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java +++ b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java @@ -19,19 +19,29 @@ package org.apache.samza.container; /** - * A Listener for {@link org.apache.samza.container.SamzaContainer} lifecycle events. + * A Listener for {@link SamzaContainer} lifecycle events. */ public interface SamzaContainerListener { /** - * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has successfully transitioned to + * Callback before the {@link SamzaContainer} is started + */ + void beforeStart(); + + /** + * Method invoked when the {@link SamzaContainer} has successfully transitioned to * the {@link org.apache.samza.SamzaContainerStatus#STARTED} state and is about to start the * {@link org.apache.samza.container.RunLoop} */ void onContainerStart(); /** - * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has successfully transitioned to + * Callback before the {@link SamzaContainer} is stopped + */ + void beforeStop(); + + /** + * Method invoked when the {@link SamzaContainer} has successfully transitioned to * {@link org.apache.samza.SamzaContainerStatus#STOPPED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    @@ -41,7 +51,7 @@ public interface SamzaContainerListener { void onContainerStop(); /** - * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has transitioned to + * Method invoked when the {@link SamzaContainer} has transitioned to * {@link org.apache.samza.SamzaContainerStatus#FAILED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    @@ -50,7 +60,4 @@ public interface SamzaContainerListener { */ void onContainerFailed(Throwable t); - void beforeStop(); - - void beforeStart(); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java index 1ef94ab876..d32380f031 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java @@ -18,14 +18,13 @@ */ package org.apache.samza.operators; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; + import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; @@ -43,6 +42,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + /** * This class defines: * 1) an implementation of {@link StreamGraph} that provides APIs for accessing {@link MessageStream}s to be used to @@ -280,5 +282,4 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } - } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index 1fe740c6f1..e1e1c5555e 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -23,7 +23,6 @@ import java.util.LinkedHashSet; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.functions.TimerFunction; @@ -104,7 +103,7 @@ public final OpCode getOpCode() { } /** - * Get the unique ID of this operator in the {@link StreamApplicationSpec}. + * Get the unique ID of this operator in the {@link org.apache.samza.operators.StreamGraph}. * @return the unique operator ID */ public final String getOpId() { 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 225c7629b7..b1bf79f7dc 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 @@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.TaskConfigJava; import org.apache.samza.container.IllegalContainerStateException; @@ -40,6 +41,8 @@ import org.apache.samza.coordinator.JobCoordinatorListener; import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.runtime.ProcessorContext; +import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.task.TaskFactory; @@ -96,7 +99,7 @@ public class StreamProcessor { private static final String CONTAINER_THREAD_NAME_FORMAT = "Samza StreamProcessor Container Thread-%d"; private final JobCoordinator jobCoordinator; - private final StreamProcessorLifecycleListener processorListener; + private final ProcessorLifecycleListener processorListener; private final TaskFactory taskFactory; private final Map customMetricsReporter; private final Config config; @@ -160,12 +163,12 @@ public State getState() { * @param processorListener listener to the StreamProcessor life cycle. */ public StreamProcessor(Config config, Map customMetricsReporters, - AsyncStreamTaskFactory asyncStreamTaskFactory, StreamProcessorLifecycleListener processorListener) { + AsyncStreamTaskFactory asyncStreamTaskFactory, ProcessorLifecycleListener processorListener) { this(config, customMetricsReporters, asyncStreamTaskFactory, processorListener, null); } /** - * Same as {@link StreamProcessor(Config, Map, AsyncStreamTaskFactory, StreamProcessorLifecycleListener)}, except task + * Same as {@link StreamProcessor(Config, Map, AsyncStreamTaskFactory, ProcessorLifecycleListener)}, except task * instances are created using the provided {@link StreamTaskFactory}. * @param config - config * @param customMetricsReporters metric Reporter @@ -173,7 +176,7 @@ public StreamProcessor(Config config, Map customMetrics * @param processorListener listener to the StreamProcessor life cycle */ public StreamProcessor(Config config, Map customMetricsReporters, - StreamTaskFactory streamTaskFactory, StreamProcessorLifecycleListener processorListener) { + StreamTaskFactory streamTaskFactory, ProcessorLifecycleListener processorListener) { this(config, customMetricsReporters, streamTaskFactory, processorListener, null); } @@ -189,7 +192,7 @@ JobCoordinator getCurrentJobCoordinator() { } public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, - StreamProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator) { + ProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator) { Preconditions.checkNotNull(processorListener, "ProcessorListener cannot be null."); this.taskFactory = taskFactory; this.config = config; @@ -272,6 +275,24 @@ public void stop() { } } + /** + * Get the {@link ProcessorContext} of this {@link StreamProcessor} + * @return the {@link ProcessorContext} object + */ + public ProcessorContext getProcessorContext() { + JobConfig jobConfig = new JobConfig(config); + return () -> String.format("%s-%s-%s", jobConfig.getName(), jobConfig.getJobId(), processorId); + } + + /** + * Get the {@code config} of this {@link StreamProcessor} + * + * @return {@code config} object + */ + public Config getConfig() { + return config; + } + SamzaContainer createSamzaContainer(String processorId, JobModel jobModel) { return SamzaContainer.apply(processorId, jobModel, config, ScalaJavaUtil.toScalaMap(customMetricsReporter), taskFactory); } @@ -348,10 +369,7 @@ public void onCoordinatorStop() { executorService.shutdownNow(); state = State.STOPPED; } - if (containerException != null) - processorListener.onFailure(containerException); - else - processorListener.onShutdown(); + processorListener.afterStop(containerException); } @@ -363,7 +381,7 @@ public void onCoordinatorFailure(Throwable throwable) { executorService.shutdownNow(); state = State.STOPPED; } - processorListener.onFailure(throwable); + processorListener.afterStop(throwable); } }; } @@ -379,7 +397,7 @@ class ContainerListener implements SamzaContainerListener { public void onContainerStart() { LOGGER.warn("Received container start notification for container: {} in stream processor: {}.", container, processorId); if (!processorOnStartCalled) { - processorListener.onStart(); + processorListener.afterStart(); processorOnStartCalled = true; } state = State.RUNNING; diff --git a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java deleted file mode 100644 index 0ac4d827a7..0000000000 --- a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessorLifecycleListener.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.samza.processor; - -import org.apache.samza.annotation.InterfaceStability; - - -/** - * This class listens to the life cycle events in a {@link StreamProcessor}, - */ -@InterfaceStability.Evolving -public interface StreamProcessorLifecycleListener { - /** - * Callback when the {@link StreamProcessor} is started - * This callback is invoked only once when {@link org.apache.samza.container.SamzaContainer} starts for the first time - * in the {@link StreamProcessor}. When there is a re-balance of tasks/partitions among the processors, the container - * may temporarily be "paused" and re-started again. For such re-starts, this callback is NOT invoked. - */ - void onStart(); - - /** - * Callback when the {@link StreamProcessor} is shut down. - */ - void onShutdown(); - - /** - * Callback when the {@link StreamProcessor} fails - * @param t Cause of the failure - */ - void onFailure(Throwable t); - - void beforeStop(); - - void beforeStart(); - -} diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 6ae0e84fe4..496e7cb39f 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -26,9 +26,10 @@ import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ApplicationConfig.ApplicationMode; import org.apache.samza.config.Config; @@ -41,7 +42,6 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.runtime.internal.ApplicationRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,11 +52,15 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { private static final Logger log = LoggerFactory.getLogger(AbstractApplicationRunner.class); + protected final AppDescriptorImpl appDesc; + protected final AppRuntimeExecutable appExecutable; protected final Config config; protected final Map metricsReporters = new HashMap<>(); - AbstractApplicationRunner(Config config) { - this.config = config; + AbstractApplicationRunner(AppDescriptorImpl appDesc) { + this.appDesc = appDesc; + this.config = appDesc.getConfig(); + this.appExecutable = getAppRuntimeExecutable(appDesc); } @Override @@ -65,29 +69,28 @@ public final void addMetricsReporters(Map metricsReport } @Override - public final void run(ApplicationSpec appSpec) { - getAppRuntimeExecutable(appSpec).run(); + public final void run() { + appExecutable.run(); } @Override - public final ApplicationStatus status(ApplicationSpec appSpec) { - return getAppRuntimeExecutable(appSpec).status(); + public final ApplicationStatus status() { + return appExecutable.status(); } @Override - public final void kill(ApplicationSpec appSpec) { - getAppRuntimeExecutable(appSpec).kill(); + public final void kill() { + appExecutable.kill(); } - @Deprecated @Override - public final void waitForFinish(ApplicationSpec appSpec) { - getAppRuntimeExecutable(appSpec).waitForFinish(Duration.ofSeconds(0)); + public final void waitForFinish() { + appExecutable.waitForFinish(Duration.ofSeconds(0)); } @Override - public final boolean waitForFinish(ApplicationSpec appSpec, Duration timeout) { - return getAppRuntimeExecutable(appSpec).waitForFinish(timeout); + public final boolean waitForFinish(Duration timeout) { + return appExecutable.waitForFinish(timeout); } interface AppRuntimeExecutable { @@ -109,22 +112,22 @@ interface AppRuntimeExecutable { } - abstract AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec); + abstract AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appSpec); - abstract AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec); + abstract AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appSpec); - final StreamManager buildAndStartStreamManager() { - StreamManager streamManager = new StreamManager(this.config); + StreamManager buildAndStartStreamManager() { + StreamManager streamManager = new StreamManager(config); streamManager.start(); return streamManager; } - final ExecutionPlan getExecutionPlan(OperatorSpecGraph graphSpec, StreamManager streamManager) throws Exception { + ExecutionPlan getExecutionPlan(OperatorSpecGraph graphSpec, StreamManager streamManager) throws Exception { return getExecutionPlan(graphSpec, null, streamManager); } /* package private */ - final ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, StreamManager streamManager) throws Exception { + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, StreamManager streamManager) throws Exception { // update application configs Map cfg = new HashMap<>(config); @@ -165,12 +168,12 @@ final void writePlanJsonFile(String planJson) { } } - private AppRuntimeExecutable getAppRuntimeExecutable(ApplicationSpec appSpec) { - if (appSpec instanceof StreamAppSpecImpl) { - return getStreamAppRuntimeExecutable((StreamAppSpecImpl) appSpec); + private AppRuntimeExecutable getAppRuntimeExecutable(ApplicationDescriptor appSpec) { + if (appSpec instanceof StreamAppDescriptorImpl) { + return getStreamAppRuntimeExecutable((StreamAppDescriptorImpl) appSpec); } - if (appSpec instanceof TaskAppSpecImpl) { - return getTaskAppRuntimeExecutable((TaskAppSpecImpl) appSpec); + if (appSpec instanceof TaskAppDescriptorImpl) { + return getTaskAppRuntimeExecutable((TaskAppDescriptorImpl) appSpec); } throw new IllegalArgumentException(String.format("The specified application %s is not valid. " + "Only StreamApplicationSpec and TaskApplicationSpec are supported.", appSpec.getClass().getName())); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java index 059d2a5b65..1e672a0376 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java @@ -27,11 +27,11 @@ * The class defines the basic command line arguments for Samza command line scripts. */ public class ApplicationRunnerCommandLine extends CommandLine { - public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; start, status, kill.") + public OptionSpec operationOpt = parser().accepts("operation", "The operation to perform; run, status, kill.") .withRequiredArg() .ofType(String.class) - .describedAs("operation=start") - .defaultsTo("start"); + .describedAs("operation=run") + .defaultsTo("run"); public ApplicationRunnerOperation getOperation(OptionSet options) { String rawOp = options.valueOf(operationOpt).toString(); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 9c43733a35..974e59507f 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -23,25 +23,23 @@ import joptsimple.OptionSpec; import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.config.Config; -import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.util.CommandLine; import org.apache.samza.util.Util; /** * This class contains the main() method used by start-app.sh. - * For a StreamApplication, it creates the {@link ApplicationRunner} based on the config, and then start the application. - * For a Samza job using low level task API, it will create the JobRunner to start it. + * It creates the {@link ApplicationRunner} based on the config, and then start the application. */ public class ApplicationRunnerMain { public static class ApplicationRunnerCommandLine extends CommandLine { public OptionSpec operationOpt = - parser().accepts("operation", "The operation to perform; start, status, kill.") + parser().accepts("operation", "The operation to perform; run, status, kill.") .withRequiredArg() .ofType(String.class) - .describedAs("operation=start") - .defaultsTo("start"); + .describedAs("operation=run") + .defaultsTo("run"); public ApplicationRunnerOperation getOperation(OptionSet options) { String rawOp = options.valueOf(operationOpt).toString(); @@ -56,17 +54,18 @@ public static void main(String[] args) throws Exception { Config config = Util.rewriteConfig(orgConfig); ApplicationRunnerOperation op = cmdLine.getOperation(options); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); + ApplicationRunner + appRunner = ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); switch (op) { case RUN: - appRuntime.run(); + appRunner.run(); break; case KILL: - appRuntime.kill(); + appRunner.kill(); break; case STATUS: - System.out.println(appRuntime.status()); + System.out.println(appRunner.status()); break; default: throw new IllegalArgumentException("Unrecognized operation: " + op); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 639e3ee263..a79f950a2f 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -31,9 +31,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; -import org.apache.samza.application.ProcessorLifecycleListener; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobCoordinatorConfig; @@ -46,11 +46,9 @@ import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; -import org.apache.samza.processor.StreamProcessorLifecycleListener; -import org.apache.samza.runtime.internal.ApplicationRunner; import org.apache.samza.system.StreamSpec; -import org.apache.samza.task.StreamOperatorTask; import org.apache.samza.task.TaskFactory; +import org.apache.samza.task.TaskFactoryUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,80 +64,63 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private final Set processors = ConcurrentHashMap.newKeySet(); private final CountDownLatch shutdownLatch = new CountDownLatch(1); private final AtomicInteger numProcessorsToStart = new AtomicInteger(); - private final AtomicInteger numProcessorsStopped = new AtomicInteger(); private final AtomicReference failure = new AtomicReference<>(); private ApplicationStatus appStatus = ApplicationStatus.New; - public LocalApplicationRunner(Config config) { - super(config); + public LocalApplicationRunner(AppDescriptorImpl appDesc) { + super(appDesc); this.uid = UUID.randomUUID().toString(); } - private final class LocalStreamProcessorLifeCycleListener implements StreamProcessorLifecycleListener { + private final class LocalStreamProcessorLifecycleListener implements ProcessorLifecycleListener { private StreamProcessor processor; - private final ProcessorLifecycleListener processorLifecycleListener; + private ProcessorLifecycleListener processorLifecycleListener; - private LocalStreamProcessorLifeCycleListener(ProcessorLifecycleListener processorLifecycleListener) { - this.processorLifecycleListener = processorLifecycleListener; + private LocalStreamProcessorLifecycleListener() { } - void setProcessor(StreamProcessor processor) { - this.processor = processor; + @Override + public void beforeStart() { + processorLifecycleListener.beforeStart(); } @Override - public void onStart() { - if (numProcessorsToStart.get() == 0) { + public void afterStart() { + processorLifecycleListener.afterStart(); + if (numProcessorsToStart.decrementAndGet() == 0) { appStatus = ApplicationStatus.Running; - processorLifecycleListener.afterStart(); } } @Override - public void onShutdown() { - processors.remove(processor); - processor = null; - - if (processors.isEmpty()) { - processorLifecycleListener.afterStop(); - shutdownAndNotify(); - } + public void beforeStop() { + processorLifecycleListener.beforeStop(); } @Override - public void onFailure(Throwable t) { + public void afterStop(Throwable t) { processors.remove(processor); processor = null; - if (failure.compareAndSet(null, t)) { - // shutdown the other processors - processors.forEach(StreamProcessor::stop); + processorLifecycleListener.afterStop(t); + if (t != null) { + // the processor stopped with failure + if (failure.compareAndSet(null, t)) { + // shutdown the other processors + processors.forEach(StreamProcessor::stop); + } } - if (processors.isEmpty()) { - // TODO: shutdown due to failure may not have the processorLifecycleListener.beforeStop() invoked. - // Hence, we don't have a corresponding processorLifecycleListener.afterStop() here either. + // successful shutdown shutdownAndNotify(); } } - @Override - public void beforeStop() { - // This is to record the number of processors that are stopped via normal shutdown sequence (i.e. calling sp.stop()) - // If this is the first call to stop in all processors in the application, we also call beforeStop() as well. - if (numProcessorsStopped.getAndIncrement() == 0) { - processorLifecycleListener.beforeStop(); - } - } - - @Override - public void beforeStart() { - // This is to record the number of processors that are to be started (i.e. calling sp.start()) - // If this is the first call to start in all processors in the application, we also call beforeStart() method as well. - if (numProcessorsToStart.getAndDecrement() == processors.size()) { - processorLifecycleListener.beforeStart(); - } + void setProcessor(StreamProcessor processor) { + this.processor = processor; + this.processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory(). + createInstance(processor.getProcessorContext(), processor.getConfig()); } private void shutdownAndNotify() { @@ -158,11 +139,11 @@ private void shutdownAndNotify() { } } - private class StreamAppExecutable implements AppRuntimeExecutable { - private final StreamAppSpecImpl streamApp; + class StreamAppExecutable implements AppRuntimeExecutable { + private final StreamAppDescriptorImpl appDesc; - private StreamAppExecutable(StreamAppSpecImpl streamApp) { - this.streamApp = streamApp; + private StreamAppExecutable(StreamAppDescriptorImpl appDesc) { + this.appDesc = appDesc; } @Override @@ -172,7 +153,7 @@ public void run() { streamManager = buildAndStartStreamManager(); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec)streamApp.getGraph()).getOperatorSpecGraph(), streamManager); + ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) appDesc.getGraph()).getOperatorSpecGraph(), streamManager); String executionPlanJson = plan.getPlanAsJson(); writePlanJsonFile(executionPlanJson); @@ -188,13 +169,13 @@ public void run() { throw new SamzaException("No jobs to start."); } plan.getJobConfigs().forEach(jobConfig -> { - LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); - LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(streamApp.getProcessorLifecycleListner()); - StreamProcessor processor = createStreamProcessor(jobConfig, ((StreamGraphSpec)streamApp.getGraph()).getOperatorSpecGraph(), - streamApp.getContextManager(), listener); - listener.setProcessor(processor); - processors.add(processor); - }); + LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); + LocalStreamProcessorLifecycleListener listener = new LocalStreamProcessorLifecycleListener(); + StreamProcessor processor = createStreamProcessor(jobConfig, ((StreamGraphSpec) appDesc.getGraph()).getOperatorSpecGraph(), + appDesc.getContextManager(), listener); + listener.setProcessor(processor); + processors.add(processor); + }); numProcessorsToStart.set(processors.size()); // 4. start the StreamProcessors @@ -202,7 +183,7 @@ public void run() { } catch (Throwable throwable) { appStatus = ApplicationStatus.unsuccessfulFinish(throwable); shutdownLatch.countDown(); - throw new SamzaException(String.format("Failed to start application: %s.", streamApp), throwable); + throw new SamzaException(String.format("Failed to start application: %s.", appDesc), throwable); } finally { if (streamManager != null) { streamManager.stop(); @@ -222,25 +203,25 @@ public ApplicationStatus status() { @Override public boolean waitForFinish(Duration timeout) { - return LocalApplicationRunner.this.waitForFinish(timeout); + return LocalApplicationRunner.this.localWaitForFinish(timeout); } } - private class TaskAppExecutable implements AppRuntimeExecutable { - private final TaskAppSpecImpl appSpec; + class TaskAppExecutable implements AppRuntimeExecutable { + private final TaskAppDescriptorImpl appDesc; private StreamProcessor sp; - private TaskAppExecutable(TaskAppSpecImpl appSpec) { - this.appSpec = appSpec; + private TaskAppExecutable(TaskAppDescriptorImpl appDesc) { + this.appDesc = appDesc; } @Override public void run() { - LOG.info("LocalApplicationRunner will start task " + appSpec.getGlobalAppId()); - LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(appSpec.getProcessorLifecycleListner()); + LOG.info("LocalApplicationRunner will start task " + appDesc.getGlobalAppId()); + LocalStreamProcessorLifecycleListener listener = new LocalStreamProcessorLifecycleListener(); - sp = createStreamProcessor(config, appSpec.getTaskFactory(), listener); + sp = createStreamProcessor(config, appDesc.getTaskFactory(), listener); numProcessorsToStart.set(1); listener.setProcessor(sp); @@ -259,12 +240,12 @@ public ApplicationStatus status() { @Override public boolean waitForFinish(Duration timeout) { - return LocalApplicationRunner.this.waitForFinish(timeout); + return LocalApplicationRunner.this.localWaitForFinish(timeout); } } - private boolean waitForFinish(Duration timeout) { + private boolean localWaitForFinish(Duration timeout) { long timeoutInMs = timeout.toMillis(); boolean finished = true; @@ -344,7 +325,7 @@ private void createStreams(String planId, StreamProcessor createStreamProcessor( Config config, TaskFactory taskFactory, - StreamProcessorLifecycleListener listener) { + ProcessorLifecycleListener listener) { return new StreamProcessor(config, this.metricsReporters, taskFactory, listener, null); } @@ -360,8 +341,8 @@ StreamProcessor createStreamProcessor( Config config, OperatorSpecGraph graph, ContextManager contextManager, - StreamProcessorLifecycleListener listener) { - TaskFactory taskFactory = () -> new StreamOperatorTask(graph, contextManager); + ProcessorLifecycleListener listener) { + TaskFactory taskFactory = TaskFactoryUtil.createTaskFactory(graph, contextManager); return new StreamProcessor(config, this.metricsReporters, taskFactory, listener, null); } @@ -376,13 +357,13 @@ CountDownLatch getShutdownLatch() { } @Override - protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { - return new TaskAppExecutable(appSpec); + protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appDesc) { + return new TaskAppExecutable(appDesc); } @Override - protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { - return new StreamAppExecutable(appSpec); + protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appDesc) { + return new StreamAppExecutable(appDesc); } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index fd7402b9a1..4a382602ee 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -27,9 +27,9 @@ import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.AppSpecImpl; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -40,8 +40,8 @@ import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.model.JobModel; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.task.StreamOperatorTask; import org.apache.samza.task.TaskFactory; +import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.SamzaUncaughtExceptionHandler; import org.apache.samza.util.ScalaJavaUtil; import org.slf4j.Logger; @@ -82,20 +82,59 @@ public static void main(String[] args) throws Exception { MDC.put("jobName", jobName); MDC.put("jobId", jobId); - AppSpecImpl appSpec = getAppSpec(config); - run(appSpec, containerId, jobModel, config); + AppDescriptorImpl appDesc = getAppDesc(config); + run(appDesc, containerId, jobModel, config); System.exit(0); } - private static AppSpecImpl getAppSpec(Config config) { + private static class LocalContainerLifecycleListener implements SamzaContainerListener { + private final ProcessorLifecycleListener pListener; + + LocalContainerLifecycleListener(ProcessorLifecycleListener pListener) { + this.pListener = pListener; + } + + @Override + public void beforeStart() { + log.info("Starting Local Container"); + pListener.beforeStart(); + } + + @Override + public void onContainerStart() { + log.info("Local Container Started"); + pListener.afterStart(); + } + + @Override + public void beforeStop() { + log.info("Stopping Local Container"); + pListener.beforeStop(); + } + + @Override + public void onContainerStop() { + log.info("Container Stopped Successfully"); + pListener.afterStop(null); + } + + @Override + public void onContainerFailed(Throwable t) { + log.info("Container Stopped with Failure"); + containerRunnerException = t; + pListener.afterStop(t); + } + } + + private static AppDescriptorImpl getAppDesc(Config config) { ApplicationBase userApp = ApplicationClassUtils.fromConfig(config); - return userApp instanceof StreamApplication ? new StreamAppSpecImpl((StreamApplication) userApp, config) : - new TaskAppSpecImpl((TaskApplication) userApp, config); + return userApp instanceof StreamApplication ? new StreamAppDescriptorImpl((StreamApplication) userApp, config) : + new TaskAppDescriptorImpl((TaskApplication) userApp, config); } - private static void run(AppSpecImpl appSpec, String containerId, JobModel jobModel, Config config) { - TaskFactory taskFactory = getTaskFactory(appSpec); + private static void run(AppDescriptorImpl appDesc, String containerId, JobModel jobModel, Config config) { + TaskFactory taskFactory = getTaskFactory(appDesc); SamzaContainer container = SamzaContainer$.MODULE$.apply( containerId, jobModel, @@ -103,37 +142,12 @@ private static void run(AppSpecImpl appSpec, String containerId, JobModel jobMod ScalaJavaUtil.toScalaMap(new HashMap<>()), taskFactory); - // TODO: this is a temporary solution to inject the lifecycle listeners before we fix SAMZA-1168 - container.setContainerListener( - new SamzaContainerListener() { - @Override - public void onContainerStart() { - log.info("Container Started"); - appSpec.getProcessorLifecycleListner().afterStart(); - } - - @Override - public void onContainerStop() { - log.info("Container Stopped"); - appSpec.getProcessorLifecycleListner().afterStop(); - } - - @Override - public void onContainerFailed(Throwable t) { - log.info("Container Failed"); - containerRunnerException = t; - } - - @Override - public void beforeStop() { - appSpec.getProcessorLifecycleListner().beforeStop(); - } + JobConfig jobConfig = new JobConfig(config); + ProcessorContext pContext = () -> String.format("%s-%s-%s", jobConfig.getName(), jobConfig.getJobId(), containerId); + ProcessorLifecycleListener pListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(pContext, config); - @Override - public void beforeStart() { - appSpec.getProcessorLifecycleListner().beforeStart(); - } - }); + // TODO: this is a temporary solution to inject the lifecycle listeners before we fix SAMZA-1168 + container.setContainerListener(new LocalContainerLifecycleListener(pListener)); ContainerHeartbeatMonitor heartbeatMonitor = createContainerHeartbeatMonitor(container); if (heartbeatMonitor != null) { @@ -152,13 +166,13 @@ public void beforeStart() { } } - private static TaskFactory getTaskFactory(AppSpecImpl appSpec) { - if (appSpec instanceof StreamAppSpecImpl) { - StreamAppSpecImpl streamAppSpec = (StreamAppSpecImpl) appSpec; - return () -> new StreamOperatorTask(((StreamGraphSpec) streamAppSpec.getGraph()).getOperatorSpecGraph(), - streamAppSpec.getContextManager()); + private static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { + if (appDesc instanceof StreamAppDescriptorImpl) { + StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; + return TaskFactoryUtil.createTaskFactory(((StreamGraphSpec) streamAppDesc.getGraph()).getOperatorSpecGraph(), + streamAppDesc.getContextManager()); } - return ((TaskAppSpecImpl) appSpec).getTaskFactory(); + return ((TaskAppDescriptorImpl) appDesc).getTaskFactory(); } /** diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 47b70f2319..33fe88fa53 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -22,8 +22,9 @@ import java.time.Duration; import java.util.UUID; import org.apache.samza.SamzaException; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -34,7 +35,6 @@ import org.apache.samza.job.JobRunner; import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.runtime.internal.ApplicationRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,16 +49,16 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; - public RemoteApplicationRunner(Config config) { - super(config); + public RemoteApplicationRunner(AppDescriptorImpl appDesc) { + super(appDesc); } class TaskAppExecutable implements AppRuntimeExecutable { - final TaskAppSpecImpl taskApp; + final TaskAppDescriptorImpl appDesc; final JobRunner jobRunner; - TaskAppExecutable(TaskAppSpecImpl appSpec) { - this.taskApp = appSpec; + TaskAppExecutable(TaskAppDescriptorImpl appDesc) { + this.appDesc = appDesc; this.jobRunner = new JobRunner(config); } @@ -79,16 +79,16 @@ public ApplicationStatus status() { @Override public boolean waitForFinish(Duration timeout) { - return RemoteApplicationRunner.this.waitForFinish(timeout); + return RemoteApplicationRunner.this.remoteWaitForFinish(timeout); } } class StreamAppExecutable implements AppRuntimeExecutable { - final StreamAppSpecImpl streamApp; + final StreamAppDescriptorImpl appDesc; - StreamAppExecutable(StreamAppSpecImpl appSpec) { - this.streamApp = appSpec; + StreamAppExecutable(StreamAppDescriptorImpl appDesc) { + this.appDesc = appDesc; } @Override @@ -102,7 +102,7 @@ public void run() { LOG.info("The start id for this start is {}", runId); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) streamApp.getGraph()).getOperatorSpecGraph(), runId, streamManager); + ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) appDesc.getGraph()).getOperatorSpecGraph(), runId, streamManager); writePlanJsonFile(plan.getPlanAsJson()); // 2. create the necessary streams @@ -113,10 +113,10 @@ public void run() { // 3. submit jobs for remote execution plan.getJobConfigs().forEach(jobConfig -> { - LOG.info("Starting job {} with config {}", jobConfig.getName(), jobConfig); - JobRunner runner = new JobRunner(jobConfig); - runner.run(true); - }); + LOG.info("Starting job {} with config {}", jobConfig.getName(), jobConfig); + JobRunner runner = new JobRunner(jobConfig); + runner.run(true); + }); } catch (Throwable t) { throw new SamzaException("Failed to start application", t); } finally { @@ -156,19 +156,19 @@ public ApplicationStatus status() { @Override public boolean waitForFinish(Duration timeout) { - return RemoteApplicationRunner.this.waitForFinish(timeout); + return RemoteApplicationRunner.this.remoteWaitForFinish(timeout); } } @Override - protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { - return new TaskAppExecutable(appSpec); + protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appDesc) { + return new TaskAppExecutable(appDesc); } @Override - protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { - return new StreamAppExecutable(appSpec); + protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appDesc) { + return new StreamAppExecutable(appDesc); } /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { @@ -178,7 +178,7 @@ protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl a return status; } - private boolean waitForFinish(Duration timeout) { + private boolean remoteWaitForFinish(Duration timeout) { JobConfig jobConfig = new JobConfig(config); boolean finished = true; long timeoutInMs = timeout.toMillis(); diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java index 6a4076cb7c..2daffeadc7 100644 --- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java +++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java @@ -18,16 +18,15 @@ */ package org.apache.samza.task; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; -import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.system.EndOfStreamMessage; -import org.apache.samza.system.MessageType; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; +import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.impl.InputOperatorImpl; import org.apache.samza.operators.impl.OperatorImplGraph; +import org.apache.samza.system.EndOfStreamMessage; import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.MessageType; import org.apache.samza.system.SystemStream; import org.apache.samza.system.WatermarkMessage; import org.apache.samza.util.Clock; @@ -72,7 +71,7 @@ public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextMan *

    * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}. * The {@link org.apache.samza.operators.StreamGraphSpec} sets the input and output streams and the task-wide - * context manager using the {@link StreamApplicationSpec} APIs, + * context manager using the {@link org.apache.samza.operators.StreamGraph} APIs, * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. After the * {@link org.apache.samza.operators.StreamGraphSpec} is initialized once by the application, it then creates * an immutable {@link OperatorSpecGraph} accordingly, which is passed in to this class to create the {@link OperatorImplGraph} diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index 3094d44ad5..62701690f2 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -18,6 +18,7 @@ */ package org.apache.samza.task; +import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; @@ -27,9 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.ExecutorService; - -import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; +import static org.apache.samza.util.ScalaJavaUtil.*; /** * This class provides utility functions to load task factory classes based on config, and to wrap {@link StreamTaskFactory} @@ -38,6 +37,17 @@ public class TaskFactoryUtil { private static final Logger log = LoggerFactory.getLogger(TaskFactoryUtil.class); + /** + * This method creates a task factory class based on the {@link OperatorSpecGraph} and {@link ContextManager} + * + * @param specGraph the {@link OperatorSpecGraph} + * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph} + * @return a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} + */ + public static TaskFactory createTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) { + return (StreamTaskFactory) () -> new StreamOperatorTask(specGraph, contextManager); + } + /** * This method creates a task factory class based on the configuration * @@ -70,28 +80,22 @@ private static TaskFactory fromTaskClassConfig(Config config) { } if (isAsyncTaskClass) { - return new AsyncStreamTaskFactory() { - @Override - public AsyncStreamTask createInstance() { - try { - return (AsyncStreamTask) Class.forName(taskClassName).newInstance(); - } catch (Throwable t) { - log.error("Error loading AsyncStreamTask class: {}. error: {}", taskClassName, t); - throw new SamzaException(String.format("Error loading AsyncStreamTask class: %s", taskClassName), t); - } + return (AsyncStreamTaskFactory) () -> { + try { + return (AsyncStreamTask) Class.forName(taskClassName).newInstance(); + } catch (Throwable t) { + log.error("Error loading AsyncStreamTask class: {}. error: {}", taskClassName, t); + throw new SamzaException(String.format("Error loading AsyncStreamTask class: %s", taskClassName), t); } }; } - return new StreamTaskFactory() { - @Override - public StreamTask createInstance() { - try { - return (StreamTask) Class.forName(taskClassName).newInstance(); - } catch (Throwable t) { - log.error("Error loading StreamTask class: {}. error: {}", taskClassName, t); - throw new SamzaException(String.format("Error loading StreamTask class: %s", taskClassName), t); - } + return (StreamTaskFactory) () -> { + try { + return (StreamTask) Class.forName(taskClassName).newInstance(); + } catch (Throwable t) { + log.error("Error loading StreamTask class: {}. error: {}", taskClassName, t); + throw new SamzaException(String.format("Error loading StreamTask class: %s", taskClassName), t); } }; } @@ -120,12 +124,7 @@ public static TaskFactory finalizeTaskFactory(TaskFactory factory, boolean singl if (!singleThreadMode && !isAsyncTaskClass) { log.info("Converting StreamTask to AsyncStreamTaskAdapter when running StreamTask with multiple threads"); - return new AsyncStreamTaskFactory() { - @Override - public AsyncStreamTask createInstance() { - return new AsyncStreamTaskAdapter(((StreamTaskFactory) factory).createInstance(), taskThreadPool); - } - }; + return (AsyncStreamTaskFactory) () -> new AsyncStreamTaskAdapter(((StreamTaskFactory) factory).createInstance(), taskThreadPool); } return factory; diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 7f506f072f..f46913e847 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -889,8 +889,8 @@ class SamzaContainer( *
    * Implementation: Stops the [[RunLoop]], which will eventually transition the container from * [[SamzaContainerStatus.STARTED]] to either [[SamzaContainerStatus.STOPPED]] or [[SamzaContainerStatus.FAILED]]]. - * Based on the final `status`, [[SamzaContainerListener#onContainerStop(boolean)]] or - * [[SamzaContainerListener#onContainerFailed(Throwable)]] will be invoked respectively. + * Based on the final `status`, [[SamzaContainerListener#onContainerStop()]] or + * [[SamzaContainerListener#onContainerFailed(Throwable]] will be invoked respectively. * * @throws SamzaException, Thrown when the container has already been stopped or failed */ diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index eb570c020b..c95def7d6c 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,7 +19,7 @@ package org.apache.samza.job.local -import org.apache.samza.application.internal.{StreamAppSpecImpl, TaskAppSpecImpl} +import org.apache.samza.application.internal.{StreamAppDescriptorImpl, TaskAppDescriptorImpl} import org.apache.samza.application.{ApplicationClassUtils, StreamApplication, TaskApplication} import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ @@ -76,11 +76,11 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val taskFactory : TaskFactory[_] = ApplicationClassUtils.fromConfig(config) match { case app if (app.isInstanceOf[TaskApplication]) => { - val appSpec = new TaskAppSpecImpl(app.asInstanceOf[TaskApplication], config) + val appSpec = new TaskAppDescriptorImpl(app.asInstanceOf[TaskApplication], config) appSpec.getTaskFactory } case app if (app.isInstanceOf[StreamApplication]) => { - val appSpec = new StreamAppSpecImpl(app.asInstanceOf[StreamApplication], config) + val appSpec = new StreamAppDescriptorImpl(app.asInstanceOf[StreamApplication], config) new StreamTaskFactory { override def createInstance(): StreamTask = new StreamOperatorTask(appSpec.getGraph.asInstanceOf[StreamGraphSpec].getOperatorSpecGraph, appSpec.getContextManager) @@ -107,6 +107,14 @@ class ThreadJobFactory extends StreamJobFactory with Logging { override def onContainerStart(): Unit = { } + + override def beforeStop(): Unit = { + + } + + override def beforeStart(): Unit = { + + } } try { diff --git a/samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java b/samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java deleted file mode 100644 index 7d40ebd9e2..0000000000 --- a/samza-core/src/test/java/org/apache/samza/application/TestAppClassUtils.java +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.samza.application; - -/** - * Created by yipan on 8/3/18. - */ -public class TestAppClassUtils { -} diff --git a/samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java new file mode 100644 index 0000000000..4b7f9abc89 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java @@ -0,0 +1,80 @@ +/* + * 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.application; + +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.task.TestStreamTask; +import org.junit.Test; + +import static org.junit.Assert.*; + + +/** + * Unit tests for {@link ApplicationClassUtils} + */ +public class TestApplicationClassUtils { + + @Test + public void testStreamAppClass() { + Map configMap = new HashMap<>(); + configMap.put(ApplicationConfig.APP_CLASS, TestStreamApplication.class.getName()); + ApplicationBase app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + assertTrue(app instanceof TestStreamApplication); + + configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); + app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + assertTrue(app instanceof TestStreamApplication); + } + + @Test + public void testTaskAppClass() { + Map configMap = new HashMap<>(); + configMap.put(ApplicationConfig.APP_CLASS, TestTaskApplication.class.getName()); + ApplicationBase app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + assertTrue(app instanceof TestTaskApplication); + + configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); + app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + assertTrue(app instanceof TestTaskApplication); + } + + @Test + public void testTaskClassOnly() { + Map configMap = new HashMap<>(); + configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); + Config config = new MapConfig(configMap); + ApplicationBase app = ApplicationClassUtils.fromConfig(config); + assertTrue(app instanceof TaskApplication); + TaskAppDescriptorImpl appSpec = new TaskAppDescriptorImpl((TaskApplication) app, config); + assertTrue(appSpec.getTaskFactory().createInstance() instanceof TestStreamTask); + } + + @Test(expected = ConfigException.class) + public void testNoAppClassNoTaskClass() { + Map configMap = new HashMap<>(); + ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java new file mode 100644 index 0000000000..8beb76f0db --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java @@ -0,0 +1,29 @@ +/* + * 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.application; + +/** + * Test class of {@link StreamApplication} for unit tests + */ +public class TestStreamApplication implements StreamApplication { + @Override + public void describe(StreamAppDescriptor appSpec) { + + } +} diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java new file mode 100644 index 0000000000..97113e84bd --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java @@ -0,0 +1,29 @@ +/* + * 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.application; + +/** + * Test class of {@link TaskApplication} for unit tests + */ +public class TestTaskApplication implements TaskApplication { + @Override + public void describe(TaskAppDescriptor appSpec) { + + } +} 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 6184af08a0..94dbcfc5fb 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 @@ -34,8 +34,8 @@ import org.apache.samza.config.TaskConfig; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.Serde; @@ -48,11 +48,8 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; public class TestExecutionPlanner { diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index cefe128070..3d77e40be1 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -19,14 +19,19 @@ package org.apache.samza.execution; +import java.time.Duration; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.SerializerConfig; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; import org.apache.samza.serializers.JsonSerdeV2; @@ -37,18 +42,10 @@ import org.apache.samza.system.StreamSpec; import org.junit.Test; -import java.time.Duration; -import java.util.Base64; -import java.util.HashMap; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class TestJobNode { diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java index 70547274d7..27f340ffa6 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java @@ -81,7 +81,7 @@ public void setUp() { @Test public void join() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -105,14 +105,14 @@ public void joinWithSelfThrowsException() throws Exception { inStream.join(inStream, new TestJoinFunction(), integerSerde, kvSerde, kvSerde, JOIN_TTL, "join"); - createStreamOperatorTask(new SystemClock(), graphSpec); // should throw an exception + createStreamOperatorTask(new SystemClock(), graphSpec, null); // should throw an exception } @Test public void joinFnInitAndClose() throws Exception { TestJoinFunction joinFn = new TestJoinFunction(); StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(joinFn); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); MessageCollector messageCollector = mock(MessageCollector.class); @@ -131,7 +131,7 @@ public void joinFnInitAndClose() throws Exception { @Test public void joinReverse() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -147,7 +147,7 @@ public void joinReverse() throws Exception { @Test public void joinNoMatch() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -162,7 +162,7 @@ public void joinNoMatch() throws Exception { @Test public void joinNoMatchReverse() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -177,7 +177,7 @@ public void joinNoMatchReverse() throws Exception { @Test public void joinRetainsLatestMessageForKey() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -195,7 +195,7 @@ public void joinRetainsLatestMessageForKey() throws Exception { @Test public void joinRetainsLatestMessageForKeyReverse() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -213,7 +213,7 @@ public void joinRetainsLatestMessageForKeyReverse() throws Exception { @Test public void joinRetainsMatchedMessages() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -236,7 +236,7 @@ public void joinRetainsMatchedMessages() throws Exception { @Test public void joinRetainsMatchedMessagesReverse() throws Exception { StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -260,7 +260,7 @@ public void joinRetainsMatchedMessagesReverse() throws Exception { public void joinRemovesExpiredMessages() throws Exception { TestClock testClock = new TestClock(); StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -280,7 +280,7 @@ public void joinRemovesExpiredMessages() throws Exception { public void joinRemovesExpiredMessagesReverse() throws Exception { TestClock testClock = new TestClock(); StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec); + StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec, null); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -296,7 +296,8 @@ public void joinRemovesExpiredMessagesReverse() throws Exception { assertTrue(output.isEmpty()); } - private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec graphSpec) throws Exception { + private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec graphSpec, ContextManager contextManager) + throws Exception { TaskContextImpl taskContext = mock(TaskContextImpl.class); when(taskContext.getSystemStreamPartitions()).thenReturn(ImmutableSet @@ -311,7 +312,7 @@ private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec when(taskContext.getStore(eq("jobName-jobId-join-j1-R"))) .thenReturn(new TestInMemoryStore(integerSerde, timestampedValueSerde)); - StreamOperatorTask sot = new StreamOperatorTask(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager(), clock); + StreamOperatorTask sot = new StreamOperatorTask(graphSpec.getOperatorSpecGraph(), contextManager, clock); sot.init(config, taskContext); return sot; } diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 77bdaadaea..f24facb0cf 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -21,7 +21,6 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; - import java.io.Serializable; import java.time.Duration; import java.util.ArrayList; @@ -55,7 +54,6 @@ import org.apache.samza.operators.functions.InitableFunction; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; -import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; @@ -71,15 +69,13 @@ import org.apache.samza.testUtils.StreamTestUtils; import org.apache.samza.util.Clock; import org.apache.samza.util.SystemClock; +import org.apache.samza.util.TimestampedValue; import org.junit.After; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class TestOperatorImplGraph { diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java index 0ef6680c66..8082b0507c 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java @@ -19,23 +19,30 @@ package org.apache.samza.operators.impl; - import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.samza.Partition; import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.impl.store.TestInMemoryStore; import org.apache.samza.operators.impl.store.TimeSeriesKeySerde; -import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.triggers.FiringType; import org.apache.samza.operators.triggers.Trigger; import org.apache.samza.operators.triggers.Triggers; @@ -57,21 +64,9 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Collections; - import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class TestWindowOperator { private final TaskCoordinator taskCoordinator = mock(TaskCoordinator.class); diff --git a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java index 052aa29554..88fb3f1935 100644 --- a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java +++ b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java @@ -19,6 +19,13 @@ package org.apache.samza.processor; import com.google.common.collect.ImmutableMap; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaContainerStatus; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; @@ -30,31 +37,23 @@ import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.processor.StreamProcessor.State; +import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.task.StreamTask; import org.apache.samza.task.StreamTaskFactory; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import org.mockito.Mockito; import org.powermock.api.mockito.PowerMockito; + import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class TestStreamProcessor { private ConcurrentMap processorListenerState; private enum ListenerCallback { - ON_START, ON_SHUTDOWN, ON_FAILURE + BEFORE_START, AFTER_START, BEFORE_STOP, AFTER_STOP, AFTER_STOP_WITH_FAILURE } @Before @@ -62,9 +61,11 @@ public void before() { Mockito.reset(); processorListenerState = new ConcurrentHashMap() { { - put(ListenerCallback.ON_START, false); - put(ListenerCallback.ON_FAILURE, false); - put(ListenerCallback.ON_SHUTDOWN, false); + put(ListenerCallback.BEFORE_START, false); + put(ListenerCallback.AFTER_START, false); + put(ListenerCallback.BEFORE_STOP, false); + put(ListenerCallback.AFTER_STOP, false); + put(ListenerCallback.AFTER_STOP_WITH_FAILURE, false); } }; } @@ -83,7 +84,7 @@ public TestableStreamProcessor( Config config, Map customMetricsReporters, StreamTaskFactory streamTaskFactory, - StreamProcessorLifecycleListener processorListener, + ProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator, SamzaContainer container) { super(config, customMetricsReporters, streamTaskFactory, processorListener, jobCoordinator); @@ -144,22 +145,31 @@ public void testStopByProcessor() throws InterruptedException { new MapConfig(), new HashMap<>(), mock(StreamTaskFactory.class), - new StreamProcessorLifecycleListener() { + new ProcessorLifecycleListener() { @Override - public void onStart() { - processorListenerState.put(ListenerCallback.ON_START, true); + public void afterStart() { + processorListenerState.put(ListenerCallback.AFTER_START, true); processorListenerStart.countDown(); } @Override - public void onShutdown() { - processorListenerState.put(ListenerCallback.ON_SHUTDOWN, true); - processorListenerStop.countDown(); + public void afterStop(Throwable t) { + if (t != null) { + processorListenerState.put(ListenerCallback.AFTER_STOP_WITH_FAILURE, true); + } else { + processorListenerState.put(ListenerCallback.AFTER_STOP, true); + processorListenerStop.countDown(); + } } @Override - public void onFailure(Throwable t) { - processorListenerState.put(ListenerCallback.ON_FAILURE, true); + public void beforeStop() { + processorListenerState.put(ListenerCallback.BEFORE_STOP, true); + } + + @Override + public void beforeStart() { + processorListenerState.put(ListenerCallback.BEFORE_START, true); } }, mockJobCoordinator, @@ -204,9 +214,11 @@ public void onFailure(Throwable t) { processorListenerStop.await(); // Assertions on which callbacks are expected to be invoked - Assert.assertTrue(processorListenerState.get(ListenerCallback.ON_START)); - Assert.assertTrue(processorListenerState.get(ListenerCallback.ON_SHUTDOWN)); - Assert.assertFalse(processorListenerState.get(ListenerCallback.ON_FAILURE)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_START)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_START)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_STOP)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_STOP)); + Assert.assertFalse(processorListenerState.get(ListenerCallback.AFTER_STOP_WITH_FAILURE)); } /** @@ -215,7 +227,7 @@ public void onFailure(Throwable t) { * * Assertions: * - JobCoordinator has been stopped from the JobCoordinatorListener callback - * - StreamProcessorLifecycleListener#onFailure(Throwable) has been invoked + * - ProcessorLifecycleListener#afterStop(Throwable) has been invoked w/ non-null Throwable */ @Test public void testContainerFailureCorrectlyStopsProcessor() throws InterruptedException { @@ -242,22 +254,32 @@ public void testContainerFailureCorrectlyStopsProcessor() throws InterruptedExce new MapConfig(), new HashMap<>(), mock(StreamTaskFactory.class), - new StreamProcessorLifecycleListener() { + new ProcessorLifecycleListener() { @Override - public void onStart() { - processorListenerState.put(ListenerCallback.ON_START, true); + public void beforeStart() { + processorListenerState.put(ListenerCallback.BEFORE_START, true); } @Override - public void onShutdown() { - processorListenerState.put(ListenerCallback.ON_SHUTDOWN, true); + public void afterStart() { + processorListenerState.put(ListenerCallback.AFTER_START, true); } @Override - public void onFailure(Throwable t) { - processorListenerState.put(ListenerCallback.ON_FAILURE, true); - actualThrowable.getAndSet(t); - processorListenerFailed.countDown(); + public void beforeStop() { + processorListenerState.put(ListenerCallback.BEFORE_STOP, true); + } + + @Override + public void afterStop(Throwable t) { + if (t == null) { + // successful stop + processorListenerState.put(ListenerCallback.AFTER_STOP, true); + } else { + processorListenerState.put(ListenerCallback.AFTER_STOP_WITH_FAILURE, true); + actualThrowable.getAndSet(t); + processorListenerFailed.countDown(); + } } }, mockJobCoordinator, @@ -296,16 +318,18 @@ public void onFailure(Throwable t) { processorListenerFailed.await(30, TimeUnit.SECONDS)); Assert.assertEquals(expectedThrowable, actualThrowable.get()); - Assert.assertFalse(processorListenerState.get(ListenerCallback.ON_SHUTDOWN)); - Assert.assertTrue(processorListenerState.get(ListenerCallback.ON_START)); - Assert.assertTrue(processorListenerState.get(ListenerCallback.ON_FAILURE)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_START)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_START)); + Assert.assertFalse(processorListenerState.get(ListenerCallback.BEFORE_STOP)); + Assert.assertFalse(processorListenerState.get(ListenerCallback.AFTER_STOP)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_STOP_WITH_FAILURE)); } @Test public void testStartOperationShouldBeIdempotent() { JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); Mockito.doNothing().when(mockJobCoordinator).start(); - StreamProcessorLifecycleListener lifecycleListener = Mockito.mock(StreamProcessorLifecycleListener.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); StreamProcessor streamProcessor = new StreamProcessor(new MapConfig(), new HashMap<>(), null, lifecycleListener, mockJobCoordinator); Assert.assertEquals(State.NEW, streamProcessor.getState()); streamProcessor.start(); @@ -322,7 +346,7 @@ public void testStartOperationShouldBeIdempotent() { @Test public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() { JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); - StreamProcessorLifecycleListener lifecycleListener = Mockito.mock(StreamProcessorLifecycleListener.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); SamzaContainer mockSamzaContainer = Mockito.mock(SamzaContainer.class); MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); StreamProcessor streamProcessor = new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator); @@ -372,7 +396,7 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() { @Test public void testOnNewJobModelShouldResultInValidStateTransitions() throws Exception { JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); - StreamProcessorLifecycleListener lifecycleListener = Mockito.mock(StreamProcessorLifecycleListener.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); SamzaContainer mockSamzaContainer = Mockito.mock(SamzaContainer.class); MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); StreamProcessor streamProcessor = PowerMockito.spy(new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator)); @@ -389,7 +413,7 @@ public void testOnNewJobModelShouldResultInValidStateTransitions() throws Except @Test public void testStopShouldBeIdempotent() { JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); - StreamProcessorLifecycleListener lifecycleListener = Mockito.mock(StreamProcessorLifecycleListener.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); SamzaContainer mockSamzaContainer = Mockito.mock(SamzaContainer.class); MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); StreamProcessor streamProcessor = PowerMockito.spy(new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator)); @@ -411,7 +435,7 @@ public void testStopShouldBeIdempotent() { @Test public void testCoordinatorFailureShouldStopTheStreamProcessor() { JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); - StreamProcessorLifecycleListener lifecycleListener = Mockito.mock(StreamProcessorLifecycleListener.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); SamzaContainer mockSamzaContainer = Mockito.mock(SamzaContainer.class); MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); StreamProcessor streamProcessor = new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator); @@ -426,14 +450,14 @@ public void testCoordinatorFailureShouldStopTheStreamProcessor() { Assert.assertEquals(State.STOPPED, streamProcessor.state); - Mockito.verify(lifecycleListener).onFailure(failureException); + Mockito.verify(lifecycleListener).afterStop(failureException); Mockito.verify(mockSamzaContainer).shutdown(); } @Test public void testCoordinatorStopShouldStopTheStreamProcessor() { JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); - StreamProcessorLifecycleListener lifecycleListener = Mockito.mock(StreamProcessorLifecycleListener.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); StreamProcessor streamProcessor = new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator); @@ -441,6 +465,6 @@ public void testCoordinatorStopShouldStopTheStreamProcessor() { streamProcessor.jobCoordinatorListener.onCoordinatorStop(); Assert.assertEquals(State.STOPPED, streamProcessor.state); - Mockito.verify(lifecycleListener).onShutdown(); + Mockito.verify(lifecycleListener).afterStop(null); } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index 0a502bb011..caed583b60 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -19,10 +19,12 @@ package org.apache.samza.runtime; import java.time.Duration; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; -import org.apache.samza.config.Config; import org.apache.samza.job.ApplicationStatus; import org.junit.Test; @@ -78,24 +80,32 @@ public void TestStatusOperation() throws Exception { assertEquals(1, TestApplicationRunnerInvocationCounts.statusCount); } + public static class TestStreamApplicationDummy implements StreamApplication { + + @Override + public void describe(StreamAppDescriptor appDesc) { + + } + } + public static class TestApplicationRunnerInvocationCounts extends AbstractApplicationRunner { protected static int runCount = 0; protected static int killCount = 0; protected static int statusCount = 0; - public TestApplicationRunnerInvocationCounts(Config config) { - super(config); + public TestApplicationRunnerInvocationCounts(AppDescriptorImpl appDesc) { + super(appDesc); } - private void run() { + private void countRun() { runCount++; } - private void kill() { + private void countKill() { killCount++; } - private ApplicationStatus status() { + private ApplicationStatus countStatus() { statusCount++; return ApplicationStatus.Running; } @@ -104,17 +114,17 @@ class TestAppExecutable implements AppRuntimeExecutable { @Override public void run() { - run(); + countRun(); } @Override public void kill() { - kill(); + countKill(); } @Override public ApplicationStatus status() { - return status(); + return countStatus(); } @Override @@ -124,12 +134,12 @@ public boolean waitForFinish(Duration timeout) { } @Override - AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppSpecImpl appSpec) { + AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appSpec) { return new TestAppExecutable(); } @Override - AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppSpecImpl appSpec) { + AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appSpec) { return new TestAppExecutable(); } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 4481130ac5..7b0382f439 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -28,10 +28,9 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.samza.application.ApplicationClassUtils; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TaskApplicationSpec; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -44,16 +43,19 @@ import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; -import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.system.StreamSpec; import org.apache.samza.task.TaskFactory; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; import static org.junit.Assert.*; import static org.mockito.Matchers.anyObject; @@ -80,13 +82,25 @@ public class TestLocalApplicationRunner { "\"%s\":{" + "\"streamSpec\":{" + "\"id\":\"%s\"," + "\"systemName\":\"%s\"," + "\"physicalName\":\"%s\"," + "\"partitionCount\":2}," + "\"sourceJobs\":[\"test-app\"]," + "\"targetJobs\":[\"test-target-app\"]},"; + private LocalApplicationRunner runner; + + @Before + public void setUp() { + Map config = new HashMap<>(); + StreamGraphSpec mockGraphSpec = mock(StreamGraphSpec.class); + OperatorSpecGraph mockOpSpecGraph = mock(OperatorSpecGraph.class); + StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); + when(appDesc.getConfig()).thenReturn(new MapConfig(config)); + when(mockGraphSpec.getOperatorSpecGraph()).thenReturn(mockOpSpecGraph); + when(appDesc.getGraph()).thenReturn(mockGraphSpec); + runner = spy(new LocalApplicationRunner(appDesc)); + AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); + Whitebox.setInternalState(runner, "appExecutable", appExecutable); + } + @Test public void testStreamCreation() throws Exception { - Map config = new HashMap<>(); - LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config))); - StreamApplicationSpec appSpec = mock(StreamApplicationSpec.class); - StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(runner).buildAndStartStreamManager(); @@ -101,8 +115,8 @@ public void testStreamCreation() PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig); try { - runner.run(appSpec); - runner.waitForFinish(appSpec); + runner.run(); + runner.waitForFinish(); } catch (Throwable t) { assertNotNull(t); //no jobs exception } @@ -118,12 +132,6 @@ public void testStreamCreation() @Test public void testStreamCreationWithCoordination() throws Exception { - Map config = new HashMap<>(); - LocalApplicationRunner localRunner = new LocalApplicationRunner(new MapConfig(config)); - LocalApplicationRunner runner = spy(localRunner); - - StreamApplicationSpec appSpec = mock(StreamApplicationSpec.class); - StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(runner).buildAndStartStreamManager(); @@ -145,8 +153,8 @@ public void testStreamCreationWithCoordination() .thenReturn(coordinationUtils); try { - runner.run(appSpec); - runner.waitForFinish(appSpec); + runner.run(); + runner.waitForFinish(); } catch (Throwable t) { assertNotNull(t); //no jobs exception } @@ -168,39 +176,51 @@ public void testRunStreamTask() config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); Config samzaConfig = new MapConfig(config); - LocalApplicationRunner runner = new LocalApplicationRunner(samzaConfig); - TaskApplicationSpec appSpec = new TaskAppSpecImpl((TaskApplication) ApplicationClassUtils.fromConfig(samzaConfig), samzaConfig); + TaskAppDescriptorImpl + appDesc = new TaskAppDescriptorImpl((TaskApplication) ApplicationClassUtils.fromConfig(samzaConfig), samzaConfig); + runner = spy(new LocalApplicationRunner(appDesc)); + LocalApplicationRunner.TaskAppExecutable taskAppExecutable = + spy((LocalApplicationRunner.TaskAppExecutable) runner.getTaskAppRuntimeExecutable(appDesc)); + Whitebox.setInternalState(runner, "appExecutable", taskAppExecutable); StreamProcessor sp = mock(StreamProcessor.class); ArgumentCaptor captor1 = ArgumentCaptor.forClass(TaskFactory.class); - ArgumentCaptor captor2 = - ArgumentCaptor.forClass(StreamProcessorLifecycleListener.class); + ArgumentCaptor captor2 = + ArgumentCaptor.forClass(ProcessorLifecycleListener.class); doAnswer(i -> { - StreamProcessorLifecycleListener listener = captor2.getValue(); - listener.onStart(); - listener.onShutdown(); + ProcessorLifecycleListener listener = captor2.getValue(); + listener.afterStart(); + listener.afterStop(null); return null; }).when(sp).start(); - LocalApplicationRunner spy = spy(runner); - doReturn(sp).when(spy).createStreamProcessor(any(Config.class), captor1.capture(), captor2.capture()); + doReturn(sp).when(runner).createStreamProcessor(any(Config.class), captor1.capture(), captor2.capture()); + doReturn(ApplicationStatus.SuccessfulFinish).when(taskAppExecutable).status(); + runner.run(); - spy.run(appSpec); - - assertEquals(ApplicationStatus.SuccessfulFinish, spy.status(appSpec)); + assertEquals(ApplicationStatus.SuccessfulFinish, runner.status()); } @Test public void testRunComplete() throws Exception { - final Map config = new HashMap<>(); + Map config = new HashMap<>(); config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); - LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config))); - StreamApplicationSpec app = mock(StreamApplicationSpec.class); + StreamGraphSpec mockGraphSpec = mock(StreamGraphSpec.class); + OperatorSpecGraph mockOpSpecGraph = mock(OperatorSpecGraph.class); + ProcessorLifecycleListenerFactory mockFactory = (pContext, cfg) -> mock(ProcessorLifecycleListener.class); + StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); + when(appDesc.getConfig()).thenReturn(new MapConfig(config)); + when(mockGraphSpec.getOperatorSpecGraph()).thenReturn(mockOpSpecGraph); + when(appDesc.getGraph()).thenReturn(mockGraphSpec); + when(appDesc.getProcessorLifecycleListenerFactory()).thenReturn(mockFactory); + runner = spy(new LocalApplicationRunner(appDesc)); + AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); + Whitebox.setInternalState(runner, "appExecutable", appExecutable); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -212,33 +232,42 @@ public void testRunComplete() doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(ProcessorLifecycleListener.class); doAnswer(i -> { - StreamProcessorLifecycleListener listener = captor.getValue(); - listener.onStart(); - listener.onShutdown(); + ProcessorLifecycleListener listener = captor.getValue(); + listener.afterStart(); + listener.afterStop(null); return null; }).when(sp).start(); - doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); + doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), anyObject(), captor.capture()); - runner.run(app); - runner.waitForFinish(app); + runner.run(); + runner.waitForFinish(); - assertEquals(runner.status(app), ApplicationStatus.SuccessfulFinish); + assertEquals(runner.status(), ApplicationStatus.SuccessfulFinish); verify(streamManager).stop(); } @Test public void testRunFailure() throws Exception { - final Map config = new HashMap<>(); + Map config = new HashMap<>(); config.put(ApplicationConfig.PROCESSOR_ID, "0"); - LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config))); - StreamApplicationSpec app = mock(StreamApplicationSpec.class); + StreamGraphSpec mockGraphSpec = mock(StreamGraphSpec.class); + OperatorSpecGraph mockOpSpecGraph = mock(OperatorSpecGraph.class); + ProcessorLifecycleListenerFactory mockFactory = (pContext, cfg) -> mock(ProcessorLifecycleListener.class); + StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); + when(appDesc.getConfig()).thenReturn(new MapConfig(config)); + when(mockGraphSpec.getOperatorSpecGraph()).thenReturn(mockOpSpecGraph); + when(appDesc.getGraph()).thenReturn(mockGraphSpec); + when(appDesc.getProcessorLifecycleListenerFactory()).thenReturn(mockFactory); + runner = spy(new LocalApplicationRunner(appDesc)); + AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); + Whitebox.setInternalState(runner, "appExecutable", appExecutable); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -250,24 +279,24 @@ public void testRunFailure() doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(ProcessorLifecycleListener.class); doAnswer(i -> { throw new Exception("test failure"); }).when(sp).start(); - doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); + doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), anyObject(), captor.capture()); try { - runner.run(app); - runner.waitForFinish(app); + runner.run(); + runner.waitForFinish(); } catch (Throwable th) { assertNotNull(th); } - assertEquals(runner.status(app), ApplicationStatus.UnsuccessfulFinish); + assertEquals(runner.status(), ApplicationStatus.UnsuccessfulFinish); verify(streamManager).stop(); } @@ -333,21 +362,17 @@ public void testGeneratePlanIdWithDifferentStreamSpecs() { @Test public void testWaitForFinishReturnsBeforeTimeout() { - LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig()); - StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); long timeoutInMs = 1000; runner.getShutdownLatch().countDown(); - boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(timeoutInMs)); + boolean finished = runner.waitForFinish(Duration.ofMillis(timeoutInMs)); assertTrue("Application did not finish before the timeout.", finished); } @Test public void testWaitForFinishTimesout() { - LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig()); - StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); long timeoutInMs = 100; - boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(timeoutInMs)); + boolean finished = runner.waitForFinish(Duration.ofMillis(timeoutInMs)); assertFalse("Application finished before the timeout.", finished); } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java index 53816a3868..60b6a431a5 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java @@ -20,9 +20,9 @@ package org.apache.samza.runtime; import java.time.Duration; -import org.apache.samza.application.StreamApplicationSpec; import java.util.HashMap; import java.util.Map; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -30,33 +30,48 @@ import org.apache.samza.job.StreamJob; import org.apache.samza.job.StreamJobFactory; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.mockito.Matchers.*; +import static org.powermock.api.mockito.PowerMockito.*; /** * A test class for {@link RemoteApplicationRunner}. */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(RemoteApplicationRunner.class) public class TestRemoteApplicationRunner { + + private RemoteApplicationRunner runner; + + @Before + public void setUp() { + Map config = new HashMap<>(); + StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); + when(appDesc.getConfig()).thenReturn(new MapConfig(config)); + runner = spy(new RemoteApplicationRunner(appDesc)); + AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); + Whitebox.setInternalState(runner, "appExecutable", appExecutable); + } + @Test public void testWaitForFinishReturnsBeforeTimeout() { - RemoteApplicationRunner runner = spy(new RemoteApplicationRunner(new MapConfig())); - StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); doReturn(ApplicationStatus.SuccessfulFinish).when(runner).getApplicationStatus(any(JobConfig.class)); - - boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(5000)); + boolean finished = runner.waitForFinish(Duration.ofMillis(5000)); assertTrue("Application did not finish before the timeout.", finished); } @Test public void testWaitForFinishTimesout() { - RemoteApplicationRunner runner = spy(new RemoteApplicationRunner(new MapConfig())); - StreamApplicationSpec mockSpec = mock(StreamApplicationSpec.class); doReturn(ApplicationStatus.Running).when(runner).getApplicationStatus(any(JobConfig.class)); - - boolean finished = runner.waitForFinish(mockSpec, Duration.ofMillis(1000)); + boolean finished = runner.waitForFinish(Duration.ofMillis(1000)); assertFalse("Application finished before the timeout.", finished); } @@ -67,11 +82,16 @@ public void testGetStatus() { m.put(JobConfig.STREAM_JOB_FACTORY_CLASS(), MockStreamJobFactory.class.getName()); m.put(JobConfig.JOB_ID(), "newJob"); - RemoteApplicationRunner runner = new RemoteApplicationRunner(new MapConfig()); + + StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); + when(appDesc.getConfig()).thenReturn(new MapConfig(m)); + runner = spy(new RemoteApplicationRunner(appDesc)); + Assert.assertEquals(ApplicationStatus.New, runner.getApplicationStatus(new JobConfig(new MapConfig(m)))); m.put(JobConfig.JOB_ID(), "runningJob"); - runner = new RemoteApplicationRunner(new JobConfig(new MapConfig(m))); + when(appDesc.getConfig()).thenReturn(new MapConfig(m)); + runner = spy(new RemoteApplicationRunner(appDesc)); Assert.assertEquals(ApplicationStatus.Running, runner.getApplicationStatus(new JobConfig(new MapConfig(m)))); } diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/TestAsyncStreamTask.java b/samza-core/src/test/java/org/apache/samza/task/TestAsyncStreamTask.java similarity index 84% rename from samza-core/src/test/java/org/apache/samza/testUtils/TestAsyncStreamTask.java rename to samza-core/src/test/java/org/apache/samza/task/TestAsyncStreamTask.java index 81f3fd4884..26e8c8579e 100644 --- a/samza-core/src/test/java/org/apache/samza/testUtils/TestAsyncStreamTask.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestAsyncStreamTask.java @@ -16,13 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.testUtils; +package org.apache.samza.task; import org.apache.samza.system.IncomingMessageEnvelope; -import org.apache.samza.task.AsyncStreamTask; -import org.apache.samza.task.MessageCollector; -import org.apache.samza.task.TaskCallback; -import org.apache.samza.task.TaskCoordinator; /** * Test implementation class for {@link AsyncStreamTask} diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/TestStreamTask.java b/samza-core/src/test/java/org/apache/samza/task/TestStreamTask.java similarity index 86% rename from samza-core/src/test/java/org/apache/samza/testUtils/TestStreamTask.java rename to samza-core/src/test/java/org/apache/samza/task/TestStreamTask.java index ce0980a8fc..1ad0398a4c 100644 --- a/samza-core/src/test/java/org/apache/samza/testUtils/TestStreamTask.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestStreamTask.java @@ -16,12 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.testUtils; +package org.apache.samza.task; import org.apache.samza.system.IncomingMessageEnvelope; -import org.apache.samza.task.MessageCollector; -import org.apache.samza.task.StreamTask; -import org.apache.samza.task.TaskCoordinator; /** * Test implementation class for {@link StreamTask} diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index c17cc6ffac..222e72ccdf 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -25,8 +25,6 @@ import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; -import org.apache.samza.testUtils.TestAsyncStreamTask; -import org.apache.samza.testUtils.TestStreamTask; import org.junit.Test; import static org.junit.Assert.*; @@ -41,7 +39,7 @@ public class TestTaskFactoryUtil { public void testStreamTaskClass() { Config config = new MapConfig(new HashMap() { { - this.put("task.class", "org.apache.samza.testUtils.TestStreamTask"); + this.put("task.class", TestStreamTask.class.getName()); } }); Object retFactory = TaskFactoryUtil.createTaskFactory(config); @@ -65,7 +63,7 @@ public void testStreamTaskClass() { public void testAsyncStreamTask() { Config config = new MapConfig(new HashMap() { { - this.put("task.class", "org.apache.samza.testUtils.TestAsyncStreamTask"); + this.put("task.class", TestAsyncStreamTask.class.getName()); } }); Object retFactory = TaskFactoryUtil.createTaskFactory(config); diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala index 9aca45e259..6abb40e18d 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala @@ -187,6 +187,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null + @volatile var beforeStartCalled = false + @volatile var beforeStopCalled = false val container = new SamzaContainer( containerContext = containerContext, @@ -210,12 +212,22 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { override def onContainerStart(): Unit = { onContainerStartCalled = true } + + override def beforeStop(): Unit = { + beforeStopCalled = true + } + + override def beforeStart(): Unit = { + beforeStartCalled = true + } } container.setContainerListener(containerListener) container.run assertTrue(task.wasShutdown) + assertTrue(beforeStartCalled) assertFalse(onContainerStartCalled) + assertTrue(beforeStopCalled) assertFalse(onContainerStopCalled) assertTrue(onContainerFailedCalled) @@ -266,6 +278,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null + @volatile var beforeStartCalled = false + @volatile var beforeStopCalled = false val mockRunLoop = mock[RunLoop] when(mockRunLoop.run).thenThrow(new RuntimeException("Trigger a shutdown, please.")) @@ -291,13 +305,23 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { override def onContainerStart(): Unit = { onContainerStartCalled = true } + + override def beforeStop(): Unit = { + beforeStopCalled = true + } + + override def beforeStart(): Unit = { + beforeStartCalled = true + } } container.setContainerListener(containerListener) container.run assertTrue(task.wasShutdown) + assertTrue(beforeStartCalled) assertTrue(onContainerStartCalled) + assertTrue(beforeStopCalled) assertFalse(onContainerStopCalled) assertTrue(onContainerFailedCalled) @@ -352,6 +376,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null + @volatile var beforeStartCalled = false + @volatile var beforeStopCalled = false val container = new SamzaContainer( containerContext = containerContext, @@ -374,6 +400,14 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { override def onContainerStart(): Unit = { onContainerStartCalled = true } + + override def beforeStop(): Unit = { + beforeStopCalled = true + } + + override def beforeStart(): Unit = { + beforeStartCalled = true + } } container.setContainerListener(containerListener) @@ -381,9 +415,11 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { assertTrue(task.wasShutdown) + assertTrue(beforeStartCalled) assertFalse(onContainerStopCalled) assertFalse(onContainerStartCalled) + assertTrue(beforeStopCalled) assertTrue(onContainerFailedCalled) assertNotNull(onContainerFailedThrowable) } @@ -429,6 +465,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null + @volatile var beforeStartCalled = false + @volatile var beforeStopCalled = false val mockRunLoop = mock[RunLoop] when(mockRunLoop.run).thenAnswer(new Answer[Unit] { @@ -458,12 +496,22 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { override def onContainerStart(): Unit = { onContainerStartCalled = true } + + override def beforeStop(): Unit = { + beforeStopCalled = true + } + + override def beforeStart(): Unit = { + beforeStartCalled = true + } } container.setContainerListener(containerListener) container.run + assertTrue(beforeStartCalled) assertFalse(onContainerFailedCalled) assertTrue(onContainerStartCalled) + assertTrue(beforeStopCalled) assertTrue(onContainerStopCalled) } @@ -507,6 +555,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null + @volatile var beforeStartCalled = false + @volatile var beforeStopCalled = false val mockRunLoop = mock[RunLoop] when(mockRunLoop.run).thenAnswer(new Answer[Unit] { @@ -537,12 +587,24 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { override def onContainerStart(): Unit = { onContainerStartCalled = true } + + override def beforeStop(): Unit = { + beforeStopCalled = true } + + override def beforeStart(): Unit = { + beforeStartCalled = true + } + } container.setContainerListener(containerListener) container.run + assertTrue(beforeStartCalled) + assertTrue(onContainerStartCalled) + assertTrue(beforeStopCalled) assertTrue(onContainerFailedCalled) + assertFalse(onContainerStopCalled) } @Test 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 668a5157b7..36e5402876 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 @@ -21,7 +21,7 @@ import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; import org.apache.samza.sql.translator.QueryTranslator; import org.slf4j.Logger; @@ -36,14 +36,14 @@ public class SamzaSqlApplication implements StreamApplication { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplication.class); @Override - public void describe(StreamApplicationSpec appSpec) { + public void describe(StreamAppDescriptor appDesc) { try { - SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(appSpec.getConfig()); + SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(appDesc.getConfig()); QueryTranslator queryTranslator = new QueryTranslator(sqlConfig); List queries = sqlConfig.getQueryInfo(); for (SamzaSqlQueryParser.QueryInfo query : queries) { LOG.info("Translating the query {} to samza stream graph", query.getSelectQuery()); - queryTranslator.translate(query, appSpec); + queryTranslator.translate(query, appDesc); } } catch (RuntimeException e) { LOG.error("SamzaSqlApplication threw exception.", e); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java similarity index 81% rename from samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java rename to samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java index 4106a6f8d8..af67b4b22a 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRuntime.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java @@ -24,17 +24,14 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.Validate; -import org.apache.samza.application.ApplicationSpec; -import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.runtime.ApplicationRuntime; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.RemoteApplicationRunner; -import org.apache.samza.runtime.internal.ApplicationRunner; -import org.apache.samza.runtime.internal.ApplicationRunners; import org.apache.samza.sql.interfaces.SqlIOConfig; import org.apache.samza.sql.interfaces.SqlIOResolver; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; @@ -49,23 +46,18 @@ * This runner invokes the SamzaSqlConfig re-writer if it is invoked on a standalone mode (i.e. localRunner == true) * otherwise directly calls the RemoteApplicationRunner which automatically performs the config rewriting . */ -public class SamzaSqlApplicationRuntime implements ApplicationRuntime { +public class SamzaSqlApplicationRunner implements ApplicationRunner { - private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplicationRuntime.class); + private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplicationRunner.class); - private final Config sqlConfig; - private final ApplicationSpec appSpec; private final ApplicationRunner runner; - private final Boolean localRunner; public static final String RUNNER_CONFIG = "app.runner.class"; public static final String CFG_FMT_SAMZA_STREAM_SYSTEM = "streams.%s.samza.system"; - public SamzaSqlApplicationRuntime(Boolean localRunner, Config config) { - this.localRunner = localRunner; - sqlConfig = computeSamzaConfigs(localRunner, config); - appSpec = new StreamAppSpecImpl(new SamzaSqlApplication(), sqlConfig); - runner = ApplicationRunners.fromConfig(sqlConfig); + public SamzaSqlApplicationRunner(Boolean localRunner, Config config) { + this.runner = ApplicationRunners.getApplicationRunner(new SamzaSqlApplication(), + computeSamzaConfigs(localRunner, config)); } public static Config computeSamzaConfigs(Boolean localRunner, Config config) { @@ -108,34 +100,34 @@ public static Config computeSamzaConfigs(Boolean localRunner, Config config) { } public void runAndWaitForFinish() { - Validate.isTrue(localRunner, "This method can be called only in standalone mode."); + Validate.isTrue(runner instanceof LocalApplicationRunner, "This method can be called only in standalone mode."); run(); waitForFinish(); } @Override public void run() { - runner.run(appSpec); + runner.run(); } @Override public void kill() { - runner.kill(appSpec); + runner.kill(); } @Override public ApplicationStatus status() { - return runner.status(appSpec); + return runner.status(); } @Override public void waitForFinish() { - runner.waitForFinish(appSpec); + runner.waitForFinish(); } @Override public boolean waitForFinish(Duration timeout) { - return runner.waitForFinish(appSpec, timeout); + return runner.waitForFinish(timeout); } @Override diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index 870fc13ac0..cd220cbaf0 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -30,8 +30,8 @@ import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalProject; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamApplicationSpec; -import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; @@ -56,7 +56,7 @@ /** * This class is used to populate the StreamGraph using the SQL queries. * This class contains the core of the SamzaSQL control code that converts the SQL statements to calcite relational graph. - * It then walks the relational graph and then populates the Samza's {@link StreamApplicationSpec} accordingly. + * It then walks the relational graph and then populates the Samza's {@link StreamAppDescriptor} accordingly. */ public class QueryTranslator { private static final Logger LOG = LoggerFactory.getLogger(QueryTranslator.class); @@ -92,13 +92,13 @@ public QueryTranslator(SamzaSqlApplicationConfig sqlConfig) { this.converters = sqlConfig.getSamzaRelConverters(); } - public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplicationSpec appSpec) { + public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamAppDescriptor appDesc) { QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getInputSystemStreamConfigBySource(), sqlConfig.getUdfMetadata()); final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig); final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery()); - final TranslatorContext context = new TranslatorContext(((StreamAppSpecImpl)appSpec).getGraph(), relRoot, executionContext, this.converters); + final TranslatorContext context = new TranslatorContext(((StreamAppDescriptorImpl)appDesc).getGraph(), relRoot, executionContext, this.converters); final RelNode node = relRoot.project(); final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver(); @@ -151,9 +151,9 @@ public RelNode visit(LogicalAggregate aggregate) { Optional tableDescriptor = sinkConfig.getTableDescriptor(); if (!tableDescriptor.isPresent()) { - outputStream.sendTo(appSpec.getOutputStream(sinkConfig.getStreamName())); + outputStream.sendTo(appDesc.getOutputStream(sinkConfig.getStreamName())); } else { - Table outputTable = appSpec.getTable(tableDescriptor.get()); + Table outputTable = appDesc.getTable(tableDescriptor.get()); if (outputTable == null) { String msg = "Failed to obtain table descriptor of " + sinkConfig.getSource(); LOG.error(msg); @@ -162,7 +162,7 @@ public RelNode visit(LogicalAggregate aggregate) { outputStream.sendTo(outputTable); } - appSpec.withContextManager(new ContextManager() { + appDesc.withContextManager(new ContextManager() { @Override public void init(Config config, TaskContext taskContext) { taskContext.setUserContext(context.clone()); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java index ec758ac3ec..9399fd656f 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java @@ -31,7 +31,7 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.SchemaPlus; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.StreamGraph; import org.apache.samza.sql.data.RexToJavaCompiler; @@ -40,7 +40,7 @@ /** - * State that is maintained while translating the Calcite relational graph to Samza {@link StreamApplicationSpec}. + * State that is maintained while translating the Calcite relational graph to Samza {@link StreamAppDescriptor}. */ public class TranslatorContext implements Cloneable { /** diff --git a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java index 8f4322c33e..cc339f14f0 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java @@ -25,7 +25,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; +import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; import org.apache.samza.sql.testutil.JsonUtil; import org.apache.samza.sql.testutil.SamzaSqlTestConfig; import org.apache.samza.sql.testutil.TestIOResolverFactory; @@ -45,7 +45,7 @@ public void testEndToEnd() throws Exception { String sql1 = "Insert into testDb.testTable.`$table` select id, name from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime appRunnable = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner appRunnable = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); appRunnable.runAndWaitForFinish(); Assert.assertEquals(numMessages, TestIOResolverFactory.TestTable.records.size()); @@ -61,7 +61,7 @@ public void testEndToEndWithKey() throws Exception { String sql1 = "Insert into testDb.testTable.`$table` select id __key__, name from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime appRunnable = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner appRunnable = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); appRunnable.runAndWaitForFinish(); Assert.assertEquals(numMessages, TestIOResolverFactory.TestTable.records.size()); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java index 32674ed8de..55be9f852d 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java @@ -38,15 +38,15 @@ public void testComputeSamzaConfigs() { Map configs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1"; configs.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql1); - configs.put(SamzaSqlApplicationRuntime.RUNNER_CONFIG, SamzaSqlApplicationRuntime.class.getName()); + configs.put(SamzaSqlApplicationRunner.RUNNER_CONFIG, SamzaSqlApplicationRunner.class.getName()); MapConfig samzaConfig = new MapConfig(configs); - Config newConfigs = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, samzaConfig); - Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRuntime.RUNNER_CONFIG), LocalApplicationRunner.class.getName()); + Config newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(true, samzaConfig); + Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), LocalApplicationRunner.class.getName()); // Check whether three new configs added. Assert.assertEquals(newConfigs.size(), configs.size() + 3); - newConfigs = SamzaSqlApplicationRuntime.computeSamzaConfigs(false, samzaConfig); - Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRuntime.RUNNER_CONFIG), RemoteApplicationRunner.class.getName()); + newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(false, samzaConfig); + Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), RemoteApplicationRunner.class.getName()); // Check whether three new configs added. Assert.assertEquals(newConfigs.size(), configs.size() + 3); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index 12193421e5..0a7e4bba9c 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -24,7 +24,7 @@ import java.util.HashSet; import java.util.Map; import org.apache.samza.SamzaException; -import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; @@ -38,7 +38,7 @@ import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; +import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; import org.apache.samza.sql.testutil.SamzaSqlTestConfig; import org.junit.Assert; @@ -88,12 +88,12 @@ public void testTranslate() { Map config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.outputTopic select MyTest(id) from testavro.level1.level2.SIMPLE1 as s where s.id = 10"); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -130,7 +130,7 @@ public void testTranslate() { validatePerTaskContextInit(mockSpec, samzaConfig); } - private void validatePerTaskContextInit(StreamAppSpecImpl graphSpec, Config samzaConfig) { + private void validatePerTaskContextInit(StreamAppDescriptorImpl graphSpec, Config samzaConfig) { // make sure that each task context would have a separate instance of cloned TranslatorContext TaskContextImpl testContext = new TaskContextImpl(new TaskName("Partition 1"), null, null, new HashSet<>(), null, null, null, null, null, null); @@ -154,13 +154,13 @@ public void testTranslateComplex() { // config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, // "Insert into testavro.foo2 select string_value, SUM(id) from testavro.COMPLEX1 " // + "GROUP BY TumbleWindow(CURRENT_TIME, INTERVAL '1' HOUR), string_value"); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -201,13 +201,13 @@ public void testTranslateSubQuery() { Map config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.outputTopic select Flatten(a), id from (select id, array_values a, string_value s from testavro.COMPLEX1)"); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -252,13 +252,13 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p" + " where p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -286,13 +286,13 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { + " full join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -320,13 +320,13 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { + " join testavro.PROFILE.`$table` as p2" + " on p1.id = p2.id"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -354,13 +354,13 @@ public void testTranslateStreamTableJoinWithThetaCondition() { + " join testavro.PROFILE.`$table` as p" + " on p.id <> pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -385,13 +385,13 @@ public void testTranslateStreamTableCrossJoin() { + " select p.name as profileName, pv.pageKey" + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -418,13 +418,13 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { + " join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId and p.name = 'John'"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -452,13 +452,13 @@ public void testTranslateStreamTableJoinWithSubQuery() { + " (select p.id from testavro.PROFILE.`$table` as p" + " where p.id = pv.profileId)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -485,13 +485,13 @@ public void testTranslateTableTableJoin() { + " join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -518,13 +518,13 @@ public void testTranslateStreamStreamJoin() { + " join testavro.PROFILE as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -551,13 +551,13 @@ public void testTranslateJoinWithIncorrectLeftJoin() { + " left join testavro.PROFILE as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -584,13 +584,13 @@ public void testTranslateJoinWithIncorrectRightJoin() { + " right join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -621,13 +621,13 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { + " join testavro.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -654,13 +654,13 @@ public void testTranslateStreamTableInnerJoinWithUdf() { + " join testavro.PROFILE.`$table` as p" + " on MyTest(p.id) = MyTest(pv.profileId)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -687,13 +687,13 @@ public void testTranslateStreamTableInnerJoin() { + " join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -755,13 +755,13 @@ public void testTranslateStreamTableLeftJoin() { + " left join testavro.PROFILE.`$table` as p" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -824,13 +824,13 @@ public void testTranslateStreamTableRightJoin() { + " right join testavro.PAGEVIEW as pv" + " on p.id = pv.profileId"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); @@ -893,13 +893,13 @@ public void testTranslateGroupBy() { + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'" + " group by (pv.pageKey)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); @@ -932,13 +932,13 @@ public void testTranslateGroupByWithSumAggregator() { + " from testavro.PAGEVIEW as pv" + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'" + " group by (pv.pageKey)"; config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql); - Config samzaConfig = SamzaSqlApplicationRuntime.computeSamzaConfigs(true, new MapConfig(config)); + Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config)); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppSpecImpl mockSpec = mock(StreamAppSpecImpl.class); + StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); // The following steps are used to simulate the actual logic in the constructor when(mockSpec.getConfig()).thenReturn(samzaConfig); when(mockSpec.getGraph()).thenReturn(graphSpec); 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 bb6035f627..c784222b46 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 @@ -21,15 +21,15 @@ import java.time.Duration; import java.util.HashMap; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -45,23 +45,23 @@ public class AppWithGlobalConfigExample implements StreamApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new AppWithGlobalConfigExample(), config); - app.addMetricsReporters(new HashMap<>()); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new AppWithGlobalConfigExample(), config); + runner.addMetricsReporters(new HashMap<>()); - app.run(); - app.waitForFinish(); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec appSpec) { + public void describe(StreamAppDescriptor appDesc) { - appSpec.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) + appDesc.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class))) .map(KV::getValue) .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(appSpec.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); + .sendTo(appDesc.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class)))); } 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 68535dc537..7320e44b8b 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 @@ -20,12 +20,12 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -41,19 +41,19 @@ public class BroadcastExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new BroadcastExample(), config); - app.run(); - app.waitForFinish(); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new BroadcastExample(), config); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec appBuilder) { + public void describe(StreamAppDescriptor appDesc) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); - MessageStream> inputStream = appBuilder.getInputStream("pageViewEventStream", pgeMsgSerde); + MessageStream> inputStream = appDesc.getInputStream("pageViewEventStream", pgeMsgSerde); - inputStream.filter(m -> m.key.equals("key1")).sendTo(appBuilder.getOutputStream("outStream1", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key2")).sendTo(appBuilder.getOutputStream("outStream2", pgeMsgSerde)); - inputStream.filter(m -> m.key.equals("key3")).sendTo(appBuilder.getOutputStream("outStream3", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key1")).sendTo(appDesc.getOutputStream("outStream1", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key2")).sendTo(appDesc.getOutputStream("outStream2", pgeMsgSerde)); + inputStream.filter(m -> m.key.equals("key3")).sendTo(appDesc.getOutputStream("outStream3", pgeMsgSerde)); } 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 4f508ecb75..8339f20e6f 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 @@ -23,14 +23,14 @@ import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.FlatMapFunction; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -48,18 +48,18 @@ public class KeyValueStoreExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new KeyValueStoreExample(), config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new KeyValueStoreExample(), config); - app.run(); - app.waitForFinish(); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec graph) { + public void describe(StreamAppDescriptor appDesc) { MessageStream pageViewEvents = - graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + appDesc.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = - graph.getOutputStream("pageViewEventPerMember", + appDesc.getOutputStream("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class))); pageViewEvents 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 db85833a93..540003826e 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 @@ -21,11 +21,11 @@ import com.google.common.collect.ImmutableList; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -37,20 +37,20 @@ public class MergeExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new MergeExample(), config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new MergeExample(), config); - app.run(); - app.waitForFinish(); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec graph) { + public void describe(StreamAppDescriptor appDesc) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); - MessageStream.mergeAll(ImmutableList.of(graph.getInputStream("viewStream1", pgeMsgSerde), - graph.getInputStream("viewStream2", pgeMsgSerde), graph.getInputStream("viewStream3", pgeMsgSerde))) - .sendTo(graph.getOutputStream("mergedStream", pgeMsgSerde)); + MessageStream.mergeAll(ImmutableList.of(appDesc.getInputStream("viewStream1", pgeMsgSerde), + appDesc.getInputStream("viewStream2", pgeMsgSerde), appDesc.getInputStream("viewStream3", pgeMsgSerde))) + .sendTo(appDesc.getOutputStream("mergedStream", pgeMsgSerde)); } 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 5f90584bfa..b09fb26efe 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 @@ -20,14 +20,14 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.functions.JoinFunction; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -43,19 +43,19 @@ public class OrderShipmentJoinExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new OrderShipmentJoinExample(), config); - app.run(); - app.waitForFinish(); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new OrderShipmentJoinExample(), config); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec graph) { + public void describe(StreamAppDescriptor appDesc) { MessageStream orders = - graph.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class)); + appDesc.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class)); MessageStream shipments = - graph.getInputStream("shipments", new JsonSerdeV2<>(ShipmentRecord.class)); + appDesc.getInputStream("shipments", new JsonSerdeV2<>(ShipmentRecord.class)); OutputStream> fulfilledOrders = - graph.getOutputStream("fulfilledOrders", + appDesc.getOutputStream("fulfilledOrders", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(FulfilledOrderRecord.class))); orders 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 3c24a2b6cb..460d4bcbb3 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 @@ -19,9 +19,8 @@ package org.apache.samza.example; import java.time.Duration; -import org.apache.samza.application.ApplicationClassUtils; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -32,8 +31,8 @@ import org.apache.samza.operators.windows.AccumulationMode; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -49,28 +48,30 @@ public class PageViewCounterExample implements StreamApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); - appRuntime.run(); - appRuntime.waitForFinish(); + PageViewCounterExample app = new PageViewCounterExample(); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config); + + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec graph) { - MessageStream pageViewEvents = null; - pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); - OutputStream> pageViewEventPerMemberStream = - graph.getOutputStream("pageViewEventPerMemberStream", - KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); + public void describe(StreamAppDescriptor appDesc) { + MessageStream pageViewEvents = null; + pageViewEvents = appDesc.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class)); + OutputStream> pageViewEventPerMemberStream = + appDesc.getOutputStream("pageViewEventPerMemberStream", + KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class))); + + SupplierFunction initialValue = () -> 0; + FoldLeftFunction foldLeftFn = (m, c) -> c + 1; + pageViewEvents + .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, null, null) + .setEarlyTrigger(Triggers.repeat(Triggers.count(5))) + .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow") + .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane))) + .sendTo(pageViewEventPerMemberStream); - SupplierFunction initialValue = () -> 0; - FoldLeftFunction foldLeftFn = (m, c) -> c + 1; - pageViewEvents - .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, - null, null) - .setEarlyTrigger(Triggers.repeat(Triggers.count(5))) - .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow") - .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane))) - .sendTo(pageViewEventPerMemberStream); } class PageViewEvent { 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 bab7e5b552..3df606f6b6 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 @@ -20,15 +20,15 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -44,18 +44,18 @@ public class RepartitionExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new RepartitionExample(), config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new RepartitionExample(), config); - app.run(); - app.waitForFinish(); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec graph) { + public void describe(StreamAppDescriptor appDesc) { MessageStream pageViewEvents = - graph.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); + appDesc.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)); OutputStream> pageViewEventPerMember = - graph.getOutputStream("pageViewEventPerMember", + appDesc.getOutputStream("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class))); pageViewEvents 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 0b507fcb56..8997c78286 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 @@ -1,39 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.samza.example; import java.util.Collections; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TaskApplicationSpec; +import org.apache.samza.application.TaskAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.storage.kv.RocksDbTableDescriptor; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.CommandLine; /** - * Created by yipan on 7/16/18. + * Test example of a low-level API application (i.e. {@link TaskApplication}) */ public class TaskApplicationExample implements TaskApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(new TaskApplicationExample(), config); - appRuntime.run(); - appRuntime.waitForFinish(); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TaskApplicationExample(), config); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(TaskApplicationSpec appBuilder) { + public void describe(TaskAppDescriptor appDesc) { // add input and output streams - appBuilder.addInputStreams(Collections.singletonList("myinput")); - appBuilder.addOutputStreams(Collections.singletonList("myoutput")); + appDesc.addInputStreams(Collections.singletonList("myinput")); + appDesc.addOutputStreams(Collections.singletonList("myoutput")); TableDescriptor td = new RocksDbTableDescriptor("mytable"); - appBuilder.addTables(Collections.singletonList(td)); + appDesc.addTables(Collections.singletonList(td)); // create the task factory based on configuration - appBuilder.setTaskFactory(TaskFactoryUtil.createTaskFactory(appBuilder.getConfig())); + appDesc.setTaskFactory(TaskFactoryUtil.createTaskFactory(appDesc.getConfig())); } } 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 6fbd9ba948..f513a70571 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 @@ -21,7 +21,7 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; @@ -30,8 +30,8 @@ import org.apache.samza.operators.triggers.Triggers; import org.apache.samza.operators.windows.WindowPane; import org.apache.samza.operators.windows.Windows; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.util.CommandLine; @@ -47,18 +47,18 @@ public class WindowExample implements StreamApplication { public static void main(String[] args) throws Exception { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRuntime app = ApplicationRuntimes.getApplicationRuntime(new WindowExample(), config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new WindowExample(), config); - app.run(); - app.waitForFinish(); + runner.run(); + runner.waitForFinish(); } @Override - public void describe(StreamApplicationSpec graph) { + public void describe(StreamAppDescriptor appDesc) { SupplierFunction initialValue = () -> 0; FoldLeftFunction counter = (m, c) -> c == null ? 1 : c + 1; - MessageStream inputStream = graph.getInputStream("inputStream", new JsonSerdeV2()); - OutputStream outputStream = graph.getOutputStream("outputStream", new IntegerSerde()); + MessageStream inputStream = appDesc.getInputStream("inputStream", new JsonSerdeV2()); + OutputStream outputStream = appDesc.getOutputStream("outputStream", new IntegerSerde()); // 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/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index 2c95473e60..887d8416f7 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -31,12 +31,11 @@ import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.samza.SamzaException; -import org.apache.samza.application.ApplicationSpec; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamAppSpecImpl; -import org.apache.samza.application.internal.TaskAppSpecImpl; +import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.Config; -import org.apache.samza.config.ConfigException; import org.apache.samza.config.InMemorySystemConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -60,7 +59,10 @@ import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.inmemory.InMemorySystemFactory; import org.apache.samza.task.AsyncStreamTask; +import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTask; +import org.apache.samza.task.StreamTaskFactory; +import org.apache.samza.task.TaskFactory; import org.apache.samza.test.framework.stream.CollectionStream; import org.apache.samza.test.framework.system.CollectionStreamSystemSpec; import org.junit.Assert; @@ -288,21 +290,15 @@ public TestRunner addOutputStream(CollectionStream stream) { public void run(Duration timeout) { Preconditions.checkState((app == null && taskClass != null) || (app != null && taskClass == null), "TestRunner should run for Low Level Task api or High Level Application Api"); - Preconditions.checkState(!timeout.isZero() || !timeout.isNegative(), - "Timeouts should be positive"); - final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - ApplicationSpec appSpec = app == null ? new TaskAppSpecImpl(spec -> spec.setTaskFactory(() -> { - try { - return taskClass.newInstance(); - } catch (InstantiationException | IllegalAccessException e) { - throw new ConfigException(e); - } - }), new MapConfig(configs)) : - new StreamAppSpecImpl(app, new MapConfig(configs)); - runner.run(appSpec); - boolean timedOut = !runner.waitForFinish(appSpec, timeout); + Preconditions.checkState(!timeout.isZero() || !timeout.isNegative(), "Timeouts should be positive"); + AppDescriptorImpl appDesc = app == null ? + new TaskAppDescriptorImpl(spec -> spec.setTaskFactory(createTaskFactory()), new MapConfig(configs)) : + new StreamAppDescriptorImpl(app, new MapConfig(configs)); + final LocalApplicationRunner runner = new LocalApplicationRunner(appDesc); + runner.run(); + boolean timedOut = !runner.waitForFinish(timeout); Assert.assertFalse("Timed out waiting for application to finish", timedOut); - ApplicationStatus status = runner.status(appSpec); + ApplicationStatus status = runner.status(); if (status.getStatusCode() == ApplicationStatus.StatusCode.UnsuccessfulFinish) { throw new SamzaException(ExceptionUtils.getStackTrace(status.getThrowable())); } @@ -372,4 +368,26 @@ public static Map> consumeStream(CollectionStream stream, D .collect(Collectors.toMap(entry -> entry.getKey().getPartition().getPartitionId(), entry -> entry.getValue().stream().map(e -> (T) e.getMessage()).collect(Collectors.toList()))); } + + private TaskFactory createTaskFactory() { + if (StreamTask.class.isAssignableFrom(taskClass)) { + return (StreamTaskFactory) () -> { + try { + return (StreamTask) taskClass.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new SamzaException(String.format("Failed to instantiate StreamTask class %s", taskClass.getName()), e); + } + }; + } else if (AsyncStreamTask.class.isAssignableFrom(taskClass)) { + return (AsyncStreamTaskFactory) () -> { + try { + return (AsyncStreamTask) taskClass.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new SamzaException(String.format("Failed to instantiate AsyncStreamTask class %s", taskClass.getName()), e); + } + }; + } + throw new SamzaException(String.format("Not supported task.class %s. task.class has to implement either StreamTask " + + "or AsyncStreamTask", taskClass.getName())); + } } diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index 31692d4c1c..d1c40d4da2 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -21,17 +21,16 @@ import joptsimple.OptionSet; import org.apache.samza.application.ApplicationClassUtils; -import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * {@link ApplicationRunnerMain} was designed for deploying {@link StreamApplication} in yarn + * {@link ApplicationRunnerMain} was designed for deploying {@link org.apache.samza.application.ApplicationBase} in yarn * and doesn't work for in standalone. * * This runner class is built for standalone failure tests and not recommended for general use. @@ -46,7 +45,8 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(ApplicationClassUtils.fromConfig(config), config); + ApplicationRunner + appRuntime = ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); try { LOGGER.info("Launching stream application: {} to start.", appRuntime); 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 f1a235ead8..aa5c49c316 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 @@ -19,7 +19,7 @@ package org.apache.samza.test.integration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,10 +32,10 @@ public class TestStandaloneIntegrationApplication implements StreamApplication { private static final Logger LOGGER = LoggerFactory.getLogger(TestStandaloneIntegrationApplication.class); @Override - public void describe(StreamApplicationSpec graph) { - String inputStream = graph.getConfig().get("input.stream.name"); + public void describe(StreamAppDescriptor appDesc) { + String inputStream = appDesc.getConfig().get("input.stream.name"); String outputStreamName = "standaloneIntegrationTestKafkaOutputTopic"; LOGGER.info("Publishing message to: {}.", outputStreamName); - graph.getInputStream(inputStream).sendTo(graph.getOutputStream(outputStreamName)); + appDesc.getInputStream(inputStream).sendTo(appDesc.getOutputStream(outputStreamName)); } } diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java index 189ae9b365..d738e63367 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java @@ -46,6 +46,7 @@ import org.apache.samza.config.ZkConfig; import org.apache.samza.coordinator.JobCoordinator; import org.apache.samza.coordinator.JobCoordinatorFactory; +import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemStream; @@ -135,9 +136,14 @@ protected StreamProcessor createStreamProcessor(final String pId, Map>getInputStream("PageView") + public void describe(StreamAppDescriptor appDesc) { + appDesc.>getInputStream("PageView") .map(Values.create()) .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") .sink((m, collector, coordinator) -> { - received.add(m.getValue()); - }); + received.add(m.getValue()); + }); } } - final ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new PipelineApplication(), new MapConfig(configs)); + final ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PipelineApplication(), new MapConfig(configs)); - app.run(); - app.waitForFinish(); + runner.run(); + runner.waitForFinish(); assertEquals(received.size(), count * partitionCount); } 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 eb1ed2254d..11427104c4 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 @@ -26,7 +26,8 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; -import org.apache.samza.application.internal.StreamAppSpecImpl; +import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -45,9 +46,8 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.TestStreamProcessorUtil; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; -import org.apache.samza.runtime.ApplicationRuntimes.AppRuntimeImpl; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.runtime.TestLocalApplicationRunner; import org.apache.samza.serializers.IntegerSerdeFactory; @@ -145,24 +145,24 @@ public void testWatermark() throws Exception { configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName()); List received = new ArrayList<>(); - class TestStreamApp { + class TestStreamApp implements StreamApplication { @Override - public void init(StreamAppSpecImpl appSpec, Config config) { - appSpec.>getInputStream("PageView") + public void describe(StreamAppDescriptor appDesc) { + appDesc.>getInputStream("PageView") .map(EndOfStreamIntegrationTest.Values.create()) .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") .sink((m, collector, coordinator) -> { - received.add(m.getValue()); - }); + received.add(m.getValue()); + }); } } - final ApplicationRuntime app = ApplicationRuntimes.createStreamApp(new TestStreamApp(), new MapConfig(configs)); - app.run(); - Map tasks = getTaskOperationGraphs(app); + final ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TestStreamApp(), new MapConfig(configs)); + runner.run(); + Map tasks = getTaskOperationGraphs((LocalApplicationRunner) runner); - app.waitForFinish(); + runner.waitForFinish(); StreamOperatorTask task0 = tasks.get("Partition 0"); OperatorImplGraph graph = TestStreamOperatorTask.getOperatorImplGraph(task0); @@ -183,10 +183,7 @@ public void init(StreamAppSpecImpl appSpec, Config config) { assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3); } - Map getTaskOperationGraphs(ApplicationRuntime app) throws Exception { - Field appRunnerField = AppRuntimeImpl.class.getDeclaredField("runner"); - appRunnerField.setAccessible(true); - LocalApplicationRunner runner = (LocalApplicationRunner) appRunnerField.get(app); + Map getTaskOperationGraphs(LocalApplicationRunner runner) throws Exception { StreamProcessor processor = TestLocalApplicationRunner.getProcessors(runner).iterator().next(); SamzaContainer container = TestStreamProcessorUtil.getContainer(processor); Map taskInstances = JavaConverters.mapAsJavaMapConverter(container.getTaskInstances()).asJava(); 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 417dfee129..208a347603 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 @@ -21,7 +21,7 @@ import java.util.Arrays; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.JsonSerdeV2; @@ -33,12 +33,12 @@ public class BroadcastAssertApp implements StreamApplication { @Override - public void describe(StreamApplicationSpec appSpec) { - Config config = appSpec.getConfig(); + public void describe(StreamAppDescriptor appDesc) { + Config config = appDesc.getConfig(); String inputTopic = config.get(INPUT_TOPIC_NAME_PROP); final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); - final MessageStream broadcastPageViews = appSpec + final MessageStream broadcastPageViews = appDesc .getInputStream(inputTopic, serde) .broadcast(serde, "pv"); 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 3675b30c03..0fcdde619c 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 @@ -32,20 +32,21 @@ import org.apache.samza.system.SystemStream; import org.apache.samza.test.controlmessages.TestData; import org.apache.samza.test.framework.stream.CollectionStream; -import static org.apache.samza.test.controlmessages.TestData.PageView; import org.junit.Assert; import org.junit.Test; +import static org.apache.samza.test.controlmessages.TestData.*; + public class StreamApplicationIntegrationTest { - final StreamApplication pageViewFilter = (streamAppSpec) -> { - streamAppSpec.>getInputStream("PageView").map( + final StreamApplication pageViewFilter = streamAppDesc -> { + streamAppDesc.>getInputStream("PageView").map( StreamApplicationIntegrationTest.Values.create()).filter(pv -> pv.getPageKey().equals("inbox")); }; - final StreamApplication pageViewParition = (streamAppSpec) -> { - streamAppSpec.>getInputStream("PageView") + final StreamApplication pageViewParition = streamAppDesc -> { + streamAppDesc.>getInputStream("PageView") .map(Values.create()) .partitionBy(pv -> pv.getMemberId(), pv -> pv, "p1") .sink((m, collector, coordinator) -> { diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java index 28a1e0de74..fe18e65d38 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java @@ -40,8 +40,8 @@ import org.apache.samza.config.KafkaConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.execution.TestStreamManager; -import org.apache.samza.runtime.ApplicationRuntime; -import org.apache.samza.runtime.ApplicationRuntimes; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.system.kafka.KafkaSystemAdmin; import org.apache.samza.test.harness.AbstractIntegrationTestHarness; import scala.Option; @@ -254,17 +254,13 @@ protected RunApplicationContext runApplication(StreamApplication streamApplicati } Config config = new MapConfig(configMap); - ApplicationRuntime appRuntime = ApplicationRuntimes.getApplicationRuntime(streamApplication, config); + ApplicationRunner appRuntime = ApplicationRunners.getApplicationRunner(streamApplication, config); appRuntime.run(); MessageStreamAssert.waitForComplete(); return new RunApplicationContext(appRuntime, config); } - public void setNumEmptyPolls(int numEmptyPolls) { - this.numEmptyPolls = numEmptyPolls; - } - /** * Shutdown and clear Zookeeper and Kafka broker state. */ @@ -281,15 +277,15 @@ public void tearDown() { * runApplication in order to do verification. */ protected static class RunApplicationContext { - private final ApplicationRuntime appRuntime; + private final ApplicationRunner appRuntime; private final Config config; - private RunApplicationContext(ApplicationRuntime appRuntime, Config config) { + private RunApplicationContext(ApplicationRunner appRuntime, Config config) { this.config = config; this.appRuntime = appRuntime; } - public ApplicationRuntime getAppRuntime() { + public ApplicationRunner getAppRuntime() { return this.appRuntime; } diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java b/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java index 9ae4f6dd37..adeca9cd02 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java @@ -25,7 +25,7 @@ import java.util.Collections; import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.FlatMapFunction; @@ -37,9 +37,9 @@ public class TestTimerApp implements StreamApplication { public static final String PAGE_VIEWS = "page-views"; @Override - public void describe(StreamApplicationSpec graph) { + public void describe(StreamAppDescriptor appDesc) { final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); - final MessageStream pageViews = graph.getInputStream(PAGE_VIEWS, serde); + final MessageStream pageViews = appDesc.getInputStream(PAGE_VIEWS, serde); final MessageStream output = pageViews.flatMap(new FlatmapTimerFn()); MessageStreamAssert.that("Output from timer function should container all complete messages", output, serde) diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/TimerTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/TimerTest.java index b1de935218..d4e0e14f02 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/TimerTest.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/TimerTest.java @@ -19,6 +19,10 @@ package org.apache.samza.test.framework; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; import org.junit.Before; import org.junit.Test; @@ -44,6 +48,13 @@ public void setup() { @Test public void testJob() throws InterruptedException { - runApplication(new TestTimerApp(), "TimerTest", null).getAppRuntime(); + Map configs = new HashMap<>(); + configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + configs.put("job.systemstreampartition.grouper.factory", "org.apache.samza.container.grouper.stream.AllSspToSingleTaskGrouperFactory"); + configs.put("task.name.grouper.factory", "org.apache.samza.container.grouper.task.SingleContainerGrouperFactory"); + configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory"); + configs.put(JobConfig.PROCESSOR_ID(), "0"); + + runApplication(new TestTimerApp(), "TimerTest", configs); } } 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 569e85c5f5..1c04a118e5 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 @@ -23,7 +23,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.functions.JoinFunction; @@ -56,13 +56,13 @@ List getIntermediateStreamIds() { } @Override - public void describe(StreamApplicationSpec appSpec) { - String inputTopicName1 = appSpec.getConfig().get(INPUT_TOPIC_NAME_1_PROP); - String inputTopicName2 = appSpec.getConfig().get(INPUT_TOPIC_NAME_2_PROP); - String outputTopic = appSpec.getConfig().get(OUTPUT_TOPIC_NAME_PROP); + public void describe(StreamAppDescriptor appDesc) { + String inputTopicName1 = appDesc.getConfig().get(INPUT_TOPIC_NAME_1_PROP); + String inputTopicName2 = appDesc.getConfig().get(INPUT_TOPIC_NAME_2_PROP); + String outputTopic = appDesc.getConfig().get(OUTPUT_TOPIC_NAME_PROP); - MessageStream pageViews = appSpec.getInputStream(inputTopicName1, new JsonSerdeV2<>(PageView.class)); - MessageStream adClicks = appSpec.getInputStream(inputTopicName2, new JsonSerdeV2<>(AdClick.class)); + MessageStream pageViews = appDesc.getInputStream(inputTopicName1, new JsonSerdeV2<>(PageView.class)); + MessageStream adClicks = appDesc.getInputStream(inputTopicName2, new JsonSerdeV2<>(AdClick.class)); MessageStream> pageViewsRepartitionedByViewId = pageViews .partitionBy(PageView::getViewId, pv -> pv, @@ -89,9 +89,9 @@ public void describe(StreamApplicationSpec appSpec) { new StringSerde(), new JsonSerdeV2<>(UserPageAdClick.class)), "userAdClickWindow") .map(windowPane -> KV.of(windowPane.getKey().getKey(), String.valueOf(windowPane.getMessage().size()))) .sink((message, messageCollector, taskCoordinator) -> { - taskCoordinator.commit(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER); - messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue())); - }); + taskCoordinator.commit(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER); + messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue())); + }); intermediateStreamIds.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamId()); intermediateStreamIds.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamId()); 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 aad35af497..28bd0b8d68 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 @@ -21,7 +21,7 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamApplicationSpec; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.operators.KV; import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.IntegerSerde; @@ -44,16 +44,16 @@ public class RepartitionWindowApp implements StreamApplication { @Override - public void describe(StreamApplicationSpec appSpec) { + public void describe(StreamAppDescriptor appDesc) { KVSerde pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageView.class)); - appSpec.getInputStream(INPUT_TOPIC, pgeMsgSerde) + appDesc.getInputStream(INPUT_TOPIC, pgeMsgSerde) .map(KV::getValue) .partitionBy(PageView::getUserId, m -> m, pgeMsgSerde, "inputByUID") .window(Windows.keyedSessionWindow(m -> m.getKey(), Duration.ofSeconds(3), () -> 0, (m, c) -> c + 1, new StringSerde("UTF-8"), new IntegerSerde()), "countWindow") - .map(wp -> KV.of(wp.getKey().getKey().toString(), wp.getMessage())) - .sendTo(appSpec.getOutputStream(OUTPUT_TOPIC)); + .map(wp -> KV.of(wp.getKey().getKey().toString(), wp.getMessage().toString())) + .sendTo(appDesc.getOutputStream(OUTPUT_TOPIC)); } } 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 3462689d4a..7a8046cd3b 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 @@ -20,11 +20,15 @@ package org.apache.samza.test.operator; import java.time.Duration; +import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.windows.Windows; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; @@ -37,7 +41,7 @@ /** * A {@link StreamApplication} that demonstrates a filter followed by a session window. */ -public class SessionWindowApp { +public class SessionWindowApp implements StreamApplication { private static final String INPUT_TOPIC = "page-views"; private static final String OUTPUT_TOPIC = "page-view-counts"; @@ -47,11 +51,16 @@ public class SessionWindowApp { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new SessionWindowApp(), config); + runner.run(); + runner.waitForFinish(); + } - MessageStream pageViews = app.openInput(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); + @Override + public void describe(StreamAppDescriptor appDesc) { + MessageStream pageViews = appDesc.getInputStream(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); OutputStream> outputStream = - app.openOutput(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); + appDesc.getOutputStream(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); pageViews .filter(m -> !FILTER_KEY.equals(m.getUserId())) @@ -60,7 +69,5 @@ public static void main(String[] args) { .map(m -> KV.of(m.getKey().getKey(), m.getMessage().size())) .sendTo(outputStream); - app.run(); - app.waitForFinish(); } } diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java index 063bb90abd..d5c0a0aa7f 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java @@ -26,6 +26,7 @@ import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.samza.Partition; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.system.SystemStreamMetadata; @@ -77,6 +78,10 @@ public void testRepartitionJoinWindowAppWithoutDeletionOnCommit() throws Excepti RepartitionJoinWindowApp app = new RepartitionJoinWindowApp(); String appName = "UserPageAdClickCounter"; Map configs = new HashMap<>(); + configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory"); + configs.put(JobConfig.PROCESSOR_ID(), "0"); + configs.put(TaskConfig.GROUPER_FACTORY(), "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"); configs.put("systems.kafka.samza.delete.committed.messages", "false"); configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_1_PROP, inputTopicName1); configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_2_PROP, inputTopicName2); @@ -105,6 +110,7 @@ public void testRepartitionJoinWindowAppAndDeleteMessagesOnCommit() throws Excep Map configs = new HashMap<>(); configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory"); + configs.put(JobConfig.PROCESSOR_ID(), "0"); configs.put(TaskConfig.GROUPER_FACTORY(), "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"); configs.put("systems.kafka.samza.delete.committed.messages", "true"); configs.put(RepartitionJoinWindowApp.INPUT_TOPIC_NAME_1_PROP, inputTopicName1); @@ -150,6 +156,10 @@ public void testBroadcastApp() { String inputTopicName2 = "ad-clicks"; String outputTopicName = "user-ad-click-counts"; Map configs = new HashMap<>(); + configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory"); + configs.put(JobConfig.PROCESSOR_ID(), "0"); + configs.put(TaskConfig.GROUPER_FACTORY(), "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"); configs.put(BroadcastAssertApp.INPUT_TOPIC_NAME_PROP, inputTopicName1); initializeTopics(inputTopicName1, inputTopicName2, outputTopicName); diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java index 8d3bdf0822..6775622010 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.test.framework.StreamApplicationIntegrationTestHarness; @@ -62,6 +63,7 @@ public void testRepartitionedSessionWindowCounter() throws Exception { Map configs = new HashMap<>(); configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory"); + configs.put(JobConfig.PROCESSOR_ID(), "0"); configs.put(TaskConfig.GROUPER_FACTORY(), "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"); configs.put(String.format("streams.%s.samza.msg.serde", INPUT_TOPIC), "string"); configs.put(String.format("streams.%s.samza.key.serde", INPUT_TOPIC), "string"); 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 977236449a..c1af59b193 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 @@ -19,13 +19,17 @@ package org.apache.samza.test.operator; -import java.time.Duration; import java.io.IOException; +import java.time.Duration; +import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.windows.Windows; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; @@ -38,7 +42,7 @@ /** * A {@link StreamApplication} that demonstrates a filter followed by a tumbling window. */ -public class TumblingWindowApp { +public class TumblingWindowApp implements StreamApplication { private static final String INPUT_TOPIC = "page-views"; private static final String OUTPUT_TOPIC = "page-view-counts"; @@ -48,12 +52,17 @@ public class TumblingWindowApp { public static void main(String[] args) throws IOException { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - StreamApplication app = StreamApplications.createStreamApp(config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TumblingWindowApp(), config); + runner.run(); + runner.waitForFinish(); + } + @Override + public void describe(StreamAppDescriptor appDesc) { MessageStream pageViews = - app.openInput(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); + appDesc.getInputStream(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class)); OutputStream> outputStream = - app.openOutput(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); + appDesc.getOutputStream(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde())); pageViews .filter(m -> !FILTER_KEY.equals(m.getUserId())) @@ -62,8 +71,5 @@ public static void main(String[] args) throws IOException { .map(m -> KV.of(m.getKey().getKey(), m.getMessage().size())) .sendTo(outputStream); - app.run(); - app.waitForFinish(); } - } 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 6035cc33b8..59cf18ee34 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 @@ -23,7 +23,7 @@ import java.io.ObjectInputStream; import java.io.Serializable; import java.util.concurrent.CountDownLatch; -import org.apache.samza.config.Config; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; @@ -78,45 +78,45 @@ public static StreamApplication getInstance( String outputTopic, CountDownLatch processedMessageLatch, StreamApplicationCallback callback, - CountDownLatch kafkaEventsConsumedLatch, - Config config) { - StreamApplication app = StreamApplications.createStreamApp(config); - String appName = app.getGlobalAppId(); - String processorName = config.get(JobConfig.PROCESSOR_ID()); - registerLatches(processedMessageLatch, kafkaEventsConsumedLatch, callback, appName, processorName); - MessageStream inputStream = null; - inputStream = app.openInput(inputTopic, new NoOpSerde()); - OutputStream outputStream = app.openOutput(outputTopic, new StringSerde()); - inputStream - .map(new MapFunction() { - transient CountDownLatch latch1; - transient CountDownLatch latch2; - transient StreamApplicationCallback callback; - - @Override - public String apply(String message) { - TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message); - if (callback != null) { - callback.onMessage(incomingMessage); - } - if (latch1 != null) { - latch1.countDown(); - } - if (latch2 != null) { - latch2.countDown(); - } - return incomingMessage.toString(); + CountDownLatch kafkaEventsConsumedLatch) { + StreamApplication app = appDesc -> { + String appName = appDesc.getGlobalAppId(); + String processorName = appDesc.getConfig().get(JobConfig.PROCESSOR_ID()); + registerLatches(processedMessageLatch, kafkaEventsConsumedLatch, callback, appName, processorName); + + MessageStream inputStream = appDesc.getInputStream(inputTopic, new NoOpSerde()); + OutputStream outputStream = appDesc.getOutputStream(outputTopic, new StringSerde()); + inputStream.map(new MapFunction() { + transient CountDownLatch latch1; + transient CountDownLatch latch2; + transient StreamApplicationCallback callback; + + @Override + public String apply(String message) { + TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message); + if (callback != null) { + callback.onMessage(incomingMessage); } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - SharedContextFactories.SharedContextFactory contextFactory = SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName); - this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch"); - this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch"); - this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback"); + if (latch1 != null) { + latch1.countDown(); + } + if (latch2 != null) { + latch2.countDown(); } - }) - .sendTo(outputStream); + return incomingMessage.toString(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + SharedContextFactories.SharedContextFactory contextFactory = + SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName); + this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch"); + this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch"); + this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback"); + } + }).sendTo(outputStream); + }; + return app; } diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java index c37132f9ed..874b243f21 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java @@ -42,7 +42,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.config.ZkConfig; import org.apache.samza.processor.StreamProcessor; -import org.apache.samza.processor.StreamProcessorLifecycleListener; +import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.task.AsyncStreamTaskAdapter; import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTaskFactory; @@ -52,10 +52,7 @@ import org.junit.Test; import scala.Option$; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.*; public class TestStreamProcessor extends StandaloneIntegrationTestHarness { @@ -232,7 +229,7 @@ private static class TestStubs { KafkaConsumer consumer; KafkaProducer producer; StreamProcessor processor; - StreamProcessorLifecycleListener listener; + ProcessorLifecycleListener listener; private TestStubs(String bootstrapServer) { shutdownLatch = new CountDownLatch(1); @@ -266,13 +263,15 @@ private void initConsumer(String bootstrapServer) { } private void initProcessorListener() { - listener = mock(StreamProcessorLifecycleListener.class); - doNothing().when(listener).onStart(); - doNothing().when(listener).onFailure(anyObject()); + listener = mock(ProcessorLifecycleListener.class); + doNothing().when(listener).afterStart(); doAnswer(invocation -> { - shutdownLatch.countDown(); + if (invocation.getArguments()[0] == null) { + // stopped successfully + shutdownLatch.countDown(); + } return null; - }).when(listener).onShutdown(); + }).when(listener).afterStop(any()); } private void initProducer(String bootstrapServer) { diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index cb0a207a16..9791a73ad0 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -48,10 +49,10 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.model.JobModel; import org.apache.samza.job.model.TaskModel; -import org.apache.samza.runtime.LocalApplicationRunner; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.test.StandaloneIntegrationTestHarness; import org.apache.samza.test.StandaloneTestUtils; -import org.apache.samza.test.processor.TestStreamApplication.StreamApplicationCallback; import org.apache.samza.util.NoOpMetricsRegistry; import org.apache.samza.zk.ZkJobCoordinatorFactory; import org.apache.samza.zk.ZkKeyBuilder; @@ -64,13 +65,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** - * Integration tests for {@link org.apache.samza.runtime.LocalApplicationRunner}. + * Integration tests for {@link org.apache.samza.runtime.LocalApplicationRunner} with {@link ZkJobCoordinatorFactory}. * - * Brings up embedded ZooKeeper, Kafka broker and launches multiple {@link StreamApplication} through - * {@link org.apache.samza.runtime.LocalApplicationRunner} to verify the guarantees made in stand alone execution environment. + * Brings up embedded ZooKeeper, Kafka broker and launches multiple {@link org.apache.samza.runtime.LocalApplicationRunner}s + * in an application to verify the guarantees made in stand alone execution environment. */ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarness { @@ -209,421 +211,421 @@ private Map buildStreamApplicationConfigMap(String systemName, S /** * sspGrouper is set to GroupBySystemStreamPartitionFactory. + * Run a stream application(appRunner1) consuming messages from input topic(effectively one container). + * + * In the callback triggered by appRunner1 after processing a message, bring up an another stream application(appRunner2). + * + * Assertions: + * A) JobModel generated before and after the addition of appRunner2 should be equal. + * B) Second stream application(appRunner2) should not join the group and process any message. + */ + + @Test + public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNumTasks() throws InterruptedException { + // Set up kafka topics. + publishKafkaEvents(inputSinglePartitionKafkaTopic, 0, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); + + // Configuration, verification variables + MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), + "org.apache.samza.container.grouper.stream.GroupBySystemStreamPartitionFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); + // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). + final JobModel[] previousJobModel = new JobModel[1]; + final String[] previousJobModelVersion = new String[1]; + AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); + final CountDownLatch secondProcessorRegistered = new CountDownLatch(1); + + zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { + // When appRunner2 with id: PROCESSOR_IDS[1] is registered, run processing message in appRunner1. + if (currentChilds.contains(PROCESSOR_IDS[1])) { + secondProcessorRegistered.countDown(); + } + }); + + // Set up stream app appRunner2. + CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS); + Config localTestConfig2 = new MapConfig(applicationConfig2, testConfig); + ApplicationRunner appRunner2 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance( + inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic, processedMessagesLatch, null, null), + localTestConfig2); + + // Callback handler for appRunner1. + TestStreamApplication.StreamApplicationCallback callback = m -> { + if (hasSecondProcessorJoined.compareAndSet(false, true)) { + previousJobModelVersion[0] = zkUtils.getJobModelVersion(); + previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); + appRunner2.run(); + try { + // Wait for appRunner2 to register with zookeeper. + secondProcessorRegistered.await(); + } catch (InterruptedException e) { + } + } + }; + + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2); + + // Set up stream app appRunner1. + Config localTestConfig1 = new MapConfig(applicationConfig1, testConfig); + ApplicationRunner appRunner1 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance( + inputSinglePartitionKafkaTopic, outputKafkaTopic, null, callback, kafkaEventsConsumedLatch), + localTestConfig1); + appRunner1.run(); + + kafkaEventsConsumedLatch.await(); + + String currentJobModelVersion = zkUtils.getJobModelVersion(); + JobModel updatedJobModel = zkUtils.getJobModel(currentJobModelVersion); + + // Job model before and after the addition of second stream processor should be the same. + assertEquals(previousJobModel[0], updatedJobModel); + assertEquals(new MapConfig(), updatedJobModel.getConfig()); + assertEquals(NUM_KAFKA_EVENTS, processedMessagesLatch.getCount()); + appRunner1.kill(); + appRunner1.waitForFinish(); + appRunner2.kill(); + appRunner2.waitForFinish(); + assertEquals(appRunner1.status(), ApplicationStatus.SuccessfulFinish); + assertEquals(appRunner2.status(), ApplicationStatus.UnsuccessfulFinish); + } + + /** + * sspGrouper is set to AllSspToSingleTaskGrouperFactory (All ssps from input kafka topic are mapped to a single task per container). + * AllSspToSingleTaskGrouperFactory should be used only with high-level consumers which do the partition management + * by themselves. Using the factory with the consumers that do not do the partition management will result in + * each processor/task consuming all the messages from all the partitions. * Run a stream application(streamApp1) consuming messages from input topic(effectively one container). * * In the callback triggered by streamApp1 after processing a message, bring up an another stream application(streamApp2). * * Assertions: - * A) JobModel generated before and after the addition of streamApp2 should be equal. - * B) Second stream application(streamApp2) should not join the group and process any message. + * A) JobModel generated before and after the addition of streamApp2 should not be equal. + * B) Second stream application(streamApp2) should join the group and process all the messages. */ + @Test + public void shouldUpdateJobModelWhenNewProcessorJoiningGroupUsingAllSspToSingleTaskGrouperFactory() throws InterruptedException { + // Set up kafka topics. + publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); + + // Configuration, verification variables + MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), "org.apache.samza.container.grouper.stream.AllSspToSingleTaskGrouperFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); + // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). + final JobModel[] previousJobModel = new JobModel[1]; + final String[] previousJobModelVersion = new String[1]; + AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); + final CountDownLatch secondProcessorRegistered = new CountDownLatch(1); + + zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { + // When appRunner2 with id: PROCESSOR_IDS[1] is registered, start processing message in appRunner1. + if (currentChilds.contains(PROCESSOR_IDS[1])) { + secondProcessorRegistered.countDown(); + } + }); + + // Set up appRunner2. + CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2); + Config testAppConfig2 = new MapConfig(applicationConfig2, testConfig); + ApplicationRunner appRunner2 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch, null, null), testAppConfig2); + + // Callback handler for appRunner1. + TestStreamApplication.StreamApplicationCallback streamApplicationCallback = message -> { + if (hasSecondProcessorJoined.compareAndSet(false, true)) { + previousJobModelVersion[0] = zkUtils.getJobModelVersion(); + previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); + appRunner2.run(); + try { + // Wait for appRunner2 to register with zookeeper. + secondProcessorRegistered.await(); + } catch (InterruptedException e) { + } + } + }; + + // This is the latch for the messages received by appRunner1. Since appRunner1 is run first, it gets one event + // redelivered due to re-balancing done by Zk after the appRunner2 joins (See the callback above). + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2 + 1); + + // Set up stream app appRunner1. + Config testAppConfig1 = new MapConfig(applicationConfig1, testConfig); + ApplicationRunner appRunner1 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance( + inputKafkaTopic, outputKafkaTopic, null, streamApplicationCallback, kafkaEventsConsumedLatch), + testAppConfig1); + appRunner1.run(); + + kafkaEventsConsumedLatch.await(); + + String currentJobModelVersion = zkUtils.getJobModelVersion(); + JobModel updatedJobModel = zkUtils.getJobModel(currentJobModelVersion); + + // JobModelVersion check to verify that leader publishes new jobModel. + assertTrue(Integer.parseInt(previousJobModelVersion[0]) < Integer.parseInt(currentJobModelVersion)); + + // Job model before and after the addition of second stream processor should not be the same. + assertTrue(!previousJobModel[0].equals(updatedJobModel)); + + // Task names in the job model should be different but the set of partitions should be the same and each task name + // should be assigned to a different container. + assertEquals(new MapConfig(), previousJobModel[0].getConfig()); + assertEquals(previousJobModel[0].getContainers().get(PROCESSOR_IDS[0]).getTasks().size(), 1); + assertEquals(new MapConfig(), updatedJobModel.getConfig()); + assertEquals(updatedJobModel.getContainers().get(PROCESSOR_IDS[0]).getTasks().size(), 1); + assertEquals(updatedJobModel.getContainers().get(PROCESSOR_IDS[1]).getTasks().size(), 1); + Map updatedTaskModelMap1 = updatedJobModel.getContainers().get(PROCESSOR_IDS[0]).getTasks(); + Map updatedTaskModelMap2 = updatedJobModel.getContainers().get(PROCESSOR_IDS[1]).getTasks(); + assertEquals(updatedTaskModelMap1.size(), 1); + assertEquals(updatedTaskModelMap2.size(), 1); + + TaskModel taskModel1 = updatedTaskModelMap1.values().stream().findFirst().get(); + TaskModel taskModel2 = updatedTaskModelMap2.values().stream().findFirst().get(); + assertEquals(taskModel1.getSystemStreamPartitions(), taskModel2.getSystemStreamPartitions()); + assertTrue(!taskModel1.getTaskName().getTaskName().equals(taskModel2.getTaskName().getTaskName())); + + processedMessagesLatch.await(); + + assertEquals(ApplicationStatus.Running, appRunner2.status()); + appRunner1.kill(); + appRunner1.waitForFinish(); + appRunner2.kill(); + appRunner2.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner1.status()); + } + + @Test + public void shouldReElectLeaderWhenLeaderDies() throws InterruptedException { + // Set up kafka topics. + publishKafkaEvents(inputKafkaTopic, 0, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); + + // Create stream applications. + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(2 * NUM_KAFKA_EVENTS); + CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); + CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); + CountDownLatch processedMessagesLatch3 = new CountDownLatch(1); + + ApplicationRunner appRunner1 = + ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, + processedMessagesLatch1, null, kafkaEventsConsumedLatch), applicationConfig1); + ApplicationRunner appRunner2 = + ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, + processedMessagesLatch2, null, kafkaEventsConsumedLatch), applicationConfig2); + ApplicationRunner appRunner3 = + ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, + processedMessagesLatch3, null, kafkaEventsConsumedLatch), applicationConfig3); + + appRunner1.run(); + appRunner2.run(); + + // Wait until all processors have processed a message. + processedMessagesLatch1.await(); + processedMessagesLatch2.await(); + + // Verifications before killing the leader. + String jobModelVersion = zkUtils.getJobModelVersion(); + JobModel jobModel = zkUtils.getJobModel(jobModelVersion); + assertEquals(2, jobModel.getContainers().size()); + assertEquals(Sets.newHashSet("0000000000", "0000000001"), jobModel.getContainers().keySet()); + assertEquals("1", jobModelVersion); + + List processorIdsFromZK = zkUtils.getActiveProcessorsIDs(Arrays.asList(PROCESSOR_IDS)); + + assertEquals(2, processorIdsFromZK.size()); + assertEquals(PROCESSOR_IDS[0], processorIdsFromZK.get(0)); + + // Kill the leader. Since appRunner1 is the first to join the cluster, it's the leader. + appRunner1.kill(); + appRunner1.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner1.status()); + + kafkaEventsConsumedLatch.await(); + publishKafkaEvents(inputKafkaTopic, 0, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); + + appRunner3.run(); + processedMessagesLatch3.await(); + + // Verifications after killing the leader. + processorIdsFromZK = zkUtils.getActiveProcessorsIDs(ImmutableList.of(PROCESSOR_IDS[1], PROCESSOR_IDS[2])); + assertEquals(2, processorIdsFromZK.size()); + assertEquals(PROCESSOR_IDS[1], processorIdsFromZK.get(0)); + jobModelVersion = zkUtils.getJobModelVersion(); + jobModel = zkUtils.getJobModel(jobModelVersion); + assertEquals(Sets.newHashSet("0000000001", "0000000002"), jobModel.getContainers().keySet()); + assertEquals(2, jobModel.getContainers().size()); + + appRunner2.kill(); + appRunner2.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner2.status()); + appRunner3.kill(); + appRunner3.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner3.status()); + } + + @Test + public void shouldFailWhenNewProcessorJoinsWithSameIdAsExistingProcessor() throws InterruptedException { + // Set up kafka topics. + publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// @Test -// public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNumTasks() throws InterruptedException { -// // Set up kafka topics. -// publishKafkaEvents(inputSinglePartitionKafkaTopic, 0, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); -// -// // Configuration, verification variables -// MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), -// "org.apache.samza.container.grouper.stream.GroupBySystemStreamPartitionFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); -// // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). -// final JobModel[] previousJobModel = new JobModel[1]; -// final String[] previousJobModelVersion = new String[1]; -// -// CountDownLatch secondProcessorRegistered = new CountDownLatch(1); -// -// zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { -// // When streamApp2 with id: PROCESSOR_IDS[1] is registered, run processing message in streamApp1. -// if (currentChilds.contains(PROCESSOR_IDS[1])) { -// secondProcessorRegistered.countDown(); -// } -// }); -// -// // Set up stream app 2. -// CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS); -// Config localTestConfig2 = new MapConfig(applicationConfig2, testConfig); -// StreamApplication -// streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null, localTestConfig2); -// -// // Set up stream app 1. -// Config localTestConfig1 = new MapConfig(applicationConfig1, testConfig); -// AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); -// CountDownLatch processedLatch = new CountDownLatch(1); -// StreamApplicationCallback callback = m -> { -// if (hasSecondProcessorJoined.compareAndSet(false, true)) { -// processedLatch.countDown(); -// try { -// secondProcessorRegistered.await(); -// } catch (InterruptedException ie) { -// throw new SamzaException(ie); -// } -// } -// }; -// -// CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2); -// StreamApplication -// streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, callback, kafkaEventsConsumedLatch, localTestConfig1); -// streamApp1.run(); -// -// processedLatch.await(); -// previousJobModelVersion[0] = zkUtils.getJobModelVersion(); -// previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); -// streamApp2.run(); -// -// kafkaEventsConsumedLatch.await(); -// -// String currentJobModelVersion = zkUtils.getJobModelVersion(); -// JobModel updatedJobModel = zkUtils.getJobModel(currentJobModelVersion); -// -// // Job model before and after the addition of second stream processor should be the same. -// assertEquals(previousJobModel[0], updatedJobModel); -// assertEquals(new MapConfig(), updatedJobModel.getConfig()); -// assertEquals(NUM_KAFKA_EVENTS, processedMessagesLatch.getCount()); -// streamApp1.kill(); -// streamApp1.waitForFinish(); -// streamApp2.kill(); -// streamApp2.waitForFinish(); -// assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); -// assertEquals(streamApp2.status(), ApplicationStatus.UnsuccessfulFinish); -//> } -// -// /** -// * sspGrouper is set to AllSspToSingleTaskGrouperFactory (All ssps from input kafka topic are mapped to a single task per container). -// * AllSspToSingleTaskGrouperFactory should be used only with high-level consumers which do the partition management -// * by themselves. Using the factory with the consumers that do not do the partition management will result in -// * each processor/task consuming all the messages from all the partitions. -// * Run a stream application(streamApp1) consuming messages from input topic(effectively one container). -// * -// * In the callback triggered by streamApp1 after processing a message, bring up an another stream application(streamApp2). -// * -// * Assertions: -// * A) JobModel generated before and after the addition of streamApp2 should not be equal. -// * B) Second stream application(streamApp2) should join the group and process all the messages. -// */ -// -// @Test -// public void shouldUpdateJobModelWhenNewProcessorJoiningGroupUsingAllSspToSingleTaskGrouperFactory() throws InterruptedException { -// // Set up kafka topics. -// publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS * 2, PROCESSOR_IDS[0]); -// -// // Configuration, verification variables -// MapConfig testConfig = new MapConfig(ImmutableMap.of(JobConfig.SSP_GROUPER_FACTORY(), "org.apache.samza.container.grouper.stream.AllSspToSingleTaskGrouperFactory", JobConfig.JOB_DEBOUNCE_TIME_MS(), "10")); -// // Declared as final array to update it from streamApplication callback(Variable should be declared final to access in lambda block). -// final JobModel[] previousJobModel = new JobModel[1]; -// final String[] previousJobModelVersion = new String[1]; -// AtomicBoolean hasSecondProcessorJoined = new AtomicBoolean(false); -// final CountDownLatch secondProcessorRegistered = new CountDownLatch(1); -// -// zkUtils.subscribeToProcessorChange((parentPath, currentChilds) -> { -// // When streamApp2 with id: PROCESSOR_IDS[1] is registered, run processing message in streamApp1. -// if (currentChilds.contains(PROCESSOR_IDS[1])) { -// secondProcessorRegistered.countDown(); -// } -// }); -// -// // Set up streamApp2. -// CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2); -// Config testAppConfig2 = new MapConfig(applicationConfig2, testConfig); -// LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(testAppConfig2); -// StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null, testAppConfig2); -// -// // Callback handler for streamApp1. -// StreamApplicationCallback streamApplicationCallback = message -> { -// if (hasSecondProcessorJoined.compareAndSet(false, true)) { -// previousJobModelVersion[0] = zkUtils.getJobModelVersion(); -// previousJobModel[0] = zkUtils.getJobModel(previousJobModelVersion[0]); -// localApplicationRunner2.run(streamApp2); -// try { -// // Wait for streamApp2 to register with zookeeper. -// secondProcessorRegistered.await(); -// } catch (InterruptedException e) { -// } -// } -// }; -// -// // This is the latch for the messages received by streamApp1. Since streamApp1 is run first, it gets one event -// // redelivered due to re-balancing done by Zk after the streamApp2 joins (See the callback above). -// CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2 + 1); -// -// // Set up stream app 1. -// Config testAppConfig1 = new MapConfig(applicationConfig1, testConfig); -// LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(testAppConfig1); -// StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, -// streamApplicationCallback, kafkaEventsConsumedLatch, testAppConfig1); -// localApplicationRunner1.run(streamApp1); -// -// kafkaEventsConsumedLatch.await(); -// -// String currentJobModelVersion = zkUtils.getJobModelVersion(); -// JobModel updatedJobModel = zkUtils.getJobModel(currentJobModelVersion); -// -// // JobModelVersion check to verify that leader publishes new jobModel. -// assertTrue(Integer.parseInt(previousJobModelVersion[0]) < Integer.parseInt(currentJobModelVersion)); -// -// // Job model before and after the addition of second stream processor should not be the same. -// assertTrue(!previousJobModel[0].equals(updatedJobModel)); -// -// // Task names in the job model should be different but the set of partitions should be the same and each task name -// // should be assigned to a different container. -// assertEquals(new MapConfig(), previousJobModel[0].getConfig()); -// assertEquals(previousJobModel[0].getContainers().get(PROCESSOR_IDS[0]).getTasks().size(), 1); -// assertEquals(new MapConfig(), updatedJobModel.getConfig()); -// assertEquals(updatedJobModel.getContainers().get(PROCESSOR_IDS[0]).getTasks().size(), 1); -// assertEquals(updatedJobModel.getContainers().get(PROCESSOR_IDS[1]).getTasks().size(), 1); -// Map updatedTaskModelMap1 = updatedJobModel.getContainers().get(PROCESSOR_IDS[0]).getTasks(); -// Map updatedTaskModelMap2 = updatedJobModel.getContainers().get(PROCESSOR_IDS[1]).getTasks(); -// assertEquals(updatedTaskModelMap1.size(), 1); -// assertEquals(updatedTaskModelMap2.size(), 1); -// -// TaskModel taskModel1 = updatedTaskModelMap1.values().stream().findFirst().get(); -// TaskModel taskModel2 = updatedTaskModelMap2.values().stream().findFirst().get(); -// assertEquals(taskModel1.getSystemStreamPartitions(), taskModel2.getSystemStreamPartitions()); -// assertTrue(!taskModel1.getTaskName().getTaskName().equals(taskModel2.getTaskName().getTaskName())); -// -// processedMessagesLatch.await(); -// -// assertEquals(ApplicationStatus.Running, localApplicationRunner2.status(streamApp2)); -// streamApp1.kill(); -// streamApp1.waitForFinish(); -// streamApp2.kill(); -// streamApp2.waitForFinish(); -// assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); -// } -// -// @Test -// public void shouldReElectLeaderWhenLeaderDies() throws InterruptedException { -// // Set up kafka topics. -// publishKafkaEvents(inputKafkaTopic, 0, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// -// // Create stream applications. -// CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(2 * NUM_KAFKA_EVENTS); -// CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); -// CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); -// CountDownLatch processedMessagesLatch3 = new CountDownLatch(1); -// -// StreamApplication -// streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); -// StreamApplication -// streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); -// StreamApplication -// streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch, applicationConfig3); -// -// streamApp1.run(); -// streamApp2.run(); -// streamApp3.run(); -// -// // Wait until all processors have processed a message. -// processedMessagesLatch1.await(); -// processedMessagesLatch2.await(); -// -// // Verifications before killing the leader. -// String jobModelVersion = zkUtils.getJobModelVersion(); -// JobModel jobModel = zkUtils.getJobModel(jobModelVersion); -// assertEquals(2, jobModel.getContainers().size()); -// assertEquals(Sets.newHashSet("0000000000", "0000000001"), jobModel.getContainers().keySet()); -// assertEquals("1", jobModelVersion); -// -// List processorIdsFromZK = zkUtils.getActiveProcessorsIDs(Arrays.asList(PROCESSOR_IDS)); -// -// assertEquals(2, processorIdsFromZK.size()); -// assertEquals(PROCESSOR_IDS[0], processorIdsFromZK.get(0)); -// -// // Kill the leader. Since streamApp1 is the first to join the cluster, it's the leader. -// streamApp1.kill(); -// streamApp1.waitForFinish(); -// -// assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); -// -// kafkaEventsConsumedLatch.await(); -// publishKafkaEvents(inputKafkaTopic, 0, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// -// processedMessagesLatch3.await(); -// -// // Verifications after killing the leader. -// assertEquals(ApplicationStatus.SuccessfulFinish, streamApp1.status()); -// processorIdsFromZK = zkUtils.getActiveProcessorsIDs(ImmutableList.of(PROCESSOR_IDS[1], PROCESSOR_IDS[2])); -// assertEquals(2, processorIdsFromZK.size()); -// assertEquals(PROCESSOR_IDS[1], processorIdsFromZK.get(0)); -// jobModelVersion = zkUtils.getJobModelVersion(); -// jobModel = zkUtils.getJobModel(jobModelVersion); -// assertEquals(Sets.newHashSet("0000000001", "0000000002"), jobModel.getContainers().keySet()); -// assertEquals(2, jobModel.getContainers().size()); -// -// streamApp2.kill(); -// streamApp2.waitForFinish(); -// assertEquals(streamApp2.status(), ApplicationStatus.SuccessfulFinish); -// streamApp3.kill(); -// streamApp3.waitForFinish(); -// assertEquals(streamApp3.status(), ApplicationStatus.SuccessfulFinish); -// } -// -// @Test -// public void shouldFailWhenNewProcessorJoinsWithSameIdAsExistingProcessor() throws InterruptedException { -// // Set up kafka topics. -// publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// -// // Create StreamApplications. -// CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); -// CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); -// CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); -// -// StreamApplication -// streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); -// StreamApplication -// streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); -// -// // Run stream applications. -// streamApp1.run(); -// streamApp2.run(); -// -// // Wait for message processing to run in both the processors. -// processedMessagesLatch1.await(); -// processedMessagesLatch2.await(); -// -// // Create a stream app with same processor id as SP2 and run it. It should fail. -// publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[2]); -// kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); -// StreamApplication -// streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, null, kafkaEventsConsumedLatch, applicationConfig2); -// // Fail when the duplicate processor joins. -// expectedException.expect(SamzaException.class); -// try { -// streamApp3.run(); -// } finally { -// streamApp1.kill(); -// streamApp2.kill(); -// -// streamApp1.waitForFinish(); -// streamApp2.waitForFinish(); -// } -// } -// -// @Test -// public void testRollingUpgradeOfStreamApplicationsShouldGenerateSameJobModel() throws Exception { -// // Set up kafka topics. -// publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// -// Map configMap = buildStreamApplicationConfigMap(TEST_SYSTEM, inputKafkaTopic, testStreamAppName, testStreamAppId); -// -// configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[0]); -// Config applicationConfig1 = new MapConfig(configMap); -// -// configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]); -// Config applicationConfig2 = new MapConfig(configMap); -// -// List messagesProcessed = new ArrayList<>(); -// StreamApplicationCallback streamApplicationCallback = messagesProcessed::add; -// -// // Create StreamApplication from configuration. -// CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); -// CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); -// CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); -// -// StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); -// StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); -// -// // Run stream application. -// streamApp1.run(); -// streamApp2.run(); -// -// processedMessagesLatch1.await(); -// processedMessagesLatch2.await(); -// -// // Read job model before rolling upgrade. -// String jobModelVersion = zkUtils.getJobModelVersion(); -// JobModel jobModel = zkUtils.getJobModel(jobModelVersion); -// -// streamApp1.kill(); -// streamApp1.waitForFinish(); -// -// int lastProcessedMessageId = -1; -// for (TestStreamApplication.TestKafkaEvent message : messagesProcessed) { -// lastProcessedMessageId = Math.max(lastProcessedMessageId, Integer.parseInt(message.getEventData())); -// } -// messagesProcessed.clear(); -// -// assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); -// -// LocalApplicationRunner applicationRunner4 = new LocalApplicationRunner(applicationConfig1); -// processedMessagesLatch1 = new CountDownLatch(1); -// publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, streamApplicationCallback, kafkaEventsConsumedLatch, applicationConfig1); -// streamApp1.run(); -// -// processedMessagesLatch1.await(); -// -// // Read new job model after rolling upgrade. -// String newJobModelVersion = zkUtils.getJobModelVersion(); -// JobModel newJobModel = zkUtils.getJobModel(newJobModelVersion); -// -// assertEquals(Integer.parseInt(jobModelVersion) + 1, Integer.parseInt(newJobModelVersion)); -// assertEquals(jobModel.getContainers(), newJobModel.getContainers()); -// -// streamApp2.kill(); -// streamApp2.waitForFinish(); -// assertEquals(streamApp2.status(), ApplicationStatus.SuccessfulFinish); -// streamApp1.kill(); -// streamApp1.waitForFinish(); -// assertEquals(streamApp1.status(), ApplicationStatus.SuccessfulFinish); -// } -// -// @Test -// public void testShouldStopStreamApplicationWhenShutdownTimeOutIsLessThanContainerShutdownTime() throws Exception { -// publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// -// Map configMap = buildStreamApplicationConfigMap(TEST_SYSTEM, inputKafkaTopic, testStreamAppName, testStreamAppId); -// configMap.put(TaskConfig.SHUTDOWN_MS(), "0"); -// -// configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[0]); -// Config applicationConfig1 = new MapConfig(configMap); -// -// configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]); -// Config applicationConfig2 = new MapConfig(configMap); -// -// LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1); -// LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2); -// -// // Create StreamApplication from configuration. -// CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); -// CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); -// CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); -// -// StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1); -// StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2); -// -// applicationRunner1.run(streamApp1); -// applicationRunner2.run(streamApp2); -// -// processedMessagesLatch1.await(); -// processedMessagesLatch2.await(); -// kafkaEventsConsumedLatch.await(); -// -// // At this stage, both the processors are running and have drained the kakfa source. -// // Trigger re-balancing phase, by manually adding a new processor. -// -// configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[2]); -// Config applicationConfig3 = new MapConfig(configMap); -// -// LocalApplicationRunner applicationRunner3 = new LocalApplicationRunner(applicationConfig3); -// CountDownLatch processedMessagesLatch3 = new CountDownLatch(1); -// -// StreamApplication streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch, applicationConfig3); -// applicationRunner3.run(streamApp3); -// -// publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); -// -// processedMessagesLatch3.await(); -// -// /** -// * If the processing has started in the third stream processor, then other two stream processors should be stopped. -// */ -// // TODO: This is a bug! Status should be unsuccessful finish. -// assertEquals(applicationRunner1.status(streamApp1), ApplicationStatus.SuccessfulFinish); -// assertEquals(applicationRunner2.status(streamApp2), ApplicationStatus.SuccessfulFinish); -// -// applicationRunner3.kill(streamApp3); -// applicationRunner3.waitForFinish(); -// assertEquals(applicationRunner3.status(streamApp3), ApplicationStatus.SuccessfulFinish); -// } + // Create StreamApplications. + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); + CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); + CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); + + ApplicationRunner appRunner1 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch), applicationConfig1); + ApplicationRunner appRunner2 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch), applicationConfig2); + + // Run stream applications. + appRunner1.run(); + appRunner2.run(); + + // Wait for message processing to run in both the processors. + processedMessagesLatch1.await(); + processedMessagesLatch2.await(); + + // Create a stream app with same processor id as SP2 and run it. It should fail. + publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[2]); + kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); + ApplicationRunner appRunner3 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, null, null, kafkaEventsConsumedLatch), applicationConfig2); + // Fail when the duplicate processor joins. + expectedException.expect(SamzaException.class); + try { + appRunner3.run(); + } finally { + appRunner1.kill(); + appRunner2.kill(); + + appRunner1.waitForFinish(); + appRunner2.waitForFinish(); + } + } + + @Test + public void testRollingUpgradeOfStreamApplicationsShouldGenerateSameJobModel() throws Exception { + // Set up kafka topics. + publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); + + Map configMap = buildStreamApplicationConfigMap(TEST_SYSTEM, inputKafkaTopic, testStreamAppName, testStreamAppId); + + configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[0]); + Config applicationConfig1 = new MapConfig(configMap); + + configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]); + Config applicationConfig2 = new MapConfig(configMap); + + List messagesProcessed = new ArrayList<>(); + TestStreamApplication.StreamApplicationCallback streamApplicationCallback = messagesProcessed::add; + + // Create StreamApplication from configuration. + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); + CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); + CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); + + ApplicationRunner appRunner1 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch), applicationConfig1); + ApplicationRunner appRunner2 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch), applicationConfig2); + + // Run stream application. + appRunner1.run(); + appRunner2.run(); + + processedMessagesLatch1.await(); + processedMessagesLatch2.await(); + + // Read job model before rolling upgrade. + String jobModelVersion = zkUtils.getJobModelVersion(); + JobModel jobModel = zkUtils.getJobModel(jobModelVersion); + + appRunner1.kill(); + appRunner1.waitForFinish(); + + int lastProcessedMessageId = -1; + for (TestStreamApplication.TestKafkaEvent message : messagesProcessed) { + lastProcessedMessageId = Math.max(lastProcessedMessageId, Integer.parseInt(message.getEventData())); + } + messagesProcessed.clear(); + + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner1.status()); + + processedMessagesLatch1 = new CountDownLatch(1); + publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); + ApplicationRunner appRunner3 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch1, streamApplicationCallback, kafkaEventsConsumedLatch), applicationConfig1); + appRunner3.run(); + + processedMessagesLatch1.await(); + + // Read new job model after rolling upgrade. + String newJobModelVersion = zkUtils.getJobModelVersion(); + JobModel newJobModel = zkUtils.getJobModel(newJobModelVersion); + + assertEquals(Integer.parseInt(jobModelVersion) + 1, Integer.parseInt(newJobModelVersion)); + assertEquals(jobModel.getContainers(), newJobModel.getContainers()); + + appRunner2.kill(); + appRunner2.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner2.status()); + appRunner3.kill(); + appRunner3.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner3.status()); + } + + @Test + public void testShouldStopStreamApplicationWhenShutdownTimeOutIsLessThanContainerShutdownTime() throws Exception { + publishKafkaEvents(inputKafkaTopic, 0, NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); + + Map configMap = buildStreamApplicationConfigMap(TEST_SYSTEM, inputKafkaTopic, testStreamAppName, testStreamAppId); + configMap.put(TaskConfig.SHUTDOWN_MS(), "0"); + + configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[0]); + Config applicationConfig1 = new MapConfig(configMap); + + configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]); + Config applicationConfig2 = new MapConfig(configMap); + + // Create StreamApplication from configuration. + CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS); + CountDownLatch processedMessagesLatch1 = new CountDownLatch(1); + CountDownLatch processedMessagesLatch2 = new CountDownLatch(1); + + ApplicationRunner appRunner1 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch), applicationConfig1); + ApplicationRunner appRunner2 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch), applicationConfig2); + + appRunner1.run(); + appRunner2.run(); + + processedMessagesLatch1.await(); + processedMessagesLatch2.await(); + kafkaEventsConsumedLatch.await(); + + // At this stage, both the processors are running and have drained the kakfa source. + // Trigger re-balancing phase, by manually adding a new processor. + + configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[2]); + Config applicationConfig3 = new MapConfig(configMap); + + CountDownLatch processedMessagesLatch3 = new CountDownLatch(1); + + ApplicationRunner appRunner3 = ApplicationRunners.getApplicationRunner(TestStreamApplication.getInstance(inputKafkaTopic, + outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch), applicationConfig3); + appRunner3.run(); + + publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]); + + processedMessagesLatch3.await(); + + /** + * If the processing has started in the third stream processor, then other two stream processors should be stopped. + */ + // TODO: This is a bug! Status should be unsuccessful finish. + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner1.status()); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner2.status()); + + appRunner3.kill(); + appRunner3.waitForFinish(); + assertEquals(ApplicationStatus.SuccessfulFinish, appRunner3.status()); + } } diff --git a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java index e776594d17..4ee42ae245 100644 --- a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java +++ b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java @@ -34,7 +34,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.serializers.JsonSerdeV2Factory; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; +import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; import org.apache.samza.sql.system.TestAvroSystemFactory; import org.apache.samza.sql.testutil.JsonUtil; import org.apache.samza.sql.testutil.MyTestUdf; @@ -78,7 +78,7 @@ public void testEndToEnd() throws Exception { String sql1 = "Insert into testavro.outputTopic select id, CURRENT_TIME as long_value from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -102,7 +102,7 @@ public void testEndToEndFlatten() throws Exception { + "from testavro.COMPLEX1"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = new ArrayList<>(TestAvroSystemFactory.messages); @@ -124,7 +124,7 @@ public void testEndToEndSubQuery() throws Exception { "Insert into testavro.outputTopic select Flatten(a) as id from (select MyTestArray(id) a from testavro.SIMPLE1)"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = new ArrayList<>(TestAvroSystemFactory.messages); @@ -145,7 +145,7 @@ public void testEndToEndUdf() throws Exception { String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); LOG.info("output Messages " + TestAvroSystemFactory.messages); @@ -169,7 +169,7 @@ public void testRegexMatchUdfInWhereClause() throws Exception { String sql1 = "Insert into testavro.outputTopic select id from testavro.SIMPLE1 where RegexMatch('.*4', Name)"; List sqlStmts = Collections.singletonList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); LOG.info("output Messages " + TestAvroSystemFactory.messages); @@ -193,7 +193,7 @@ public void testEndToEndStreamTableInnerJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -222,7 +222,7 @@ public void testEndToEndStreamTableInnerJoinWithNestedRecord() throws Exception List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -257,7 +257,7 @@ public void testEndToEndStreamTableInnerJoinWithFilter() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -289,7 +289,7 @@ public void testEndToEndStreamTableInnerJoinWithNullForeignKeys() throws Excepti List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -318,7 +318,7 @@ public void testEndToEndStreamTableLeftJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -347,7 +347,7 @@ public void testEndToEndStreamTableRightJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -378,7 +378,7 @@ public void testEndToEndStreamTableTableJoin() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -408,7 +408,7 @@ public void testEndToEndStreamTableTableJoinWithCompositeKey() throws Exception List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); List outMessages = TestAvroSystemFactory.messages.stream() @@ -442,7 +442,7 @@ public void testEndToEndGroupBy() throws Exception { List sqlStmts = Arrays.asList(sql); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime runner = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); runner.runAndWaitForFinish(); // Let's capture the list of windows/counts per key. 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 12b85a6352..7434fd3eeb 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 @@ -24,8 +24,9 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; - import java.util.concurrent.atomic.AtomicInteger; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -58,8 +59,7 @@ import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; /** * This test class tests sendTo() and join() for local tables @@ -81,18 +81,18 @@ public void testSendTo() throws Exception { MyMapFunction mapFn = new MyMapFunction(); - final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - final StreamApplication app = (streamGraph, cfg) -> { + final StreamApplication app = appDesc -> { - Table> table = streamGraph.getTable(new InMemoryTableDescriptor("t1") + Table> table = appDesc.getTable(new InMemoryTableDescriptor("t1") .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))); - streamGraph.getInputStream("Profile", new NoOpSerde()) + appDesc.getInputStream("Profile", new NoOpSerde()) .map(mapFn) .sendTo(table); }; - runner.run(app); + final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + runner.run(); runner.waitForFinish(); for (int i = 0; i < partitionCount; i++) { @@ -116,17 +116,16 @@ static class TestStreamTableJoin { } void runTest() { - final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - final StreamApplication app = (streamGraph, cfg) -> { + final StreamApplication app = appDesc -> { - Table> table = streamGraph.getTable( + Table> table = appDesc.getTable( new InMemoryTableDescriptor("t1").withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))); - streamGraph.getInputStream("Profile", new NoOpSerde()) + appDesc.getInputStream("Profile", new NoOpSerde()) .map(m -> new KV(m.getMemberId(), m)) .sendTo(table); - streamGraph.getInputStream("PageView", new NoOpSerde()) + appDesc.getInputStream("PageView", new NoOpSerde()) .map(pv -> { received.add(pv); return pv; @@ -136,7 +135,8 @@ void runTest() { .sink((m, collector, coordinator) -> joined.add(m)); }; - runner.run(app); + final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + runner.run(); runner.waitForFinish(); assertEquals(count * partitionCount, received.size()); @@ -190,14 +190,13 @@ void runTest() { PageViewToProfileJoinFunction joinFn1 = new PageViewToProfileJoinFunction(); PageViewToProfileJoinFunction joinFn2 = new PageViewToProfileJoinFunction(); - final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - final StreamApplication app = (streamGraph, cfg) -> { + final StreamApplication app = appDesc -> { - Table> profileTable = streamGraph.getTable(new InMemoryTableDescriptor("t1") + Table> profileTable = appDesc.getTable(new InMemoryTableDescriptor("t1") .withSerde(profileKVSerde)); - MessageStream profileStream1 = streamGraph.getInputStream("Profile1", new NoOpSerde()); - MessageStream profileStream2 = streamGraph.getInputStream("Profile2", new NoOpSerde()); + MessageStream profileStream1 = appDesc.getInputStream("Profile1", new NoOpSerde()); + MessageStream profileStream2 = appDesc.getInputStream("Profile2", new NoOpSerde()); profileStream1 .map(m -> { @@ -212,8 +211,8 @@ void runTest() { }) .sendTo(profileTable); - MessageStream pageViewStream1 = streamGraph.getInputStream("PageView1", new NoOpSerde()); - MessageStream pageViewStream2 = streamGraph.getInputStream("PageView2", new NoOpSerde()); + MessageStream pageViewStream1 = appDesc.getInputStream("PageView1", new NoOpSerde()); + MessageStream pageViewStream2 = appDesc.getInputStream("PageView2", new NoOpSerde()); pageViewStream1 .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p1") @@ -226,7 +225,8 @@ void runTest() { .sink((m, collector, coordinator) -> joinedPageViews2.add(m)); }; - runner.run(app); + final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + runner.run(); runner.waitForFinish(); assertEquals(count * partitionCount, sentToProfileTable1.size()); 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 d9016b1888..2e34d33044 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 @@ -26,13 +26,12 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; import org.apache.samza.operators.KV; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; @@ -121,16 +120,6 @@ private void runTest(String systemName, StreamApplication app, List> table = graph.getTable(getTableDescriptor()); - - graph.getInputStream(PAGEVIEW_STREAM, new NoOpSerde()) - .partitionBy(TestTableData.PageView::getMemberId, v -> v, "partition-page-view") - .join(table, new TestLocalTable.PageViewToProfileJoinFunction()) - .sendTo(graph.getOutputStream(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<>())); - } - protected TableDescriptor getTableDescriptor() { return new InMemoryTableDescriptor(PROFILE_TABLE) .withSerde(KVSerde.of(new IntegerSerde(), new TestTableData.ProfileJsonSerde())) @@ -142,6 +131,16 @@ public void init(StreamGraph graph, Config config) { return ImmutableList.of(new Entry<>(key, profile)); }); } + + @Override + public void describe(StreamAppDescriptor appDesc) { + Table> table = appDesc.getTable(getTableDescriptor()); + + appDesc.getInputStream(PAGEVIEW_STREAM, new NoOpSerde()) + .partitionBy(TestTableData.PageView::getMemberId, v -> v, "partition-page-view") + .join(table, new TestLocalTable.PageViewToProfileJoinFunction()) + .sendTo(appDesc.getOutputStream(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<>())); + } } static class DurablePageViewProfileJoin extends PageViewProfileJoin { diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java index 8990b3f054..eed0ffe765 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java @@ -19,6 +19,7 @@ package org.apache.samza.test.table; +import com.google.common.cache.CacheBuilder; import java.io.IOException; import java.io.ObjectInputStream; import java.time.Duration; @@ -31,25 +32,26 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; - import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.MapConfig; import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; import org.apache.samza.operators.KV; -import org.apache.samza.application.StreamApplicationSpec; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.table.Table; import org.apache.samza.table.caching.CachingTableDescriptor; import org.apache.samza.table.caching.guava.GuavaCacheTableDescriptor; -import org.apache.samza.table.remote.TableReadFunction; -import org.apache.samza.table.remote.TableWriteFunction; +import org.apache.samza.table.remote.RemoteReadWriteTable; import org.apache.samza.table.remote.RemoteReadableTable; import org.apache.samza.table.remote.RemoteTableDescriptor; -import org.apache.samza.table.remote.RemoteReadWriteTable; +import org.apache.samza.table.remote.TableReadFunction; +import org.apache.samza.table.remote.TableWriteFunction; import org.apache.samza.task.TaskContext; import org.apache.samza.test.harness.AbstractIntegrationTestHarness; import org.apache.samza.test.util.Base64Serializer; @@ -57,13 +59,9 @@ import org.junit.Assert; import org.junit.Test; -import com.google.common.cache.CacheBuilder; - import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.*; public class TestRemoteTable extends AbstractIntegrationTestHarness { @@ -126,7 +124,7 @@ public void deleteAll(Collection keys) { } } - private Table> getCachingTable(Table> actualTable, boolean defaultCache, String id, StreamApplicationSpec streamGraph) { + private Table> getCachingTable(Table> actualTable, boolean defaultCache, String id, StreamAppDescriptor streamGraph) { CachingTableDescriptor cachingDesc = new CachingTableDescriptor<>("caching-table-" + id); if (defaultCache) { cachingDesc.withReadTtl(Duration.ofMinutes(5)); @@ -160,8 +158,7 @@ private void doTestStreamTableJoinRemoteTable(boolean withCache, boolean default final RateLimiter readRateLimiter = mock(RateLimiter.class); final RateLimiter writeRateLimiter = mock(RateLimiter.class); - final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs)); - final StreamApplication app = (streamGraph, cfg) -> { + final StreamApplication app = appDesc -> { RemoteTableDescriptor inputTableDesc = new RemoteTableDescriptor<>("profile-table-1"); inputTableDesc .withReadFunction(InMemoryReadFunction.getInMemoryReadFunction(profiles)) @@ -173,19 +170,19 @@ private void doTestStreamTableJoinRemoteTable(boolean withCache, boolean default .withWriteFunction(writer) .withRateLimiter(writeRateLimiter, null, null); - Table> outputTable = streamGraph.getTable(outputTableDesc); + Table> outputTable = appDesc.getTable(outputTableDesc); if (withCache) { - outputTable = getCachingTable(outputTable, defaultCache, "output", streamGraph); + outputTable = getCachingTable(outputTable, defaultCache, "output", appDesc); } - Table> inputTable = streamGraph.getTable(inputTableDesc); + Table> inputTable = appDesc.getTable(inputTableDesc); if (withCache) { - inputTable = getCachingTable(inputTable, defaultCache, "input", streamGraph); + inputTable = getCachingTable(inputTable, defaultCache, "input", appDesc); } - streamGraph.getInputStream("PageView", new NoOpSerde()) + appDesc.getInputStream("PageView", new NoOpSerde()) .map(pv -> { return new KV(pv.getMemberId(), pv); }) @@ -194,7 +191,8 @@ private void doTestStreamTableJoinRemoteTable(boolean withCache, boolean default .sendTo(outputTable); }; - runner.run(app); + final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + runner.run(); runner.waitForFinish(); int numExpected = count * partitionCount; diff --git a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java index 9064e9fb08..df68f089a2 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java @@ -42,7 +42,7 @@ import org.apache.samza.sql.impl.ConfigBasedUdfResolver; import org.apache.samza.sql.interfaces.SqlIOConfig; import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; -import org.apache.samza.sql.runner.SamzaSqlApplicationRuntime; +import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; import org.apache.samza.sql.testutil.JsonUtil; import org.apache.samza.sql.testutil.SqlFileParser; import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; @@ -107,7 +107,7 @@ public static void main(String[] args) { public static void executeSql(List sqlStmts) { Map staticConfigs = fetchSamzaSqlConfig(); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRuntime appRunnable = new SamzaSqlApplicationRuntime(true, new MapConfig(staticConfigs)); + SamzaSqlApplicationRunner appRunnable = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); appRunnable.runAndWaitForFinish(); } diff --git a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java index ad2155c3a6..6b8fb74d1f 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java @@ -30,13 +30,14 @@ import java.util.stream.IntStream; import org.apache.commons.cli.ParseException; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamAppSpecImpl; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.functions.MapFunction; +import org.apache.samza.runtime.ApplicationRunner; +import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.runtime.LocalApplicationRunner; import org.apache.samza.standalone.PassthroughJobCoordinatorFactory; @@ -67,16 +68,15 @@ public void addMoreSystemConfigs(Properties props) { } public void start() throws IOException, InterruptedException { - LocalApplicationRunner runner = new LocalApplicationRunner(config); super.start(); MessageConsumer consumeFn = new MessageConsumer(); StreamApplication app = spec -> { MessageStream stream = spec.getInputStream(streamId); stream.map(consumeFn); }; - StreamAppSpecImpl appSpec = new StreamAppSpecImpl(app, new MapConfig()); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, new MapConfig()); - runner.run(appSpec); + runner.run(); while (consumeFn.getEventsConsumed() < totalEvents) { Thread.sleep(10); @@ -84,7 +84,7 @@ public void start() throws IOException, InterruptedException { Instant endTime = Instant.now(); - runner.kill(appSpec); + runner.kill(); System.out.println("\n*******************"); System.out.println(String.format("Started at %s Ending at %s ", consumeFn.startTime, endTime)); From db96da830a11d001eec7c068bf8a7d03a9a11799 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Thu, 16 Aug 2018 17:15:48 -0700 Subject: [PATCH 10/38] SAMZA-1789: WIP - revision to address review feedbacks. --- .../versioned/hello-samza-high-level-code.md | 16 +- .../samza/application/ApplicationBase.java | 13 +- .../application/ApplicationDescriptor.java | 23 +- .../application/StreamAppDescriptor.java | 117 +++++- .../samza/application/StreamApplication.java | 4 + .../samza/application/TaskAppDescriptor.java | 29 +- .../samza/application/TaskApplication.java | 4 + .../internal/StreamAppDescriptorImpl.java | 114 ------ .../apache/samza/operators/MessageStream.java | 19 +- .../apache/samza/operators/StreamGraph.java | 137 ------- .../operators/functions/ClosableFunction.java | 2 +- .../operators/functions/InitableFunction.java | 2 +- .../samza/runtime/ApplicationRunner.java | 1 + .../samza/runtime/ApplicationRunners.java | 46 +-- .../samza/runtime/ProcessorContext.java | 9 + .../runtime/ProcessorLifecycleListener.java | 21 +- .../ProcessorLifecycleListenerFactory.java | 2 + .../samza/task/AsyncStreamTaskFactory.java | 7 +- .../apache/samza/task/StreamTaskFactory.java | 3 +- .../org/apache/samza/task/TaskFactory.java | 2 + .../internal/TestStreamAppDescriptorImpl.java | 122 ------ .../application/internal/TestStreamGraph.java | 89 ---- .../samza/runtime/TestApplicationRunner.java | 10 +- .../samza/runtime/TestApplicationRunners.java | 6 +- .../MockEventHubClientManagerFactory.java | 4 +- .../samza/application}/AppDescriptorImpl.java | 67 +-- .../application/ApplicationDescriptors.java | 42 ++ .../application/LegacyTaskApplication.java | 51 +++ .../StreamAppDescriptorImpl.java} | 62 +-- .../application}/TaskAppDescriptorImpl.java | 41 +- .../container/SamzaContainerListener.java | 16 +- .../samza/execution/ExecutionPlanner.java | 6 +- .../samza/operators/MessageStreamImpl.java | 13 +- .../samza/operators/OperatorSpecGraph.java | 29 +- .../samza/operators/spec/OperatorSpec.java | 2 +- .../stream/IntermediateMessageStreamImpl.java | 6 +- .../samza/processor/StreamProcessor.java | 49 ++- .../runtime/AbstractApplicationRunner.java | 117 +++--- .../ApplicationClassUtils.java | 16 +- .../samza/runtime/ApplicationRunnerMain.java | 5 +- .../runtime/ApplicationRunnerOperation.java | 2 +- .../samza/runtime/LocalApplicationRunner.java | 270 +++++------- .../samza/runtime/LocalContainerRunner.java | 78 ++-- .../runtime/RemoteApplicationRunner.java | 200 ++++----- .../apache/samza/task/StreamOperatorTask.java | 12 +- .../apache/samza/task/TaskFactoryUtil.java | 5 +- .../org/apache/samza/util/StreamUtil.java | 3 +- .../samza/container/SamzaContainer.scala | 11 +- .../samza/job/local/ThreadJobFactory.scala | 18 +- .../TestStreamAppDescriptorImpl.java} | 295 +++++++------- .../TestTaskAppDescriptorImpl.java | 15 +- .../samza/execution/TestExecutionPlanner.java | 192 +++++---- .../execution/TestJobGraphJsonGenerator.java | 84 ++-- .../apache/samza/execution/TestJobNode.java | 27 +- .../samza/operators/TestJoinOperator.java | 91 +++-- .../operators/TestMessageStreamImpl.java | 29 +- .../operators/TestOperatorSpecGraph.java | 13 +- .../operators/impl/TestOperatorImplGraph.java | 133 +++--- .../operators/impl/TestWindowOperator.java | 113 +++--- .../spec/TestPartitionByOperatorSpec.java | 54 ++- .../samza/processor/TestStreamProcessor.java | 47 +-- .../TestApplicationClassUtils.java | 8 +- .../runtime/TestApplicationRunnerMain.java | 57 +-- .../runtime/TestLocalApplicationRunner.java | 95 ++--- .../runtime/TestRemoteApplicationRunner.java | 18 +- .../samza/container/TestSamzaContainer.scala | 74 ++-- .../samza/sql/translator/JoinTranslator.java | 2 +- .../samza/sql/translator/QueryTranslator.java | 6 +- .../samza/sql/translator/ScanTranslator.java | 6 +- .../sql/translator/TranslatorContext.java | 16 +- .../sql/translator/TestFilterTranslator.java | 4 +- .../sql/translator/TestJoinTranslator.java | 6 +- .../sql/translator/TestProjectTranslator.java | 6 +- .../sql/translator/TestQueryTranslator.java | 383 ++---------------- .../samza/example/TaskApplicationExample.java | 20 +- .../samza/test/framework/TestRunner.java | 11 +- .../LocalApplicationRunnerMain.java | 2 +- .../processor/TestZkStreamProcessorBase.java | 22 +- .../TestZkStreamProcessorSession.java | 4 +- .../test/processor/TestStreamApplication.java | 3 +- .../test/processor/TestStreamProcessor.java | 9 +- .../samza/test/table/TestLocalTable.java | 15 +- .../samza/test/table/TestRemoteTable.java | 15 +- 83 files changed, 1553 insertions(+), 2245 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java delete mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java delete mode 100644 samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java rename {samza-api/src/main/java/org/apache/samza/application/internal => samza-core/src/main/java/org/apache/samza/application}/AppDescriptorImpl.java (65%) create mode 100644 samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java create mode 100644 samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java rename samza-core/src/main/java/org/apache/samza/{operators/StreamGraphSpec.java => application/StreamAppDescriptorImpl.java} (85%) rename {samza-api/src/main/java/org/apache/samza/application/internal => samza-core/src/main/java/org/apache/samza/application}/TaskAppDescriptorImpl.java (68%) rename samza-core/src/main/java/org/apache/samza/{application => runtime}/ApplicationClassUtils.java (82%) rename samza-core/src/test/java/org/apache/samza/{operators/TestStreamGraphSpec.java => application/TestStreamAppDescriptorImpl.java} (64%) rename {samza-api/src/test/java/org/apache/samza/application/internal => samza-core/src/test/java/org/apache/samza/application}/TestTaskAppDescriptorImpl.java (90%) rename samza-core/src/test/java/org/apache/samza/{application => runtime}/TestApplicationClassUtils.java (90%) diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md index a1a98396c2..2f6a4a6094 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-code.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-code.md @@ -176,7 +176,7 @@ package samza.examples.wikipedia.application; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; -import org.apache.samza.application.StreamGraph; +import org.apache.samza.operators.StreamGraph; public class MyWikipediaApplication implements StreamApplication{ @Override @@ -188,12 +188,12 @@ public class MyWikipediaApplication implements StreamApplication{ Be sure to include the Apache header. The project will not compile without it. -The [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method is where the application logic is defined. The [Config](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/config/Config.html) argument is the runtime configuration loaded from the properties file we defined earlier. The [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) argument provides methods to declare input streams. You can then invoke a number of flexible operations on those streams. The result of each operation is another stream, so you can keep chaining more operations or direct the result to an output stream. +The [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method is where the application logic is defined. The [Config](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/config/Config.html) argument is the runtime configuration loaded from the properties file we defined earlier. The [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) argument provides methods to declare input streams. You can then invoke a number of flexible operations on those streams. The result of each operation is another stream, so you can keep chaining more operations or direct the result to an output stream. Next, we will declare the input streams for the Wikipedia application. #### Inputs -The Wikipedia application consumes events from three channels. Let's declare each of those channels as an input streams via the [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) in the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method. +The Wikipedia application consumes events from three channels. Let's declare each of those channels as an input streams via the [StreamGraph](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/StreamGraph.html) in the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method. {% highlight java %} MessageStream wikipediaEvents = streamGraph.getInputStream("en-wikipedia", new NoOpSerde<>()); MessageStream wiktionaryEvents = streamGraph.getInputStream("en-wiktionary", new NoOpSerde<>()); @@ -208,7 +208,7 @@ Note the streams are all MessageStreams of type WikipediaFeedEvent. [MessageStre #### Merge We'd like to use the same processing logic for all three input streams, so we will use the [mergeAll](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#mergeAll-java.util.Collection-) operator to merge them together. Note: this is not the same as a [join](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#join-org.apache.samza.operators.MessageStream-org.apache.samza.operators.functions.JoinFunction-java.time.Duration-) because we are not associating events by key. We are simply combining three streams into one, like a union. -Add the following snippet to the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method. It merges all the input streams into a new one called _allWikipediaEvents_ +Add the following snippet to the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method. It merges all the input streams into a new one called _allWikipediaEvents_ {% highlight java %} MessageStream allWikipediaEvents = MessageStream.mergeAll(ImmutableList.of(wikipediaEvents, wiktionaryEvents, wikiNewsEvents)); {% endhighlight %} @@ -218,7 +218,7 @@ Note there is a [merge](/learn/documentation/{{site.version}}/api/javadocs/org/a #### Parse The next step is to parse the events and extract some information. We will use the pre-existing `WikipediaParser.parseEvent()' method to do this. The parser extracts some flags we want to monitor as well as some metadata about the event. Inspect the method signature. The input is a WikipediaFeedEvents and the output is a Map. These types will be reflected in the types of the streams before and after the operation. -In the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method, invoke the [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation on `allWikipediaEvents`, passing the `WikipediaParser::parseEvent` method reference as follows: +In the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method, invoke the [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation on `allWikipediaEvents`, passing the `WikipediaParser::parseEvent` method reference as follows: {% highlight java %} allWikipediaEvents.map(WikipediaParser::parseEvent); @@ -227,7 +227,7 @@ allWikipediaEvents.map(WikipediaParser::parseEvent); #### Window Now that we have the relevant information extracted, let's perform some aggregations over a 10-second [window](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/windows/Window.html). -First, we need a container class for statistics we want to track. Add the following static class after the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-) method. +First, we need a container class for statistics we want to track. Add the following static class after the [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-) method. {% highlight java %} private static class WikipediaStats { int edits = 0; @@ -262,7 +262,7 @@ private class WikipediaStatsAggregator implements FoldLeftFunction(WikipediaStats.class))); @@ -301,7 +301,7 @@ Paste the following after the aggregator class: } {% endhighlight %} -Now, we can invoke the method by adding another [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation to the chain in [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.application.StreamGraph-org.apache.samza.config.Config-). The operator chain should now look like this: +Now, we can invoke the method by adding another [map](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/operators/MessageStream.html#map-org.apache.samza.operators.functions.MapFunction-) operation to the chain in [init](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/application/StreamApplication.html#init-org.apache.samza.operators.StreamGraph-org.apache.samza.config.Config-). The operator chain should now look like this: {% highlight java %} allWikipediaEvents.map(WikipediaParser::parseEvent) .window(Windows.tumblingWindow(Duration.ofSeconds(10), WikipediaStats::new, new WikipediaStatsAggregator())) diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java index eaf982c641..4ae7e2dfc9 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java @@ -18,12 +18,17 @@ */ package org.apache.samza.application; +import org.apache.samza.annotation.InterfaceStability; + + /** - * The base interface for all user-implemented applications in Samza. The main processing logic of the user application - * should be implemented in {@link ApplicationBase#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. + * The base interface for all user-implemented applications in Samza. + *

    + * The main processing logic of the user application should be implemented in {@link ApplicationBase#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. */ +@InterfaceStability.Evolving public interface ApplicationBase { /** diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index aafe3ff5ab..f3e5dcae43 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -18,22 +18,23 @@ */ package org.apache.samza.application; +import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; /** - * The base interface class to describe a user application in Samza. Sub-classes {@link StreamAppDescriptor} - * and {@link TaskAppDescriptor} are specific interfaces for applications written in high-level DAG and low-level task APIs, - * respectively. + * The base interface class to describe a user application in Samza. + *

    + * Sub-classes {@link StreamAppDescriptor} and {@link TaskAppDescriptor} are specific interfaces for applications written + * in high-level DAG and low-level task APIs, respectively. + * + * @param type of user application this descriptor describes. It has to be either {@link StreamApplication} or + * {@link TaskApplication} */ +@InterfaceStability.Evolving public interface ApplicationDescriptor { - /** - * Get the global unique application ID in the runtime process - * @return globally unique application ID - */ - String getGlobalAppId(); /** * Get the user defined {@link Config} @@ -47,6 +48,8 @@ public interface ApplicationDescriptor { * The provided {@link ContextManager} can be used to setup shared context between the operator functions * within a task instance * + * TODO: this should be replaced by the shared context factory when SAMZA-1714 is fixed. + * @param contextManager the {@link ContextManager} to use for the application * @return the {@link ApplicationDescriptor} with {@code contextManager} set as its {@link ContextManager} */ @@ -55,8 +58,8 @@ public interface ApplicationDescriptor { /** * Sets the {@link ProcessorLifecycleListenerFactory} for this application. * - * @param listenerFactory the user implemented {@link ProcessorLifecycleListenerFactory} that creates lifecycle aware - * methods to be invoked before and after the start/stop of the StreamProcessor(s) in the application + * @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 the {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory} */ ApplicationDescriptor withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java index 4cfc7cd851..51bb36325a 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java @@ -19,12 +19,123 @@ package org.apache.samza.application; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.operators.StreamGraph; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.serializers.Serde; +import org.apache.samza.table.Table; /** - * The interface class to describe a user applicationin high-level DAG in Samza. + * The interface class to describe a user application in high-level DAG in Samza. */ @InterfaceStability.Evolving -public interface StreamAppDescriptor extends ApplicationDescriptor, StreamGraph { +public interface StreamAppDescriptor extends ApplicationDescriptor { + + /** + * Sets the default {@link Serde} to use for (de)serializing messages. + *

    . + * If the default serde is set, it must be set before creating any input or output streams. + *

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

    + * Providing an incompatible message type for the input/output streams that use the default serde will result in + * {@link ClassCastException}s at runtime. + * + * @param serde the default message {@link Serde} to use + */ + void setDefaultSerde(Serde serde); + + /** + * Gets the input {@link MessageStream} corresponding to the {@code streamId}. + *

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

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param serde the {@link Serde} to use for deserializing incoming messages + * @param the type of messages in the input {@link MessageStream} + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + MessageStream getInputStream(String streamId, Serde serde); + + /** + * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via + * {@link #setDefaultSerde(Serde)} for deserializing input messages. + *

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param the type of message in the input {@link MessageStream} + * @return the input {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + MessageStream getInputStream(String streamId); + + /** + * Gets the {@link OutputStream} corresponding to the {@code streamId}. + *

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

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

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param serde the {@link Serde} to use for serializing outgoing messages + * @param the type of messages in the {@link OutputStream} + * @return the output {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + OutputStream getOutputStream(String streamId, Serde serde); + + /** + * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via + * {@link #setDefaultSerde(Serde)} for serializing output messages. + *

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

    + * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. + * + * @param streamId the unique ID for the stream + * @param the type of messages in the {@link OutputStream} + * @return the output {@link MessageStream} + * @throws IllegalStateException when invoked multiple times with the same {@code streamId} + */ + OutputStream getOutputStream(String streamId); + + /** + * Gets the {@link Table} corresponding to the {@link TableDescriptor}. + *

    + * Multiple invocations of this method with the same {@link TableDescriptor} will throw an + * {@link IllegalStateException}. + * + * @param tableDesc the {@link TableDescriptor} + * @param the type of the key + * @param the type of the value + * @return the {@link Table} corresponding to the {@code tableDesc} + * @throws IllegalStateException when invoked multiple times with the same {@link TableDescriptor} + */ + Table> getTable(TableDescriptor tableDesc); } 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 848c26ac1b..3bac7ea0f2 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 @@ -18,8 +18,12 @@ */ package org.apache.samza.application; +import org.apache.samza.annotation.InterfaceStability; + + /** * The interface to implement user applications defining a high-level DAG as the main stream processing logic. */ +@InterfaceStability.Evolving public interface StreamApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java index aad0f75806..5cc0308f50 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java @@ -18,7 +18,7 @@ */ package org.apache.samza.application; -import java.util.List; +import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.task.TaskFactory; @@ -26,6 +26,7 @@ /** * The interface class to describe a user application as low-level task in Samza. */ +@InterfaceStability.Evolving public interface TaskAppDescriptor extends ApplicationDescriptor { /** @@ -37,26 +38,34 @@ public interface TaskAppDescriptor extends ApplicationDescriptor inputStreams); + void addInputStream(String inputStream); /** - * Adds the output streams to the user application. + * Adds the input stream to the user application. * - * @param outputStreams the list of streamIds for the output streams + * @param inputStream streamId of the input stream + */ + // TODO: needs to be replaced by InputStreamDescriptor after SAMZA-1804 is implemented + void addBroadcastStream(String inputStream); + + /** + * Adds the output stream to the user application. + * + * @param outputStream streamId of the output stream */ // TODO: needs to be replaced by OutputStreamDescriptor after SAMZA-1804 is implemented - void addOutputStreams(List outputStreams); + void addOutputStream(String outputStream); /** - * Adds the {@link TableDescriptor}s used in the application + * Adds the {@link TableDescriptor} used in the application * - * @param tables the list of {@link TableDescriptor}s + * @param table {@link TableDescriptor} */ - void addTables(List tables); + void addTable(TableDescriptor table); } 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 ef580b2759..7841385c92 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 @@ -18,8 +18,12 @@ */ package org.apache.samza.application; +import org.apache.samza.annotation.InterfaceStability; + + /** * The interface to implement user applications defining a low-level task as the main stream processing logic. */ +@InterfaceStability.Evolving public interface TaskApplication extends ApplicationBase { } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java b/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java deleted file mode 100644 index a62eb66a0e..0000000000 --- a/samza-api/src/main/java/org/apache/samza/application/internal/StreamAppDescriptorImpl.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import java.lang.reflect.Constructor; -import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptor; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.Config; -import org.apache.samza.config.ConfigException; -import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.serializers.Serde; -import org.apache.samza.table.Table; - - -/** - * This class implements interface {@link StreamAppDescriptor}. In addition to the common objects for an application - * defined in {@link AppDescriptorImpl}, this class also includes the high-level DAG {@link StreamGraph} object that user will - * use to create the processing logic in DAG. - */ -public class StreamAppDescriptorImpl extends AppDescriptorImpl - implements StreamAppDescriptor { - final StreamGraph graph; - - // this config variable is for unit test in samza-api only. *MUST NOT* be set by the user - private static final String TEST_GRAPH_CLASS_CFG = "app.test.graph.class"; - private static final String DEFAULT_GRAPH_CLASS = "org.apache.samza.operators.StreamGraphSpec"; - - public StreamAppDescriptorImpl(StreamApplication userApp, Config config) { - super(config); - this.graph = createDefaultGraph(config); - userApp.describe(this); - } - - @Override - public void setDefaultSerde(Serde serde) { - this.graph.setDefaultSerde(serde); - } - - @Override - public MessageStream getInputStream(String streamId, Serde serde) { - return this.graph.getInputStream(streamId, serde); - } - - @Override - public MessageStream getInputStream(String streamId) { - return this.graph.getInputStream(streamId); - } - - @Override - public OutputStream getOutputStream(String streamId, Serde serde) { - return this.graph.getOutputStream(streamId, serde); - } - - @Override - public OutputStream getOutputStream(String streamId) { - return this.graph.getOutputStream(streamId); - } - - @Override - public Table> getTable(TableDescriptor tableDesc) { - return this.graph.getTable(tableDesc); - } - - /** - * Get the user-defined high-level DAG {@link StreamGraph} object - * - * @return the {@link StreamGraph} object defined by the user application - */ - public StreamGraph getGraph() { - return graph; - } - - /** - * Helper method to load the implementation class of {@link StreamGraph} interface - * - * @param config the configuration of the application - * @return an object implements {@link StreamGraph} interface - */ - private StreamGraph createDefaultGraph(Config config) { - String graphClass = config.getOrDefault(TEST_GRAPH_CLASS_CFG, DEFAULT_GRAPH_CLASS); - try { - if (StreamGraph.class.isAssignableFrom(Class.forName(graphClass))) { - Constructor constructor = Class.forName(graphClass).getConstructor(Config.class); // *sigh* - return (StreamGraph) constructor.newInstance(config); - } else { - throw new ConfigException(String.format("Incompatible class %s is invalid. Must implement StreamGraph.", graphClass)); - } - } catch (Exception e) { - throw new SamzaException("Cannot instantiate an empty StreamGraph to start user application.", e); - } - } - -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index 7797f9a255..ec664d6c7c 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -39,11 +39,12 @@ /** * A stream of messages that can be transformed into another {@link MessageStream}. *

    - * A {@link MessageStream} corresponding to an input stream can be obtained using {@link StreamGraph#getInputStream}. + * A {@link MessageStream} corresponding to an input stream can be obtained using + * {@link org.apache.samza.application.StreamAppDescriptor#getInputStream}. * * @param the type of messages in this stream */ -@InterfaceStability.Unstable +@InterfaceStability.Evolving public interface MessageStream { /** @@ -217,9 +218,10 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, - * uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default - * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided - * before calling this method, a {@code KVSerde} is used. + * uses the default serde provided via {@link org.apache.samza.application.StreamAppDescriptor#setDefaultSerde}, + * which must be a KVSerde. If the default serde is not a {@link KVSerde}, a runtime exception will be thrown. + * If no default serde has been provided before calling this method, a {@code KVSerde} + * is used. *

    * The number of partitions for this intermediate stream is determined as follows: * If the stream is an eventual input to a {@link #join}, and the number of partitions for the other stream is known, @@ -251,9 +253,10 @@ MessageStream> partitionBy(MapFunction k /** * Same as calling {@link #partitionBy(MapFunction, MapFunction, KVSerde, String)} with a null KVSerde. *

    - * Uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default - * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided - * before calling this method, a {@code KVSerde} is used. + * Uses the default serde provided via {@link org.apache.samza.application.StreamAppDescriptor#setDefaultSerde}, + * which must be a KVSerde. If the default serde is not a {@link KVSerde}, a runtime exception will be thrown. + * If no default serde has been provided before calling this method, a {@code KVSerde} + * is used. * * @param keyExtractor the {@link MapFunction} to extract the message and partition key from the input message * @param valueExtractor the {@link MapFunction} to extract the value from the input message diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java deleted file mode 100644 index f19ef07b82..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators; - -import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.serializers.Serde; -import org.apache.samza.table.Table; - - -/** - * Provides access to {@link MessageStream}s, {@link OutputStream}s, and {@link Table}s used to describe application logic. - */ -@InterfaceStability.Evolving -public interface StreamGraph { - - /** - * Sets the default {@link Serde} to use for (de)serializing messages. - *

    . - * If the default serde is set, it must be set before creating any input or output streams. - *

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

    - * Providing an incompatible message type for the input/output streams that use the default serde will result in - * {@link ClassCastException}s at runtime. - * - * @param serde the default message {@link Serde} to use - */ - void setDefaultSerde(Serde serde); - - /** - * Gets the input {@link MessageStream} corresponding to the {@code streamId}. - *

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

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param serde the {@link Serde} to use for deserializing incoming messages - * @param the type of messages in the input {@link MessageStream} - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - MessageStream getInputStream(String streamId, Serde serde); - - /** - * Same as {@link #getInputStream(String, Serde)}, but uses the default {@link Serde} provided via - * {@link #setDefaultSerde(Serde)} for deserializing input messages. - *

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param the type of message in the input {@link MessageStream} - * @return the input {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - MessageStream getInputStream(String streamId); - - /** - * Gets the {@link OutputStream} corresponding to the {@code streamId}. - *

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

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

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param serde the {@link Serde} to use for serializing outgoing messages - * @param the type of messages in the {@link OutputStream} - * @return the output {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - OutputStream getOutputStream(String streamId, Serde serde); - - /** - * Same as {@link #getOutputStream(String, Serde)}, but uses the default {@link Serde} provided via - * {@link #setDefaultSerde(Serde)} for serializing output messages. - *

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

    - * Multiple invocations of this method with the same {@code streamId} will throw an {@link IllegalStateException}. - * - * @param streamId the unique ID for the stream - * @param the type of messages in the {@link OutputStream} - * @return the output {@link MessageStream} - * @throws IllegalStateException when invoked multiple times with the same {@code streamId} - */ - OutputStream getOutputStream(String streamId); - - /** - * Gets the {@link Table} corresponding to the {@link TableDescriptor}. - *

    - * Multiple invocations of this method with the same {@link TableDescriptor} will throw an - * {@link IllegalStateException}. - * - * @param tableDesc the {@link TableDescriptor} - * @param the type of the key - * @param the type of the value - * @return the {@link Table} corresponding to the {@code tableDesc} - * @throws IllegalStateException when invoked multiple times with the same {@link TableDescriptor} - */ - Table> getTable(TableDescriptor tableDesc); -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java index faf9fc5a35..5f9f0d1b0c 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java @@ -27,7 +27,7 @@ *

    Implement {@link #close()} to free resources used during the execution of the function, clean up state etc. * *

    Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the - * {@link org.apache.samza.operators.StreamGraph}. For any two operators A and B in the graph, if operator B consumes results + * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B consumes results * from operator A, then operator B is guaranteed to be closed before operator A. * */ diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index 6651819780..843924330a 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -27,7 +27,7 @@ * A function that can be initialized before execution. * *

    Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the - * {@link org.apache.samza.operators.StreamGraph}. For any two operators A and B in the graph, if operator B consumes results + * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B consumes results * from operator A, then operator A is guaranteed to be initialized before operator B. * */ diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index c52cfe6e8c..a3180fcd32 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -30,6 +30,7 @@ */ @InterfaceStability.Evolving public interface ApplicationRunner { + /** * Deploy and run the Samza jobs to execute {@link org.apache.samza.application.ApplicationBase}. * It is non-blocking so it doesn't wait for the application running. diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java index 9c589d8733..5e5202aec5 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java @@ -20,11 +20,6 @@ import java.lang.reflect.Constructor; import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; @@ -46,14 +41,19 @@ private ApplicationRunners() { * @return the {@link ApplicationRunner} object that will run the {@code userApp} */ public static final ApplicationRunner getApplicationRunner(ApplicationBase userApp, Config config) { - if (userApp instanceof StreamApplication) { - return getRunner(new StreamAppDescriptorImpl((StreamApplication) userApp, config)); - } - if (userApp instanceof TaskApplication) { - return getRunner(new TaskAppDescriptorImpl((TaskApplication) userApp, config)); + AppRunnerConfig runnerConfig = new AppRunnerConfig(config); + try { + Class runnerClass = Class.forName(runnerConfig.getAppRunnerClass()); + if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { + Constructor constructor = runnerClass.getConstructor(ApplicationBase.class, Config.class); // *sigh* + return (ApplicationRunner) constructor.newInstance(userApp, config); + } + } catch (Exception e) { + throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", + runnerConfig.getAppRunnerClass()), e); } - throw new IllegalArgumentException(String.format("User application instance has to be either StreamApplicationFactory or TaskApplicationFactory. " - + "Invalid userApp class %s.", userApp.getClass().getName())); + throw new ConfigException(String.format("Class %s does not extend ApplicationRunner properly", + runnerConfig.getAppRunnerClass())); } static class AppRunnerConfig { @@ -72,26 +72,4 @@ String getAppRunnerClass() { } - /** - * Static method to get the {@link ApplicationRunner} - * - * @param appSpec configuration passed in to initialize the Samza processes - * @return the configure-driven {@link ApplicationRunner} to run the user-defined stream applications - */ - static ApplicationRunner getRunner(AppDescriptorImpl appSpec) { - AppRunnerConfig appRunnerCfg = new AppRunnerConfig(appSpec.getConfig()); - try { - Class runnerClass = Class.forName(appRunnerCfg.getAppRunnerClass()); - if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { - Constructor constructor = runnerClass.getConstructor(AppDescriptorImpl.class); // *sigh* - return (ApplicationRunner) constructor.newInstance(appSpec); - } - } catch (Exception e) { - throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", - appRunnerCfg.getAppRunnerClass()), e); - } - throw new ConfigException(String.format( - "Class %s does not extend ApplicationRunner properly", - appRunnerCfg.getAppRunnerClass())); - } } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java index a255858a5c..82c76ee6f3 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java @@ -18,10 +18,19 @@ */ package org.apache.samza.runtime; +import org.apache.samza.annotation.InterfaceStability; + + /** * The context for a StreamProcessor. Currently, only defines a method to report the processorId * */ +@InterfaceStability.Evolving public interface ProcessorContext { + /** + * Get the unique processorID for the corresponding StreamProcessor + * + * @return the processorId + */ String getProcessorId(); } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java index 5ce1157859..f4e49edfc9 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListener.java @@ -18,11 +18,17 @@ */ package org.apache.samza.runtime; +import org.apache.samza.annotation.InterfaceStability; + + /** * This interface defines methods that are invoked in different stages of StreamProcessor's lifecycle in local - * process (i.e. as a standalone process, or a container process in YARN NodeManager). User can implement this interface - * to instantiate/release shared objects in the local process. + * process (i.e. as a standalone process, or a container process in YARN NodeManager). + * + *

    + * User can implement this interface to instantiate/release shared objects in the local process. */ +@InterfaceStability.Evolving public interface ProcessorLifecycleListener { /** * User defined initialization before a StreamProcessor is started @@ -36,15 +42,14 @@ default void beforeStart() {} default void afterStart() {} /** - * User defined callback before a StreamProcessor is stopped - * + * User defined callback after a StreamProcessor is stopped successfully */ - default void beforeStop() {} + default void afterStop() {} /** - * User defined callback after a StreamProcessor is stopped + * User defined callback after a StreamProcessor is stopped with failure * - * @param t the error causing the stop of the StreamProcessor. null value of this parameter indicates a successful completion. + * @param t the error causing the stop of the StreamProcessor. */ - default void afterStop(Throwable t) {} + default void afterFailure(Throwable t) {} } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java index cbc49fee08..b52161b7fb 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java @@ -19,12 +19,14 @@ package org.apache.samza.runtime; import java.io.Serializable; +import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; /** * This interface class defines the factory method to create an instance of {@link ProcessorLifecycleListener}. */ +@InterfaceStability.Evolving public interface ProcessorLifecycleListenerFactory extends Serializable { /** * Create an instance of {@link ProcessorLifecycleListener} for the StreamProcessor diff --git a/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java b/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java index ebd64b2fb0..1879ce8268 100644 --- a/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java +++ b/samza-api/src/main/java/org/apache/samza/task/AsyncStreamTaskFactory.java @@ -19,9 +19,14 @@ package org.apache.samza.task; +import org.apache.samza.annotation.InterfaceStability; + + /** * Build {@link AsyncStreamTask} instances. - * Implementations should return a new instance for each {@link #createInstance()} invocation. + *

    + * Implementations should return a new instance of {@link AsyncStreamTask} for each {@link #createInstance()} invocation. */ +@InterfaceStability.Stable public interface AsyncStreamTaskFactory extends TaskFactory { } diff --git a/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java b/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java index 02087eee09..8588a0d641 100644 --- a/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java +++ b/samza-api/src/main/java/org/apache/samza/task/StreamTaskFactory.java @@ -23,7 +23,8 @@ /** * Build {@link StreamTask} instances. - * Implementations should return a new instance for each {@link #createInstance()} invocation. + *

    + * Implementations should return a new instance of {@link StreamTask} for each {@link #createInstance()} invocation. */ @InterfaceStability.Stable public interface StreamTaskFactory extends TaskFactory { 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 49277928dd..dd088def6b 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 @@ -24,6 +24,8 @@ /** * The base interface class for all task factories (i.e. {@link StreamTaskFactory} and {@link AsyncStreamTaskFactory} + * + * @param the type of task instances */ @InterfaceStability.Stable public interface TaskFactory extends Serializable { diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java deleted file mode 100644 index ef90a98556..0000000000 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamAppDescriptorImpl.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import java.util.ArrayList; -import java.util.HashMap; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; -import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; -import org.apache.samza.serializers.Serde; -import org.apache.samza.serializers.StringSerde; -import org.junit.Test; - -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - - -/** - * Unit test for {@link StreamAppDescriptorImpl} - */ -public class TestStreamAppDescriptorImpl { - private Config config = new MapConfig(new HashMap() { - { - this.put("app.test.graph.class", TestStreamGraph.class.getName()); - } - }); - - @Test - public void testConstructor() { - StreamApplication mockApp = mock(StreamApplication.class); - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(mockApp, config); - verify(mockApp, times(1)).describe(appDesc); - assertEquals(config, appDesc.config); - assertTrue(appDesc.graph instanceof TestStreamGraph); - } - - @Test - public void testGetInputStream() { - Serde testSerde = new StringSerde(); - StreamApplication testApp = appDesc -> { - appDesc.getInputStream("myinput1"); - appDesc.getInputStream("myinput2", testSerde); - }; - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); - assertEquals(((TestStreamGraph) appDesc.graph).inputStreams, - new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }); - assertEquals(((TestStreamGraph) appDesc.graph).inputSerdes.get("myinput2"), testSerde); - } - - @Test - public void testGetOutputStream() { - Serde testSerde = new StringSerde(); - StreamApplication testApp = appDesc -> { - appDesc.getOutputStream("myoutput1"); - appDesc.getOutputStream("myoutput2", testSerde); - }; - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); - assertEquals(((TestStreamGraph) appDesc.graph).outputStreams, - new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }); - assertEquals(((TestStreamGraph) appDesc.graph).outputSerdes.get("myoutput2"), testSerde); - } - - @Test - public void testGetTable() { - TableDescriptor mockTd = mock(TableDescriptor.class); - StreamApplication testApp = appDesc -> { - appDesc.getTable(mockTd); - }; - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); - assertEquals(((TestStreamGraph) appDesc.graph).tables, - new ArrayList() { { this.add(mockTd); } }); - } - - @Test - public void testSetDefaultSerde() { - Serde testSerde = new StringSerde(); - StreamApplication testApp = appDesc -> { - appDesc.setDefaultSerde(testSerde); - }; - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); - assertEquals(((TestStreamGraph) appDesc.graph).defaultSerde, testSerde); - } - - @Test - public void testContextManager() { - ContextManager cntxMan = mock(ContextManager.class); - StreamApplication testApp = appDesc -> { - appDesc.withContextManager(cntxMan); - }; - StreamAppDescriptorImpl appSpec = new StreamAppDescriptorImpl(testApp, config); - assertEquals(appSpec.getContextManager(), cntxMan); - } - - @Test - public void testProcessorLifecycleListenerFactory() { - ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class); - StreamApplication testApp = appSpec -> { - appSpec.withProcessorLifecycleListenerFactory(mockFactory); - }; - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, config); - assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java b/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java deleted file mode 100644 index 1925062812..0000000000 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestStreamGraph.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application.internal; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.samza.config.Config; -import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraph; -import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.serializers.Serde; -import org.apache.samza.table.Table; - -import static org.mockito.Mockito.*; - - -/** - * Test implementation of {@link StreamGraph} used only in unit test - */ -public class TestStreamGraph implements StreamGraph { - final Config config; - final List inputStreams = new ArrayList<>(); - final List outputStreams = new ArrayList<>(); - final List tables = new ArrayList<>(); - final Map inputSerdes = new HashMap<>(); - final Map outputSerdes = new HashMap<>(); - Serde defaultSerde; - - public TestStreamGraph(Config config) { - this.config = config; - } - - @Override - public void setDefaultSerde(Serde serde) { - this.defaultSerde = serde; - } - - @Override - public MessageStream getInputStream(String streamId, Serde serde) { - this.inputStreams.add(streamId); - this.inputSerdes.put(streamId, serde); - return mock(MessageStream.class); - } - - @Override - public MessageStream getInputStream(String streamId) { - this.inputStreams.add(streamId); - return mock(MessageStream.class); - } - - @Override - public OutputStream getOutputStream(String streamId, Serde serde) { - this.outputStreams.add(streamId); - this.outputSerdes.put(streamId, serde); - return mock(OutputStream.class); - } - - @Override - public OutputStream getOutputStream(String streamId) { - this.outputStreams.add(streamId); - return mock(OutputStream.class); - } - - @Override - public Table> getTable(TableDescriptor tableDesc) { - this.tables.add(tableDesc); - return mock(Table.class); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java index 999e51fa33..edfef7bdc6 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java @@ -20,7 +20,8 @@ import java.time.Duration; import java.util.Map; -import org.apache.samza.application.internal.AppDescriptorImpl; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.config.Config; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; @@ -29,9 +30,12 @@ * Test class for {@link org.apache.samza.runtime.ApplicationRunners} unit test */ public class TestApplicationRunner implements ApplicationRunner { + private final ApplicationBase userApp; + private final Config config; - public TestApplicationRunner(AppDescriptorImpl appDesc) { - + public TestApplicationRunner(ApplicationBase userApp, Config config) { + this.userApp = userApp; + this.config = config; } @Override diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java index 772a5b23f5..b74d94dfe3 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java @@ -21,13 +21,12 @@ import java.util.HashMap; import java.util.Map; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.TestStreamGraph; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; /** @@ -39,7 +38,6 @@ public class TestApplicationRunners { public void testGetAppRunner() { Map configMap = new HashMap<>(); configMap.put("app.runner.class", TestApplicationRunner.class.getName()); - configMap.put("app.test.graph.class", TestStreamGraph.class.getName()); Config config = new MapConfig(configMap); StreamApplication app = mock(StreamApplication.class); ApplicationRunner appRunner = ApplicationRunners.getApplicationRunner(app, config); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java index a9eefeb523..3b4f1ec3ff 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java @@ -173,7 +173,7 @@ public void init() { @Override public EventHubClient getEventHubClient() { if (!initiated) { - Assert.fail("Should have called bootstrap() on EventHubClient before getEventHubClient()"); + Assert.fail("Should have called init() on EventHubClient before getEventHubClient()"); } return mockEventHubClient; } @@ -181,7 +181,7 @@ public EventHubClient getEventHubClient() { @Override public void close(long timeoutMS) { if (!initiated) { - Assert.fail("Should have called bootstrap() on EventHubClient before close()"); + Assert.fail("Should have called init() on EventHubClient before close()"); } initiated = false; } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/AppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java similarity index 65% rename from samza-api/src/main/java/org/apache/samza/application/internal/AppDescriptorImpl.java rename to samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java index cb7a1252d9..2e62030384 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/AppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java @@ -16,12 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application.internal; +package org.apache.samza.application; -import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; import org.apache.samza.operators.ContextManager; import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; @@ -29,14 +26,20 @@ /** - * This is the base class that implements interface {@link ApplicationDescriptor}. This base class contains the common objects - * that are used by both high-level and low-level API applications, such as {@link Config}, {@link ContextManager}, and - * {@link ProcessorLifecycleListener}. + * This is the base class that implements interface {@link ApplicationDescriptor}. + *

    + * This base class contains the common objects that are used by both high-level and low-level API applications, such as + * {@link Config}, {@link ContextManager}, and {@link ProcessorLifecycleListenerFactory}. + * + * @param the type of user application + * @param the type of {@link ApplicationDescriptor} interface this implements. It has to be either + * {@link org.apache.samza.application.StreamAppDescriptor} or {@link org.apache.samza.application.TaskAppDescriptor} */ public abstract class AppDescriptorImpl> implements ApplicationDescriptor { final Config config; + final Class appClass; // Default to no-op functions in ContextManager // TODO: this should be replaced by shared context factory defined in SAMZA-1714 @@ -53,44 +56,9 @@ public void close() { // Default to no-op ProcessorLifecycleListenerFactory ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { }; - AppDescriptorImpl(Config config) { + AppDescriptorImpl(ApplicationBase userApp, Config config) { this.config = config; - } - - static class AppConfig extends MapConfig { - - static final String APP_NAME = "app.name"; - static final String APP_ID = "app.id"; - - static final String JOB_NAME = "job.name"; - static final String JOB_ID = "job.id"; - - AppConfig(Config config) { - super(config); - } - - String getAppName() { - return get(APP_NAME, get(JOB_NAME)); - } - - String getAppId() { - return get(APP_ID, get(JOB_ID, "1")); - } - - /** - * Returns full application id - * - * @return full app id - */ - String getGlobalAppId() { - return String.format("app-%s-%s", getAppName(), getAppId()); - } - - } - - @Override - public String getGlobalAppId() { - return new AppConfig(config).getGlobalAppId(); + this.appClass = userApp.getClass(); } @Override @@ -110,6 +78,13 @@ public S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory return (S) this; } + /** + * Get the user application class + */ + public Class getAppClass() { + return appClass; + } + /** * Get the user-implemented {@link ContextManager} object associated with this application * @@ -120,9 +95,9 @@ public ContextManager getContextManager() { } /** - * Get the user-implemented {@link ProcessorLifecycleListener} object associated with this application + * Get the user-implemented {@link ProcessorLifecycleListenerFactory} object associated with this application * - * @return the {@link ProcessorLifecycleListener} object + * @return the {@link ProcessorLifecycleListenerFactory} object */ public ProcessorLifecycleListenerFactory getProcessorLifecycleListenerFactory() { return listenerFactory; diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java new file mode 100644 index 0000000000..3ce8c5d5e0 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.application; + +import org.apache.samza.config.Config; + + +/** + * Util class to help creating {@link AppDescriptorImpl} instance from {@link ApplicationBase} and {@link Config} + */ +public class ApplicationDescriptors { + private ApplicationDescriptors() { + + } + + public static AppDescriptorImpl getAppDescriptor(ApplicationBase userApp, Config config) { + if (userApp instanceof StreamApplication) { + return new StreamAppDescriptorImpl((StreamApplication) userApp, config); + } + if (userApp instanceof TaskApplication) { + return new TaskAppDescriptorImpl((TaskApplication) userApp, config); + } + throw new IllegalArgumentException(String.format("User application class %s is not supported. Only StreamApplication " + + "and TaskApplication are supported.", userApp.getClass().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 new file mode 100644 index 0000000000..f0fb866a40 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/application/LegacyTaskApplication.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.application; + +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.task.TaskFactoryUtil; + +import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; + + +/** + * Default {@link TaskApplication} for legacy applications w/ only task.class implemented + */ +public final class LegacyTaskApplication implements TaskApplication { + private final Config config; + + public LegacyTaskApplication(Config config) { + this.config = validate(config); + } + + private Config validate(Config config) { + new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( + () -> { + throw new ConfigException("No task class defined in the configuration."); + })); + return config; + } + + @Override + public void describe(TaskAppDescriptor appDesc) { + appDesc.setTaskFactory(TaskFactoryUtil.createTaskFactory(config)); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java similarity index 85% rename from samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java rename to samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java index d32380f031..c29b5ee722 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java @@ -16,21 +16,30 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.operators; +package org.apache.samza.application; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; - import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; +import org.apache.samza.operators.BaseTableDescriptor; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.MessageStreamImpl; +import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.operators.TableImpl; import org.apache.samza.operators.spec.InputOperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec.OpCode; +import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; @@ -42,17 +51,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; - /** * This class defines: - * 1) an implementation of {@link StreamGraph} that provides APIs for accessing {@link MessageStream}s to be used to - * create the DAG of transforms. + * 1) an implementation of {@link StreamAppDescriptor} that provides APIs to access {@link MessageStream}, {@link OutputStream}, + * and {@link Table} to create the DAG of transforms. * 2) a builder that creates a serializable {@link OperatorSpecGraph} from user-defined DAG */ -public class StreamGraphSpec implements StreamGraph { - private static final Logger LOGGER = LoggerFactory.getLogger(StreamGraphSpec.class); +public class StreamAppDescriptorImpl extends AppDescriptorImpl + implements StreamAppDescriptor { + private static final Logger LOGGER = LoggerFactory.getLogger(StreamAppDescriptorImpl.class); private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); // We use a LHM for deterministic order in initializing and closing operators. @@ -60,19 +67,19 @@ public class StreamGraphSpec implements StreamGraph { private final Map outputStreams = new LinkedHashMap<>(); private final Set broadcastStreams = new HashSet<>(); private final Map tables = new LinkedHashMap<>(); - private final Config config; /** * The 0-based position of the next operator in the graph. * Part of the unique ID for each OperatorSpec in the graph. - * Should only accessed and incremented via {@link #getNextOpId(OpCode, String)}. + * Should only accessed and incremented via {@link #getNextOpId(OperatorSpec.OpCode, String)}. */ private int nextOpNum = 0; private final Set operatorIds = new HashSet<>(); private Serde defaultSerde = new KVSerde(new NoOpSerde(), new NoOpSerde()); - public StreamGraphSpec(Config config) { - this.config = config; + public StreamAppDescriptorImpl(StreamApplication userApp, Config config) { + super(userApp, config); + userApp.describe(this); } @Override @@ -104,7 +111,7 @@ public MessageStream getInputStream(String streamId, Serde serde) { boolean isKeyed = serde instanceof KVSerde; InputOperatorSpec inputOperatorSpec = OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), - isKeyed, this.getNextOpId(OpCode.INPUT, null)); + isKeyed, this.getNextOpId(OperatorSpec.OpCode.INPUT, null)); inputOperators.put(streamId, inputOperatorSpec); return new MessageStreamImpl<>(this, inputOperators.get(streamId)); } @@ -155,18 +162,18 @@ public Table> getTable(TableDescriptor tableDesc) { } public OperatorSpecGraph getOperatorSpecGraph() { - return new OperatorSpecGraph(this); + return OperatorSpecGraph.getInstance(this); } /** * Gets the unique ID for the next operator in the graph. The ID is of the following format: * jobName-jobId-opCode-(userDefinedId|nextOpNum); * - * @param opCode the {@link OpCode} of the next operator + * @param opCode the {@link OperatorSpec.OpCode} of the next operator * @param userDefinedId the optional user-provided name of the next operator or null * @return the unique ID for the next operator in the graph */ - public String getNextOpId(OpCode opCode, String userDefinedId) { + public String getNextOpId(OperatorSpec.OpCode opCode, String userDefinedId) { if (StringUtils.isNotBlank(userDefinedId) && !ID_PATTERN.matcher(userDefinedId).matches()) { throw new SamzaException("Operator ID must not contain spaces or special characters: " + userDefinedId); } @@ -188,15 +195,15 @@ public String getNextOpId(OpCode opCode, String userDefinedId) { * Gets the unique ID for the next operator in the graph. The ID is of the following format: * jobName-jobId-opCode-nextOpNum; * - * @param opCode the {@link OpCode} of the next operator + * @param opCode the {@link OperatorSpec.OpCode} of the next operator * @return the unique ID for the next operator in the graph */ - public String getNextOpId(OpCode opCode) { + public String getNextOpId(OperatorSpec.OpCode opCode) { return getNextOpId(opCode, null); } /** - * See {@link StreamGraphSpec#getIntermediateStream(String, Serde, boolean)}. + * See {@link StreamAppDescriptorImpl#getIntermediateStream(String, Serde, boolean)}. * * @param type of messages in the intermediate stream * @param streamId the id of the stream to be created @@ -220,7 +227,7 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI * @return the intermediate {@link MessageStreamImpl} */ @VisibleForTesting - IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serde serde, boolean isBroadcast) { + public IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serde serde, boolean isBroadcast) { Preconditions.checkState(!inputOperators.containsKey(streamId) && !outputStreams.containsKey(streamId), "getIntermediateStream must not be called multiple times with the same streamId: " + streamId); @@ -234,25 +241,25 @@ IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serd KV kvSerdes = getKVSerdes(streamId, serde); InputOperatorSpec inputOperatorSpec = OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), - isKeyed, this.getNextOpId(OpCode.INPUT, null)); + isKeyed, this.getNextOpId(OperatorSpec.OpCode.INPUT, null)); inputOperators.put(streamId, inputOperatorSpec); outputStreams.put(streamId, new OutputStreamImpl(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamId), outputStreams.get(streamId)); } - Map getInputOperators() { + public Map getInputOperators() { return Collections.unmodifiableMap(inputOperators); } - Map getOutputStreams() { + public Map getOutputStreams() { return Collections.unmodifiableMap(outputStreams); } - Set getBroadcastStreams() { + public Set getBroadcastStreams() { return Collections.unmodifiableSet(broadcastStreams); } - Map getTables() { + public Map getTables() { return Collections.unmodifiableMap(tables); } @@ -282,4 +289,5 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } + } diff --git a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java similarity index 68% rename from samza-api/src/main/java/org/apache/samza/application/internal/TaskAppDescriptorImpl.java rename to samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java index 441cc6d9ce..e258c19151 100644 --- a/samza-api/src/main/java/org/apache/samza/application/internal/TaskAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java @@ -16,23 +16,22 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application.internal; +package org.apache.samza.application; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TaskAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.task.TaskFactory; /** - * This class implements interface {@link TaskAppDescriptor}. In addition to the common objects for an application - * defined in {@link AppDescriptorImpl}, this class also includes the low-level {@link TaskFactory} object that creates - * user-defined task instances, the lists of input/output streams, and the list of {@link TableDescriptor}s used in - * the application. + * This class implements interface {@link TaskAppDescriptor}. + *

    + * In addition to the common objects for an application defined in {@link AppDescriptorImpl}, this class also includes + * the low-level {@link TaskFactory} object that creates user-defined task instances, the lists of input/output streams, + * and the list of {@link TableDescriptor}s used in the application. */ public class TaskAppDescriptorImpl extends AppDescriptorImpl implements TaskAppDescriptor { @@ -41,10 +40,11 @@ public class TaskAppDescriptorImpl extends AppDescriptorImpl inputStreams = new ArrayList<>(); final List outputStreams = new ArrayList<>(); + final List broadcastStreams = new ArrayList<>(); final List tables = new ArrayList<>(); public TaskAppDescriptorImpl(TaskApplication userApp, Config config) { - super(config); + super(userApp, config); userApp.describe(this); } @@ -54,18 +54,23 @@ public void setTaskFactory(TaskFactory factory) { } @Override - public void addInputStreams(List inputStreams) { - this.inputStreams.addAll(inputStreams); + public void addInputStream(String inputStream) { + this.inputStreams.add(inputStream); } @Override - public void addOutputStreams(List outputStreams) { - this.outputStreams.addAll(outputStreams); + public void addBroadcastStream(String broadcastStream) { + this.broadcastStreams.add(broadcastStream); } @Override - public void addTables(List tables) { - this.tables.addAll(tables); + public void addOutputStream(String outputStream) { + this.outputStreams.add(outputStream); + } + + @Override + public void addTable(TableDescriptor table) { + this.tables.add(table); } /** @@ -79,6 +84,8 @@ public TaskFactory getTaskFactory() { /** * Get the input streams to this application * + * TODO: need to change to InputStreamDescriptors after SAMZA-1804 + * * @return the list of input streamIds */ public List getInputStreams() { @@ -88,6 +95,8 @@ public List getInputStreams() { /** * Get the output streams to this application * + * TODO: need to change to OutputStreamDescriptors after SAMZA-1804 + * * @return the list of output streamIds */ public List getOutputStreams() { @@ -102,4 +111,8 @@ public List getOutputStreams() { public List getTables() { return Collections.unmodifiableList(this.tables); } + + public List getBroadcastStreams() { + return Collections.unmodifiableList(this.broadcastStreams); + } } diff --git a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java index 3ef6dbe680..e9f3558a93 100644 --- a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java +++ b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java @@ -19,29 +19,24 @@ package org.apache.samza.container; /** - * A Listener for {@link SamzaContainer} lifecycle events. + * A Listener for {@link org.apache.samza.container.SamzaContainer} lifecycle events. */ public interface SamzaContainerListener { /** - * Callback before the {@link SamzaContainer} is started + * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started */ void beforeStart(); /** - * Method invoked when the {@link SamzaContainer} has successfully transitioned to + * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has successfully transitioned to * the {@link org.apache.samza.SamzaContainerStatus#STARTED} state and is about to start the * {@link org.apache.samza.container.RunLoop} */ void onContainerStart(); /** - * Callback before the {@link SamzaContainer} is stopped - */ - void beforeStop(); - - /** - * Method invoked when the {@link SamzaContainer} has successfully transitioned to + * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has successfully transitioned to * {@link org.apache.samza.SamzaContainerStatus#STOPPED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    @@ -51,7 +46,7 @@ public interface SamzaContainerListener { void onContainerStop(); /** - * Method invoked when the {@link SamzaContainer} has transitioned to + * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has transitioned to * {@link org.apache.samza.SamzaContainerStatus#FAILED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    @@ -59,5 +54,4 @@ public interface SamzaContainerListener { * @param t Throwable that caused the container failure. */ void onContainerFailed(Throwable t); - } diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index ef52e90e9c..230f46fdad 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -48,7 +48,7 @@ /** - * The ExecutionPlanner creates the physical execution graph for the StreamGraph, and + * The ExecutionPlanner creates the physical execution graph for the {@link OperatorSpecGraph}, and * the intermediate topics needed for the execution. */ public class ExecutionPlanner { @@ -92,7 +92,7 @@ private void validateConfig() { } /** - * Create the physical graph from StreamGraph + * Create the physical graph from {@link OperatorSpecGraph} */ /* package private */ JobGraph createJobGraph(OperatorSpecGraph specGraph) { JobGraph jobGraph = new JobGraph(config, specGraph); @@ -189,7 +189,7 @@ private void validateConfig() { jobGraph.getSpecGraph().getInputOperators().entrySet().forEach(entry -> { StreamConfig streamConfig = new StreamConfig(config); StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(getStreamSpec(entry.getKey(), streamConfig)); - // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge + // Traverses the graph to find and update mappings for all Joins reachable from this input StreamEdge findReachableJoins(entry.getValue(), streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, visited); }); diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index 6922c763e6..01e62821f9 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -19,10 +19,12 @@ package org.apache.samza.operators; +import com.google.common.annotations.VisibleForTesting; import java.time.Duration; import java.util.Collection; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; @@ -53,7 +55,7 @@ /** * The {@link MessageStream} implementation that lets users describe their logical DAG. - * Users can obtain an instance by calling {@link StreamGraph#getInputStream}. + * Users can obtain an instance by calling {@link StreamAppDescriptorImpl#getInputStream}. *

    * Each {@link MessageStreamImpl} is associated with a single {@link OperatorSpec} in the DAG and allows * users to chain further operators on its {@link OperatorSpec}. In other words, a {@link MessageStreamImpl} @@ -63,16 +65,16 @@ */ public class MessageStreamImpl implements MessageStream { /** - * The {@link StreamGraphSpec} that contains this {@link MessageStreamImpl} + * The {@link StreamAppDescriptorImpl} that contains this {@link MessageStreamImpl} */ - private final StreamGraphSpec graph; + private final StreamAppDescriptorImpl graph; /** * The {@link OperatorSpec} associated with this {@link MessageStreamImpl} */ private final OperatorSpec operatorSpec; - public MessageStreamImpl(StreamGraphSpec graph, OperatorSpec operatorSpec) { + public MessageStreamImpl(StreamAppDescriptorImpl graph, OperatorSpec operatorSpec) { this.graph = graph; this.operatorSpec = operatorSpec; } @@ -210,7 +212,8 @@ public MessageStream broadcast(String userDefinedId) { * Get the {@link OperatorSpec} associated with this {@link MessageStreamImpl}. * @return the {@link OperatorSpec} associated with this {@link MessageStreamImpl}. */ - protected OperatorSpec getOperatorSpec() { + @VisibleForTesting + public OperatorSpec getOperatorSpec() { return this.operatorSpec; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java index b6c3dae298..c37b9e382a 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; @@ -33,9 +34,9 @@ /** - * Defines the serialized format of {@link StreamGraphSpec}. This class encapsulates all getter methods to get the {@link OperatorSpec} - * initialized in the {@link StreamGraphSpec} and constructsthe corresponding serialized instances of {@link OperatorSpec}. - * The {@link StreamGraphSpec} and {@link OperatorSpec} instances included in this class are considered as immutable and read-only. + * Defines the serialized format of {@link StreamAppDescriptorImpl}. This class encapsulates all getter methods to get the {@link OperatorSpec} + * initialized in the {@link StreamAppDescriptorImpl} and constructs the corresponding serialized instances of {@link OperatorSpec}. + * The {@link StreamAppDescriptorImpl} and {@link OperatorSpec} instances included in this class are considered as immutable and read-only. * The instance of {@link OperatorSpecGraph} should only be used in runtime to construct {@link org.apache.samza.task.StreamOperatorTask}. */ public class OperatorSpecGraph implements Serializable { @@ -51,16 +52,22 @@ public class OperatorSpecGraph implements Serializable { private transient final SerializableSerde opSpecGraphSerde = new SerializableSerde<>(); private transient final byte[] serializedOpSpecGraph; - OperatorSpecGraph(StreamGraphSpec graphSpec) { - this.inputOperators = graphSpec.getInputOperators(); - this.outputStreams = graphSpec.getOutputStreams(); - this.broadcastStreams = graphSpec.getBroadcastStreams(); - this.tables = graphSpec.getTables(); + OperatorSpecGraph(Map inputOperators, Map outputStreams, + Set broadcastStreams, Map tables) { + this.inputOperators = inputOperators; + this.outputStreams = outputStreams; + this.broadcastStreams = broadcastStreams; + this.tables = tables; this.allOpSpecs = Collections.unmodifiableSet(this.findAllOperatorSpecs()); this.hasWindowOrJoins = checkWindowOrJoins(); this.serializedOpSpecGraph = opSpecGraphSerde.toBytes(this); } + public static OperatorSpecGraph getInstance(StreamAppDescriptorImpl appDesc) { + return new OperatorSpecGraph(appDesc.getInputOperators(), appDesc.getOutputStreams(), appDesc.getBroadcastStreams(), + appDesc.getTables()); + } + public Map getInputOperators() { return inputOperators; } @@ -78,7 +85,7 @@ public Map getTables() { } /** - * Get all {@link OperatorSpec}s available in this {@link StreamGraphSpec} + * Get all {@link OperatorSpec}s available in this {@link StreamAppDescriptorImpl} * * @return all available {@link OperatorSpec}s */ @@ -87,9 +94,9 @@ public Collection getAllOperatorSpecs() { } /** - * Returns true iff this {@link StreamGraphSpec} contains a join or a window operator + * Returns true iff this {@link StreamAppDescriptorImpl} contains a join or a window operator * - * @return true iff this {@link StreamGraphSpec} contains a join or a window operator + * @return true iff this {@link StreamAppDescriptorImpl} contains a join or a window operator */ public boolean hasWindowOrJoins() { return hasWindowOrJoins; diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index e1e1c5555e..019046a171 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -103,7 +103,7 @@ public final OpCode getOpCode() { } /** - * Get the unique ID of this operator in the {@link org.apache.samza.operators.StreamGraph}. + * Get the unique ID of this operator in the {@link org.apache.samza.application.StreamAppDescriptorImpl}. * @return the unique operator ID */ public final String getOpId() { diff --git a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java index 3bb8713b97..0b06d6d6b9 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java @@ -18,9 +18,9 @@ */ package org.apache.samza.operators.stream; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; @@ -44,9 +44,9 @@ public class IntermediateMessageStreamImpl extends MessageStreamImpl imple private final OutputStreamImpl outputStream; private final boolean isKeyed; - public IntermediateMessageStreamImpl(StreamGraphSpec graph, InputOperatorSpec inputOperatorSpec, + public IntermediateMessageStreamImpl(StreamAppDescriptorImpl appDesc, InputOperatorSpec inputOperatorSpec, OutputStreamImpl outputStream) { - super(graph, (OperatorSpec) inputOperatorSpec); + super(appDesc, (OperatorSpec) inputOperatorSpec); this.outputStream = outputStream; if (inputOperatorSpec.isKeyed() != outputStream.isKeyed()) { LOGGER.error("Input and output streams for intermediate stream {} aren't keyed consistently. Input: {}, Output: {}", 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 b1bf79f7dc..b030f12868 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 @@ -28,6 +28,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -168,8 +169,8 @@ public StreamProcessor(Config config, Map customMetrics } /** - * Same as {@link StreamProcessor(Config, Map, AsyncStreamTaskFactory, ProcessorLifecycleListener)}, except task - * instances are created using the provided {@link StreamTaskFactory}. + * Same as {@link StreamProcessor#StreamProcessor(Config, Map, StreamTaskFactory, ProcessorLifecycleListener)}, + * except task instances are created using the provided {@link StreamTaskFactory}. * @param config - config * @param customMetricsReporters metric Reporter * @param streamTaskFactory task factory to instantiate the Task @@ -207,6 +208,21 @@ public StreamProcessor(Config config, Map customMetrics this.processorId = this.jobCoordinator.getProcessorId(); } + public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, + StreamProcessorListenerSupplier getListenerFn, JobCoordinator jobCoordinator) { + Preconditions.checkNotNull(getListenerFn, "getListenerFn cannot be null."); + this.taskFactory = taskFactory; + this.config = config; + this.taskShutdownMs = new TaskConfigJava(config).getShutdownMs(); + this.customMetricsReporter = customMetricsReporters; + this.jobCoordinator = (jobCoordinator != null) ? jobCoordinator : getJobCoordinator(); + this.jobCoordinatorListener = createJobCoordinatorListener(); + this.jobCoordinator.setListener(jobCoordinatorListener); + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(CONTAINER_THREAD_NAME_FORMAT).setDaemon(true).build(); + this.executorService = Executors.newSingleThreadExecutor(threadFactory); + this.processorId = this.jobCoordinator.getProcessorId(); + this.processorListener = getListenerFn.apply(this); + } /** * Asynchronously starts this {@link StreamProcessor}. *

    @@ -253,7 +269,6 @@ public void start() { * */ public void stop() { - processorListener.beforeStop(); synchronized (lock) { if (state != State.STOPPING && state != State.STOPPED) { state = State.STOPPING; @@ -369,7 +384,10 @@ public void onCoordinatorStop() { executorService.shutdownNow(); state = State.STOPPED; } - processorListener.afterStop(containerException); + if (containerException != null) + processorListener.afterFailure(containerException); + else + processorListener.afterStop(); } @@ -381,7 +399,7 @@ public void onCoordinatorFailure(Throwable throwable) { executorService.shutdownNow(); state = State.STOPPED; } - processorListener.afterStop(throwable); + processorListener.afterFailure(throwable); } }; } @@ -393,6 +411,11 @@ SamzaContainer getContainer() { class ContainerListener implements SamzaContainerListener { + @Override + public void beforeStart() { + // processorListener.beforeStart() is invoked at the StreamProcessorLifecycleListener + } + @Override public void onContainerStart() { LOGGER.warn("Received container start notification for container: {} in stream processor: {}.", container, processorId); @@ -427,17 +450,13 @@ public void onContainerFailed(Throwable t) { jobCoordinator.stop(); } } + } - @Override - public void beforeStop() { - // There is no need to invoked user defined logic before shutting down container here, since the container lifecycle - // inside a StreamProcessor is managed internally. User defined beforeStop() method is called in StreamProcessor.stop() - } + /** + * Supplier interface to allow creation of {@link ProcessorLifecycleListener} for a {@link StreamProcessor} + */ + public interface StreamProcessorListenerSupplier extends Function { - @Override - public void beforeStart() { - // There is no need to invoked user defined logic before starting container here, since the container lifecycle - // inside a StreamProcessor is managed internally. User defined beforeStart() method is called in StreamProcessor.start() - } + ProcessorLifecycleListener apply(StreamProcessor sp); } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 496e7cb39f..870b904bea 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -20,28 +20,34 @@ import java.io.File; import java.io.PrintWriter; -import java.time.Duration; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.ApplicationDescriptor; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ApplicationConfig.ApplicationMode; import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.ShellCommandConfig; import org.apache.samza.config.StreamConfig; import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.ExecutionPlanner; import org.apache.samza.execution.StreamManager; -import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.table.TableConfigGenerator; +import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,14 +59,12 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { private static final Logger log = LoggerFactory.getLogger(AbstractApplicationRunner.class); protected final AppDescriptorImpl appDesc; - protected final AppRuntimeExecutable appExecutable; protected final Config config; protected final Map metricsReporters = new HashMap<>(); - AbstractApplicationRunner(AppDescriptorImpl appDesc) { - this.appDesc = appDesc; + AbstractApplicationRunner(ApplicationBase userApp, Config config) { + this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); this.config = appDesc.getConfig(); - this.appExecutable = getAppRuntimeExecutable(appDesc); } @Override @@ -68,62 +72,27 @@ public final void addMetricsReporters(Map metricsReport this.metricsReporters.putAll(metricsReporters); } - @Override - public final void run() { - appExecutable.run(); - } - - @Override - public final ApplicationStatus status() { - return appExecutable.status(); - } - - @Override - public final void kill() { - appExecutable.kill(); - } - - @Override - public final void waitForFinish() { - appExecutable.waitForFinish(Duration.ofSeconds(0)); - } - - @Override - public final boolean waitForFinish(Duration timeout) { - return appExecutable.waitForFinish(timeout); - } - - interface AppRuntimeExecutable { - - void run(); - - void kill(); - - ApplicationStatus status(); - - /** - * Waits for {@code timeout} duration for the application to finish. - * - * @param timeout time to wait for the application to finish - * @return true - application finished before timeout - * false - otherwise - */ - boolean waitForFinish(Duration timeout); - - } - - abstract AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appSpec); - - abstract AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appSpec); - StreamManager buildAndStartStreamManager() { StreamManager streamManager = new StreamManager(config); streamManager.start(); return streamManager; } - ExecutionPlan getExecutionPlan(OperatorSpecGraph graphSpec, StreamManager streamManager) throws Exception { - return getExecutionPlan(graphSpec, null, streamManager); + abstract List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc); + + List createJobConfigs() { + if (appDesc instanceof TaskAppDescriptorImpl) { + // low-level task application only needs a simple single job configuration + return Collections.singletonList(getTaskJobConfig((TaskAppDescriptorImpl) appDesc)); + } else if (appDesc instanceof StreamAppDescriptorImpl) { + return getJobConfigsFromPlan((StreamAppDescriptorImpl) appDesc); + } + + throw new IllegalArgumentException("ApplicationDescriptor class " + appDesc.getClass().getName() + " is not supported"); + } + + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, StreamManager streamManager) throws Exception { + return getExecutionPlan(specGraph, null, streamManager); } /* package private */ @@ -141,6 +110,7 @@ ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, Stream ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) ? ApplicationMode.BATCH : ApplicationMode.STREAM; cfg.put(ApplicationConfig.APP_MODE, mode.name()); + validateAppClassCfg(cfg, appDesc.getAppClass()); // create the physical execution plan ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager); @@ -168,15 +138,26 @@ final void writePlanJsonFile(String planJson) { } } - private AppRuntimeExecutable getAppRuntimeExecutable(ApplicationDescriptor appSpec) { - if (appSpec instanceof StreamAppDescriptorImpl) { - return getStreamAppRuntimeExecutable((StreamAppDescriptorImpl) appSpec); - } - if (appSpec instanceof TaskAppDescriptorImpl) { - return getTaskAppRuntimeExecutable((TaskAppDescriptorImpl) appSpec); + // helper method to generate a single node job configuration for low level task applications + private JobConfig getTaskJobConfig(TaskAppDescriptorImpl taskAppDesc) { + Map cfg = new HashMap<>(taskAppDesc.getConfig()); + //TODO: add stream and system descriptor to configuration conversion here when SAMZA-1804 is fixed. + // adding table configuration + List tableSpecs = taskAppDesc.getTables().stream() + .map(td -> ((BaseTableDescriptor) td).getTableSpec()) + .collect(Collectors.toList()); + cfg.putAll(TableConfigGenerator.generateConfigsForTableSpecs(tableSpecs)); + validateAppClassCfg(cfg, taskAppDesc.getAppClass()); + return new JobConfig(new MapConfig(cfg)); + } + + private void validateAppClassCfg(Map cfg, Class appClass) { + if (cfg.get(ApplicationConfig.APP_CLASS) != null && !cfg.get(ApplicationConfig.APP_CLASS).isEmpty()) { + // app.class is already set + return; } - throw new IllegalArgumentException(String.format("The specified application %s is not valid. " - + "Only StreamApplicationSpec and TaskApplicationSpec are supported.", appSpec.getClass().getName())); + // adding app.class in the configuration + cfg.put(ApplicationConfig.APP_CLASS, appClass.getCanonicalName()); } } diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java similarity index 82% rename from samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java rename to samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java index 63b473d0ae..8dd903ace0 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationClassUtils.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java @@ -16,15 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application; +package org.apache.samza.runtime; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.LegacyTaskApplication; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.TaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; -import org.apache.samza.config.TaskConfig; -import org.apache.samza.task.TaskFactoryUtil; - -import static org.apache.samza.util.ScalaJavaUtil.*; /** @@ -53,11 +53,7 @@ public static ApplicationBase fromConfig(Config config) { } } // no app.class defined. It has to be a legacy application with task.class configuration - new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( - () -> { - throw new ConfigException("No task class defined in the configuration."); - })); - return (TaskApplication) (appSpec) -> appSpec.setTaskFactory(TaskFactoryUtil.createTaskFactory(config)); + return new LegacyTaskApplication(config); } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 974e59507f..3e14412a46 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -21,15 +21,14 @@ import joptsimple.OptionSet; import joptsimple.OptionSpec; -import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.config.Config; import org.apache.samza.util.CommandLine; import org.apache.samza.util.Util; /** - * This class contains the main() method used by start-app.sh. - * It creates the {@link ApplicationRunner} based on the config, and then start the application. + * This class contains the main() method used by run-app.sh. + * It creates the {@link ApplicationRunner} based on the config, and then run the application. */ public class ApplicationRunnerMain { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java index 1602681645..1fd60fc83a 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerOperation.java @@ -20,7 +20,7 @@ package org.apache.samza.runtime; /** - * Operation to perform in the {@link RemoteApplicationRunner} + * Operation to perform in the {@link ApplicationRunnerMain} */ public enum ApplicationRunnerOperation { RUN("run"), KILL("kill"), STATUS("status"); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index a79f950a2f..510594fb86 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -31,20 +31,20 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.coordinator.CoordinationUtils; import org.apache.samza.coordinator.DistributedLockWithState; import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.system.StreamSpec; import org.apache.samza.task.TaskFactory; @@ -68,17 +68,9 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private ApplicationStatus appStatus = ApplicationStatus.New; - public LocalApplicationRunner(AppDescriptorImpl appDesc) { - super(appDesc); - this.uid = UUID.randomUUID().toString(); - } - private final class LocalStreamProcessorLifecycleListener implements ProcessorLifecycleListener { - private StreamProcessor processor; - private ProcessorLifecycleListener processorLifecycleListener; - - private LocalStreamProcessorLifecycleListener() { - } + private final StreamProcessor processor; + private final ProcessorLifecycleListener processorLifecycleListener; @Override public void beforeStart() { @@ -94,30 +86,35 @@ public void afterStart() { } @Override - public void beforeStop() { - processorLifecycleListener.beforeStop(); + public void afterStop() { + processors.remove(processor); + //processor = null; + + processorLifecycleListener.afterStop(); + if (processors.isEmpty()) { + // successful shutdown + shutdownAndNotify(); + } } @Override - public void afterStop(Throwable t) { + public void afterFailure(Throwable t) { processors.remove(processor); - processor = null; - - processorLifecycleListener.afterStop(t); - if (t != null) { - // the processor stopped with failure - if (failure.compareAndSet(null, t)) { - // shutdown the other processors - processors.forEach(StreamProcessor::stop); - } + //processor = null; + + processorLifecycleListener.afterFailure(t); + // the processor stopped with failure + if (failure.compareAndSet(null, t)) { + // shutdown the other processors + processors.forEach(StreamProcessor::stop); } + if (processors.isEmpty()) { - // successful shutdown shutdownAndNotify(); } } - void setProcessor(StreamProcessor processor) { + LocalStreamProcessorLifecycleListener(StreamProcessor processor) { this.processor = processor; this.processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory(). createInstance(processor.getProcessorContext(), processor.getConfig()); @@ -137,115 +134,55 @@ private void shutdownAndNotify() { shutdownLatch.countDown(); } - } - class StreamAppExecutable implements AppRuntimeExecutable { - private final StreamAppDescriptorImpl appDesc; + } - private StreamAppExecutable(StreamAppDescriptorImpl appDesc) { - this.appDesc = appDesc; - } + public LocalApplicationRunner(ApplicationBase userApp, Config config) { + super(userApp, config); + this.uid = UUID.randomUUID().toString(); + } - @Override - public void run() { - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) appDesc.getGraph()).getOperatorSpecGraph(), streamManager); - - String executionPlanJson = plan.getPlanAsJson(); - writePlanJsonFile(executionPlanJson); - LOG.info("Execution Plan: \n" + executionPlanJson); - - // 2. create the necessary streams - // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 - String planId = String.valueOf(executionPlanJson.hashCode()); - createStreams(planId, plan.getIntermediateStreams(), streamManager); - - // 3. create the StreamProcessors - if (plan.getJobConfigs().isEmpty()) { - throw new SamzaException("No jobs to start."); - } - plan.getJobConfigs().forEach(jobConfig -> { - LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); - LocalStreamProcessorLifecycleListener listener = new LocalStreamProcessorLifecycleListener(); - StreamProcessor processor = createStreamProcessor(jobConfig, ((StreamGraphSpec) appDesc.getGraph()).getOperatorSpecGraph(), - appDesc.getContextManager(), listener); - listener.setProcessor(processor); - processors.add(processor); - }); - numProcessorsToStart.set(processors.size()); - - // 4. start the StreamProcessors - processors.forEach(StreamProcessor::start); - } catch (Throwable throwable) { - appStatus = ApplicationStatus.unsuccessfulFinish(throwable); - shutdownLatch.countDown(); - throw new SamzaException(String.format("Failed to start application: %s.", appDesc), throwable); - } finally { - if (streamManager != null) { - streamManager.stop(); - } + @Override + public void run() { + try { + List jobConfigs = createJobConfigs(); + // 3. create the StreamProcessors + if (jobConfigs.isEmpty()) { + throw new SamzaException("No jobs to run."); } + jobConfigs.forEach(jobConfig -> { + LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); + StreamProcessor processor = createStreamProcessor(jobConfig, appDesc, sp -> new LocalStreamProcessorLifecycleListener(sp)); + processors.add(processor); + }); + numProcessorsToStart.set(processors.size()); + + // 4. start the StreamProcessors + processors.forEach(StreamProcessor::start); + } catch (Throwable throwable) { + appStatus = ApplicationStatus.unsuccessfulFinish(throwable); + shutdownLatch.countDown(); + throw new SamzaException("Failed to start application.", throwable); } - - @Override - public void kill() { - processors.forEach(StreamProcessor::stop); - } - - @Override - public ApplicationStatus status() { - return appStatus; - } - - @Override - public boolean waitForFinish(Duration timeout) { - return LocalApplicationRunner.this.localWaitForFinish(timeout); - } - } - class TaskAppExecutable implements AppRuntimeExecutable { - private final TaskAppDescriptorImpl appDesc; - private StreamProcessor sp; - - private TaskAppExecutable(TaskAppDescriptorImpl appDesc) { - this.appDesc = appDesc; - } - - @Override - public void run() { - LOG.info("LocalApplicationRunner will start task " + appDesc.getGlobalAppId()); - LocalStreamProcessorLifecycleListener listener = new LocalStreamProcessorLifecycleListener(); - - sp = createStreamProcessor(config, appDesc.getTaskFactory(), listener); - - numProcessorsToStart.set(1); - listener.setProcessor(sp); - sp.start(); - } - - @Override - public void kill() { - sp.stop(); - } - - @Override - public ApplicationStatus status() { - return appStatus; - } + @Override + public void kill() { + processors.forEach(StreamProcessor::stop); + } - @Override - public boolean waitForFinish(Duration timeout) { - return LocalApplicationRunner.this.localWaitForFinish(timeout); - } + @Override + public ApplicationStatus status() { + return appStatus; + } + @Override + public void waitForFinish() { + this.waitForFinish(Duration.ofSeconds(0)); } - private boolean localWaitForFinish(Duration timeout) { + @Override + public boolean waitForFinish(Duration timeout) { long timeoutInMs = timeout.toMillis(); boolean finished = true; @@ -267,6 +204,37 @@ private boolean localWaitForFinish(Duration timeout) { return finished; } + @Override + List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc) { + // for high-level DAG, generating the plan and job configs + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), streamManager); + + String executionPlanJson = plan.getPlanAsJson(); + writePlanJsonFile(executionPlanJson); + LOG.info("Execution Plan: \n" + executionPlanJson); + + // 2. create the necessary streams + // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 + String planId = String.valueOf(executionPlanJson.hashCode()); + createStreams(planId, plan.getIntermediateStreams(), streamManager); + + return plan.getJobConfigs(); + } catch (Throwable throwable) { + appStatus = ApplicationStatus.unsuccessfulFinish(throwable); + shutdownLatch.countDown(); + throw new SamzaException("Failed to start application.", throwable); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + } + /** * Create intermediate streams using {@link org.apache.samza.execution.StreamManager}. * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader @@ -317,33 +285,27 @@ private void createStreams(String planId, } /** - * Create {@link StreamProcessor} based on the config + * Create {@link StreamProcessor} based on config, {@link AppDescriptorImpl}, and {@link ProcessorLifecycleListener} * @param config config + * @param appDesc {@link AppDescriptorImpl} + * @param listenerFn the {@link ProcessorLifecycleListener} * @return {@link StreamProcessor]} */ /* package private */ - StreamProcessor createStreamProcessor( - Config config, - TaskFactory taskFactory, - ProcessorLifecycleListener listener) { - return new StreamProcessor(config, this.metricsReporters, taskFactory, listener, null); + StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, StreamProcessor.StreamProcessorListenerSupplier listenerFn) { + TaskFactory taskFactory = getTaskFactory(appDesc); + return new StreamProcessor(config, this.metricsReporters, taskFactory, listenerFn, null); } - /** - * Create {@link StreamProcessor} based on {@link OperatorSpecGraph}, {@link ContextManager} and the config - * @param config config - * @param graph {@link OperatorSpecGraph} - * @param contextManager {@link ContextManager} - * @return {@link StreamProcessor]} - */ - /* package private */ - StreamProcessor createStreamProcessor( - Config config, - OperatorSpecGraph graph, - ContextManager contextManager, - ProcessorLifecycleListener listener) { - TaskFactory taskFactory = TaskFactoryUtil.createTaskFactory(graph, contextManager); - return new StreamProcessor(config, this.metricsReporters, taskFactory, listener, null); + TaskFactory getTaskFactory(ApplicationDescriptor appDesc) { + if (appDesc instanceof StreamAppDescriptorImpl) { + StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; + return TaskFactoryUtil.createTaskFactory(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()); + } else if (appDesc instanceof TaskAppDescriptorImpl) { + TaskAppDescriptorImpl taskAppDescriptor = (TaskAppDescriptorImpl) appDesc; + return taskAppDescriptor.getTaskFactory(); + } + throw new IllegalArgumentException("Invalid ApplicationDescriptor " + appDesc.getClass().getName()); } /* package private for testing */ @@ -356,14 +318,4 @@ CountDownLatch getShutdownLatch() { return shutdownLatch; } - @Override - protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appDesc) { - return new TaskAppExecutable(appDesc); - } - - @Override - protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appDesc) { - return new StreamAppExecutable(appDesc); - } - } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 4a382602ee..ddc65f5389 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -24,12 +24,11 @@ import org.apache.log4j.MDC; import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.ApplicationClassUtils; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -39,7 +38,6 @@ import org.apache.samza.container.SamzaContainer$; import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.model.JobModel; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.SamzaUncaughtExceptionHandler; @@ -88,45 +86,6 @@ public static void main(String[] args) throws Exception { System.exit(0); } - private static class LocalContainerLifecycleListener implements SamzaContainerListener { - private final ProcessorLifecycleListener pListener; - - LocalContainerLifecycleListener(ProcessorLifecycleListener pListener) { - this.pListener = pListener; - } - - @Override - public void beforeStart() { - log.info("Starting Local Container"); - pListener.beforeStart(); - } - - @Override - public void onContainerStart() { - log.info("Local Container Started"); - pListener.afterStart(); - } - - @Override - public void beforeStop() { - log.info("Stopping Local Container"); - pListener.beforeStop(); - } - - @Override - public void onContainerStop() { - log.info("Container Stopped Successfully"); - pListener.afterStop(null); - } - - @Override - public void onContainerFailed(Throwable t) { - log.info("Container Stopped with Failure"); - containerRunnerException = t; - pListener.afterStop(t); - } - } - private static AppDescriptorImpl getAppDesc(Config config) { ApplicationBase userApp = ApplicationClassUtils.fromConfig(config); return userApp instanceof StreamApplication ? new StreamAppDescriptorImpl((StreamApplication) userApp, config) : @@ -146,8 +105,33 @@ private static void run(AppDescriptorImpl appDesc, String containerId, JobModel ProcessorContext pContext = () -> String.format("%s-%s-%s", jobConfig.getName(), jobConfig.getJobId(), containerId); ProcessorLifecycleListener pListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(pContext, config); - // TODO: this is a temporary solution to inject the lifecycle listeners before we fix SAMZA-1168 - container.setContainerListener(new LocalContainerLifecycleListener(pListener)); + container.setContainerListener( + new SamzaContainerListener() { + @Override + public void beforeStart() { + log.info("Before starting the container."); + pListener.beforeStart(); + } + + @Override + public void onContainerStart() { + log.info("Container Started"); + pListener.afterStart(); + } + + @Override + public void onContainerStop() { + log.info("Container Stopped"); + pListener.afterStop(); + } + + @Override + public void onContainerFailed(Throwable t) { + log.info("Container Failed"); + containerRunnerException = t; + pListener.afterFailure(t); + } + }); ContainerHeartbeatMonitor heartbeatMonitor = createContainerHeartbeatMonitor(container); if (heartbeatMonitor != null) { @@ -169,7 +153,7 @@ private static void run(AppDescriptorImpl appDesc, String containerId, JobModel private static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { if (appDesc instanceof StreamAppDescriptorImpl) { StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; - return TaskFactoryUtil.createTaskFactory(((StreamGraphSpec) streamAppDesc.getGraph()).getOperatorSpecGraph(), + return TaskFactoryUtil.createTaskFactory(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()); } return ((TaskAppDescriptorImpl) appDesc).getTaskFactory(); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 33fe88fa53..995132c731 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -20,11 +20,11 @@ package org.apache.samza.runtime; import java.time.Duration; +import java.util.List; import java.util.UUID; import org.apache.samza.SamzaException; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -34,7 +34,6 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.JobRunner; import org.apache.samza.metrics.MetricsRegistryMap; -import org.apache.samza.operators.StreamGraphSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,136 +48,63 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; - public RemoteApplicationRunner(AppDescriptorImpl appDesc) { - super(appDesc); + RemoteApplicationRunner(ApplicationBase userApp, Config config) { + super(userApp, config); } - class TaskAppExecutable implements AppRuntimeExecutable { - final TaskAppDescriptorImpl appDesc; - final JobRunner jobRunner; - - TaskAppExecutable(TaskAppDescriptorImpl appDesc) { - this.appDesc = appDesc; - this.jobRunner = new JobRunner(config); - } - - @Override - public void run() { - jobRunner.run(true); - } - - @Override - public void kill() { - jobRunner.kill(); - } - - @Override - public ApplicationStatus status() { - return jobRunner.status(); - } - - @Override - public boolean waitForFinish(Duration timeout) { - return RemoteApplicationRunner.this.remoteWaitForFinish(timeout); - } - - } - - class StreamAppExecutable implements AppRuntimeExecutable { - final StreamAppDescriptorImpl appDesc; - - StreamAppExecutable(StreamAppDescriptorImpl appDesc) { - this.appDesc = appDesc; - } - - @Override - public void run() { - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - // TODO: start.id needs to be set for standalone: SAMZA-1531 - // start.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision - String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The start id for this start is {}", runId); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(((StreamGraphSpec) appDesc.getGraph()).getOperatorSpecGraph(), runId, streamManager); - writePlanJsonFile(plan.getPlanAsJson()); - - // 2. create the necessary streams - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - streamManager.createStreams(plan.getIntermediateStreams()); - - // 3. submit jobs for remote execution - plan.getJobConfigs().forEach(jobConfig -> { - LOG.info("Starting job {} with config {}", jobConfig.getName(), jobConfig); - JobRunner runner = new JobRunner(jobConfig); - runner.run(true); - }); - } catch (Throwable t) { - throw new SamzaException("Failed to start application", t); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - } - - @Override - public void kill() { - - // since currently we only support single actual remote job, we can get its status without - // building the execution plan. - try { - JobConfig jc = new JobConfig(config); - LOG.info("Killing job {}", jc.getName()); - JobRunner runner = new JobRunner(jc); - runner.kill(); - } catch (Throwable t) { - throw new SamzaException("Failed to kill application", t); + @Override + public void run() { + try { + List jobConfigs = createJobConfigs(); + if (jobConfigs.isEmpty()) { + throw new SamzaException("No jobs to run."); } - } - @Override - public ApplicationStatus status() { - - // since currently we only support single actual remote job, we can get its status without - // building the execution plan - try { - JobConfig jc = new JobConfig(config); - return getApplicationStatus(jc); - } catch (Throwable t) { - throw new SamzaException("Failed to get status for application", t); - } - } + // 3. submit jobs for remote execution + jobConfigs.forEach(jobConfig -> { + LOG.info("Starting job {} with config {}", jobConfig.getName(), jobConfig); + JobRunner runner = new JobRunner(jobConfig); + runner.run(true); + }); - @Override - public boolean waitForFinish(Duration timeout) { - return RemoteApplicationRunner.this.remoteWaitForFinish(timeout); + } catch (Throwable t) { + throw new SamzaException("Failed to run application", t); } - } @Override - protected AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appDesc) { - return new TaskAppExecutable(appDesc); + public void kill() { + // since currently we only support single actual remote job, we can get its status without + // building the execution plan. + try { + JobConfig jc = new JobConfig(config); + LOG.info("Killing job {}", jc.getName()); + JobRunner runner = new JobRunner(jc); + runner.kill(); + } catch (Throwable t) { + throw new SamzaException("Failed to kill application", t); + } } @Override - protected AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appDesc) { - return new StreamAppExecutable(appDesc); + public ApplicationStatus status() { + // since currently we only support single actual remote job, we can get its status without + // building the execution plan + try { + JobConfig jc = new JobConfig(config); + return getApplicationStatus(jc); + } catch (Throwable t) { + throw new SamzaException("Failed to get status for application", t); + } } - /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { - JobRunner runner = new JobRunner(jobConfig); - ApplicationStatus status = runner.status(); - LOG.debug("Status is {} for job {}", new Object[]{status, jobConfig.getName()}); - return status; + @Override + public void waitForFinish() { + this.waitForFinish(Duration.ofSeconds(0)); } - private boolean remoteWaitForFinish(Duration timeout) { + @Override + public boolean waitForFinish(Duration timeout) { JobConfig jobConfig = new JobConfig(config); boolean finished = true; long timeoutInMs = timeout.toMillis(); @@ -213,6 +139,44 @@ private boolean remoteWaitForFinish(Duration timeout) { return finished; } + @Override + List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc) { + // for high-level DAG, generate the plan and job configs + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + // TODO: run.id needs to be set for standalone: SAMZA-1531 + // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); + LOG.info("The run id for this run is {}", runId); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId, streamManager); + writePlanJsonFile(plan.getPlanAsJson()); + + // 2. create the necessary streams + if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { + streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); + } + streamManager.createStreams(plan.getIntermediateStreams()); + + return plan.getJobConfigs(); + } catch (Throwable t) { + throw new SamzaException("Failed to run application", t); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + } + + /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { + JobRunner runner = new JobRunner(jobConfig); + ApplicationStatus status = runner.status(); + LOG.debug("Status is {} for job {}", new Object[]{status, jobConfig.getName()}); + return status; + } + private Config getConfigFromPrevRun() { CoordinatorStreamSystemConsumer consumer = new CoordinatorStreamSystemConsumer(config, new MetricsRegistryMap()); consumer.register(); diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java index 2daffeadc7..2872f3d6bc 100644 --- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java +++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java @@ -19,14 +19,14 @@ package org.apache.samza.task; import org.apache.samza.config.Config; +import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.system.EndOfStreamMessage; +import org.apache.samza.system.MessageType; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; -import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.impl.InputOperatorImpl; import org.apache.samza.operators.impl.OperatorImplGraph; -import org.apache.samza.system.EndOfStreamMessage; import org.apache.samza.system.IncomingMessageEnvelope; -import org.apache.samza.system.MessageType; import org.apache.samza.system.SystemStream; import org.apache.samza.system.WatermarkMessage; import org.apache.samza.util.Clock; @@ -70,10 +70,10 @@ public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextMan * Initializes this task during startup. *

    * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}. - * The {@link org.apache.samza.operators.StreamGraphSpec} sets the input and output streams and the task-wide - * context manager using the {@link org.apache.samza.operators.StreamGraph} APIs, + * The {@link org.apache.samza.application.StreamAppDescriptorImpl} sets the input and output streams and the task-wide + * context manager using the {@link org.apache.samza.application.StreamAppDescriptor} APIs, * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. After the - * {@link org.apache.samza.operators.StreamGraphSpec} is initialized once by the application, it then creates + * {@link org.apache.samza.application.StreamAppDescriptorImpl} is initialized once by the application, it then creates * an immutable {@link OperatorSpecGraph} accordingly, which is passed in to this class to create the {@link OperatorImplGraph} * corresponding to the logical DAG. * diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index 62701690f2..1d3d40be18 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -18,7 +18,6 @@ */ package org.apache.samza.task; -import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; @@ -28,7 +27,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.samza.util.ScalaJavaUtil.*; +import java.util.concurrent.ExecutorService; + +import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; /** * This class provides utility functions to load task factory classes based on config, and to wrap {@link StreamTaskFactory} diff --git a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java index e7a1e5460e..31796d95ed 100644 --- a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java @@ -18,6 +18,7 @@ */ package org.apache.samza.util; +import java.util.Collection; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -77,7 +78,7 @@ public static String getNameFromSystemStream(SystemStream systemStream) { return systemStream.getSystem() + "." + systemStream.getStream(); } - public static Set getStreamSpecs(Set streamIds, StreamConfig streamConfig) { + public static Set getStreamSpecs(Collection streamIds, StreamConfig streamConfig) { return streamIds.stream().map(streamId -> getStreamSpec(streamId, streamConfig)).collect(Collectors.toSet()); } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index f46913e847..b3372c6fca 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -792,14 +792,15 @@ class SamzaContainer( try { info("Starting container.") + if (containerListener != null) { + containerListener.beforeStart() + } + val startTime = System.nanoTime() status = SamzaContainerStatus.STARTING jmxServer = new JmxServer() - if (containerListener != null) { - containerListener.beforeStart() - } startMetrics startDiagnostics startAdmins @@ -837,10 +838,6 @@ class SamzaContainer( info("Shutting down.") removeShutdownHook - if (containerListener != null) { - containerListener.beforeStop() - } - jmxServer.stop shutdownConsumers diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index c95def7d6c..439dd117d6 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,8 +19,7 @@ package org.apache.samza.job.local -import org.apache.samza.application.internal.{StreamAppDescriptorImpl, TaskAppDescriptorImpl} -import org.apache.samza.application.{ApplicationClassUtils, StreamApplication, TaskApplication} +import org.apache.samza.application._ import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ @@ -30,8 +29,9 @@ import org.apache.samza.coordinator.JobModelManager import org.apache.samza.coordinator.stream.CoordinatorStreamManager import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} -import org.apache.samza.operators.StreamGraphSpec +import org.apache.samza.runtime.ApplicationClassUtils import org.apache.samza.storage.ChangelogStreamManager +import org.apache.samza.task.TaskFactoryUtil import org.apache.samza.task._ import org.apache.samza.util.Logging @@ -81,10 +81,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging { } case app if (app.isInstanceOf[StreamApplication]) => { val appSpec = new StreamAppDescriptorImpl(app.asInstanceOf[StreamApplication], config) - new StreamTaskFactory { - override def createInstance(): StreamTask = - new StreamOperatorTask(appSpec.getGraph.asInstanceOf[StreamGraphSpec].getOperatorSpecGraph, appSpec.getContextManager) - } + TaskFactoryUtil.createTaskFactory(appSpec.getOperatorSpecGraph, appSpec.getContextManager) } } @@ -108,10 +105,9 @@ class ThreadJobFactory extends StreamJobFactory with Logging { } - override def beforeStop(): Unit = { - - } - + /** + * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started + */ override def beforeStart(): Unit = { } diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java similarity index 64% rename from samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java rename to samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java index c1008bfc18..943e790acf 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java @@ -12,55 +12,62 @@ * Unless required by applicable law or agreed to in writing, * software distributed under the License is distributed on an * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for THE + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.operators; +package org.apache.samza.application; import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.HashMap; import java.util.List; - import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; +import org.apache.samza.operators.BaseTableDescriptor; +import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.data.TestMessageEnvelope; import org.apache.samza.operators.spec.InputOperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec.OpCode; +import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; +import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.table.TableSpec; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + + +/** + * Unit test for {@link StreamAppDescriptorImpl} + */ +public class TestStreamAppDescriptorImpl { -public class TestStreamGraphSpec { + @Test + public void testConstructor() { + StreamApplication mockApp = mock(StreamApplication.class); + Config mockConfig = mock(Config.class); + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(mockApp, mockConfig); + verify(mockApp, times(1)).describe(appDesc); + assertEquals(mockConfig, appDesc.config); + } @Test public void testGetInputStreamWithValueSerde() { - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - MessageStream inputStream = graphSpec.getInputStream(streamId, mockValueSerde); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId, mockValueSerde); + }, mock(Config.class)); - InputOperatorSpec inputOpSpec = - (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); - assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); + assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -68,7 +75,6 @@ public void testGetInputStreamWithValueSerde() { @Test public void testGetInputStreamWithKeyValueSerde() { - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); String streamId = "test-stream-1"; KVSerde mockKVSerde = mock(KVSerde.class); @@ -76,12 +82,12 @@ public void testGetInputStreamWithKeyValueSerde() { Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - MessageStream inputStream = graphSpec.getInputStream(streamId, mockKVSerde); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId, mockKVSerde); + }, mock(Config.class)); - InputOperatorSpec inputOpSpec = - (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); - assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); + assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertEquals(mockKeySerde, inputOpSpec.getKeySerde()); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -89,24 +95,23 @@ public void testGetInputStreamWithKeyValueSerde() { @Test(expected = NullPointerException.class) public void testGetInputStreamWithNullSerde() { - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - - graphSpec.getInputStream("test-stream-1", null); + new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream("test-stream-1", null); + }, mock(Config.class)); } @Test public void testGetInputStreamWithDefaultValueSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); Serde mockValueSerde = mock(Serde.class); - graphSpec.setDefaultSerde(mockValueSerde); - MessageStream inputStream = graphSpec.getInputStream(streamId); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(mockValueSerde); + appDesc.getInputStream(streamId); + }, mock(Config.class)); - InputOperatorSpec inputOpSpec = - (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); - assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); + assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -115,20 +120,19 @@ public void testGetInputStreamWithDefaultValueSerde() { @Test public void testGetInputStreamWithDefaultKeyValueSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - graphSpec.setDefaultSerde(mockKVSerde); - MessageStream inputStream = graphSpec.getInputStream(streamId); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(mockKVSerde); + appDesc.getInputStream(streamId); + }, mock(Config.class)); - InputOperatorSpec inputOpSpec = - (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); - assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); + assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertEquals(mockKeySerde, inputOpSpec.getKeySerde()); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -139,13 +143,12 @@ public void testGetInputStreamWithDefaultDefaultSerde() { String streamId = "test-stream-1"; // default default serde == user hasn't provided a default serde - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - MessageStream inputStream = graphSpec.getInputStream(streamId); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId); + }, mock(Config.class)); - InputOperatorSpec inputOpSpec = - (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); - assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); + assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde); @@ -154,14 +157,12 @@ public void testGetInputStreamWithDefaultDefaultSerde() { @Test public void testGetInputStreamWithRelaxedTypes() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId); + }, mock(Config.class)); - MessageStream inputStream = graphSpec.getInputStream(streamId); - - InputOperatorSpec inputOpSpec = - (InputOperatorSpec) ((MessageStreamImpl) inputStream).getOperatorSpec(); - assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); - assertEquals(graphSpec.getInputOperators().get(streamId), inputOpSpec); + InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); + assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); } @@ -170,40 +171,38 @@ public void testMultipleGetInputStreams() { String streamId1 = "test-stream-1"; String streamId2 = "test-stream-2"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - MessageStream inputStream1 = graphSpec.getInputStream(streamId1); - MessageStream inputStream2 = graphSpec.getInputStream(streamId2); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId1); + appDesc.getInputStream(streamId2); + }, mock(Config.class)); - InputOperatorSpec inputOpSpec1 = - (InputOperatorSpec) ((MessageStreamImpl) inputStream1).getOperatorSpec(); - InputOperatorSpec inputOpSpec2 = - (InputOperatorSpec) ((MessageStreamImpl) inputStream2).getOperatorSpec(); + InputOperatorSpec inputOpSpec1 = graphSpec.getInputOperators().get(streamId1); + InputOperatorSpec inputOpSpec2 = graphSpec.getInputOperators().get(streamId2); assertEquals(graphSpec.getInputOperators().size(), 2); - assertEquals(graphSpec.getInputOperators().get(streamId1), inputOpSpec1); - assertEquals(graphSpec.getInputOperators().get(streamId2), inputOpSpec2); + assertEquals(streamId1, inputOpSpec1.getStreamId()); + assertEquals(streamId2, inputOpSpec2.getStreamId()); } @Test(expected = IllegalStateException.class) public void testGetSameInputStreamTwice() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - graphSpec.getInputStream(streamId); - // should throw exception - graphSpec.getInputStream(streamId); + new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId); + // should throw exception + appDesc.getInputStream(streamId); + }, mock(Config.class)); } @Test public void testGetOutputStreamWithValueSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - Serde mockValueSerde = mock(Serde.class); - OutputStream outputStream = - graphSpec.getOutputStream(streamId, mockValueSerde); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getOutputStream(streamId, mockValueSerde); + }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -212,17 +211,17 @@ public void testGetOutputStreamWithValueSerde() { @Test public void testGetOutputStreamWithKeyValueSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - graphSpec.setDefaultSerde(mockKVSerde); - OutputStream outputStream = graphSpec.getOutputStream(streamId, mockKVSerde); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(mockKVSerde); + appDesc.getOutputStream(streamId, mockKVSerde); + }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertEquals(mockKeySerde, outputStreamImpl.getKeySerde()); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -231,9 +230,9 @@ public void testGetOutputStreamWithKeyValueSerde() { @Test(expected = NullPointerException.class) public void testGetOutputStreamWithNullSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - - graphSpec.getOutputStream(streamId, null); + new StreamAppDescriptorImpl(appDesc -> { + appDesc.getOutputStream(streamId, null); + }, mock(Config.class)); } @Test @@ -241,12 +240,12 @@ public void testGetOutputStreamWithDefaultValueSerde() { String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - graphSpec.setDefaultSerde(mockValueSerde); - OutputStream outputStream = graphSpec.getOutputStream(streamId); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(mockValueSerde); + appDesc.getOutputStream(streamId); + }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -256,18 +255,17 @@ public void testGetOutputStreamWithDefaultValueSerde() { public void testGetOutputStreamWithDefaultKeyValueSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - graphSpec.setDefaultSerde(mockKVSerde); - - OutputStream outputStream = graphSpec.getOutputStream(streamId); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(mockKVSerde); + appDesc.getOutputStream(streamId); + }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertEquals(mockKeySerde, outputStreamImpl.getKeySerde()); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -277,12 +275,12 @@ public void testGetOutputStreamWithDefaultKeyValueSerde() { public void testGetOutputStreamWithDefaultDefaultSerde() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getOutputStream(streamId); + }, mock(Config.class)); - OutputStream outputStream = graphSpec.getOutputStream(streamId); - OutputStreamImpl outputStreamImpl = (OutputStreamImpl) outputStream; - assertEquals(graphSpec.getOutputStreams().get(streamId), outputStreamImpl); + OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertTrue(outputStreamImpl.getValueSerde() instanceof NoOpSerde); @@ -292,23 +290,25 @@ public void testGetOutputStreamWithDefaultDefaultSerde() { public void testSetDefaultSerdeAfterGettingStreams() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - graphSpec.getInputStream(streamId); - graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception + new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(streamId); + appDesc.setDefaultSerde(mock(Serde.class)); // should throw exception + }, mock(Config.class)); } @Test(expected = IllegalStateException.class) public void testSetDefaultSerdeAfterGettingOutputStream() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - graphSpec.getOutputStream(streamId); - graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception + new StreamAppDescriptorImpl(appDesc -> { + appDesc.getOutputStream(streamId); + appDesc.setDefaultSerde(mock(Serde.class)); // should throw exception + }, mock(Config.class)); } @Test(expected = IllegalStateException.class) public void testSetDefaultSerdeAfterGettingIntermediateStream() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); graphSpec.getIntermediateStream(streamId, null); graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception } @@ -316,15 +316,16 @@ public void testSetDefaultSerdeAfterGettingIntermediateStream() { @Test(expected = IllegalStateException.class) public void testGetSameOutputStreamTwice() { String streamId = "test-stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); - graphSpec.getOutputStream(streamId); - graphSpec.getOutputStream(streamId); // should throw exception + new StreamAppDescriptorImpl(appDesc -> { + appDesc.getOutputStream(streamId); + appDesc.getOutputStream(streamId); // should throw exception + }, mock(Config.class)); } @Test public void testGetIntermediateStreamWithValueSerde() { String streamId = "stream-1"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); Serde mockValueSerde = mock(Serde.class); IntermediateMessageStreamImpl intermediateStreamImpl = @@ -342,7 +343,7 @@ public void testGetIntermediateStreamWithValueSerde() { @Test public void testGetIntermediateStreamWithKeyValueSerde() { String streamId = "streamId"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -364,7 +365,7 @@ public void testGetIntermediateStreamWithKeyValueSerde() { @Test public void testGetIntermediateStreamWithDefaultValueSerde() { String streamId = "streamId"; - StreamGraphSpec graph = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graph = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); Serde mockValueSerde = mock(Serde.class); graph.setDefaultSerde(mockValueSerde); @@ -385,7 +386,7 @@ public void testGetIntermediateStreamWithDefaultKeyValueSerde() { Config mockConfig = mock(Config.class); String streamId = "streamId"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -410,7 +411,7 @@ public void testGetIntermediateStreamWithDefaultDefaultSerde() { Config mockConfig = mock(Config.class); String streamId = "streamId"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); IntermediateMessageStreamImpl intermediateStreamImpl = graphSpec.getIntermediateStream(streamId, null); @@ -425,7 +426,7 @@ public void testGetIntermediateStreamWithDefaultDefaultSerde() { @Test(expected = IllegalStateException.class) public void testGetSameIntermediateStreamTwice() { - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class)); graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class)); } @@ -436,10 +437,10 @@ public void testGetNextOpIdIncrementsId() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); - assertEquals("jobName-1234-merge-0", graphSpec.getNextOpId(OpCode.MERGE, null)); - assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OpCode.JOIN, "customName")); - assertEquals("jobName-1234-map-2", graphSpec.getNextOpId(OpCode.MAP, null)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + assertEquals("jobName-1234-merge-0", graphSpec.getNextOpId(OperatorSpec.OpCode.MERGE, null)); + assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OperatorSpec.OpCode.JOIN, "customName")); + assertEquals("jobName-1234-map-2", graphSpec.getNextOpId(OperatorSpec.OpCode.MAP, null)); } @Test(expected = SamzaException.class) @@ -448,9 +449,9 @@ public void testGetNextOpIdRejectsDuplicates() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); - assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OpCode.JOIN, "customName")); - graphSpec.getNextOpId(OpCode.JOIN, "customName"); // should throw + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OperatorSpec.OpCode.JOIN, "customName")); + graphSpec.getNextOpId(OperatorSpec.OpCode.JOIN, "customName"); // should throw } @Test @@ -459,14 +460,14 @@ public void testUserDefinedIdValidation() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); // null and empty userDefinedIDs should fall back to autogenerated IDs. try { - graphSpec.getNextOpId(OpCode.FILTER, null); - graphSpec.getNextOpId(OpCode.FILTER, ""); - graphSpec.getNextOpId(OpCode.FILTER, " "); - graphSpec.getNextOpId(OpCode.FILTER, "\t"); + graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, null); + graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, ""); + graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, " "); + graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, "\t"); } catch (SamzaException e) { fail("Received an error with a null or empty operator ID instead of defaulting to auto-generated ID."); } @@ -474,7 +475,7 @@ public void testUserDefinedIdValidation() { List validOpIds = ImmutableList.of("op.id", "op_id", "op-id", "1000", "op_1", "OP_ID"); for (String validOpId: validOpIds) { try { - graphSpec.getNextOpId(OpCode.FILTER, validOpId); + graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, validOpId); } catch (Exception e) { fail("Received an exception with a valid operator ID: " + validOpId); } @@ -483,7 +484,7 @@ public void testUserDefinedIdValidation() { List invalidOpIds = ImmutableList.of("op id", "op#id"); for (String invalidOpId: invalidOpIds) { try { - graphSpec.getNextOpId(OpCode.FILTER, invalidOpId); + graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, invalidOpId); fail("Did not receive an exception with an invalid operator ID: " + invalidOpId); } catch (SamzaException e) { } } @@ -493,15 +494,15 @@ public void testUserDefinedIdValidation() { public void testGetInputStreamPreservesInsertionOrder() { Config mockConfig = mock(Config.class); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); - String testStreamId1 = "test-stream-1"; String testStreamId2 = "test-stream-2"; String testStreamId3 = "test-stream-3"; - graphSpec.getInputStream("test-stream-1"); - graphSpec.getInputStream("test-stream-2"); - graphSpec.getInputStream("test-stream-3"); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getInputStream(testStreamId1); + appDesc.getInputStream(testStreamId2); + appDesc.getInputStream(testStreamId3); + }, mockConfig); List inputSpecs = new ArrayList<>(graphSpec.getInputOperators().values()); assertEquals(inputSpecs.size(), 3); @@ -511,13 +512,31 @@ public void testGetInputStreamPreservesInsertionOrder() { } @Test - public void testGetTable() { + public void testGetTable() throws Exception { Config mockConfig = mock(Config.class); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); - when(mockTableDescriptor.getTableSpec()).thenReturn( - new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>())); - assertNotNull(graphSpec.getTable(mockTableDescriptor)); + TableSpec testTableSpec = new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()); + when(mockTableDescriptor.getTableSpec()).thenReturn(testTableSpec); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getTable(mockTableDescriptor); + }, mockConfig); + assertNotNull(graphSpec.getTables().get(testTableSpec)); + } + + @Test + public void testContextManager() { + ContextManager cntxMan = mock(ContextManager.class); + StreamApplication testApp = appDesc -> appDesc.withContextManager(cntxMan); + StreamAppDescriptorImpl appSpec = new StreamAppDescriptorImpl(testApp, mock(Config.class)); + assertEquals(appSpec.getContextManager(), cntxMan); + } + + @Test + public void testProcessorLifecycleListenerFactory() { + ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class); + StreamApplication testApp = appSpec -> appSpec.withProcessorLifecycleListenerFactory(mockFactory); + StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, mock(Config.class)); + assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); } } diff --git a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java similarity index 90% rename from samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java rename to samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java index 48145b056a..10f93f92b9 100644 --- a/samza-api/src/test/java/org/apache/samza/application/internal/TestTaskAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java @@ -16,11 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application.internal; +package org.apache.samza.application; import java.util.ArrayList; import java.util.List; -import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.TableDescriptor; @@ -50,7 +49,9 @@ public void testConstructor() { @Test public void testAddInputStreams() { List testInputs = new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }; - TaskApplication testApp = appDesc -> appDesc.addInputStreams(testInputs); + TaskApplication testApp = appDesc -> { + testInputs.forEach(input -> appDesc.addInputStream(input)); + }; TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); assertEquals(appDesc.getInputStreams(), testInputs); } @@ -58,7 +59,9 @@ public void testAddInputStreams() { @Test public void testAddOutputStreams() { List testOutputs = new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }; - TaskApplication testApp = appDesc -> appDesc.addOutputStreams(testOutputs); + TaskApplication testApp = appDesc -> { + testOutputs.forEach(output -> appDesc.addOutputStream(output)); + }; TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); assertEquals(appDesc.getOutputStreams(), testOutputs); } @@ -66,7 +69,9 @@ public void testAddOutputStreams() { @Test public void testAddTables() { List testTables = new ArrayList() { { this.add(mock(TableDescriptor.class)); } }; - TaskApplication testApp = appDesc -> appDesc.addTables(testTables); + TaskApplication testApp = appDesc -> { + testTables.forEach(table -> appDesc.addTable(table)); + }; TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); assertEquals(appDesc.getTables(), testTables); } 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 94dbcfc5fb..0d66956c46 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 @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -35,7 +36,6 @@ import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.Serde; @@ -96,24 +96,24 @@ public Integer offsetComparator(String offset1, String offset2) { }; } - private StreamGraphSpec createSimpleGraph() { + private StreamAppDescriptorImpl createSimpleGraph() { /** * a simple graph of partitionBy and map * * input1 -> partitionBy -> map -> output1 * */ - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream> input1 = graphSpec.getInputStream("input1"); - OutputStream> output1 = graphSpec.getOutputStream("output1"); - input1 - .partitionBy(m -> m.key, m -> m.value, "p1") - .map(kv -> kv) - .sendTo(output1); - return graphSpec; + return new StreamAppDescriptorImpl(appDesc-> { + MessageStream> input1 = appDesc.getInputStream("input1"); + OutputStream> output1 = appDesc.getOutputStream("output1"); + input1 + .partitionBy(m -> m.key, m -> m.value, "p1") + .map(kv -> kv) + .sendTo(output1); + }, config); } - private StreamGraphSpec createStreamGraphWithJoin() { + private StreamAppDescriptorImpl createStreamGraphWithJoin() { /** * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value. @@ -125,80 +125,77 @@ private StreamGraphSpec createStreamGraphWithJoin() { * input3 (32) -> filter -> partitionBy ("64") -> map -> join -> output2 (16) * */ - - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream> messageStream1 = - graphSpec.>getInputStream("input1") - .map(m -> m); - MessageStream> messageStream2 = - graphSpec.>getInputStream("input2") - .partitionBy(m -> m.key, m -> m.value, "p1") - .filter(m -> true); - MessageStream> messageStream3 = - graphSpec.>getInputStream("input3") - .filter(m -> true) - .partitionBy(m -> m.key, m -> m.value, "p2") - .map(m -> m); - OutputStream> output1 = graphSpec.getOutputStream("output1"); - OutputStream> output2 = graphSpec.getOutputStream("output2"); - - messageStream1 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") - .sendTo(output1); - messageStream3 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") - .sendTo(output2); - - return graphSpec; + return new StreamAppDescriptorImpl(appDesc -> { + MessageStream> messageStream1 = + appDesc.>getInputStream("input1") + .map(m -> m); + MessageStream> messageStream2 = + appDesc.>getInputStream("input2") + .partitionBy(m -> m.key, m -> m.value, "p1") + .filter(m -> true); + MessageStream> messageStream3 = + appDesc.>getInputStream("input3") + .filter(m -> true) + .partitionBy(m -> m.key, m -> m.value, "p2") + .map(m -> m); + OutputStream> output1 = appDesc.getOutputStream("output1"); + OutputStream> output2 = appDesc.getOutputStream("output2"); + + messageStream1 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") + .sendTo(output1); + messageStream3 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") + .sendTo(output2); + }, config); } - private StreamGraphSpec createStreamGraphWithJoinAndWindow() { - - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream> messageStream1 = - graphSpec.>getInputStream("input1") - .map(m -> m); - MessageStream> messageStream2 = - graphSpec.>getInputStream("input2") - .partitionBy(m -> m.key, m -> m.value, "p1") - .filter(m -> true); - MessageStream> messageStream3 = - graphSpec.>getInputStream("input3") - .filter(m -> true) - .partitionBy(m -> m.key, m -> m.value, "p2") - .map(m -> m); - OutputStream> output1 = graphSpec.getOutputStream("output1"); - OutputStream> output2 = graphSpec.getOutputStream("output2"); - - messageStream1.map(m -> m) - .filter(m->true) - .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), mock(Serde.class), mock(Serde.class)), "w1"); - - messageStream2.map(m -> m) - .filter(m->true) - .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), mock(Serde.class), mock(Serde.class)), "w2"); - - messageStream1 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(1600), "j1") - .sendTo(output1); - messageStream3 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(100), "j2") - .sendTo(output2); - messageStream3 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(252), "j3") - .sendTo(output2); - - return graphSpec; + private StreamAppDescriptorImpl createStreamGraphWithJoinAndWindow() { + + return new StreamAppDescriptorImpl(appDesc -> { + MessageStream> messageStream1 = + appDesc.>getInputStream("input1") + .map(m -> m); + MessageStream> messageStream2 = + appDesc.>getInputStream("input2") + .partitionBy(m -> m.key, m -> m.value, "p1") + .filter(m -> true); + MessageStream> messageStream3 = + appDesc.>getInputStream("input3") + .filter(m -> true) + .partitionBy(m -> m.key, m -> m.value, "p2") + .map(m -> m); + OutputStream> output1 = appDesc.getOutputStream("output1"); + OutputStream> output2 = appDesc.getOutputStream("output2"); + + messageStream1.map(m -> m) + .filter(m->true) + .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), mock(Serde.class), mock(Serde.class)), "w1"); + + messageStream2.map(m -> m) + .filter(m->true) + .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), mock(Serde.class), mock(Serde.class)), "w2"); + + messageStream1 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(1600), "j1") + .sendTo(output1); + messageStream3 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(100), "j2") + .sendTo(output2); + messageStream3 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(252), "j3") + .sendTo(output2); + }, config); } @Before @@ -243,7 +240,7 @@ public void setup() { @Test public void testCreateProcessorGraph() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamGraphSpec graphSpec = createStreamGraphWithJoin(); + StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); assertTrue(jobGraph.getSources().size() == 3); @@ -254,7 +251,7 @@ public void testCreateProcessorGraph() { @Test public void testFetchExistingStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamGraphSpec graphSpec = createStreamGraphWithJoin(); + StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); @@ -272,7 +269,7 @@ public void testFetchExistingStreamPartitions() { @Test public void testCalculateJoinInputPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamGraphSpec graphSpec = createStreamGraphWithJoin(); + StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); @@ -291,7 +288,7 @@ public void testDefaultPartitions() { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamGraphSpec graphSpec = createSimpleGraph(); + StreamAppDescriptorImpl graphSpec = createSimpleGraph(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); planner.calculatePartitions(jobGraph); @@ -308,7 +305,7 @@ public void testTriggerIntervalForJoins() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamGraphSpec graphSpec = createStreamGraphWithJoin(); + StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); for (JobConfig config : jobConfigs) { @@ -323,7 +320,7 @@ public void testTriggerIntervalForWindowsAndJoins() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamGraphSpec graphSpec = createStreamGraphWithJoinAndWindow(); + StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -340,7 +337,7 @@ public void testTriggerIntervalWithInvalidWindowMs() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamGraphSpec graphSpec = createStreamGraphWithJoinAndWindow(); + StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -357,7 +354,7 @@ public void testTriggerIntervalForStatelessOperators() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamGraphSpec graphSpec = createSimpleGraph(); + StreamAppDescriptorImpl graphSpec = createSimpleGraph(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -372,7 +369,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamGraphSpec graphSpec = createSimpleGraph(); + StreamAppDescriptorImpl graphSpec = createSimpleGraph(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -382,7 +379,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { @Test public void testCalculateIntStreamPartitions() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamGraphSpec graphSpec = createSimpleGraph(); + StreamAppDescriptorImpl graphSpec = createSimpleGraph(); JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph()); // the partitions should be the same as input1 @@ -415,11 +412,12 @@ public void testMaxPartitionLimit() throws Exception { int partitionLimit = ExecutionPlanner.MAX_INFERRED_PARTITIONS; ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream> input1 = appDesc.getInputStream("input4"); + OutputStream> output1 = appDesc.getOutputStream("output1"); + input1.partitionBy(m -> m.key, m -> m.value, "p1").map(kv -> kv).sendTo(output1); + }, config); - MessageStream> input1 = graphSpec.getInputStream("input4"); - OutputStream> output1 = graphSpec.getOutputStream("output1"); - input1.partitionBy(m -> m.key, m -> m.value, "p1").map(kv -> kv).sendTo(output1); JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph()); // the partitions should be the same as input1 diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index b0f3843eaa..151b611cd0 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -22,13 +22,13 @@ import java.time.Duration; import java.util.HashMap; import java.util.Map; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.KVSerde; @@ -98,34 +98,35 @@ public void test() throws Exception { when(systemAdmins.getSystemAdmin("system2")).thenReturn(systemAdmin2); StreamManager streamManager = new StreamManager(systemAdmins); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - graphSpec.setDefaultSerde(KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); - MessageStream> messageStream1 = - graphSpec.>getInputStream("input1") - .map(m -> m); - MessageStream> messageStream2 = - graphSpec.>getInputStream("input2") - .partitionBy(m -> m.key, m -> m.value, "p1") - .filter(m -> true); - MessageStream> messageStream3 = - graphSpec.>getInputStream("input3") - .filter(m -> true) - .partitionBy(m -> m.key, m -> m.value, "p2") - .map(m -> m); - OutputStream> outputStream1 = graphSpec.getOutputStream("output1"); - OutputStream> outputStream2 = graphSpec.getOutputStream("output2"); - - messageStream1 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") - .sendTo(outputStream1); - messageStream2.sink((message, collector, coordinator) -> { }); - messageStream3 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") - .sendTo(outputStream2); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); + MessageStream> messageStream1 = + appDesc.>getInputStream("input1") + .map(m -> m); + MessageStream> messageStream2 = + appDesc.>getInputStream("input2") + .partitionBy(m -> m.key, m -> m.value, "p1") + .filter(m -> true); + MessageStream> messageStream3 = + appDesc.>getInputStream("input3") + .filter(m -> true) + .partitionBy(m -> m.key, m -> m.value, "p2") + .map(m -> m); + OutputStream> outputStream1 = appDesc.getOutputStream("output1"); + OutputStream> outputStream2 = appDesc.getOutputStream("output2"); + + messageStream1 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") + .sendTo(outputStream1); + messageStream2.sink((message, collector, coordinator) -> { }); + messageStream3 + .join(messageStream2, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") + .sendTo(outputStream2); + }, config); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); @@ -164,18 +165,19 @@ public void test2() throws Exception { when(systemAdmins.getSystemAdmin("kafka")).thenReturn(systemAdmin2); StreamManager streamManager = new StreamManager(systemAdmins); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream> inputStream = graphSpec.getInputStream("PageView"); - inputStream - .partitionBy(kv -> kv.getValue().getCountry(), kv -> kv.getValue(), "keyed-by-country") - .window(Windows.keyedTumblingWindow(kv -> kv.getValue().getCountry(), - Duration.ofSeconds(10L), - () -> 0L, - (m, c) -> c + 1L, - new StringSerde(), - new LongSerde()), "count-by-country") - .map(pane -> new KV<>(pane.getKey().getKey(), pane.getMessage())) - .sendTo(graphSpec.getOutputStream("PageViewCount")); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream> inputStream = appDesc.getInputStream("PageView"); + inputStream + .partitionBy(kv -> kv.getValue().getCountry(), kv -> kv.getValue(), "keyed-by-country") + .window(Windows.keyedTumblingWindow(kv -> kv.getValue().getCountry(), + Duration.ofSeconds(10L), + () -> 0L, + (m, c) -> c + 1L, + new StringSerde(), + new LongSerde()), "count-by-country") + .map(pane -> new KV<>(pane.getKey().getKey(), pane.getMessage())) + .sendTo(appDesc.getOutputStream("PageViewCount")); + }, config); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index 3d77e40be1..71677f0ad8 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -31,7 +32,6 @@ import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; import org.apache.samza.serializers.JsonSerdeV2; @@ -61,18 +61,19 @@ public void testAddSerdeConfigs() { when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); - graphSpec.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>())); - MessageStream> input1 = graphSpec.getInputStream("input1"); - MessageStream> input2 = graphSpec.getInputStream("input2"); - OutputStream> output = graphSpec.getOutputStream("output"); - JoinFunction> mockJoinFn = mock(JoinFunction.class); - input1 - .partitionBy(KV::getKey, KV::getValue, "p1").map(kv -> kv.value) - .join(input2.map(kv -> kv.value), mockJoinFn, - new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), - Duration.ofHours(1), "j1") - .sendTo(output); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + appDesc.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>())); + MessageStream> input1 = appDesc.getInputStream("input1"); + MessageStream> input2 = appDesc.getInputStream("input2"); + OutputStream> output = appDesc.getOutputStream("output"); + JoinFunction> mockJoinFn = mock(JoinFunction.class); + input1 + .partitionBy(KV::getKey, KV::getValue, "p1").map(kv -> kv.value) + .join(input2.map(kv -> kv.value), mockJoinFn, + new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), + Duration.ofHours(1), "j1") + .sendTo(output); + }, mockConfig); JobNode jobNode = new JobNode("jobName", "jobId", graphSpec.getOperatorSpecGraph(), mockConfig); Config config = new MapConfig(); diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java index 27f340ffa6..d59171d2e3 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.samza.Partition; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.container.TaskContextImpl; @@ -80,8 +81,8 @@ public void setUp() { @Test public void join() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -98,21 +99,22 @@ public void join() throws Exception { public void joinWithSelfThrowsException() throws Exception { config.put("streams.instream.system", "insystem"); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - IntegerSerde integerSerde = new IntegerSerde(); - KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); - MessageStream> inStream = graphSpec.getInputStream("instream", kvSerde); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + IntegerSerde integerSerde = new IntegerSerde(); + KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); + MessageStream> inStream = appDesc.getInputStream("instream", kvSerde); - inStream.join(inStream, new TestJoinFunction(), integerSerde, kvSerde, kvSerde, JOIN_TTL, "join"); + inStream.join(inStream, new TestJoinFunction(), integerSerde, kvSerde, kvSerde, JOIN_TTL, "join"); + }, config); - createStreamOperatorTask(new SystemClock(), graphSpec, null); // should throw an exception + createStreamOperatorTask(new SystemClock(), graphSpec); // should throw an exception } @Test public void joinFnInitAndClose() throws Exception { TestJoinFunction joinFn = new TestJoinFunction(); - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(joinFn); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(joinFn); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); MessageCollector messageCollector = mock(MessageCollector.class); @@ -130,8 +132,8 @@ public void joinFnInitAndClose() throws Exception { @Test public void joinReverse() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -146,8 +148,8 @@ public void joinReverse() throws Exception { @Test public void joinNoMatch() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -161,8 +163,8 @@ public void joinNoMatch() throws Exception { @Test public void joinNoMatchReverse() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -176,8 +178,8 @@ public void joinNoMatchReverse() throws Exception { @Test public void joinRetainsLatestMessageForKey() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -194,8 +196,8 @@ public void joinRetainsLatestMessageForKey() throws Exception { @Test public void joinRetainsLatestMessageForKeyReverse() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -212,8 +214,8 @@ public void joinRetainsLatestMessageForKeyReverse() throws Exception { @Test public void joinRetainsMatchedMessages() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -235,8 +237,8 @@ public void joinRetainsMatchedMessages() throws Exception { @Test public void joinRetainsMatchedMessagesReverse() throws Exception { - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -259,8 +261,8 @@ public void joinRetainsMatchedMessagesReverse() throws Exception { @Test public void joinRemovesExpiredMessages() throws Exception { TestClock testClock = new TestClock(); - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -279,8 +281,8 @@ public void joinRemovesExpiredMessages() throws Exception { @Test public void joinRemovesExpiredMessagesReverse() throws Exception { TestClock testClock = new TestClock(); - StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec, null); + StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -296,7 +298,7 @@ public void joinRemovesExpiredMessagesReverse() throws Exception { assertTrue(output.isEmpty()); } - private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec graphSpec, ContextManager contextManager) + private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamAppDescriptorImpl graphSpec) throws Exception { TaskContextImpl taskContext = mock(TaskContextImpl.class); @@ -312,26 +314,25 @@ private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec when(taskContext.getStore(eq("jobName-jobId-join-j1-R"))) .thenReturn(new TestInMemoryStore(integerSerde, timestampedValueSerde)); - StreamOperatorTask sot = new StreamOperatorTask(graphSpec.getOperatorSpecGraph(), contextManager, clock); + StreamOperatorTask sot = new StreamOperatorTask(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager(), clock); sot.init(config, taskContext); return sot; } - private StreamGraphSpec getTestJoinStreamGraph(TestJoinFunction joinFn) throws IOException { - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - IntegerSerde integerSerde = new IntegerSerde(); - KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); - MessageStream> inStream = graphSpec.getInputStream("instream", kvSerde); - MessageStream> inStream2 = graphSpec.getInputStream("instream2", kvSerde); - - inStream - .join(inStream2, joinFn, integerSerde, kvSerde, kvSerde, JOIN_TTL, "j1") - .sink((message, messageCollector, taskCoordinator) -> { - SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); - messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); - }); - - return graphSpec; + private StreamAppDescriptorImpl getTestJoinStreamGraph(TestJoinFunction joinFn) throws IOException { + return new StreamAppDescriptorImpl(appDesc -> { + IntegerSerde integerSerde = new IntegerSerde(); + KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); + MessageStream> inStream = appDesc.getInputStream("instream", kvSerde); + MessageStream> inStream2 = appDesc.getInputStream("instream2", kvSerde); + + inStream + .join(inStream2, joinFn, integerSerde, kvSerde, kvSerde, JOIN_TTL, "j1") + .sink((message, messageCollector, taskCoordinator) -> { + SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); + messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); + }); + }, config); } private static class TestJoinFunction diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java index fff85e8f6f..090b032b68 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java @@ -24,6 +24,7 @@ import java.util.Collection; import java.util.Collections; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.data.TestMessageEnvelope; import org.apache.samza.operators.data.TestOutputMessageEnvelope; import org.apache.samza.operators.functions.FilterFunction; @@ -70,7 +71,7 @@ public class TestMessageStreamImpl { @Test public void testMap() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -95,7 +96,7 @@ public void testMap() { @Test public void testFlatMap() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -112,7 +113,7 @@ public void testFlatMap() { @Test public void testFlatMapWithRelaxedTypes() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -132,7 +133,7 @@ public void testFlatMapWithRelaxedTypes() { @Test public void testFilter() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -157,7 +158,7 @@ public void testFilter() { @Test public void testSink() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -174,7 +175,7 @@ public void testSink() { @Test public void testSendTo() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); OutputStreamImpl mockOutputStreamImpl = mock(OutputStreamImpl.class); @@ -200,7 +201,7 @@ public void testSendTo() { @Test public void testPartitionBy() throws IOException { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); String mockOpName = "mockName"; when(mockGraph.getNextOpId(anyObject(), anyObject())).thenReturn(mockOpName); @@ -231,7 +232,7 @@ public void testPartitionBy() throws IOException { @Test public void testRepartitionWithoutSerde() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); String mockOpName = "mockName"; when(mockGraph.getNextOpId(anyObject(), anyObject())).thenReturn(mockOpName); @@ -261,7 +262,7 @@ public void testRepartitionWithoutSerde() { @Test public void testWindowWithRelaxedTypes() throws Exception { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStream inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -285,7 +286,7 @@ public void testWindowWithRelaxedTypes() throws Exception { @Test public void testJoin() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec leftInputOpSpec = mock(OperatorSpec.class); MessageStreamImpl source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec); OperatorSpec rightInputOpSpec = mock(OperatorSpec.class); @@ -317,7 +318,7 @@ public void testJoin() { @Test public void testSendToTable() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec inputOpSpec = mock(OperatorSpec.class); MessageStreamImpl source = new MessageStreamImpl<>(mockGraph, inputOpSpec); @@ -339,7 +340,7 @@ public void testSendToTable() { @Test public void testStreamTableJoin() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec leftInputOpSpec = mock(OperatorSpec.class); MessageStreamImpl> source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec); OperatorSpec rightInputOpSpec = mock(OperatorSpec.class); @@ -367,7 +368,7 @@ public void testStreamTableJoin() { @Test public void testMerge() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockOpSpec1 = mock(OperatorSpec.class); MessageStream inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec1); @@ -407,7 +408,7 @@ public void testMerge() { @Test public void testMergeWithRelaxedTypes() { - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); MessageStream inputStream = new MessageStreamImpl<>(mockGraph, mock(OperatorSpec.class)); // other streams have the same message type T as input stream message type M diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java index 4cfc66a6ad..11d19e9983 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -57,14 +58,14 @@ @PrepareForTest(OperatorSpec.class) public class TestOperatorSpecGraph { - private StreamGraphSpec mockGraph; + private StreamAppDescriptorImpl mockGraph; private Map inputOpSpecMap; private Map outputStrmMap; private Set allOpSpecs; @Before public void setUp() { - this.mockGraph = mock(StreamGraphSpec.class); + this.mockGraph = mock(StreamAppDescriptorImpl.class); /** * Setup two linear transformation pipelines: @@ -113,7 +114,7 @@ public void tearDown() { @Test public void testConstructor() { - OperatorSpecGraph specGraph = new OperatorSpecGraph(mockGraph); + OperatorSpecGraph specGraph = OperatorSpecGraph.getInstance(mockGraph); assertEquals(specGraph.getInputOperators(), inputOpSpecMap); assertEquals(specGraph.getOutputStreams(), outputStrmMap); assertTrue(specGraph.getTables().isEmpty()); @@ -123,7 +124,7 @@ public void testConstructor() { @Test public void testClone() { - OperatorSpecGraph operatorSpecGraph = new OperatorSpecGraph(mockGraph); + OperatorSpecGraph operatorSpecGraph = OperatorSpecGraph.getInstance(mockGraph); OperatorSpecGraph clonedSpecGraph = operatorSpecGraph.clone(); OperatorSpecTestUtils.assertClonedGraph(operatorSpecGraph, clonedSpecGraph); } @@ -137,7 +138,7 @@ public void testCloneWithSerializationError() throws Throwable { //failed with serialization error try { - new OperatorSpecGraph(mockGraph); + OperatorSpecGraph.getInstance(mockGraph); fail("Should have failed with serialization error"); } catch (SamzaException se) { throw se.getCause(); @@ -150,7 +151,7 @@ public void testCloneWithDeserializationError() throws Throwable { this.allOpSpecs.add(testOp); inputOpSpecMap.values().stream().findFirst().get().registerNextOperatorSpec(testOp); - OperatorSpecGraph operatorSpecGraph = new OperatorSpecGraph(mockGraph); + OperatorSpecGraph operatorSpecGraph = OperatorSpecGraph.getInstance(mockGraph); //failed with serialization error try { operatorSpecGraph.clone(); diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index f24facb0cf..5a39007e5f 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -34,6 +34,7 @@ import java.util.function.BiFunction; import java.util.function.Function; import org.apache.samza.Partition; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -48,7 +49,6 @@ import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.ClosableFunction; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.InitableFunction; @@ -213,7 +213,7 @@ public void tearDown() { @Test public void testEmptyChain() { - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); OperatorImplGraph opGraph = new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mock(TaskContextImpl.class), mock(Clock.class)); assertEquals(0, opGraph.getAllInputOperators().size()); @@ -237,15 +237,16 @@ public void testLinearChain() { StreamTestUtils.addStreamConfigs(configs, outputStreamId, outputSystem, outputPhysicalName); Config config = new MapConfig(configs); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(inputStreamId); + OutputStream outputStream = appDesc.getOutputStream(outputStreamId); - MessageStream inputStream = graphSpec.getInputStream(inputStreamId); - OutputStream outputStream = graphSpec.getOutputStream(outputStreamId); + inputStream + .filter(mock(FilterFunction.class)) + .map(mock(MapFunction.class)) + .sendTo(outputStream); + }, config); - inputStream - .filter(mock(FilterFunction.class)) - .map(mock(MapFunction.class)) - .sendTo(outputStream); TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); @@ -288,15 +289,16 @@ public void testPartitionByChain() { StreamTestUtils.addStreamConfigs(configs, outputStreamId, outputSystem, outputPhysicalName); Config config = new MapConfig(configs); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream inputStream = graphSpec.getInputStream(inputStreamId); - OutputStream> outputStream = graphSpec - .getOutputStream(outputStreamId, KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class))); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(inputStreamId); + OutputStream> outputStream = appDesc + .getOutputStream(outputStreamId, KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class))); - inputStream - .partitionBy(Object::hashCode, Object::toString, - KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class)), "p1") - .sendTo(outputStream); + inputStream + .partitionBy(Object::hashCode, Object::toString, + KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class)), "p1") + .sendTo(outputStream); + }, config); TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); @@ -336,11 +338,11 @@ public void testBroadcastChain() { HashMap configMap = new HashMap<>(); StreamTestUtils.addStreamConfigs(configMap, "input", "input-system", "input-stream"); Config config = new MapConfig(configMap); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - - MessageStream inputStream = graphSpec.getInputStream(inputStreamId); - inputStream.filter(mock(FilterFunction.class)); - inputStream.map(mock(MapFunction.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(inputStreamId); + inputStream.filter(mock(FilterFunction.class)); + inputStream.map(mock(MapFunction.class)); + }, config); TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); @@ -358,21 +360,21 @@ public void testBroadcastChain() { @Test public void testMergeChain() { String inputStreamId = "input"; - StreamGraphSpec graphSpec = new StreamGraphSpec(mock(Config.class)); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(inputStreamId); + MessageStream stream1 = inputStream.filter(mock(FilterFunction.class)); + MessageStream stream2 = inputStream.map(mock(MapFunction.class)); + MessageStream mergedStream = stream1.merge(Collections.singleton(stream2)); + MapFunction testMapFunction = new TestMapFunction("test-map-1", (Function & Serializable) m -> m); + mergedStream.map(testMapFunction); + }, mock(Config.class)); - MessageStream inputStream = graphSpec.getInputStream(inputStreamId); - MessageStream stream1 = inputStream.filter(mock(FilterFunction.class)); - MessageStream stream2 = inputStream.map(mock(MapFunction.class)); - MessageStream mergedStream = stream1.merge(Collections.singleton(stream2)); TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); TaskName mockTaskName = mock(TaskName.class); when(mockTaskContext.getTaskName()).thenReturn(mockTaskName); when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); - MapFunction testMapFunction = new TestMapFunction("test-map-1", (Function & Serializable) m -> m); - mergedStream.map(testMapFunction); - OperatorImplGraph opImplGraph = new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mockTaskContext, mock(Clock.class)); @@ -399,16 +401,18 @@ public void testJoinChain() { StreamTestUtils.addStreamConfigs(configs, "input1", "input-system", "input-stream1"); StreamTestUtils.addStreamConfigs(configs, "input2", "input-system", "input-stream2"); Config config = new MapConfig(configs); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); Integer joinKey = new Integer(1); Function keyFn = (Function & Serializable) m -> joinKey; JoinFunction testJoinFunction = new TestJoinFunction("jobName-jobId-join-j1", (BiFunction & Serializable) (m1, m2) -> KV.of(m1, m2), keyFn, keyFn); - MessageStream inputStream1 = graphSpec.getInputStream(inputStreamId1, new NoOpSerde<>()); - MessageStream inputStream2 = graphSpec.getInputStream(inputStreamId2, new NoOpSerde<>()); - inputStream1.join(inputStream2, testJoinFunction, - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j1"); + + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream1 = appDesc.getInputStream(inputStreamId1, new NoOpSerde<>()); + MessageStream inputStream2 = appDesc.getInputStream(inputStreamId2, new NoOpSerde<>()); + inputStream1.join(inputStream2, testJoinFunction, + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j1"); + }, config); TaskName mockTaskName = mock(TaskName.class); TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); @@ -462,17 +466,17 @@ public void testOperatorGraphInitAndClose() { TaskContextImpl mockContext = mock(TaskContextImpl.class); when(mockContext.getTaskName()).thenReturn(mockTaskName); when(mockContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); - StreamGraphSpec graphSpec = new StreamGraphSpec(mockConfig); - - MessageStream inputStream1 = graphSpec.getInputStream(inputStreamId1); - MessageStream inputStream2 = graphSpec.getInputStream(inputStreamId2); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream1 = appDesc.getInputStream(inputStreamId1); + MessageStream inputStream2 = appDesc.getInputStream(inputStreamId2); - Function mapFn = (Function & Serializable) m -> m; - inputStream1.map(new TestMapFunction("1", mapFn)) - .map(new TestMapFunction("2", mapFn)); + Function mapFn = (Function & Serializable) m -> m; + inputStream1.map(new TestMapFunction("1", mapFn)) + .map(new TestMapFunction("2", mapFn)); - inputStream2.map(new TestMapFunction("3", mapFn)) - .map(new TestMapFunction("4", mapFn)); + inputStream2.map(new TestMapFunction("3", mapFn)) + .map(new TestMapFunction("4", mapFn)); + }, mockConfig); OperatorImplGraph opImplGraph = new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mockConfig, mockContext, SystemClock.instance()); @@ -555,26 +559,27 @@ public void testGetOutputToInputStreams() { StreamTestUtils.addStreamConfigs(configs, outputStreamId2, outputSystem, outputStreamId2); Config config = new MapConfig(configs); - StreamGraphSpec graphSpec = new StreamGraphSpec(config); - MessageStream messageStream1 = graphSpec.getInputStream(inputStreamId1).map(m -> m); - MessageStream messageStream2 = graphSpec.getInputStream(inputStreamId2).filter(m -> true); - MessageStream messageStream3 = - graphSpec.getInputStream(inputStreamId3) - .filter(m -> true) - .partitionBy(m -> "m", m -> m, "p1") - .map(m -> m); - OutputStream outputStream1 = graphSpec.getOutputStream(outputStreamId1); - OutputStream outputStream2 = graphSpec.getOutputStream(outputStreamId2); - - messageStream1 - .join(messageStream2, mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") - .partitionBy(m -> "m", m -> m, "p2") - .sendTo(outputStream1); - messageStream3 - .join(messageStream2, mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") - .sendTo(outputStream2); + StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream messageStream1 = appDesc.getInputStream(inputStreamId1).map(m -> m); + MessageStream messageStream2 = appDesc.getInputStream(inputStreamId2).filter(m -> true); + MessageStream messageStream3 = + appDesc.getInputStream(inputStreamId3) + .filter(m -> true) + .partitionBy(m -> "m", m -> m, "p1") + .map(m -> m); + OutputStream outputStream1 = appDesc.getOutputStream(outputStreamId1); + OutputStream outputStream2 = appDesc.getOutputStream(outputStreamId2); + + messageStream1 + .join(messageStream2, mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") + .partitionBy(m -> "m", m -> m, "p2") + .sendTo(outputStream1); + messageStream3 + .join(messageStream2, mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2") + .sendTo(outputStream2); + }, config); Multimap outputToInput = OperatorImplGraph.getIntermediateToInputStreamsMap(graphSpec.getOperatorSpecGraph(), new StreamConfig(config)); diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java index 8082b0507c..dd1792229b 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java @@ -30,6 +30,8 @@ import java.util.List; import java.util.Map; import org.apache.samza.Partition; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -39,7 +41,6 @@ import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.impl.store.TestInMemoryStore; import org.apache.samza.operators.impl.store.TimeSeriesKeySerde; @@ -539,68 +540,72 @@ public void testEndOfStreamFlushesWithNoTriggerFirings() throws Exception { verify(taskCoordinator, times(1)).shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); } - private StreamGraphSpec getKeyedTumblingWindowStreamGraph(AccumulationMode mode, + private StreamAppDescriptorImpl getKeyedTumblingWindowStreamGraph(AccumulationMode mode, Duration duration, Trigger> earlyTrigger) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(config); - - KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); - graph.getInputStream("integers", kvSerde) - .window(Windows.keyedTumblingWindow(KV::getKey, duration, new IntegerSerde(), kvSerde) - .setEarlyTrigger(earlyTrigger).setAccumulationMode(mode), "w1") - .sink((message, messageCollector, taskCoordinator) -> { - SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); - messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); - }); - - return graph; + + StreamApplication userApp = appDesc -> { + KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); + appDesc.getInputStream("integers", kvSerde) + .window(Windows.keyedTumblingWindow(KV::getKey, duration, new IntegerSerde(), kvSerde) + .setEarlyTrigger(earlyTrigger).setAccumulationMode(mode), "w1") + .sink((message, messageCollector, taskCoordinator) -> { + SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); + messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); + }); + }; + + return new StreamAppDescriptorImpl(userApp, config); } - private StreamGraphSpec getTumblingWindowStreamGraph(AccumulationMode mode, + private StreamAppDescriptorImpl getTumblingWindowStreamGraph(AccumulationMode mode, Duration duration, Trigger> earlyTrigger) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(config); - - KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); - graph.getInputStream("integers", kvSerde) - .window(Windows.tumblingWindow(duration, kvSerde).setEarlyTrigger(earlyTrigger) - .setAccumulationMode(mode), "w1") - .sink((message, messageCollector, taskCoordinator) -> { - SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); - messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); - }); - return graph; + StreamApplication userApp = appDesc -> { + KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); + appDesc.getInputStream("integers", kvSerde) + .window(Windows.tumblingWindow(duration, kvSerde).setEarlyTrigger(earlyTrigger) + .setAccumulationMode(mode), "w1") + .sink((message, messageCollector, taskCoordinator) -> { + SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); + messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); + }); + }; + + return new StreamAppDescriptorImpl(userApp, config); } - private StreamGraphSpec getKeyedSessionWindowStreamGraph(AccumulationMode mode, Duration duration) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(config); - - KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); - graph.getInputStream("integers", kvSerde) - .window(Windows.keyedSessionWindow(KV::getKey, duration, new IntegerSerde(), kvSerde) - .setAccumulationMode(mode), "w1") - .sink((message, messageCollector, taskCoordinator) -> { - SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); - messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); - }); - return graph; + private StreamAppDescriptorImpl getKeyedSessionWindowStreamGraph(AccumulationMode mode, Duration duration) throws IOException { + StreamApplication userApp = appDesc -> { + KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); + appDesc.getInputStream("integers", kvSerde) + .window(Windows.keyedSessionWindow(KV::getKey, duration, new IntegerSerde(), kvSerde) + .setAccumulationMode(mode), "w1") + .sink((message, messageCollector, taskCoordinator) -> { + SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); + messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); + }); + }; + + return new StreamAppDescriptorImpl(userApp, config); } - private StreamGraphSpec getAggregateTumblingWindowStreamGraph(AccumulationMode mode, Duration timeDuration, + private StreamAppDescriptorImpl getAggregateTumblingWindowStreamGraph(AccumulationMode mode, Duration timeDuration, Trigger earlyTrigger) throws IOException { - StreamGraphSpec graph = new StreamGraphSpec(config); - - MessageStream> integers = graph.getInputStream("integers", - KVSerde.of(new IntegerSerde(), new IntegerSerde())); - - integers - .map(new KVMapFunction()) - .window(Windows.tumblingWindow(timeDuration, () -> 0, (m, c) -> c + 1, new IntegerSerde()) - .setEarlyTrigger(earlyTrigger) - .setAccumulationMode(mode), "w1") - .sink((message, messageCollector, taskCoordinator) -> { - SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); - messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); - }); - return graph; + StreamApplication userApp = appDesc -> { + MessageStream> integers = appDesc.getInputStream("integers", + KVSerde.of(new IntegerSerde(), new IntegerSerde())); + + integers + .map(new KVMapFunction()) + .window(Windows.tumblingWindow(timeDuration, () -> 0, (m, c) -> c + 1, new IntegerSerde()) + .setEarlyTrigger(earlyTrigger) + .setAccumulationMode(mode), "w1") + .sink((message, messageCollector, taskCoordinator) -> { + SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream"); + messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message)); + }); + }; + + return new StreamAppDescriptorImpl(userApp, config); } private static class IntegerEnvelope extends IncomingMessageEnvelope { diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java index 9bbcbfaa01..a55b42e50e 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java @@ -19,12 +19,12 @@ package org.apache.samza.operators.spec; import java.util.Collection; +import java.util.Map; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.TimerFunction; @@ -32,7 +32,6 @@ import org.apache.samza.serializers.NoOpSerde; import org.junit.Before; import org.junit.Test; -import org.mockito.internal.util.reflection.Whitebox; import static org.junit.Assert.*; import static org.mockito.Mockito.*; @@ -48,7 +47,7 @@ public class TestPartitionByOperatorSpec { private final String testJobName = "testJob"; private final String testJobId = "1"; private final String testReparStreamName = "parByKey"; - private StreamGraphSpec graphSpec = null; + private StreamAppDescriptorImpl graphSpec = null; class TimerMapFn implements MapFunction, TimerFunction { @@ -90,24 +89,27 @@ public Long getOutputWatermark() { public void setup() { when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn(testJobName); when(mockConfig.get(JobConfig.JOB_ID(), "1")).thenReturn(testJobId); - graphSpec = new StreamGraphSpec(mockConfig); } @Test public void testPartitionBy() { - MessageStream inputStream = graphSpec.getInputStream(testInputId); MapFunction keyFn = m -> m.toString(); MapFunction valueFn = m -> m; - MessageStream> - reparStream = inputStream.partitionBy(keyFn, valueFn, testReparStreamName); - InputOperatorSpec inputOpSpec = (InputOperatorSpec) Whitebox.getInternalState(reparStream, "operatorSpec"); - assertEquals(inputOpSpec.getStreamId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); + graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(testInputId); + inputStream.partitionBy(keyFn, valueFn, testReparStreamName); + }, mockConfig); + assertEquals(2, graphSpec.getInputOperators().size()); + Map inputOpSpecs = graphSpec.getInputOperators(); + assertTrue(inputOpSpecs.keySet().contains(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName))); + InputOperatorSpec inputOpSpec = inputOpSpecs.get(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); + assertEquals(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName), inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde); assertTrue(inputOpSpec.isKeyed()); assertNull(inputOpSpec.getTimerFn()); assertNull(inputOpSpec.getWatermarkFn()); - InputOperatorSpec originInputSpec = (InputOperatorSpec) Whitebox.getInternalState(inputStream, "operatorSpec"); + InputOperatorSpec originInputSpec = inputOpSpecs.get(testInputId); assertTrue(originInputSpec.getRegisteredOperatorSpecs().toArray()[0] instanceof PartitionByOperatorSpec); PartitionByOperatorSpec reparOpSpec = (PartitionByOperatorSpec) originInputSpec.getRegisteredOperatorSpecs().toArray()[0]; assertEquals(reparOpSpec.getOpId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); @@ -120,8 +122,10 @@ public void testPartitionBy() { @Test public void testCopy() { - MessageStream inputStream = graphSpec.getInputStream(testInputId); - inputStream.partitionBy(m -> m.toString(), m -> m, testReparStreamName); + graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(testInputId); + inputStream.partitionBy(m -> m.toString(), m -> m, testReparStreamName); + }, mockConfig); OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); OperatorSpecGraph clonedGraph = specGraph.clone(); OperatorSpecTestUtils.assertClonedGraph(specGraph, clonedGraph); @@ -130,28 +134,36 @@ public void testCopy() { @Test(expected = IllegalArgumentException.class) public void testTimerFunctionAsKeyFn() { TimerMapFn keyFn = new TimerMapFn(); - MessageStream inputStream = graphSpec.getInputStream(testInputId); - inputStream.partitionBy(keyFn, m -> m, "parByKey"); + graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(testInputId); + inputStream.partitionBy(keyFn, m -> m, "parByKey"); + }, mockConfig); } @Test(expected = IllegalArgumentException.class) public void testWatermarkFunctionAsKeyFn() { WatermarkMapFn keyFn = new WatermarkMapFn(); - MessageStream inputStream = graphSpec.getInputStream(testInputId); - inputStream.partitionBy(keyFn, m -> m, "parByKey"); + graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(testInputId); + inputStream.partitionBy(keyFn, m -> m, "parByKey"); + }, mockConfig); } @Test(expected = IllegalArgumentException.class) public void testTimerFunctionAsValueFn() { TimerMapFn valueFn = new TimerMapFn(); - MessageStream inputStream = graphSpec.getInputStream(testInputId); - inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); + graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(testInputId); + inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); + }, mockConfig); } @Test(expected = IllegalArgumentException.class) public void testWatermarkFunctionAsValueFn() { WatermarkMapFn valueFn = new WatermarkMapFn(); - MessageStream inputStream = graphSpec.getInputStream(testInputId); - inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); + graphSpec = new StreamAppDescriptorImpl(appDesc -> { + MessageStream inputStream = appDesc.getInputStream(testInputId); + inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); + }, mockConfig); } } diff --git a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java index 88fb3f1935..338ef0efdf 100644 --- a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java +++ b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java @@ -53,7 +53,7 @@ public class TestStreamProcessor { private ConcurrentMap processorListenerState; private enum ListenerCallback { - BEFORE_START, AFTER_START, BEFORE_STOP, AFTER_STOP, AFTER_STOP_WITH_FAILURE + BEFORE_START, AFTER_START, AFTER_STOP, AFTER_FAILURE } @Before @@ -63,9 +63,8 @@ public void before() { { put(ListenerCallback.BEFORE_START, false); put(ListenerCallback.AFTER_START, false); - put(ListenerCallback.BEFORE_STOP, false); put(ListenerCallback.AFTER_STOP, false); - put(ListenerCallback.AFTER_STOP_WITH_FAILURE, false); + put(ListenerCallback.AFTER_FAILURE, false); } }; } @@ -153,18 +152,14 @@ public void afterStart() { } @Override - public void afterStop(Throwable t) { - if (t != null) { - processorListenerState.put(ListenerCallback.AFTER_STOP_WITH_FAILURE, true); - } else { - processorListenerState.put(ListenerCallback.AFTER_STOP, true); - processorListenerStop.countDown(); - } + public void afterFailure(Throwable t) { + processorListenerState.put(ListenerCallback.AFTER_FAILURE, true); } @Override - public void beforeStop() { - processorListenerState.put(ListenerCallback.BEFORE_STOP, true); + public void afterStop() { + processorListenerState.put(ListenerCallback.AFTER_STOP, true); + processorListenerStop.countDown(); } @Override @@ -216,9 +211,8 @@ public void beforeStart() { // Assertions on which callbacks are expected to be invoked Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_START)); Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_START)); - Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_STOP)); Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_STOP)); - Assert.assertFalse(processorListenerState.get(ListenerCallback.AFTER_STOP_WITH_FAILURE)); + Assert.assertFalse(processorListenerState.get(ListenerCallback.AFTER_FAILURE)); } /** @@ -266,20 +260,16 @@ public void afterStart() { } @Override - public void beforeStop() { - processorListenerState.put(ListenerCallback.BEFORE_STOP, true); + public void afterStop() { + // successful stop + processorListenerState.put(ListenerCallback.AFTER_STOP, true); } @Override - public void afterStop(Throwable t) { - if (t == null) { - // successful stop - processorListenerState.put(ListenerCallback.AFTER_STOP, true); - } else { - processorListenerState.put(ListenerCallback.AFTER_STOP_WITH_FAILURE, true); - actualThrowable.getAndSet(t); - processorListenerFailed.countDown(); - } + public void afterFailure(Throwable t) { + processorListenerState.put(ListenerCallback.AFTER_FAILURE, true); + actualThrowable.getAndSet(t); + processorListenerFailed.countDown(); } }, mockJobCoordinator, @@ -320,9 +310,8 @@ public void afterStop(Throwable t) { Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_START)); Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_START)); - Assert.assertFalse(processorListenerState.get(ListenerCallback.BEFORE_STOP)); Assert.assertFalse(processorListenerState.get(ListenerCallback.AFTER_STOP)); - Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_STOP_WITH_FAILURE)); + Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_FAILURE)); } @Test @@ -450,7 +439,7 @@ public void testCoordinatorFailureShouldStopTheStreamProcessor() { Assert.assertEquals(State.STOPPED, streamProcessor.state); - Mockito.verify(lifecycleListener).afterStop(failureException); + Mockito.verify(lifecycleListener).afterFailure(failureException); Mockito.verify(mockSamzaContainer).shutdown(); } @@ -465,6 +454,6 @@ public void testCoordinatorStopShouldStopTheStreamProcessor() { streamProcessor.jobCoordinatorListener.onCoordinatorStop(); Assert.assertEquals(State.STOPPED, streamProcessor.state); - Mockito.verify(lifecycleListener).afterStop(null); + Mockito.verify(lifecycleListener).afterStop(); } } diff --git a/samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java similarity index 90% rename from samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java rename to samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java index 4b7f9abc89..69d5280fd1 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java @@ -16,11 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.application; +package org.apache.samza.runtime; import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.TaskAppDescriptorImpl; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.TestStreamApplication; +import org.apache.samza.application.TestTaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index caed583b60..902566aa62 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -19,12 +19,14 @@ package org.apache.samza.runtime; import java.time.Duration; +import java.util.List; +import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.job.ApplicationStatus; import org.junit.Test; @@ -93,54 +95,39 @@ public static class TestApplicationRunnerInvocationCounts extends AbstractApplic protected static int killCount = 0; protected static int statusCount = 0; - public TestApplicationRunnerInvocationCounts(AppDescriptorImpl appDesc) { - super(appDesc); + public TestApplicationRunnerInvocationCounts(ApplicationBase userApp, Config config) { + super(userApp, config); } - private void countRun() { + @Override + protected List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc) { + return null; + } + + @Override + public void run() { runCount++; } - private void countKill() { + @Override + public void kill() { killCount++; } - private ApplicationStatus countStatus() { + @Override + public ApplicationStatus status() { statusCount++; return ApplicationStatus.Running; } - class TestAppExecutable implements AppRuntimeExecutable { - - @Override - public void run() { - countRun(); - } - - @Override - public void kill() { - countKill(); - } - - @Override - public ApplicationStatus status() { - return countStatus(); - } - - @Override - public boolean waitForFinish(Duration timeout) { - return false; - } - } - @Override - AppRuntimeExecutable getTaskAppRuntimeExecutable(TaskAppDescriptorImpl appSpec) { - return new TestAppExecutable(); + public void waitForFinish() { + waitForFinish(Duration.ofSeconds(0)); } @Override - AppRuntimeExecutable getStreamAppRuntimeExecutable(StreamAppDescriptorImpl appSpec) { - return new TestAppExecutable(); + public boolean waitForFinish(Duration timeout) { + return false; } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 7b0382f439..079e7b36ad 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -27,10 +27,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.ApplicationClassUtils; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -43,11 +40,8 @@ import org.apache.samza.execution.ExecutionPlan; import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.system.StreamSpec; -import org.apache.samza.task.TaskFactory; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -55,7 +49,6 @@ import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; import static org.junit.Assert.*; import static org.mockito.Matchers.anyObject; @@ -87,15 +80,8 @@ public class TestLocalApplicationRunner { @Before public void setUp() { Map config = new HashMap<>(); - StreamGraphSpec mockGraphSpec = mock(StreamGraphSpec.class); - OperatorSpecGraph mockOpSpecGraph = mock(OperatorSpecGraph.class); - StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); - when(appDesc.getConfig()).thenReturn(new MapConfig(config)); - when(mockGraphSpec.getOperatorSpecGraph()).thenReturn(mockOpSpecGraph); - when(appDesc.getGraph()).thenReturn(mockGraphSpec); - runner = spy(new LocalApplicationRunner(appDesc)); - AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); - Whitebox.setInternalState(runner, "appExecutable", appExecutable); + StreamApplication mockApp = mock(StreamApplication.class); + runner = spy(new LocalApplicationRunner(mockApp, new MapConfig(config))); } @Test @@ -174,31 +160,24 @@ public void testRunStreamTask() final Map config = new HashMap<>(); config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); - Config samzaConfig = new MapConfig(config); - TaskAppDescriptorImpl - appDesc = new TaskAppDescriptorImpl((TaskApplication) ApplicationClassUtils.fromConfig(samzaConfig), samzaConfig); - runner = spy(new LocalApplicationRunner(appDesc)); - LocalApplicationRunner.TaskAppExecutable taskAppExecutable = - spy((LocalApplicationRunner.TaskAppExecutable) runner.getTaskAppRuntimeExecutable(appDesc)); - Whitebox.setInternalState(runner, "appExecutable", taskAppExecutable); + runner = spy(new LocalApplicationRunner(ApplicationClassUtils.fromConfig(samzaConfig), samzaConfig)); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor1 = - ArgumentCaptor.forClass(TaskFactory.class); - ArgumentCaptor captor2 = - ArgumentCaptor.forClass(ProcessorLifecycleListener.class); + + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); doAnswer(i -> - { - ProcessorLifecycleListener listener = captor2.getValue(); - listener.afterStart(); - listener.afterStop(null); - return null; - }).when(sp).start(); + { + ProcessorLifecycleListener listener = captor.getValue().apply(sp); + listener.afterStart(); + listener.afterStop(); + return null; + }).when(sp).start(); - doReturn(sp).when(runner).createStreamProcessor(any(Config.class), captor1.capture(), captor2.capture()); - doReturn(ApplicationStatus.SuccessfulFinish).when(taskAppExecutable).status(); + doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); + doReturn(ApplicationStatus.SuccessfulFinish).when(runner).status(); runner.run(); @@ -210,17 +189,11 @@ public void testRunComplete() throws Exception { Map config = new HashMap<>(); config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); - StreamGraphSpec mockGraphSpec = mock(StreamGraphSpec.class); - OperatorSpecGraph mockOpSpecGraph = mock(OperatorSpecGraph.class); ProcessorLifecycleListenerFactory mockFactory = (pContext, cfg) -> mock(ProcessorLifecycleListener.class); - StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); - when(appDesc.getConfig()).thenReturn(new MapConfig(config)); - when(mockGraphSpec.getOperatorSpecGraph()).thenReturn(mockOpSpecGraph); - when(appDesc.getGraph()).thenReturn(mockGraphSpec); - when(appDesc.getProcessorLifecycleListenerFactory()).thenReturn(mockFactory); - runner = spy(new LocalApplicationRunner(appDesc)); - AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); - Whitebox.setInternalState(runner, "appExecutable", appExecutable); + StreamApplication userApp = appDesc -> { + appDesc.withProcessorLifecycleListenerFactory(mockFactory); + }; + runner = spy(new LocalApplicationRunner(userApp, new MapConfig(config))); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -232,18 +205,18 @@ public void testRunComplete() doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(ProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); doAnswer(i -> { - ProcessorLifecycleListener listener = captor.getValue(); + ProcessorLifecycleListener listener = captor.getValue().apply(sp); listener.afterStart(); - listener.afterStop(null); + listener.afterStop(); return null; }).when(sp).start(); - doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), anyObject(), captor.capture()); + doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); runner.run(); runner.waitForFinish(); @@ -257,17 +230,11 @@ public void testRunFailure() throws Exception { Map config = new HashMap<>(); config.put(ApplicationConfig.PROCESSOR_ID, "0"); - StreamGraphSpec mockGraphSpec = mock(StreamGraphSpec.class); - OperatorSpecGraph mockOpSpecGraph = mock(OperatorSpecGraph.class); ProcessorLifecycleListenerFactory mockFactory = (pContext, cfg) -> mock(ProcessorLifecycleListener.class); - StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); - when(appDesc.getConfig()).thenReturn(new MapConfig(config)); - when(mockGraphSpec.getOperatorSpecGraph()).thenReturn(mockOpSpecGraph); - when(appDesc.getGraph()).thenReturn(mockGraphSpec); - when(appDesc.getProcessorLifecycleListenerFactory()).thenReturn(mockFactory); - runner = spy(new LocalApplicationRunner(appDesc)); - AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); - Whitebox.setInternalState(runner, "appExecutable", appExecutable); + StreamApplication userApp = appDesc -> { + appDesc.withProcessorLifecycleListenerFactory(mockFactory); + }; + runner = spy(new LocalApplicationRunner(userApp, new MapConfig(config))); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -279,15 +246,15 @@ public void testRunFailure() doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(ProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); doAnswer(i -> { throw new Exception("test failure"); }).when(sp).start(); - doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), anyObject(), captor.capture()); + doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); try { runner.run(); diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java index 60b6a431a5..eb0c5e16c4 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java @@ -22,7 +22,7 @@ import java.time.Duration; import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -35,7 +35,6 @@ import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; import static org.junit.Assert.*; import static org.mockito.Matchers.*; @@ -54,11 +53,8 @@ public class TestRemoteApplicationRunner { @Before public void setUp() { Map config = new HashMap<>(); - StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); - when(appDesc.getConfig()).thenReturn(new MapConfig(config)); - runner = spy(new RemoteApplicationRunner(appDesc)); - AbstractApplicationRunner.AppRuntimeExecutable appExecutable = runner.getStreamAppRuntimeExecutable(appDesc); - Whitebox.setInternalState(runner, "appExecutable", appExecutable); + StreamApplication userApp = appDesc -> { }; + runner = spy(new RemoteApplicationRunner(userApp, new MapConfig(config))); } @Test @@ -83,15 +79,13 @@ public void testGetStatus() { m.put(JobConfig.JOB_ID(), "newJob"); - StreamAppDescriptorImpl appDesc = mock(StreamAppDescriptorImpl.class); - when(appDesc.getConfig()).thenReturn(new MapConfig(m)); - runner = spy(new RemoteApplicationRunner(appDesc)); + StreamApplication userApp = appDesc -> { }; + runner = spy(new RemoteApplicationRunner(userApp, new MapConfig(m))); Assert.assertEquals(ApplicationStatus.New, runner.getApplicationStatus(new JobConfig(new MapConfig(m)))); m.put(JobConfig.JOB_ID(), "runningJob"); - when(appDesc.getConfig()).thenReturn(new MapConfig(m)); - runner = spy(new RemoteApplicationRunner(appDesc)); + runner = spy(new RemoteApplicationRunner(userApp, new MapConfig(m))); Assert.assertEquals(ApplicationStatus.Running, runner.getApplicationStatus(new JobConfig(new MapConfig(m)))); } diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala index 6abb40e18d..09f410f9da 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala @@ -187,8 +187,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null - @volatile var beforeStartCalled = false - @volatile var beforeStopCalled = false + @volatile var onContainerBeforeStartCalled = false val container = new SamzaContainer( containerContext = containerContext, @@ -213,21 +212,17 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { onContainerStartCalled = true } - override def beforeStop(): Unit = { - beforeStopCalled = true - } - override def beforeStart(): Unit = { - beforeStartCalled = true + onContainerBeforeStartCalled = true } + } container.setContainerListener(containerListener) container.run assertTrue(task.wasShutdown) - assertTrue(beforeStartCalled) + assertTrue(onContainerBeforeStartCalled) assertFalse(onContainerStartCalled) - assertTrue(beforeStopCalled) assertFalse(onContainerStopCalled) assertTrue(onContainerFailedCalled) @@ -278,8 +273,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null - @volatile var beforeStartCalled = false - @volatile var beforeStopCalled = false + @volatile var onContainerBeforeStartCalled = false val mockRunLoop = mock[RunLoop] when(mockRunLoop.run).thenThrow(new RuntimeException("Trigger a shutdown, please.")) @@ -306,22 +300,20 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { onContainerStartCalled = true } - override def beforeStop(): Unit = { - beforeStopCalled = true - } - + /** + * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started + */ override def beforeStart(): Unit = { - beforeStartCalled = true + onContainerBeforeStartCalled = true } } container.setContainerListener(containerListener) container.run assertTrue(task.wasShutdown) - assertTrue(beforeStartCalled) + assertTrue(onContainerBeforeStartCalled) assertTrue(onContainerStartCalled) - assertTrue(beforeStopCalled) assertFalse(onContainerStopCalled) assertTrue(onContainerFailedCalled) @@ -376,8 +368,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null - @volatile var beforeStartCalled = false - @volatile var beforeStopCalled = false + @volatile var onContainerBeforeStartCalled = false val container = new SamzaContainer( containerContext = containerContext, @@ -401,12 +392,11 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { onContainerStartCalled = true } - override def beforeStop(): Unit = { - beforeStopCalled = true - } - + /** + * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started + */ override def beforeStart(): Unit = { - beforeStartCalled = true + onContainerBeforeStartCalled = true } } container.setContainerListener(containerListener) @@ -414,12 +404,10 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { container.run assertTrue(task.wasShutdown) - - assertTrue(beforeStartCalled) + assertTrue(onContainerBeforeStartCalled) assertFalse(onContainerStopCalled) assertFalse(onContainerStartCalled) - assertTrue(beforeStopCalled) assertTrue(onContainerFailedCalled) assertNotNull(onContainerFailedThrowable) } @@ -465,8 +453,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null - @volatile var beforeStartCalled = false - @volatile var beforeStopCalled = false + @volatile var onContainerBeforeStartCalled = false val mockRunLoop = mock[RunLoop] when(mockRunLoop.run).thenAnswer(new Answer[Unit] { @@ -497,21 +484,19 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { onContainerStartCalled = true } - override def beforeStop(): Unit = { - beforeStopCalled = true - } - + /** + * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started + */ override def beforeStart(): Unit = { - beforeStartCalled = true + onContainerBeforeStartCalled = true } } container.setContainerListener(containerListener) container.run - assertTrue(beforeStartCalled) + assertTrue(onContainerBeforeStartCalled) assertFalse(onContainerFailedCalled) assertTrue(onContainerStartCalled) - assertTrue(beforeStopCalled) assertTrue(onContainerStopCalled) } @@ -555,8 +540,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { @volatile var onContainerStopCalled = false @volatile var onContainerStartCalled = false @volatile var onContainerFailedThrowable: Throwable = null - @volatile var beforeStartCalled = false - @volatile var beforeStopCalled = false + @volatile var onContainerBeforeStartCalled = false val mockRunLoop = mock[RunLoop] when(mockRunLoop.run).thenAnswer(new Answer[Unit] { @@ -588,21 +572,19 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { onContainerStartCalled = true } - override def beforeStop(): Unit = { - beforeStopCalled = true - } - + /** + * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started + */ override def beforeStart(): Unit = { - beforeStartCalled = true + onContainerBeforeStartCalled = true } } container.setContainerListener(containerListener) container.run - assertTrue(beforeStartCalled) + assertTrue(onContainerBeforeStartCalled) assertTrue(onContainerStartCalled) - assertTrue(beforeStopCalled) assertTrue(onContainerFailedCalled) assertFalse(onContainerStopCalled) } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java index 6df3421e09..5aaad260a7 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java @@ -283,7 +283,7 @@ private Table loadLocalTable(boolean isTablePosOnRight, List tableKeyId // Create a table backed by RocksDb store with the fields in the join condition as composite key and relational // message as the value. Send the messages from the input stream denoted as 'table' to the created table store. Table> table = - context.getStreamGraph().getTable(sourceConfig.getTableDescriptor().get()); + context.getStreamAppDescriptor().getTable(sourceConfig.getTableDescriptor().get()); relOutputStream .map(m -> new KV(createSamzaSqlCompositeKey(m, tableKeyIds), m)) diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index cd220cbaf0..77a6bf0daf 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -31,7 +31,7 @@ import org.apache.calcite.rel.logical.LogicalProject; import org.apache.samza.SamzaException; import org.apache.samza.application.StreamAppDescriptor; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; @@ -54,7 +54,7 @@ /** - * This class is used to populate the StreamGraph using the SQL queries. + * This class is used to populate the {@link StreamAppDescriptor} using the SQL queries. * This class contains the core of the SamzaSQL control code that converts the SQL statements to calcite relational graph. * It then walks the relational graph and then populates the Samza's {@link StreamAppDescriptor} accordingly. */ @@ -98,7 +98,7 @@ public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamAppDescript sqlConfig.getUdfMetadata()); final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig); final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery()); - final TranslatorContext context = new TranslatorContext(((StreamAppDescriptorImpl)appDesc).getGraph(), relRoot, executionContext, this.converters); + final TranslatorContext context = new TranslatorContext((StreamAppDescriptorImpl)appDesc, relRoot, executionContext, this.converters); final RelNode node = relRoot.project(); final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver(); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java index cfa4185060..2e0933f4e2 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java @@ -23,10 +23,10 @@ import java.util.Map; import org.apache.calcite.rel.core.TableScan; import org.apache.commons.lang.Validate; +import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.sql.data.SamzaSqlRelMessage; import org.apache.samza.sql.interfaces.SamzaRelConverter; @@ -69,14 +69,14 @@ public SamzaSqlRelMessage apply(KV message) { } void translate(final TableScan tableScan, final TranslatorContext context) { - StreamGraph streamGraph = context.getStreamGraph(); + StreamAppDescriptor streamAppDesc = context.getStreamAppDescriptor(); List tableNameParts = tableScan.getTable().getQualifiedName(); String sourceName = SqlIOConfig.getSourceFromSourceParts(tableNameParts); Validate.isTrue(relMsgConverters.containsKey(sourceName), String.format("Unknown source %s", sourceName)); final String streamName = systemStreamConfig.get(sourceName).getStreamName(); - MessageStream> inputStream = streamGraph.getInputStream(streamName); + MessageStream> inputStream = streamAppDesc.getInputStream(streamName); MessageStream samzaSqlRelMessageStream = inputStream.map(new ScanMapFunction(sourceName)); context.registerMessageStream(tableScan.getId(), samzaSqlRelMessageStream); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java index 9399fd656f..88c93b544d 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java @@ -32,8 +32,8 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.SchemaPlus; import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StreamGraph; import org.apache.samza.sql.data.RexToJavaCompiler; import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.sql.interfaces.SamzaRelConverter; @@ -46,7 +46,7 @@ public class TranslatorContext implements Cloneable { /** * The internal variables that are shared among all cloned {@link TranslatorContext} */ - private final StreamGraph streamGraph; + private final StreamAppDescriptorImpl streamAppDesc; private final RexToJavaCompiler compiler; private final Map relSamzaConverters; private final Map messsageStreams; @@ -112,7 +112,7 @@ private RexToJavaCompiler createExpressionCompiler(RelRoot relRoot) { * @param other the original object to copy from */ private TranslatorContext(TranslatorContext other) { - this.streamGraph = other.streamGraph; + this.streamAppDesc = other.streamAppDesc; this.compiler = other.compiler; this.relSamzaConverters = other.relSamzaConverters; this.messsageStreams = other.messsageStreams; @@ -123,13 +123,13 @@ private TranslatorContext(TranslatorContext other) { /** * Create the instance of TranslatorContext - * @param streamGraph Samza's streamGraph that is populated during the translation. + * @param stramAppDesc Samza's streamGraph that is populated during the translation. * @param relRoot Root of the relational graph from calcite. * @param executionContext the execution context * @param converters the map of schema to RelData converters */ - TranslatorContext(StreamGraph streamGraph, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map converters) { - this.streamGraph = streamGraph; + TranslatorContext(StreamAppDescriptorImpl stramAppDesc, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map converters) { + this.streamAppDesc = stramAppDesc; this.compiler = createExpressionCompiler(relRoot); this.executionContext = executionContext; this.dataContext = new DataContextImpl(); @@ -143,8 +143,8 @@ private TranslatorContext(TranslatorContext other) { * * @return the stream graph */ - public StreamGraph getStreamGraph() { - return streamGraph; + public StreamAppDescriptor getStreamAppDescriptor() { + return streamAppDesc; } /** diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java index ea67100d03..2a9c06c6d4 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java @@ -25,12 +25,12 @@ import org.apache.calcite.DataContext; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; @@ -73,7 +73,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { when(mockFilter.getInput()).thenReturn(mockInput); when(mockInput.getId()).thenReturn(1); when(mockFilter.getId()).thenReturn(2); - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockInputOp = mock(OperatorSpec.class); MessageStream mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp); when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java index 2de4856586..4a6cca95a8 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java @@ -33,9 +33,9 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.functions.StreamTableJoinFunction; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -132,14 +132,14 @@ public void testTranslateStreamToTableJoin() throws IOException, ClassNotFoundEx when(mockRightInput.getRowType()).thenReturn(mockRightRowType); when(mockRightRowType.getFieldNames()).thenReturn(rightStreamFieldNames); - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockLeftInputOp = mock(OperatorSpec.class); MessageStream mockLeftInputStream = new MessageStreamImpl<>(mockGraph, mockLeftInputOp); when(mockContext.getMessageStream(eq(mockLeftInput.getId()))).thenReturn(mockLeftInputStream); OperatorSpec mockRightInputOp = mock(OperatorSpec.class); MessageStream mockRightInputStream = new MessageStreamImpl<>(mockGraph, mockRightInputOp); when(mockContext.getMessageStream(eq(mockRightInput.getId()))).thenReturn(mockRightInputStream); - when(mockContext.getStreamGraph()).thenReturn(mockGraph); + when(mockContext.getStreamAppDescriptor()).thenReturn(mockGraph); InputOperatorSpec mockInputOp = mock(InputOperatorSpec.class); OutputStreamImpl mockOutputStream = mock(OutputStreamImpl.class); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java index eb23b46c0c..24f963ecc0 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java @@ -32,12 +32,12 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.validate.SqlUserDefinedFunction; import org.apache.calcite.util.Pair; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.StreamGraphSpec; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; @@ -91,7 +91,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { List> namedProjects = new ArrayList<>(); namedProjects.add(Pair.of(mockRexField, "test_field")); when(mockProject.getNamedProjects()).thenReturn(namedProjects); - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockInputOp = mock(OperatorSpec.class); MessageStream mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp); when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream); @@ -183,7 +183,7 @@ public void testTranslateWithFlatten() throws IOException, ClassNotFoundExceptio flattenProjects.add(mockFlattenProject); when(mockProject.getProjects()).thenReturn(flattenProjects); - StreamGraphSpec mockGraph = mock(StreamGraphSpec.class); + StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); OperatorSpec mockInputOp = new OperatorSpec(OperatorSpec.OpCode.INPUT, "1") { @Override diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index 0a7e4bba9c..e31e2f21a6 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -24,16 +24,13 @@ import java.util.HashSet; import java.util.Map; import org.apache.samza.SamzaException; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; -import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.StreamGraphSpec; -import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.sql.data.SamzaSqlExecutionContext; import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory; @@ -44,7 +41,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.ArgumentCaptor; import org.mockito.internal.util.reflection.Whitebox; import static org.mockito.Mockito.*; @@ -56,7 +52,7 @@ public class TestQueryTranslator { private void validateClonedTranslatorContext(TranslatorContext originContext, TranslatorContext clonedContext) { Assert.assertNotEquals(originContext, clonedContext); Assert.assertTrue(originContext.getExpressionCompiler() == clonedContext.getExpressionCompiler()); - Assert.assertTrue(originContext.getStreamGraph() == clonedContext.getStreamGraph()); + Assert.assertTrue(originContext.getStreamAppDescriptor() == clonedContext.getStreamAppDescriptor()); Assert.assertTrue(originContext.getExpressionCompiler() == clonedContext.getExpressionCompiler()); Assert.assertTrue(Whitebox.getInternalState(originContext, "relSamzaConverters") == Whitebox.getInternalState(clonedContext, "relSamzaConverters")); Assert.assertTrue(Whitebox.getInternalState(originContext, "messsageStreams") == Whitebox.getInternalState(clonedContext, "messsageStreams")); @@ -92,24 +88,10 @@ public void testTranslate() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl appDesc = spy(new StreamAppDescriptorImpl(descriptor -> { },samzaConfig)); - translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = ((StreamGraphSpec) mockSpec.getGraph()).getOperatorSpecGraph(); + translator.translate(queryInfo, appDesc); + OperatorSpecGraph specGraph = appDesc.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -127,7 +109,7 @@ public void testTranslate() { Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("SIMPLE1", inputPhysicalName); - validatePerTaskContextInit(mockSpec, samzaConfig); + validatePerTaskContextInit(appDesc, samzaConfig); } private void validatePerTaskContextInit(StreamAppDescriptorImpl graphSpec, Config samzaConfig) { @@ -158,25 +140,10 @@ public void testTranslateComplex() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -205,25 +172,10 @@ public void testTranslateSubQuery() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -256,22 +208,7 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -290,22 +227,7 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -324,22 +246,7 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -358,22 +265,7 @@ public void testTranslateStreamTableJoinWithThetaCondition() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -389,22 +281,7 @@ public void testTranslateStreamTableCrossJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -422,22 +299,7 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -456,22 +318,7 @@ public void testTranslateStreamTableJoinWithSubQuery() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -489,22 +336,7 @@ public void testTranslateTableTableJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -522,22 +354,7 @@ public void testTranslateStreamStreamJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -555,22 +372,7 @@ public void testTranslateJoinWithIncorrectLeftJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -588,22 +390,7 @@ public void testTranslateJoinWithIncorrectRightJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -625,22 +412,7 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -658,22 +430,7 @@ public void testTranslateStreamTableInnerJoinWithUdf() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } @@ -691,25 +448,10 @@ public void testTranslateStreamTableInnerJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -759,26 +501,11 @@ public void testTranslateStreamTableLeftJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -828,26 +555,10 @@ public void testTranslateStreamTableRightJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -897,25 +608,10 @@ public void testTranslateGroupBy() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); Assert.assertEquals(1, specGraph.getInputOperators().size()); Assert.assertEquals(1, specGraph.getOutputStreams().size()); @@ -936,22 +632,7 @@ public void testTranslateGroupByWithSumAggregator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamGraphSpec - graphSpec = new StreamGraphSpec(samzaConfig); - StreamAppDescriptorImpl mockSpec = mock(StreamAppDescriptorImpl.class); - // The following steps are used to simulate the actual logic in the constructor - when(mockSpec.getConfig()).thenReturn(samzaConfig); - when(mockSpec.getGraph()).thenReturn(graphSpec); - ArgumentCaptor streamCaptor = ArgumentCaptor.forClass(String.class); - when(mockSpec.getOutputStream(streamCaptor.capture())).then( - invocation -> graphSpec.getOutputStream(streamCaptor.getValue())); - ArgumentCaptor tableCaptor = ArgumentCaptor.forClass(TableDescriptor.class); - when(mockSpec.getTable(tableCaptor.capture())).then(invocation -> graphSpec.getTable(tableCaptor.getValue())); - ArgumentCaptor contextManagerCaptor = ArgumentCaptor.forClass(ContextManager.class); - when(mockSpec.withContextManager(contextManagerCaptor.capture())).then(invocation -> { - when(mockSpec.getContextManager()).thenReturn(contextManagerCaptor.getValue()); - return null; - }); + StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); translator.translate(queryInfo, mockSpec); } } 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 8997c78286..0d4683a027 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 @@ -18,15 +18,16 @@ */ package org.apache.samza.example; -import java.util.Collections; -import org.apache.samza.application.TaskApplication; import org.apache.samza.application.TaskAppDescriptor; +import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.runtime.ApplicationClassUtils; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.storage.kv.RocksDbTableDescriptor; -import org.apache.samza.task.TaskFactoryUtil; +import org.apache.samza.task.StreamTask; +import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.util.CommandLine; @@ -38,7 +39,9 @@ public class TaskApplicationExample implements TaskApplication { public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TaskApplicationExample(), config); + //ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TaskApplicationExample(), config); + // ==> shouldn't be in user code example + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); runner.run(); runner.waitForFinish(); } @@ -46,12 +49,13 @@ public static void main(String[] args) { @Override public void describe(TaskAppDescriptor appDesc) { // add input and output streams - appDesc.addInputStreams(Collections.singletonList("myinput")); - appDesc.addOutputStreams(Collections.singletonList("myoutput")); + appDesc.addInputStream("myinput"); + appDesc.addOutputStream("myoutput"); TableDescriptor td = new RocksDbTableDescriptor("mytable"); - appDesc.addTables(Collections.singletonList(td)); + appDesc.addTable(td); // create the task factory based on configuration - appDesc.setTaskFactory(TaskFactoryUtil.createTaskFactory(appDesc.getConfig())); + appDesc.setTaskFactory((StreamTaskFactory) () -> (StreamTask) (envelope, collector, coordinator) -> { + }); } } diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index 887d8416f7..9824065319 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -31,10 +31,9 @@ import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.AppDescriptorImpl; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; -import org.apache.samza.application.internal.TaskAppDescriptorImpl; +import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; import org.apache.samza.config.InMemorySystemConfig; import org.apache.samza.config.JobConfig; @@ -291,10 +290,8 @@ public void run(Duration timeout) { Preconditions.checkState((app == null && taskClass != null) || (app != null && taskClass == null), "TestRunner should run for Low Level Task api or High Level Application Api"); Preconditions.checkState(!timeout.isZero() || !timeout.isNegative(), "Timeouts should be positive"); - AppDescriptorImpl appDesc = app == null ? - new TaskAppDescriptorImpl(spec -> spec.setTaskFactory(createTaskFactory()), new MapConfig(configs)) : - new StreamAppDescriptorImpl(app, new MapConfig(configs)); - final LocalApplicationRunner runner = new LocalApplicationRunner(appDesc); + ApplicationBase testApp = app == null ? (TaskApplication) appDesc -> appDesc.setTaskFactory(createTaskFactory()) : app; + final LocalApplicationRunner runner = new LocalApplicationRunner(testApp, new MapConfig(configs)); runner.run(); boolean timedOut = !runner.waitForFinish(timeout); Assert.assertFalse("Timed out waiting for application to finish", timedOut); diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index d1c40d4da2..cad3af3968 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -20,7 +20,7 @@ package org.apache.samza.test.integration; import joptsimple.OptionSet; -import org.apache.samza.application.ApplicationClassUtils; +import org.apache.samza.runtime.ApplicationClassUtils; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; import org.apache.samza.runtime.ApplicationRunner; diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java index d738e63367..66cf0611fe 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorBase.java @@ -151,22 +151,18 @@ public void afterStart() { } @Override - public void beforeStop() { - + public void afterStop() { + // stopped w/o failure + if (waitStop != null) { + waitStop.countDown(); + } + LOG.info("afterStop is called for pid=" + pId + " with successful shutdown"); } @Override - public void afterStop(Throwable t) { - if (t == null) { - // stopped w/o failure - if (waitStop != null) { - waitStop.countDown(); - } - LOG.info("afterStop is called for pid=" + pId + " with successful shutdown"); - } else { - // stopped w/ failure - LOG.info("afterStop is called for pid=" + pId + " with failure"); - } + public void afterFailure(Throwable t) { + // stopped w/ failure + LOG.info("afterStop is called for pid=" + pId + " with failure"); } }; diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java index 1533fc32b2..f518c0a14c 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java @@ -79,7 +79,7 @@ private void testStreamProcessorWithSessionRestart(String[] processorIds) { jobCoordinators[i] = (ZkJobCoordinator) streamProcessors[i].getCurrentJobCoordinator(); } - // start the processors in separate threads + // run the processors in separate threads for (int i = 0; i < processorIds.length; i++) { threadStopLatches[i] = new CountDownLatch(1); // is used int stopProcessor threads[i] = runInThread(streamProcessors[i], threadStopLatches[i]); @@ -97,7 +97,7 @@ private void testStreamProcessorWithSessionRestart(String[] processorIds) { // make sure it consumes all the messages from the first batch waitUntilMessagesLeftN(totalEventsToGenerate - messageCount); - // Get the container kill latch to be able to check when a container is stopped. + // Get the container stop latch to be able to check when a container is stopped. // New containerShutdownLatch is created after each onNewJobModel, // so we need to get the current one, before it changed.. for (int i = 0; i < processorIds.length; i++) { 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 59cf18ee34..217ee51e0d 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 @@ -24,6 +24,7 @@ import java.io.Serializable; import java.util.concurrent.CountDownLatch; import org.apache.samza.application.StreamApplication; +import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; @@ -80,7 +81,7 @@ public static StreamApplication getInstance( StreamApplicationCallback callback, CountDownLatch kafkaEventsConsumedLatch) { StreamApplication app = appDesc -> { - String appName = appDesc.getGlobalAppId(); + String appName = new ApplicationConfig(appDesc.getConfig()).getAppName(); String processorName = appDesc.getConfig().get(JobConfig.PROCESSOR_ID()); registerLatches(processedMessageLatch, kafkaEventsConsumedLatch, callback, appName, processorName); diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java index 874b243f21..fc62b0a6cf 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamProcessor.java @@ -265,13 +265,12 @@ private void initConsumer(String bootstrapServer) { private void initProcessorListener() { listener = mock(ProcessorLifecycleListener.class); doNothing().when(listener).afterStart(); + doNothing().when(listener).afterFailure(any()); doAnswer(invocation -> { - if (invocation.getArguments()[0] == null) { - // stopped successfully - shutdownLatch.countDown(); - } + // stopped successfully + shutdownLatch.countDown(); return null; - }).when(listener).afterStop(any()); + }).when(listener).afterStop(); } private void initProducer(String bootstrapServer) { 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 0a823df4c1..b9417bd8cb 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 @@ -27,7 +27,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -68,15 +67,11 @@ import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyList; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.*; /** @@ -109,7 +104,7 @@ public void testSendTo() throws Exception { .sendTo(table); }; - final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + final LocalApplicationRunner runner = new LocalApplicationRunner(app, new MapConfig(configs)); runner.run(); runner.waitForFinish(); @@ -153,7 +148,7 @@ void runTest() { .sink((m, collector, coordinator) -> joined.add(m)); }; - final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + final LocalApplicationRunner runner = new LocalApplicationRunner(app, new MapConfig(configs)); runner.run(); runner.waitForFinish(); @@ -243,7 +238,7 @@ void runTest() { .sink((m, collector, coordinator) -> joinedPageViews2.add(m)); }; - final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + final LocalApplicationRunner runner = new LocalApplicationRunner(app, new MapConfig(configs)); runner.run(); runner.waitForFinish(); diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java index 24fd8eed3c..0cb355cfbf 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java @@ -19,6 +19,7 @@ package org.apache.samza.test.table; +import com.google.common.cache.CacheBuilder; import java.io.IOException; import java.io.ObjectInputStream; import java.time.Duration; @@ -35,7 +36,6 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.internal.StreamAppDescriptorImpl; import org.apache.samza.config.MapConfig; import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.metrics.Counter; @@ -47,12 +47,12 @@ import org.apache.samza.table.Table; import org.apache.samza.table.caching.CachingTableDescriptor; import org.apache.samza.table.caching.guava.GuavaCacheTableDescriptor; +import org.apache.samza.table.remote.RemoteReadWriteTable; +import org.apache.samza.table.remote.RemoteReadableTable; +import org.apache.samza.table.remote.RemoteTableDescriptor; import org.apache.samza.table.remote.TableRateLimiter; import org.apache.samza.table.remote.TableReadFunction; import org.apache.samza.table.remote.TableWriteFunction; -import org.apache.samza.table.remote.RemoteReadableTable; -import org.apache.samza.table.remote.RemoteTableDescriptor; -import org.apache.samza.table.remote.RemoteReadWriteTable; import org.apache.samza.task.TaskContext; import org.apache.samza.test.harness.AbstractIntegrationTestHarness; import org.apache.samza.test.util.Base64Serializer; @@ -60,12 +60,9 @@ import org.junit.Assert; import org.junit.Test; -import com.google.common.cache.CacheBuilder; - import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.*; public class TestRemoteTable extends AbstractIntegrationTestHarness { @@ -190,7 +187,7 @@ private void doTestStreamTableJoinRemoteTable(boolean withCache, boolean default .sendTo(outputTable); }; - final LocalApplicationRunner runner = new LocalApplicationRunner(new StreamAppDescriptorImpl(app, new MapConfig(configs))); + final LocalApplicationRunner runner = new LocalApplicationRunner(app, new MapConfig(configs)); runner.run(); runner.waitForFinish(); From 3b2f04d5434fba4a3793f85e7e57d09802d5d627 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Sun, 19 Aug 2018 09:18:23 -0700 Subject: [PATCH 11/38] SAMZA-1789: moved all impl classes from samza-api to samza-core. --- .../samza/application/StreamApplication.java | 48 ++++- .../samza/application/TaskAppDescriptor.java | 8 +- .../samza/application/TaskApplication.java | 51 +++++- .../operators/functions/ClosableFunction.java | 6 +- .../operators/functions/InitableFunction.java | 6 +- .../samza/runtime/ApplicationRunners.java | 13 ++ .../org/apache/samza/task/TaskFactory.java | 2 +- .../samza/runtime/TestApplicationRunner.java | 2 +- .../samza/runtime/TestApplicationRunners.java | 2 +- .../application/ApplicationDescriptors.java | 9 +- .../application/LegacyTaskApplication.java | 6 +- .../application/StreamAppDescriptorImpl.java | 20 +-- .../application/TaskAppDescriptorImpl.java | 17 +- .../samza/execution/ExecutionPlanner.java | 1 + .../samza/operators/MessageStreamImpl.java | 48 ++--- .../samza/operators/OperatorSpecGraph.java | 25 ++- .../samza/processor/StreamProcessor.java | 74 ++++---- .../runtime/AbstractApplicationRunner.java | 166 +++++++++--------- .../samza/runtime/ApplicationClassUtils.java | 2 +- .../runtime/ApplicationRunnerCommandLine.java | 2 +- .../samza/runtime/LocalApplicationRunner.java | 101 +++++------ .../samza/runtime/LocalContainerRunner.java | 11 +- .../runtime/RemoteApplicationRunner.java | 75 ++++---- .../apache/samza/task/TaskFactoryUtil.java | 37 ++-- .../org/apache/samza/util/StreamUtil.java | 3 +- .../org/apache/samza/job/JobRunner.scala | 2 - .../samza/job/local/ThreadJobFactory.scala | 16 +- .../TestStreamAppDescriptorImpl.java | 165 +++++++++-------- .../TestTaskAppDescriptorImpl.java | 2 +- .../application/TestTaskApplication.java | 2 +- .../samza/operators/TestJoinOperator.java | 48 ++--- .../operators/TestOperatorSpecGraph.java | 18 +- .../operators/impl/TestOperatorImplGraph.java | 15 +- .../operators/impl/TestWindowOperator.java | 5 +- .../spec/TestPartitionByOperatorSpec.java | 26 +-- .../samza/processor/TestStreamProcessor.java | 6 +- .../runtime/TestApplicationClassUtils.java | 4 +- .../runtime/TestApplicationRunnerMain.java | 25 +-- .../runtime/TestLocalApplicationRunner.java | 92 ++++++---- .../runtime/TestRemoteApplicationRunner.java | 8 +- .../samza/task/TestTaskFactoryUtil.java | 6 +- ...ava => TestSamzaSqlApplicationRunner.java} | 2 +- .../sql/translator/TestFilterTranslator.java | 2 +- .../sql/translator/TestJoinTranslator.java | 14 +- .../sql/translator/TestProjectTranslator.java | 10 +- .../sql/translator/TestQueryTranslator.java | 9 +- .../example/OrderShipmentJoinExample.java | 1 + .../samza/example/TaskApplicationExample.java | 22 ++- .../samza/system/mock/MockSystemConsumer.java | 2 +- .../LocalApplicationRunnerMain.java | 13 +- .../processor/TestZkStreamProcessor.java | 8 +- .../EndOfStreamIntegrationTest.java | 4 +- .../WatermarkIntegrationTest.java | 5 +- .../StreamApplicationIntegrationTest.java | 3 +- ...reamApplicationIntegrationTestHarness.java | 16 +- .../operator/RepartitionJoinWindowApp.java | 10 +- .../test/operator/TumblingWindowApp.java | 5 +- .../TestZkLocalApplicationRunner.java | 6 +- .../samza/test/table/TestLocalTable.java | 8 +- .../table/TestLocalTableWithSideInputs.java | 20 +-- .../samza/test/table/TestRemoteTable.java | 3 +- .../test/integration/StreamTaskTestUtil.scala | 2 +- .../apache/samza/tools/SamzaSqlConsole.java | 4 +- .../SystemConsumerWithSamzaBench.java | 4 +- 64 files changed, 756 insertions(+), 592 deletions(-) rename samza-sql/src/test/java/org/apache/samza/sql/runner/{TestSamzaSqlApplicationRuntime.java => TestSamzaSqlApplicationRunner.java} (98%) 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 3bac7ea0f2..37e7394ce2 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 @@ -20,9 +20,53 @@ import org.apache.samza.annotation.InterfaceStability; - /** - * The interface to implement user applications defining a high-level DAG as the main stream processing logic. + * Describes and initializes the transforms for processing message streams and generating results. + *

    + * The following example removes page views older than 1 hour from the input stream: + *

    {@code
    + * public class PageViewCounter implements StreamApplication {
    + *   public void describe(StreamAppDescriptor appDesc) {
    + *     MessageStream pageViewEvents =
    + *       appDesc.getInputStream("pageViewEvents", (k, m) -> (PageViewEvent) m);
    + *     OutputStream recentPageViewEvents =
    + *       appDesc.getOutputStream("recentPageViewEvents", m -> m.memberId, m -> m);
    + *
    + *     pageViewEvents
    + *       .filter(m -> m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis())
    + *       .sendTo(filteredPageViewEvents);
    + *   }
    + * }
    + * }
    + *

    + * 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));
    + *     PageViewCounter app = new PageViewCounter();
    + *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PageViewCounter(), config);
    + *     runner.run();
    + *     runner.waitForFinish();
    + *   }
    + * }
    + * + *

    + * Implementation Notes: Currently StreamApplications are wrapped in a {@link org.apache.samza.task.StreamTask} during + * execution. A new {@link StreamAppDescriptor} instance will be created and described by the user-defined + * {@link StreamApplication} when planning the execution. All user-defined transformation functions and descriptors for + * data entities used in the transformations (e.g. {@link org.apache.samza.operators.TableDescriptor}) are required to + * be serializable. 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. The user-defined application + * class that implements {@link StreamApplication} also needs to be a class with proper fully-qualified class name to ensure + * successful instantiation in both local and remote environments. Execution is synchronous and thread-safe within each + * {@link org.apache.samza.task.StreamTask}. + * + *

    + * 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}. */ @InterfaceStability.Evolving public interface StreamApplication extends ApplicationBase { diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java index 5cc0308f50..60edc002cf 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java @@ -31,7 +31,7 @@ public interface TaskAppDescriptor extends ApplicationDescriptor + * The following example removes page views older than 1 hour from the input stream: + *

    {@code
    + * public class PageViewCounter implements TaskApplication {
    + *   public void describe(TaskAppDescriptor appDesc) {
    + *     appDesc.addInputStream(PageViewTask.TASK_INPUT);
    + *     appDesc.addOutputStream(PageViewTask.TASK_OUTPUT);
    + *     appDesc.setTaskFactory((StreamTaskFactory) () -> new PageViewTask());
    + *   }
    + * }
    + *
    + * public class PageViewTask implements StreamTask {
    + *   final static String TASK_INPUT = "pageViewEvents";
    + *   final static String TASK_OUTPUT = "recentPageViewEvents";
    + *   final static String OUTPUT_SYSTEM = "kafka";
    + *
    + *   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(OUTPUT_SYSTEM, TASK_OUTPUT),
    + *           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));
    + *     PageViewCounter app = new PageViewCounter();
    + *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PageViewCounter(), config);
    + *     runner.run();
    + *     runner.waitForFinish();
    + *   }
    + * }
    + * + *

    + * Implementation Notes: 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 TaskAppDescriptor} + * instance will be created and described by the user-defined {@link TaskApplication} when planning the execution. + * {@link org.apache.samza.task.TaskFactory} and descriptors for data entities used in the task (e.g. + * {@link org.apache.samza.operators.TableDescriptor}) are required to be serializable. The user-defined application + * class that implements {@link TaskApplication} needs to be a class with proper fully-qualified class name to ensure + * successful instantiation in both local and remote environments. */ @InterfaceStability.Evolving public interface TaskApplication extends ApplicationBase { -} +} \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java index 5f9f0d1b0c..f29d4767d4 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java @@ -27,11 +27,11 @@ *

    Implement {@link #close()} to free resources used during the execution of the function, clean up state etc. * *

    Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the - * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B consumes results - * from operator A, then operator B is guaranteed to be closed before operator A. + * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B + * consumes results from operator A, then operator B is guaranteed to be closed before operator A. * */ -@InterfaceStability.Unstable +@InterfaceStability.Evolving public interface ClosableFunction { /** * Frees any resource acquired by the operators in {@link InitableFunction} diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index 843924330a..4f82b9fbae 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -27,11 +27,11 @@ * A function that can be initialized before execution. * *

    Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the - * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B consumes results - * from operator A, then operator A is guaranteed to be initialized before operator B. + * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B + * consumes results from operator A, then operator A is guaranteed to be initialized before operator B. * */ -@InterfaceStability.Unstable +@InterfaceStability.Evolving public interface InitableFunction { /** diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java index 5e5202aec5..53601a1a50 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java @@ -26,6 +26,19 @@ /** * Creates {@link ApplicationRunner} instances based on configuration and user-implemented {@link ApplicationBase} + * + *

    This class is usually used in main() function to create an instance of {@link ApplicationRunner}, as in the example + * below: + *

    {@code
    + *   public static void main(String[] args) {
    + *     CommandLine cmdLine = new CommandLine();
    + *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
    + *     PageViewCounter app = new PageViewCounter();
    + *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PageViewCounter(), config);
    + *     runner.run();
    + *     runner.waitForFinish();
    + *   }
    + * }
    */ public class ApplicationRunners { 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 dd088def6b..e3e372b386 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 @@ -35,4 +35,4 @@ public interface TaskFactory extends Serializable { * @return task of type T */ T createInstance(); -} +} \ No newline at end of file diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java index edfef7bdc6..c38999b4bd 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java @@ -67,4 +67,4 @@ public boolean waitForFinish(Duration timeout) { public void addMetricsReporters(Map metricsReporters) { } -} +} \ No newline at end of file diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java index b74d94dfe3..780c23ae87 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java @@ -43,4 +43,4 @@ public void testGetAppRunner() { ApplicationRunner appRunner = ApplicationRunners.getApplicationRunner(app, config); assertTrue(appRunner instanceof TestApplicationRunner); } -} +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java index 3ce8c5d5e0..a9cb1bcd0f 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java @@ -29,6 +29,13 @@ private ApplicationDescriptors() { } + /** + * Create a new instance of {@link AppDescriptorImpl} based on {@link ApplicationBase} and {@link Config} + * + * @param userApp the user-implemented {@link ApplicationBase}. The {@code userApp} has to have a proper fully-qualified class name. + * @param config the user-supplied {@link Config} for the the application + * @return the {@link AppDescriptorImpl} instance containing the user processing logic and the config + */ public static AppDescriptorImpl getAppDescriptor(ApplicationBase userApp, Config config) { if (userApp instanceof StreamApplication) { return new StreamAppDescriptorImpl((StreamApplication) userApp, config); @@ -39,4 +46,4 @@ public static AppDescriptorImpl getAppDescriptor(ApplicationBase userApp, Config throw new IllegalArgumentException(String.format("User application class %s is not supported. Only StreamApplication " + "and TaskApplication are supported.", userApp.getClass().getName())); } -} +} \ No newline at end of file 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 f0fb866a40..575ccae854 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 @@ -39,8 +39,8 @@ public LegacyTaskApplication(Config config) { private Config validate(Config config) { new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( () -> { - throw new ConfigException("No task class defined in the configuration."); - })); + throw new ConfigException("No task class defined in the configuration."); + })); return config; } @@ -48,4 +48,4 @@ private Config validate(Config config) { public void describe(TaskAppDescriptor appDesc) { appDesc.setTaskFactory(TaskFactoryUtil.createTaskFactory(config)); } -} +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java index c29b5ee722..9343f1be09 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java @@ -39,7 +39,7 @@ import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.TableImpl; import org.apache.samza.operators.spec.InputOperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; @@ -71,7 +71,7 @@ public class StreamAppDescriptorImpl extends AppDescriptorImpl operatorIds = new HashSet<>(); @@ -111,7 +111,7 @@ public MessageStream getInputStream(String streamId, Serde serde) { boolean isKeyed = serde instanceof KVSerde; InputOperatorSpec inputOperatorSpec = OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), - isKeyed, this.getNextOpId(OperatorSpec.OpCode.INPUT, null)); + isKeyed, this.getNextOpId(OpCode.INPUT, null)); inputOperators.put(streamId, inputOperatorSpec); return new MessageStreamImpl<>(this, inputOperators.get(streamId)); } @@ -162,18 +162,18 @@ public Table> getTable(TableDescriptor tableDesc) { } public OperatorSpecGraph getOperatorSpecGraph() { - return OperatorSpecGraph.getInstance(this); + return new OperatorSpecGraph(this); } /** * Gets the unique ID for the next operator in the graph. The ID is of the following format: * jobName-jobId-opCode-(userDefinedId|nextOpNum); * - * @param opCode the {@link OperatorSpec.OpCode} of the next operator + * @param opCode the {@link OpCode} of the next operator * @param userDefinedId the optional user-provided name of the next operator or null * @return the unique ID for the next operator in the graph */ - public String getNextOpId(OperatorSpec.OpCode opCode, String userDefinedId) { + public String getNextOpId(OpCode opCode, String userDefinedId) { if (StringUtils.isNotBlank(userDefinedId) && !ID_PATTERN.matcher(userDefinedId).matches()) { throw new SamzaException("Operator ID must not contain spaces or special characters: " + userDefinedId); } @@ -195,10 +195,10 @@ public String getNextOpId(OperatorSpec.OpCode opCode, String userDefinedId) { * Gets the unique ID for the next operator in the graph. The ID is of the following format: * jobName-jobId-opCode-nextOpNum; * - * @param opCode the {@link OperatorSpec.OpCode} of the next operator + * @param opCode the {@link OpCode} of the next operator * @return the unique ID for the next operator in the graph */ - public String getNextOpId(OperatorSpec.OpCode opCode) { + public String getNextOpId(OpCode opCode) { return getNextOpId(opCode, null); } @@ -226,7 +226,6 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI * @param the type of messages in the intermediate {@link MessageStream} * @return the intermediate {@link MessageStreamImpl} */ - @VisibleForTesting public IntermediateMessageStreamImpl getIntermediateStream(String streamId, Serde serde, boolean isBroadcast) { Preconditions.checkState(!inputOperators.containsKey(streamId) && !outputStreams.containsKey(streamId), "getIntermediateStream must not be called multiple times with the same streamId: " + streamId); @@ -241,7 +240,7 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI KV kvSerdes = getKVSerdes(streamId, serde); InputOperatorSpec inputOperatorSpec = OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), - isKeyed, this.getNextOpId(OperatorSpec.OpCode.INPUT, null)); + isKeyed, this.getNextOpId(OpCode.INPUT, null)); inputOperators.put(streamId, inputOperatorSpec); outputStreams.put(streamId, new OutputStreamImpl(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamId), outputStreams.get(streamId)); @@ -289,5 +288,4 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } - } diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java index e258c19151..8d07a6e970 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java @@ -30,7 +30,7 @@ * This class implements interface {@link TaskAppDescriptor}. *

    * In addition to the common objects for an application defined in {@link AppDescriptorImpl}, this class also includes - * the low-level {@link TaskFactory} object that creates user-defined task instances, the lists of input/output streams, + * the low-level {@link TaskFactory} that creates user-defined task instances, the lists of input/broadcast/output streams, * and the list of {@link TableDescriptor}s used in the application. */ public class TaskAppDescriptorImpl extends AppDescriptorImpl @@ -92,6 +92,15 @@ public List getInputStreams() { return Collections.unmodifiableList(this.inputStreams); } + /** + * Get the broadcast streams to this application + * + * @return the list of broadcast streamIds + */ + public List getBroadcastStreams() { + return Collections.unmodifiableList(this.broadcastStreams); + } + /** * Get the output streams to this application * @@ -111,8 +120,4 @@ public List getOutputStreams() { public List getTables() { return Collections.unmodifiableList(this.tables); } - - public List getBroadcastStreams() { - return Collections.unmodifiableList(this.broadcastStreams); - } -} +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 230f46fdad..766b9037a2 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -51,6 +51,7 @@ * The ExecutionPlanner creates the physical execution graph for the {@link OperatorSpecGraph}, and * the intermediate topics needed for the execution. */ +// TODO: ExecutionPlanner needs to be able to generate single node JobGraph for low-level TaskApplication as well (SAMZA-1811) public class ExecutionPlanner { private static final Logger log = LoggerFactory.getLogger(ExecutionPlanner.class); diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index 01e62821f9..bc3b8f1c3b 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -67,52 +67,52 @@ public class MessageStreamImpl implements MessageStream { /** * The {@link StreamAppDescriptorImpl} that contains this {@link MessageStreamImpl} */ - private final StreamAppDescriptorImpl graph; + private final StreamAppDescriptorImpl streamAppDesc; /** * The {@link OperatorSpec} associated with this {@link MessageStreamImpl} */ private final OperatorSpec operatorSpec; - public MessageStreamImpl(StreamAppDescriptorImpl graph, OperatorSpec operatorSpec) { - this.graph = graph; + public MessageStreamImpl(StreamAppDescriptorImpl streamAppDesc, OperatorSpec operatorSpec) { + this.streamAppDesc = streamAppDesc; this.operatorSpec = operatorSpec; } @Override public MessageStream map(MapFunction mapFn) { - String opId = this.graph.getNextOpId(OpCode.MAP); + String opId = this.streamAppDesc.getNextOpId(OpCode.MAP); StreamOperatorSpec op = OperatorSpecs.createMapOperatorSpec(mapFn, opId); this.operatorSpec.registerNextOperatorSpec(op); - return new MessageStreamImpl<>(this.graph, op); + return new MessageStreamImpl<>(this.streamAppDesc, op); } @Override public MessageStream filter(FilterFunction filterFn) { - String opId = this.graph.getNextOpId(OpCode.FILTER); + String opId = this.streamAppDesc.getNextOpId(OpCode.FILTER); StreamOperatorSpec op = OperatorSpecs.createFilterOperatorSpec(filterFn, opId); this.operatorSpec.registerNextOperatorSpec(op); - return new MessageStreamImpl<>(this.graph, op); + return new MessageStreamImpl<>(this.streamAppDesc, op); } @Override public MessageStream flatMap(FlatMapFunction flatMapFn) { - String opId = this.graph.getNextOpId(OpCode.FLAT_MAP); + String opId = this.streamAppDesc.getNextOpId(OpCode.FLAT_MAP); StreamOperatorSpec op = OperatorSpecs.createFlatMapOperatorSpec(flatMapFn, opId); this.operatorSpec.registerNextOperatorSpec(op); - return new MessageStreamImpl<>(this.graph, op); + return new MessageStreamImpl<>(this.streamAppDesc, op); } @Override public void sink(SinkFunction sinkFn) { - String opId = this.graph.getNextOpId(OpCode.SINK); + String opId = this.streamAppDesc.getNextOpId(OpCode.SINK); SinkOperatorSpec op = OperatorSpecs.createSinkOperatorSpec(sinkFn, opId); this.operatorSpec.registerNextOperatorSpec(op); } @Override public void sendTo(OutputStream outputStream) { - String opId = this.graph.getNextOpId(OpCode.SEND_TO); + String opId = this.streamAppDesc.getNextOpId(OpCode.SEND_TO); OutputOperatorSpec op = OperatorSpecs.createSendToOperatorSpec( (OutputStreamImpl) outputStream, opId); this.operatorSpec.registerNextOperatorSpec(op); @@ -120,10 +120,10 @@ public void sendTo(OutputStream outputStream) { @Override public MessageStream> window(Window window, String userDefinedId) { - String opId = this.graph.getNextOpId(OpCode.WINDOW, userDefinedId); + String opId = this.streamAppDesc.getNextOpId(OpCode.WINDOW, userDefinedId); OperatorSpec> op = OperatorSpecs.createWindowOperatorSpec((WindowInternal) window, opId); this.operatorSpec.registerNextOperatorSpec(op); - return new MessageStreamImpl<>(this.graph, op); + return new MessageStreamImpl<>(this.streamAppDesc, op); } @Override @@ -132,7 +132,7 @@ public MessageStream join(MessageStream otherStream, Serde keySerde, Serde messageSerde, Serde otherMessageSerde, Duration ttl, String userDefinedId) { if (otherStream.equals(this)) throw new SamzaException("Cannot join a MessageStream with itself."); - String opId = this.graph.getNextOpId(OpCode.JOIN, userDefinedId); + String opId = this.streamAppDesc.getNextOpId(OpCode.JOIN, userDefinedId); OperatorSpec otherOpSpec = ((MessageStreamImpl) otherStream).getOperatorSpec(); JoinOperatorSpec op = OperatorSpecs.createJoinOperatorSpec(this.operatorSpec, otherOpSpec, (JoinFunction) joinFn, keySerde, @@ -140,35 +140,35 @@ public MessageStream join(MessageStream otherStream, this.operatorSpec.registerNextOperatorSpec(op); otherOpSpec.registerNextOperatorSpec((OperatorSpec) op); - return new MessageStreamImpl<>(this.graph, op); + return new MessageStreamImpl<>(this.streamAppDesc, op); } @Override public MessageStream join(Table table, StreamTableJoinFunction joinFn) { - String opId = this.graph.getNextOpId(OpCode.JOIN); + String opId = this.streamAppDesc.getNextOpId(OpCode.JOIN); TableSpec tableSpec = ((TableImpl) table).getTableSpec(); StreamTableJoinOperatorSpec joinOpSpec = OperatorSpecs.createStreamTableJoinOperatorSpec( tableSpec, (StreamTableJoinFunction) joinFn, opId); this.operatorSpec.registerNextOperatorSpec(joinOpSpec); - return new MessageStreamImpl<>(this.graph, joinOpSpec); + return new MessageStreamImpl<>(this.streamAppDesc, joinOpSpec); } @Override public MessageStream merge(Collection> otherStreams) { if (otherStreams.isEmpty()) return this; - String opId = this.graph.getNextOpId(OpCode.MERGE); + String opId = this.streamAppDesc.getNextOpId(OpCode.MERGE); StreamOperatorSpec op = OperatorSpecs.createMergeOperatorSpec(opId); this.operatorSpec.registerNextOperatorSpec(op); otherStreams.forEach(other -> ((MessageStreamImpl) other).getOperatorSpec().registerNextOperatorSpec(op)); - return new MessageStreamImpl<>(this.graph, op); + return new MessageStreamImpl<>(this.streamAppDesc, op); } @Override public MessageStream> partitionBy(MapFunction keyExtractor, MapFunction valueExtractor, KVSerde serde, String userDefinedId) { - String opId = this.graph.getNextOpId(OpCode.PARTITION_BY, userDefinedId); - IntermediateMessageStreamImpl> intermediateStream = this.graph.getIntermediateStream(opId, serde); + String opId = this.streamAppDesc.getNextOpId(OpCode.PARTITION_BY, userDefinedId); + IntermediateMessageStreamImpl> intermediateStream = this.streamAppDesc.getIntermediateStream(opId, serde); if (!intermediateStream.isKeyed()) { // this can only happen when the default serde partitionBy variant is being used throw new SamzaException("partitionBy can not be used with a default serde that is not a KVSerde."); @@ -187,7 +187,7 @@ public MessageStream> partitionBy(MapFunction void sendTo(Table> table) { - String opId = this.graph.getNextOpId(OpCode.SEND_TO); + String opId = this.streamAppDesc.getNextOpId(OpCode.SEND_TO); SendToTableOperatorSpec op = OperatorSpecs.createSendToTableOperatorSpec(((TableImpl) table).getTableSpec(), opId); this.operatorSpec.registerNextOperatorSpec(op); @@ -195,8 +195,8 @@ public void sendTo(Table> table) { @Override public MessageStream broadcast(Serde serde, String userDefinedId) { - String opId = this.graph.getNextOpId(OpCode.BROADCAST, userDefinedId); - IntermediateMessageStreamImpl intermediateStream = this.graph.getIntermediateStream(opId, serde, true); + String opId = this.streamAppDesc.getNextOpId(OpCode.BROADCAST, userDefinedId); + IntermediateMessageStreamImpl intermediateStream = this.streamAppDesc.getIntermediateStream(opId, serde, true); BroadcastOperatorSpec broadcastOperatorSpec = OperatorSpecs.createBroadCastOperatorSpec(intermediateStream.getOutputStream(), opId); this.operatorSpec.registerNextOperatorSpec(broadcastOperatorSpec); diff --git a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java index c37b9e382a..f1eec24b4c 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java @@ -34,10 +34,11 @@ /** - * Defines the serialized format of {@link StreamAppDescriptorImpl}. This class encapsulates all getter methods to get the {@link OperatorSpec} - * initialized in the {@link StreamAppDescriptorImpl} and constructs the corresponding serialized instances of {@link OperatorSpec}. - * The {@link StreamAppDescriptorImpl} and {@link OperatorSpec} instances included in this class are considered as immutable and read-only. - * The instance of {@link OperatorSpecGraph} should only be used in runtime to construct {@link org.apache.samza.task.StreamOperatorTask}. + * Defines the serialized format of the operator graph in {@link StreamAppDescriptorImpl}. This class encapsulates all + * getter methods to get the {@link OperatorSpec} initialized in the {@link StreamAppDescriptorImpl} and constructs the + * corresponding serialized instances of {@link OperatorSpec}. The {@link StreamAppDescriptorImpl} and {@link OperatorSpec} + * instances included in this class are considered as immutable and read-only. The instance of {@link OperatorSpecGraph} + * should only be used in runtime to construct {@link org.apache.samza.task.StreamOperatorTask}. */ public class OperatorSpecGraph implements Serializable { // We use a LHM for deterministic order in initializing and closing operators. @@ -52,22 +53,16 @@ public class OperatorSpecGraph implements Serializable { private transient final SerializableSerde opSpecGraphSerde = new SerializableSerde<>(); private transient final byte[] serializedOpSpecGraph; - OperatorSpecGraph(Map inputOperators, Map outputStreams, - Set broadcastStreams, Map tables) { - this.inputOperators = inputOperators; - this.outputStreams = outputStreams; - this.broadcastStreams = broadcastStreams; - this.tables = tables; + public OperatorSpecGraph(StreamAppDescriptorImpl streamAppDesc) { + this.inputOperators = streamAppDesc.getInputOperators(); + this.outputStreams = streamAppDesc.getOutputStreams(); + this.broadcastStreams = streamAppDesc.getBroadcastStreams(); + this.tables = streamAppDesc.getTables(); this.allOpSpecs = Collections.unmodifiableSet(this.findAllOperatorSpecs()); this.hasWindowOrJoins = checkWindowOrJoins(); this.serializedOpSpecGraph = opSpecGraphSerde.toBytes(this); } - public static OperatorSpecGraph getInstance(StreamAppDescriptorImpl appDesc) { - return new OperatorSpecGraph(appDesc.getInputOperators(), appDesc.getOutputStreams(), appDesc.getBroadcastStreams(), - appDesc.getTables()); - } - public Map getInputOperators() { return inputOperators; } 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 b030f12868..c4a4276b04 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 @@ -44,8 +44,6 @@ import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.runtime.ProcessorContext; import org.apache.samza.runtime.ProcessorLifecycleListener; -import org.apache.samza.task.AsyncStreamTaskFactory; -import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.task.TaskFactory; import org.apache.samza.util.ScalaJavaUtil; import org.apache.samza.util.Util; @@ -132,6 +130,14 @@ public String toString() { } } + /** + * Supplier interface to allow creation of {@link ProcessorLifecycleListener} for a {@link StreamProcessor} + */ + public interface StreamProcessorListenerSupplier extends Function { + + ProcessorLifecycleListener apply(StreamProcessor sp); + } + /** * @return the current state of StreamProcessor. */ @@ -160,38 +166,23 @@ public State getState() { * * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer}. * @param customMetricsReporters metricReporter instances that will be used by SamzaContainer and JobCoordinator to report metrics. - * @param asyncStreamTaskFactory The {@link AsyncStreamTaskFactory} to be used for creating task instances. + * @param taskFactory The {@link TaskFactory} to be used for creating task instances. * @param processorListener listener to the StreamProcessor life cycle. */ public StreamProcessor(Config config, Map customMetricsReporters, - AsyncStreamTaskFactory asyncStreamTaskFactory, ProcessorLifecycleListener processorListener) { - this(config, customMetricsReporters, asyncStreamTaskFactory, processorListener, null); + TaskFactory taskFactory, ProcessorLifecycleListener processorListener) { + this(config, customMetricsReporters, taskFactory, processorListener, null); } /** - * Same as {@link StreamProcessor#StreamProcessor(Config, Map, StreamTaskFactory, ProcessorLifecycleListener)}, - * except task instances are created using the provided {@link StreamTaskFactory}. + * Same as {@link StreamProcessor#StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener)}, + * except the {@link JobCoordinator} is given for this {@link StreamProcessor}. * @param config - config * @param customMetricsReporters metric Reporter - * @param streamTaskFactory task factory to instantiate the Task + * @param taskFactory task factory to instantiate the Task * @param processorListener listener to the StreamProcessor life cycle + * @param jobCoordinator the instance of {@link JobCoordinator} */ - public StreamProcessor(Config config, Map customMetricsReporters, - StreamTaskFactory streamTaskFactory, ProcessorLifecycleListener processorListener) { - this(config, customMetricsReporters, streamTaskFactory, processorListener, null); - } - - /* package private */ - private JobCoordinator getJobCoordinator() { - String jobCoordinatorFactoryClassName = new JobCoordinatorConfig(config).getJobCoordinatorFactoryClassName(); - return Util.getObj(jobCoordinatorFactoryClassName, JobCoordinatorFactory.class).getJobCoordinator(config); - } - - @VisibleForTesting - JobCoordinator getCurrentJobCoordinator() { - return jobCoordinator; - } - public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, ProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator) { Preconditions.checkNotNull(processorListener, "ProcessorListener cannot be null."); @@ -208,9 +199,18 @@ public StreamProcessor(Config config, Map customMetrics this.processorId = this.jobCoordinator.getProcessorId(); } + /** + * Same as {@link StreamProcessor#StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener, JobCoordinator)}, + * except user supplies a {@link StreamProcessorListenerSupplier} to create the listener for this {@link StreamProcessor}. + * @param config - config + * @param customMetricsReporters metric Reporter + * @param taskFactory task factory to instantiate the Task + * @param listenerSupplier a {@link Function} to create the {@link ProcessorLifecycleListener} for this {@link StreamProcessor} + * @param jobCoordinator the instance of {@link JobCoordinator} + */ public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, - StreamProcessorListenerSupplier getListenerFn, JobCoordinator jobCoordinator) { - Preconditions.checkNotNull(getListenerFn, "getListenerFn cannot be null."); + StreamProcessorListenerSupplier listenerSupplier, JobCoordinator jobCoordinator) { + Preconditions.checkNotNull(listenerSupplier, "listenerSupplier cannot be null."); this.taskFactory = taskFactory; this.config = config; this.taskShutdownMs = new TaskConfigJava(config).getShutdownMs(); @@ -221,8 +221,10 @@ public StreamProcessor(Config config, Map customMetrics ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(CONTAINER_THREAD_NAME_FORMAT).setDaemon(true).build(); this.executorService = Executors.newSingleThreadExecutor(threadFactory); this.processorId = this.jobCoordinator.getProcessorId(); - this.processorListener = getListenerFn.apply(this); + // create and attach the listener to this StreamProcessor + this.processorListener = listenerSupplier.apply(this); } + /** * Asynchronously starts this {@link StreamProcessor}. *

    @@ -308,10 +310,20 @@ public Config getConfig() { return config; } + @VisibleForTesting + JobCoordinator getCurrentJobCoordinator() { + return jobCoordinator; + } + SamzaContainer createSamzaContainer(String processorId, JobModel jobModel) { return SamzaContainer.apply(processorId, jobModel, config, ScalaJavaUtil.toScalaMap(customMetricsReporter), taskFactory); } + private JobCoordinator getJobCoordinator() { + String jobCoordinatorFactoryClassName = new JobCoordinatorConfig(config).getJobCoordinatorFactoryClassName(); + return Util.getObj(jobCoordinatorFactoryClassName, JobCoordinatorFactory.class).getJobCoordinator(config); + } + /** * Stops the {@link SamzaContainer}. * @return true if {@link SamzaContainer} had shutdown within task.shutdown.ms. false otherwise. @@ -451,12 +463,4 @@ public void onContainerFailed(Throwable t) { } } } - - /** - * Supplier interface to allow creation of {@link ProcessorLifecycleListener} for a {@link StreamProcessor} - */ - public interface StreamProcessorListenerSupplier extends Function { - - ProcessorLifecycleListener apply(StreamProcessor sp); - } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 870b904bea..8fa4d26dc1 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -34,7 +34,6 @@ import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; -import org.apache.samza.config.ApplicationConfig.ApplicationMode; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -62,102 +61,111 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { protected final Config config; protected final Map metricsReporters = new HashMap<>(); - AbstractApplicationRunner(ApplicationBase userApp, Config config) { - this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); - this.config = appDesc.getConfig(); - } + /** + * This is a temporary helper class to include all common logic to generate {@link JobConfig}s for high- and low-level + * applications in {@link LocalApplicationRunner} and {@link RemoteApplicationRunner} before we fix SAMZA-1811. + * + * TODO: Fix SAMZA-1811 to consolidate the planning into {@link ExecutionPlanner} + */ + abstract class JobConfigPlanner { - @Override - public final void addMetricsReporters(Map metricsReporters) { - this.metricsReporters.putAll(metricsReporters); - } + abstract List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throws Exception; - StreamManager buildAndStartStreamManager() { - StreamManager streamManager = new StreamManager(config); - streamManager.start(); - return streamManager; - } + List createJobConfigs() throws Exception { + if (appDesc instanceof TaskAppDescriptorImpl) { + // low-level task application only needs a simple single job configuration + return Collections.singletonList(getTaskJobConfig((TaskAppDescriptorImpl) appDesc)); + } else if (appDesc instanceof StreamAppDescriptorImpl) { + return getStreamJobConfigs((StreamAppDescriptorImpl) appDesc); + } - abstract List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc); + throw new IllegalArgumentException("ApplicationDescriptor class " + appDesc.getClass().getName() + " is not supported"); + } - List createJobConfigs() { - if (appDesc instanceof TaskAppDescriptorImpl) { - // low-level task application only needs a simple single job configuration - return Collections.singletonList(getTaskJobConfig((TaskAppDescriptorImpl) appDesc)); - } else if (appDesc instanceof StreamAppDescriptorImpl) { - return getJobConfigsFromPlan((StreamAppDescriptorImpl) appDesc); + StreamManager buildAndStartStreamManager() { + StreamManager streamManager = new StreamManager(config); + streamManager.start(); + return streamManager; } - throw new IllegalArgumentException("ApplicationDescriptor class " + appDesc.getClass().getName() + " is not supported"); - } + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, StreamManager streamManager) throws Exception { + return getExecutionPlan(specGraph, null, streamManager); + } - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, StreamManager streamManager) throws Exception { - return getExecutionPlan(specGraph, null, streamManager); - } + /* package private */ + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, StreamManager streamManager) throws Exception { - /* package private */ - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, StreamManager streamManager) throws Exception { + // update application configs + Map cfg = new HashMap<>(config); + if (StringUtils.isNoneEmpty(runId)) { + cfg.put(ApplicationConfig.APP_RUN_ID, runId); + } - // update application configs - Map cfg = new HashMap<>(config); - if (StringUtils.isNoneEmpty(runId)) { - cfg.put(ApplicationConfig.APP_RUN_ID, runId); + StreamConfig streamConfig = new StreamConfig(config); + Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); + inputStreams.removeAll(specGraph.getOutputStreams().keySet()); + ApplicationConfig.ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) + ? ApplicationConfig.ApplicationMode.BATCH : ApplicationConfig.ApplicationMode.STREAM; + cfg.put(ApplicationConfig.APP_MODE, mode.name()); + validateAppClassCfg(cfg, appDesc.getAppClass()); + + // create the physical execution plan + ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager); + return planner.plan(specGraph); } - StreamConfig streamConfig = new StreamConfig(config); - Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); - inputStreams.removeAll(specGraph.getOutputStreams().keySet()); - ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) - ? ApplicationMode.BATCH : ApplicationMode.STREAM; - cfg.put(ApplicationConfig.APP_MODE, mode.name()); - validateAppClassCfg(cfg, appDesc.getAppClass()); - - // create the physical execution plan - ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager); - return planner.plan(specGraph); - } + /** + * Write the execution plan JSON to a file + * @param planJson JSON representation of the plan + */ + final void writePlanJsonFile(String planJson) { + try { + String content = "plan='" + planJson + "'"; + String planPath = System.getenv(ShellCommandConfig.EXECUTION_PLAN_DIR()); + if (planPath != null && !planPath.isEmpty()) { + // Write the plan json to plan path + File file = new File(planPath + "/plan.json"); + file.setReadable(true, false); + PrintWriter writer = new PrintWriter(file, "UTF-8"); + writer.println(content); + writer.close(); + } + } catch (Exception e) { + log.warn("Failed to write execution plan json to file", e); + } + } - /** - * Write the execution plan JSON to a file - * @param planJson JSON representation of the plan - */ - final void writePlanJsonFile(String planJson) { - try { - String content = "plan='" + planJson + "'"; - String planPath = System.getenv(ShellCommandConfig.EXECUTION_PLAN_DIR()); - if (planPath != null && !planPath.isEmpty()) { - // Write the plan json to plan path - File file = new File(planPath + "/plan.json"); - file.setReadable(true, false); - PrintWriter writer = new PrintWriter(file, "UTF-8"); - writer.println(content); - writer.close(); + // helper method to generate a single node job configuration for low level task applications + private JobConfig getTaskJobConfig(TaskAppDescriptorImpl taskAppDesc) { + Map cfg = new HashMap<>(taskAppDesc.getConfig()); + //TODO: add stream and system descriptor to configuration conversion here when SAMZA-1804 is fixed. + // adding table configuration + List tableSpecs = taskAppDesc.getTables().stream() + .map(td -> ((BaseTableDescriptor) td).getTableSpec()) + .collect(Collectors.toList()); + cfg.putAll(TableConfigGenerator.generateConfigsForTableSpecs(tableSpecs)); + validateAppClassCfg(cfg, taskAppDesc.getAppClass()); + return new JobConfig(new MapConfig(cfg)); + } + + private void validateAppClassCfg(Map cfg, Class appClass) { + if (cfg.get(ApplicationConfig.APP_CLASS) != null && !cfg.get(ApplicationConfig.APP_CLASS).isEmpty()) { + // app.class is already set + return; } - } catch (Exception e) { - log.warn("Failed to write execution plan json to file", e); + // adding app.class in the configuration + cfg.put(ApplicationConfig.APP_CLASS, appClass.getCanonicalName()); } } - // helper method to generate a single node job configuration for low level task applications - private JobConfig getTaskJobConfig(TaskAppDescriptorImpl taskAppDesc) { - Map cfg = new HashMap<>(taskAppDesc.getConfig()); - //TODO: add stream and system descriptor to configuration conversion here when SAMZA-1804 is fixed. - // adding table configuration - List tableSpecs = taskAppDesc.getTables().stream() - .map(td -> ((BaseTableDescriptor) td).getTableSpec()) - .collect(Collectors.toList()); - cfg.putAll(TableConfigGenerator.generateConfigsForTableSpecs(tableSpecs)); - validateAppClassCfg(cfg, taskAppDesc.getAppClass()); - return new JobConfig(new MapConfig(cfg)); + AbstractApplicationRunner(ApplicationBase userApp, Config config) { + this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); + this.config = appDesc.getConfig(); } - private void validateAppClassCfg(Map cfg, Class appClass) { - if (cfg.get(ApplicationConfig.APP_CLASS) != null && !cfg.get(ApplicationConfig.APP_CLASS).isEmpty()) { - // app.class is already set - return; - } - // adding app.class in the configuration - cfg.put(ApplicationConfig.APP_CLASS, appClass.getCanonicalName()); + @Override + public final void addMetricsReporters(Map metricsReporters) { + this.metricsReporters.putAll(metricsReporters); } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java index 8dd903ace0..44d40f771b 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java @@ -56,4 +56,4 @@ public static ApplicationBase fromConfig(Config config) { return new LegacyTaskApplication(config); } -} +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java index 1e672a0376..ca5e65733e 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerCommandLine.java @@ -37,4 +37,4 @@ public ApplicationRunnerOperation getOperation(OptionSet options) { String rawOp = options.valueOf(operationOpt).toString(); return ApplicationRunnerOperation.fromString(rawOp); } -} +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 510594fb86..3b8dd91bf5 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -33,9 +33,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.AppDescriptorImpl; import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -46,6 +44,7 @@ import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.processor.StreamProcessor; +import org.apache.samza.processor.StreamProcessor.StreamProcessorListenerSupplier; import org.apache.samza.system.StreamSpec; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; @@ -65,9 +64,13 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private final CountDownLatch shutdownLatch = new CountDownLatch(1); private final AtomicInteger numProcessorsToStart = new AtomicInteger(); private final AtomicReference failure = new AtomicReference<>(); + private final LocalJobConfigPlanner planner; private ApplicationStatus appStatus = ApplicationStatus.New; + /** + * Defines a specific implementation of {@link ProcessorLifecycleListener} for local {@link StreamProcessor}s. + */ private final class LocalStreamProcessorLifecycleListener implements ProcessorLifecycleListener { private final StreamProcessor processor; private final ProcessorLifecycleListener processorLifecycleListener; @@ -88,7 +91,6 @@ public void afterStart() { @Override public void afterStop() { processors.remove(processor); - //processor = null; processorLifecycleListener.afterStop(); if (processors.isEmpty()) { @@ -100,7 +102,6 @@ public void afterStop() { @Override public void afterFailure(Throwable t) { processors.remove(processor); - //processor = null; processorLifecycleListener.afterFailure(t); // the processor stopped with failure @@ -137,15 +138,52 @@ private void shutdownAndNotify() { } + /** + * Defines a {@link JobConfigPlanner} with specific implementation of {@link JobConfigPlanner#getStreamJobConfigs(StreamAppDescriptorImpl)} + * for standalone Samza processors. + * + * TODO: we need to consolidate all planning logic into {@link org.apache.samza.execution.ExecutionPlanner} after SAMZA-1811. + */ + @VisibleForTesting + class LocalJobConfigPlanner extends JobConfigPlanner { + @Override + List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + // for high-level DAG, generating the plan and job configs + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), streamManager); + + String executionPlanJson = plan.getPlanAsJson(); + writePlanJsonFile(executionPlanJson); + LOG.info("Execution Plan: \n" + executionPlanJson); + + // 2. create the necessary streams + // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 + String planId = String.valueOf(executionPlanJson.hashCode()); + createStreams(planId, plan.getIntermediateStreams(), streamManager); + + return plan.getJobConfigs(); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + } + } + public LocalApplicationRunner(ApplicationBase userApp, Config config) { super(userApp, config); this.uid = UUID.randomUUID().toString(); + this.planner = new LocalJobConfigPlanner(); } @Override public void run() { try { - List jobConfigs = createJobConfigs(); + List jobConfigs = planner.createJobConfigs(); // 3. create the StreamProcessors if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs to run."); @@ -204,37 +242,6 @@ public boolean waitForFinish(Duration timeout) { return finished; } - @Override - List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc) { - // for high-level DAG, generating the plan and job configs - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), streamManager); - - String executionPlanJson = plan.getPlanAsJson(); - writePlanJsonFile(executionPlanJson); - LOG.info("Execution Plan: \n" + executionPlanJson); - - // 2. create the necessary streams - // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 - String planId = String.valueOf(executionPlanJson.hashCode()); - createStreams(planId, plan.getIntermediateStreams(), streamManager); - - return plan.getJobConfigs(); - } catch (Throwable throwable) { - appStatus = ApplicationStatus.unsuccessfulFinish(throwable); - shutdownLatch.countDown(); - throw new SamzaException("Failed to start application.", throwable); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - } - /** * Create intermediate streams using {@link org.apache.samza.execution.StreamManager}. * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader @@ -285,27 +292,16 @@ private void createStreams(String planId, } /** - * Create {@link StreamProcessor} based on config, {@link AppDescriptorImpl}, and {@link ProcessorLifecycleListener} + * Create {@link StreamProcessor} based on config, {@link AppDescriptorImpl}, and {@link StreamProcessorListenerSupplier} * @param config config * @param appDesc {@link AppDescriptorImpl} - * @param listenerFn the {@link ProcessorLifecycleListener} + * @param listenerSupplier {@link StreamProcessorListenerSupplier} to create {@link ProcessorLifecycleListener} * @return {@link StreamProcessor]} */ /* package private */ - StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, StreamProcessor.StreamProcessorListenerSupplier listenerFn) { - TaskFactory taskFactory = getTaskFactory(appDesc); - return new StreamProcessor(config, this.metricsReporters, taskFactory, listenerFn, null); - } - - TaskFactory getTaskFactory(ApplicationDescriptor appDesc) { - if (appDesc instanceof StreamAppDescriptorImpl) { - StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; - return TaskFactoryUtil.createTaskFactory(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()); - } else if (appDesc instanceof TaskAppDescriptorImpl) { - TaskAppDescriptorImpl taskAppDescriptor = (TaskAppDescriptorImpl) appDesc; - return taskAppDescriptor.getTaskFactory(); - } - throw new IllegalArgumentException("Invalid ApplicationDescriptor " + appDesc.getClass().getName()); + StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, StreamProcessorListenerSupplier listenerSupplier) { + TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); + return new StreamProcessor(config, this.metricsReporters, taskFactory, listenerSupplier, null); } /* package private for testing */ @@ -317,5 +313,4 @@ Set getProcessors() { CountDownLatch getShutdownLatch() { return shutdownLatch; } - } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index ddc65f5389..7ec7baf45a 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -93,7 +93,7 @@ private static AppDescriptorImpl getAppDesc(Config config) { } private static void run(AppDescriptorImpl appDesc, String containerId, JobModel jobModel, Config config) { - TaskFactory taskFactory = getTaskFactory(appDesc); + TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); SamzaContainer container = SamzaContainer$.MODULE$.apply( containerId, jobModel, @@ -150,15 +150,6 @@ public void onContainerFailed(Throwable t) { } } - private static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { - if (appDesc instanceof StreamAppDescriptorImpl) { - StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; - return TaskFactoryUtil.createTaskFactory(streamAppDesc.getOperatorSpecGraph(), - streamAppDesc.getContextManager()); - } - return ((TaskAppDescriptorImpl) appDesc).getTaskFactory(); - } - /** * Creates a new container heartbeat monitor if possible. * @param container the container to monitor diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 995132c731..19b20be7d1 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -47,15 +47,54 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; + private final RemoteJobConfigPlanner planner; + + /** + * Defines a {@link JobConfigPlanner} with specific implementation of {@link JobConfigPlanner#getStreamJobConfigs(StreamAppDescriptorImpl)} + * for remote-launched Samza processors (e.g. in YARN). + * + * TODO: we need to consolidate all planning logic into {@link org.apache.samza.execution.ExecutionPlanner} after SAMZA-1811. + */ + private class RemoteJobConfigPlanner extends JobConfigPlanner { + @Override + List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + // for high-level DAG, generate the plan and job configs + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(); + // TODO: run.id needs to be set for standalone: SAMZA-1531 + // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); + LOG.info("The run id for this run is {}", runId); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId, streamManager); + writePlanJsonFile(plan.getPlanAsJson()); + + // 2. create the necessary streams + if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { + streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); + } + streamManager.createStreams(plan.getIntermediateStreams()); + + return plan.getJobConfigs(); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + } + } RemoteApplicationRunner(ApplicationBase userApp, Config config) { super(userApp, config); + this.planner = new RemoteJobConfigPlanner(); } @Override public void run() { try { - List jobConfigs = createJobConfigs(); + List jobConfigs = planner.createJobConfigs(); if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs to run."); } @@ -66,7 +105,6 @@ public void run() { JobRunner runner = new JobRunner(jobConfig); runner.run(true); }); - } catch (Throwable t) { throw new SamzaException("Failed to run application", t); } @@ -100,7 +138,7 @@ public ApplicationStatus status() { @Override public void waitForFinish() { - this.waitForFinish(Duration.ofSeconds(0)); + waitForFinish(Duration.ofMillis(0)); } @Override @@ -139,37 +177,6 @@ public boolean waitForFinish(Duration timeout) { return finished; } - @Override - List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc) { - // for high-level DAG, generate the plan and job configs - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - // TODO: run.id needs to be set for standalone: SAMZA-1531 - // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision - String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The run id for this run is {}", runId); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId, streamManager); - writePlanJsonFile(plan.getPlanAsJson()); - - // 2. create the necessary streams - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - streamManager.createStreams(plan.getIntermediateStreams()); - - return plan.getJobConfigs(); - } catch (Throwable t) { - throw new SamzaException("Failed to run application", t); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - } - /* package private */ ApplicationStatus getApplicationStatus(JobConfig jobConfig) { JobRunner runner = new JobRunner(jobConfig); ApplicationStatus status = runner.status(); diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index 1d3d40be18..cc847dc3b5 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -19,6 +19,9 @@ package org.apache.samza.task; import org.apache.samza.SamzaException; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.TaskConfig; @@ -39,7 +42,24 @@ public class TaskFactoryUtil { private static final Logger log = LoggerFactory.getLogger(TaskFactoryUtil.class); /** - * This method creates a task factory class based on the {@link OperatorSpecGraph} and {@link ContextManager} + * Creates a {@link TaskFactory} based on {@link AppDescriptorImpl} + * + * @param appDesc {@link AppDescriptorImpl} for this application + * @return {@link TaskFactory} object + */ + public static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { + if (appDesc instanceof StreamAppDescriptorImpl) { + StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; + return (StreamTaskFactory) () -> new StreamOperatorTask(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()); + } else if (appDesc instanceof TaskAppDescriptorImpl) { + TaskAppDescriptorImpl taskAppDescriptor = (TaskAppDescriptorImpl) appDesc; + return taskAppDescriptor.getTaskFactory(); + } + throw new IllegalArgumentException("Invalid ApplicationDescriptor class " + appDesc.getClass().getName()); + } + + /** + * Creates a {@link TaskFactory} based on the {@link OperatorSpecGraph} and {@link ContextManager} * * @param specGraph the {@link OperatorSpecGraph} * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph} @@ -50,21 +70,14 @@ public static TaskFactory createTaskFactory(OperatorSpecGraph specGraph, Context } /** - * This method creates a task factory class based on the configuration + * Creates a {@link TaskFactory} based on the configuration. + *

    + * This should only be used to create {@link TaskFactory} defined in task.class * * @param config the {@link Config} for this job - * @return a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} + * @return a {@link TaskFactory} object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} */ public static TaskFactory createTaskFactory(Config config) { - return fromTaskClassConfig(config); - } - - /** - * Create {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} based on the configured task.class. - * @param config the {@link Config} - * @return task factory instance - */ - private static TaskFactory fromTaskClassConfig(Config config) { // if there is configuration to set the job w/ a specific type of task, instantiate the corresponding task factory String taskClassName = new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( () -> { diff --git a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java index 31796d95ed..e7a1e5460e 100644 --- a/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/StreamUtil.java @@ -18,7 +18,6 @@ */ package org.apache.samza.util; -import java.util.Collection; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -78,7 +77,7 @@ public static String getNameFromSystemStream(SystemStream systemStream) { return systemStream.getSystem() + "." + systemStream.getStream(); } - public static Set getStreamSpecs(Collection streamIds, StreamConfig streamConfig) { + public static Set getStreamSpecs(Set streamIds, StreamConfig streamConfig) { return streamIds.stream().map(streamId -> getStreamSpec(streamId, streamConfig)).collect(Collectors.toSet()); } diff --git a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala index 22b80cda5b..239352082f 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala @@ -20,8 +20,6 @@ package org.apache.samza.job -import java.util.concurrent.TimeUnit - import org.apache.samza.SamzaException import org.apache.samza.config.Config import org.apache.samza.config.JobConfig.Config2Job diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index 439dd117d6..a6b2d282a9 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,7 +19,7 @@ package org.apache.samza.job.local -import org.apache.samza.application._ +import org.apache.samza.application.ApplicationDescriptors import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ @@ -31,8 +31,8 @@ import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} import org.apache.samza.runtime.ApplicationClassUtils import org.apache.samza.storage.ChangelogStreamManager +import org.apache.samza.task.TaskFactory import org.apache.samza.task.TaskFactoryUtil -import org.apache.samza.task._ import org.apache.samza.util.Logging import scala.collection.JavaConversions._ @@ -74,16 +74,8 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val containerId = "0" val jmxServer = new JmxServer - val taskFactory : TaskFactory[_] = ApplicationClassUtils.fromConfig(config) match { - case app if (app.isInstanceOf[TaskApplication]) => { - val appSpec = new TaskAppDescriptorImpl(app.asInstanceOf[TaskApplication], config) - appSpec.getTaskFactory - } - case app if (app.isInstanceOf[StreamApplication]) => { - val appSpec = new StreamAppDescriptorImpl(app.asInstanceOf[StreamApplication], config) - TaskFactoryUtil.createTaskFactory(appSpec.getOperatorSpecGraph, appSpec.getContextManager) - } - } + val taskFactory : TaskFactory[_] = TaskFactoryUtil.getTaskFactory(ApplicationDescriptors.getAppDescriptor( + ApplicationClassUtils.fromConfig(config), config)) // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job. config.getTaskOpts match { diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java index 943e790acf..982ff7e29c 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java @@ -29,7 +29,7 @@ import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.data.TestMessageEnvelope; import org.apache.samza.operators.spec.InputOperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; @@ -39,8 +39,17 @@ import org.apache.samza.table.TableSpec; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** @@ -62,12 +71,12 @@ public void testGetInputStreamWithValueSerde() { String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(streamId, mockValueSerde); }, mock(Config.class)); - InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); - assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); + InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId); + assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -82,12 +91,12 @@ public void testGetInputStreamWithKeyValueSerde() { Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(streamId, mockKVSerde); }, mock(Config.class)); - InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); - assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); + InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId); + assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertEquals(mockKeySerde, inputOpSpec.getKeySerde()); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -105,13 +114,13 @@ public void testGetInputStreamWithDefaultValueSerde() { String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.setDefaultSerde(mockValueSerde); appDesc.getInputStream(streamId); }, mock(Config.class)); - InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); - assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); + InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId); + assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -126,13 +135,13 @@ public void testGetInputStreamWithDefaultKeyValueSerde() { Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.setDefaultSerde(mockKVSerde); appDesc.getInputStream(streamId); }, mock(Config.class)); - InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); - assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); + InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId); + assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertEquals(mockKeySerde, inputOpSpec.getKeySerde()); assertEquals(mockValueSerde, inputOpSpec.getValueSerde()); @@ -143,12 +152,12 @@ public void testGetInputStreamWithDefaultDefaultSerde() { String streamId = "test-stream-1"; // default default serde == user hasn't provided a default serde - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(streamId); }, mock(Config.class)); - InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); - assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); + InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId); + assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde); @@ -157,12 +166,12 @@ public void testGetInputStreamWithDefaultDefaultSerde() { @Test public void testGetInputStreamWithRelaxedTypes() { String streamId = "test-stream-1"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(streamId); }, mock(Config.class)); - InputOperatorSpec inputOpSpec = graphSpec.getInputOperators().get(streamId); - assertEquals(OperatorSpec.OpCode.INPUT, inputOpSpec.getOpCode()); + InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get(streamId); + assertEquals(OpCode.INPUT, inputOpSpec.getOpCode()); assertEquals(streamId, inputOpSpec.getStreamId()); } @@ -171,15 +180,15 @@ public void testMultipleGetInputStreams() { String streamId1 = "test-stream-1"; String streamId2 = "test-stream-2"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(streamId1); appDesc.getInputStream(streamId2); }, mock(Config.class)); - InputOperatorSpec inputOpSpec1 = graphSpec.getInputOperators().get(streamId1); - InputOperatorSpec inputOpSpec2 = graphSpec.getInputOperators().get(streamId2); + InputOperatorSpec inputOpSpec1 = streamAppDesc.getInputOperators().get(streamId1); + InputOperatorSpec inputOpSpec2 = streamAppDesc.getInputOperators().get(streamId2); - assertEquals(graphSpec.getInputOperators().size(), 2); + assertEquals(streamAppDesc.getInputOperators().size(), 2); assertEquals(streamId1, inputOpSpec1.getStreamId()); assertEquals(streamId2, inputOpSpec2.getStreamId()); } @@ -198,11 +207,11 @@ public void testGetSameInputStreamTwice() { public void testGetOutputStreamWithValueSerde() { String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getOutputStream(streamId, mockValueSerde); }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); + OutputStreamImpl outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -216,12 +225,12 @@ public void testGetOutputStreamWithKeyValueSerde() { Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.setDefaultSerde(mockKVSerde); appDesc.getOutputStream(streamId, mockKVSerde); }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); + OutputStreamImpl outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertEquals(mockKeySerde, outputStreamImpl.getKeySerde()); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -240,12 +249,12 @@ public void testGetOutputStreamWithDefaultValueSerde() { String streamId = "test-stream-1"; Serde mockValueSerde = mock(Serde.class); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.setDefaultSerde(mockValueSerde); appDesc.getOutputStream(streamId); }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); + OutputStreamImpl outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -260,12 +269,12 @@ public void testGetOutputStreamWithDefaultKeyValueSerde() { Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.setDefaultSerde(mockKVSerde); appDesc.getOutputStream(streamId); }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); + OutputStreamImpl outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertEquals(mockKeySerde, outputStreamImpl.getKeySerde()); assertEquals(mockValueSerde, outputStreamImpl.getValueSerde()); @@ -275,12 +284,12 @@ public void testGetOutputStreamWithDefaultKeyValueSerde() { public void testGetOutputStreamWithDefaultDefaultSerde() { String streamId = "test-stream-1"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getOutputStream(streamId); }, mock(Config.class)); - OutputStreamImpl outputStreamImpl = graphSpec.getOutputStreams().get(streamId); + OutputStreamImpl outputStreamImpl = streamAppDesc.getOutputStreams().get(streamId); assertEquals(streamId, outputStreamImpl.getStreamId()); assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde); assertTrue(outputStreamImpl.getValueSerde() instanceof NoOpSerde); @@ -308,9 +317,9 @@ public void testSetDefaultSerdeAfterGettingOutputStream() { @Test(expected = IllegalStateException.class) public void testSetDefaultSerdeAfterGettingIntermediateStream() { String streamId = "test-stream-1"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); - graphSpec.getIntermediateStream(streamId, null); - graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + streamAppDesc.getIntermediateStream(streamId, null); + streamAppDesc.setDefaultSerde(mock(Serde.class)); // should throw exception } @Test(expected = IllegalStateException.class) @@ -325,14 +334,14 @@ public void testGetSameOutputStreamTwice() { @Test public void testGetIntermediateStreamWithValueSerde() { String streamId = "stream-1"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); Serde mockValueSerde = mock(Serde.class); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(streamId, mockValueSerde); + streamAppDesc.getIntermediateStream(streamId, mockValueSerde); - assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamAppDesc.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(streamAppDesc.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); @@ -343,7 +352,7 @@ public void testGetIntermediateStreamWithValueSerde() { @Test public void testGetIntermediateStreamWithKeyValueSerde() { String streamId = "streamId"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -351,10 +360,10 @@ public void testGetIntermediateStreamWithKeyValueSerde() { doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(streamId, mockKVSerde); + streamAppDesc.getIntermediateStream(streamId, mockKVSerde); - assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamAppDesc.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(streamAppDesc.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde()); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); @@ -386,19 +395,19 @@ public void testGetIntermediateStreamWithDefaultKeyValueSerde() { Config mockConfig = mock(Config.class); String streamId = "streamId"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); Serde mockValueSerde = mock(Serde.class); doReturn(mockKeySerde).when(mockKVSerde).getKeySerde(); doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); - graphSpec.setDefaultSerde(mockKVSerde); + streamAppDesc.setDefaultSerde(mockKVSerde); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(streamId, null); + streamAppDesc.getIntermediateStream(streamId, null); - assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamAppDesc.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(streamAppDesc.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde()); assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde()); @@ -411,12 +420,12 @@ public void testGetIntermediateStreamWithDefaultDefaultSerde() { Config mockConfig = mock(Config.class); String streamId = "streamId"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); IntermediateMessageStreamImpl intermediateStreamImpl = - graphSpec.getIntermediateStream(streamId, null); + streamAppDesc.getIntermediateStream(streamId, null); - assertEquals(graphSpec.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); - assertEquals(graphSpec.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); + assertEquals(streamAppDesc.getInputOperators().get(streamId), intermediateStreamImpl.getOperatorSpec()); + assertEquals(streamAppDesc.getOutputStreams().get(streamId), intermediateStreamImpl.getOutputStream()); assertEquals(streamId, intermediateStreamImpl.getStreamId()); assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde); assertTrue(intermediateStreamImpl.getOutputStream().getValueSerde() instanceof NoOpSerde); @@ -426,9 +435,9 @@ public void testGetIntermediateStreamWithDefaultDefaultSerde() { @Test(expected = IllegalStateException.class) public void testGetSameIntermediateStreamTwice() { - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); - graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class)); - graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + streamAppDesc.getIntermediateStream("test-stream-1", mock(Serde.class)); + streamAppDesc.getIntermediateStream("test-stream-1", mock(Serde.class)); } @Test @@ -437,10 +446,10 @@ public void testGetNextOpIdIncrementsId() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); - assertEquals("jobName-1234-merge-0", graphSpec.getNextOpId(OperatorSpec.OpCode.MERGE, null)); - assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OperatorSpec.OpCode.JOIN, "customName")); - assertEquals("jobName-1234-map-2", graphSpec.getNextOpId(OperatorSpec.OpCode.MAP, null)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + assertEquals("jobName-1234-merge-0", streamAppDesc.getNextOpId(OpCode.MERGE, null)); + assertEquals("jobName-1234-join-customName", streamAppDesc.getNextOpId(OpCode.JOIN, "customName")); + assertEquals("jobName-1234-map-2", streamAppDesc.getNextOpId(OpCode.MAP, null)); } @Test(expected = SamzaException.class) @@ -449,9 +458,9 @@ public void testGetNextOpIdRejectsDuplicates() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); - assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OperatorSpec.OpCode.JOIN, "customName")); - graphSpec.getNextOpId(OperatorSpec.OpCode.JOIN, "customName"); // should throw + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + assertEquals("jobName-1234-join-customName", streamAppDesc.getNextOpId(OpCode.JOIN, "customName")); + streamAppDesc.getNextOpId(OpCode.JOIN, "customName"); // should throw } @Test @@ -460,14 +469,14 @@ public void testUserDefinedIdValidation() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); // null and empty userDefinedIDs should fall back to autogenerated IDs. try { - graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, null); - graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, ""); - graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, " "); - graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, "\t"); + streamAppDesc.getNextOpId(OpCode.FILTER, null); + streamAppDesc.getNextOpId(OpCode.FILTER, ""); + streamAppDesc.getNextOpId(OpCode.FILTER, " "); + streamAppDesc.getNextOpId(OpCode.FILTER, "\t"); } catch (SamzaException e) { fail("Received an error with a null or empty operator ID instead of defaulting to auto-generated ID."); } @@ -475,7 +484,7 @@ public void testUserDefinedIdValidation() { List validOpIds = ImmutableList.of("op.id", "op_id", "op-id", "1000", "op_1", "OP_ID"); for (String validOpId: validOpIds) { try { - graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, validOpId); + streamAppDesc.getNextOpId(OpCode.FILTER, validOpId); } catch (Exception e) { fail("Received an exception with a valid operator ID: " + validOpId); } @@ -484,7 +493,7 @@ public void testUserDefinedIdValidation() { List invalidOpIds = ImmutableList.of("op id", "op#id"); for (String invalidOpId: invalidOpIds) { try { - graphSpec.getNextOpId(OperatorSpec.OpCode.FILTER, invalidOpId); + streamAppDesc.getNextOpId(OpCode.FILTER, invalidOpId); fail("Did not receive an exception with an invalid operator ID: " + invalidOpId); } catch (SamzaException e) { } } @@ -498,13 +507,13 @@ public void testGetInputStreamPreservesInsertionOrder() { String testStreamId2 = "test-stream-2"; String testStreamId3 = "test-stream-3"; - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(testStreamId1); appDesc.getInputStream(testStreamId2); appDesc.getInputStream(testStreamId3); }, mockConfig); - List inputSpecs = new ArrayList<>(graphSpec.getInputOperators().values()); + List inputSpecs = new ArrayList<>(streamAppDesc.getInputOperators().values()); assertEquals(inputSpecs.size(), 3); assertEquals(inputSpecs.get(0).getStreamId(), testStreamId1); assertEquals(inputSpecs.get(1).getStreamId(), testStreamId2); @@ -518,10 +527,10 @@ public void testGetTable() throws Exception { BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); TableSpec testTableSpec = new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()); when(mockTableDescriptor.getTableSpec()).thenReturn(testTableSpec); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { - appDesc.getTable(mockTableDescriptor); - }, mockConfig); - assertNotNull(graphSpec.getTables().get(testTableSpec)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + appDesc.getTable(mockTableDescriptor); + }, mockConfig); + assertNotNull(streamAppDesc.getTables().get(testTableSpec)); } @Test diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java index 10f93f92b9..33bec07c5c 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java @@ -104,4 +104,4 @@ public void testProcessorLifecycleListener() { assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); } -} +} \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java index 97113e84bd..87381628e6 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java @@ -26,4 +26,4 @@ public class TestTaskApplication implements TaskApplication { public void describe(TaskAppDescriptor appSpec) { } -} +} \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java index d59171d2e3..128d948a98 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java @@ -81,8 +81,8 @@ public void setUp() { @Test public void join() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -99,7 +99,7 @@ public void join() throws Exception { public void joinWithSelfThrowsException() throws Exception { config.put("streams.instream.system", "insystem"); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { IntegerSerde integerSerde = new IntegerSerde(); KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); MessageStream> inStream = appDesc.getInputStream("instream", kvSerde); @@ -107,14 +107,14 @@ public void joinWithSelfThrowsException() throws Exception { inStream.join(inStream, new TestJoinFunction(), integerSerde, kvSerde, kvSerde, JOIN_TTL, "join"); }, config); - createStreamOperatorTask(new SystemClock(), graphSpec); // should throw an exception + createStreamOperatorTask(new SystemClock(), streamAppDesc); // should throw an exception } @Test public void joinFnInitAndClose() throws Exception { TestJoinFunction joinFn = new TestJoinFunction(); - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(joinFn); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(joinFn); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); MessageCollector messageCollector = mock(MessageCollector.class); @@ -132,8 +132,8 @@ public void joinFnInitAndClose() throws Exception { @Test public void joinReverse() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -148,8 +148,8 @@ public void joinReverse() throws Exception { @Test public void joinNoMatch() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -163,8 +163,8 @@ public void joinNoMatch() throws Exception { @Test public void joinNoMatchReverse() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -178,8 +178,8 @@ public void joinNoMatchReverse() throws Exception { @Test public void joinRetainsLatestMessageForKey() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -196,8 +196,8 @@ public void joinRetainsLatestMessageForKey() throws Exception { @Test public void joinRetainsLatestMessageForKeyReverse() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -214,8 +214,8 @@ public void joinRetainsLatestMessageForKeyReverse() throws Exception { @Test public void joinRetainsMatchedMessages() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -237,8 +237,8 @@ public void joinRetainsMatchedMessages() throws Exception { @Test public void joinRetainsMatchedMessagesReverse() throws Exception { - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -261,8 +261,8 @@ public void joinRetainsMatchedMessagesReverse() throws Exception { @Test public void joinRemovesExpiredMessages() throws Exception { TestClock testClock = new TestClock(); - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(testClock, streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -281,8 +281,8 @@ public void joinRemovesExpiredMessages() throws Exception { @Test public void joinRemovesExpiredMessagesReverse() throws Exception { TestClock testClock = new TestClock(); - StreamAppDescriptorImpl graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction()); - StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec); + StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamOperatorTask sot = createStreamOperatorTask(testClock, streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java index 11d19e9983..86308e752b 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java @@ -58,14 +58,14 @@ @PrepareForTest(OperatorSpec.class) public class TestOperatorSpecGraph { - private StreamAppDescriptorImpl mockGraph; + private StreamAppDescriptorImpl mockAppDesc; private Map inputOpSpecMap; private Map outputStrmMap; private Set allOpSpecs; @Before public void setUp() { - this.mockGraph = mock(StreamAppDescriptorImpl.class); + this.mockAppDesc = mock(StreamAppDescriptorImpl.class); /** * Setup two linear transformation pipelines: @@ -92,8 +92,8 @@ public void setUp() { inputOpSpecMap.put(streamId2, testInput2); this.outputStrmMap = new LinkedHashMap<>(); outputStrmMap.put(outputStreamId, outputStream1); - when(mockGraph.getInputOperators()).thenReturn(Collections.unmodifiableMap(inputOpSpecMap)); - when(mockGraph.getOutputStreams()).thenReturn(Collections.unmodifiableMap(outputStrmMap)); + when(mockAppDesc.getInputOperators()).thenReturn(Collections.unmodifiableMap(inputOpSpecMap)); + when(mockAppDesc.getOutputStreams()).thenReturn(Collections.unmodifiableMap(outputStrmMap)); this.allOpSpecs = new HashSet() { { this.add(testInput); this.add(filterOp); @@ -106,7 +106,7 @@ public void setUp() { @After public void tearDown() { - this.mockGraph = null; + this.mockAppDesc = null; this.inputOpSpecMap = null; this.outputStrmMap = null; this.allOpSpecs = null; @@ -114,7 +114,7 @@ public void tearDown() { @Test public void testConstructor() { - OperatorSpecGraph specGraph = OperatorSpecGraph.getInstance(mockGraph); + OperatorSpecGraph specGraph = new OperatorSpecGraph(mockAppDesc); assertEquals(specGraph.getInputOperators(), inputOpSpecMap); assertEquals(specGraph.getOutputStreams(), outputStrmMap); assertTrue(specGraph.getTables().isEmpty()); @@ -124,7 +124,7 @@ public void testConstructor() { @Test public void testClone() { - OperatorSpecGraph operatorSpecGraph = OperatorSpecGraph.getInstance(mockGraph); + OperatorSpecGraph operatorSpecGraph = new OperatorSpecGraph(mockAppDesc); OperatorSpecGraph clonedSpecGraph = operatorSpecGraph.clone(); OperatorSpecTestUtils.assertClonedGraph(operatorSpecGraph, clonedSpecGraph); } @@ -138,7 +138,7 @@ public void testCloneWithSerializationError() throws Throwable { //failed with serialization error try { - OperatorSpecGraph.getInstance(mockGraph); + new OperatorSpecGraph(mockAppDesc); fail("Should have failed with serialization error"); } catch (SamzaException se) { throw se.getCause(); @@ -151,7 +151,7 @@ public void testCloneWithDeserializationError() throws Throwable { this.allOpSpecs.add(testOp); inputOpSpecMap.values().stream().findFirst().get().registerNextOperatorSpec(testOp); - OperatorSpecGraph operatorSpecGraph = OperatorSpecGraph.getInstance(mockGraph); + OperatorSpecGraph operatorSpecGraph = new OperatorSpecGraph(mockAppDesc); //failed with serialization error try { operatorSpecGraph.clone(); diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 5a39007e5f..bf52a9ea4b 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -73,9 +73,12 @@ import org.junit.After; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestOperatorImplGraph { @@ -171,10 +174,10 @@ static public List getCloseListByTaskName(TaskName taskName) { @Override public void close() { if (this.taskName == null) { - throw new IllegalStateException("Close called before bootstrap"); + throw new IllegalStateException("Close called before init"); } if (perTaskFunctionMap.get(this.taskName) == null || !perTaskFunctionMap.get(this.taskName).containsKey(opId)) { - throw new IllegalStateException("Close called before bootstrap"); + throw new IllegalStateException("Close called before init"); } if (perTaskCloseList.get(this.taskName) == null) { @@ -192,7 +195,7 @@ public void init(Config config, TaskContext context) { perTaskFunctionMap.put(context.getTaskName(), new HashMap() { { this.put(opId, BaseTestFunction.this); } }); } else { if (perTaskFunctionMap.get(context.getTaskName()).containsKey(opId)) { - throw new IllegalStateException(String.format("Multiple bootstrap called for op %s in the same task instance %s", opId, this.taskName.getTaskName())); + throw new IllegalStateException(String.format("Multiple init called for op %s in the same task instance %s", opId, this.taskName.getTaskName())); } perTaskFunctionMap.get(context.getTaskName()).put(opId, this); } @@ -247,7 +250,6 @@ public void testLinearChain() { .sendTo(outputStream); }, config); - TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); when(mockTaskContext.getTaskName()).thenReturn(new TaskName("task 0")); @@ -369,7 +371,6 @@ public void testMergeChain() { mergedStream.map(testMapFunction); }, mock(Config.class)); - TaskContextImpl mockTaskContext = mock(TaskContextImpl.class); TaskName mockTaskName = mock(TaskName.class); when(mockTaskContext.getTaskName()).thenReturn(mockTaskName); diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java index dd1792229b..e314a3bcb6 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java @@ -67,7 +67,10 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class TestWindowOperator { private final TaskCoordinator taskCoordinator = mock(TaskCoordinator.class); diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java index a55b42e50e..a2eb8348de 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java @@ -33,8 +33,11 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** @@ -47,7 +50,6 @@ public class TestPartitionByOperatorSpec { private final String testJobName = "testJob"; private final String testJobId = "1"; private final String testReparStreamName = "parByKey"; - private StreamAppDescriptorImpl graphSpec = null; class TimerMapFn implements MapFunction, TimerFunction { @@ -95,12 +97,12 @@ public void setup() { public void testPartitionBy() { MapFunction keyFn = m -> m.toString(); MapFunction valueFn = m -> m; - graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputId); inputStream.partitionBy(keyFn, valueFn, testReparStreamName); }, mockConfig); - assertEquals(2, graphSpec.getInputOperators().size()); - Map inputOpSpecs = graphSpec.getInputOperators(); + assertEquals(2, streamAppDesc.getInputOperators().size()); + Map inputOpSpecs = streamAppDesc.getInputOperators(); assertTrue(inputOpSpecs.keySet().contains(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName))); InputOperatorSpec inputOpSpec = inputOpSpecs.get(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName)); assertEquals(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName), inputOpSpec.getStreamId()); @@ -122,11 +124,11 @@ public void testPartitionBy() { @Test public void testCopy() { - graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputId); inputStream.partitionBy(m -> m.toString(), m -> m, testReparStreamName); }, mockConfig); - OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); OperatorSpecGraph clonedGraph = specGraph.clone(); OperatorSpecTestUtils.assertClonedGraph(specGraph, clonedGraph); } @@ -134,7 +136,7 @@ public void testCopy() { @Test(expected = IllegalArgumentException.class) public void testTimerFunctionAsKeyFn() { TimerMapFn keyFn = new TimerMapFn(); - graphSpec = new StreamAppDescriptorImpl(appDesc -> { + new StreamAppDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputId); inputStream.partitionBy(keyFn, m -> m, "parByKey"); }, mockConfig); @@ -143,7 +145,7 @@ public void testTimerFunctionAsKeyFn() { @Test(expected = IllegalArgumentException.class) public void testWatermarkFunctionAsKeyFn() { WatermarkMapFn keyFn = new WatermarkMapFn(); - graphSpec = new StreamAppDescriptorImpl(appDesc -> { + new StreamAppDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputId); inputStream.partitionBy(keyFn, m -> m, "parByKey"); }, mockConfig); @@ -152,7 +154,7 @@ public void testWatermarkFunctionAsKeyFn() { @Test(expected = IllegalArgumentException.class) public void testTimerFunctionAsValueFn() { TimerMapFn valueFn = new TimerMapFn(); - graphSpec = new StreamAppDescriptorImpl(appDesc -> { + new StreamAppDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputId); inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); }, mockConfig); @@ -161,7 +163,7 @@ public void testTimerFunctionAsValueFn() { @Test(expected = IllegalArgumentException.class) public void testWatermarkFunctionAsValueFn() { WatermarkMapFn valueFn = new WatermarkMapFn(); - graphSpec = new StreamAppDescriptorImpl(appDesc -> { + new StreamAppDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputId); inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); }, mockConfig); diff --git a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java index 338ef0efdf..b407b5a2fc 100644 --- a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java +++ b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java @@ -48,7 +48,10 @@ import org.powermock.api.mockito.PowerMockito; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestStreamProcessor { private ConcurrentMap processorListenerState; @@ -261,7 +264,6 @@ public void afterStart() { @Override public void afterStop() { - // successful stop processorListenerState.put(ListenerCallback.AFTER_STOP, true); } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java index 69d5280fd1..07fb895b9d 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java @@ -33,7 +33,7 @@ import org.apache.samza.task.TestStreamTask; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertTrue; /** @@ -81,4 +81,4 @@ public void testNoAppClassNoTaskClass() { Map configMap = new HashMap<>(); ApplicationClassUtils.fromConfig(new MapConfig(configMap)); } -} +} \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index 902566aa62..a590dd2fdf 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -19,18 +19,15 @@ package org.apache.samza.runtime; import java.time.Duration; -import java.util.List; import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.StreamAppDescriptor; -import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; import org.apache.samza.job.ApplicationStatus; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class TestApplicationRunnerMain { @@ -82,14 +79,6 @@ public void TestStatusOperation() throws Exception { assertEquals(1, TestApplicationRunnerInvocationCounts.statusCount); } - public static class TestStreamApplicationDummy implements StreamApplication { - - @Override - public void describe(StreamAppDescriptor appDesc) { - - } - } - public static class TestApplicationRunnerInvocationCounts extends AbstractApplicationRunner { protected static int runCount = 0; protected static int killCount = 0; @@ -99,11 +88,6 @@ public TestApplicationRunnerInvocationCounts(ApplicationBase userApp, Config con super(userApp, config); } - @Override - protected List getJobConfigsFromPlan(StreamAppDescriptorImpl streamAppDesc) { - return null; - } - @Override public void run() { runCount++; @@ -132,4 +116,11 @@ public boolean waitForFinish(Duration timeout) { } + public static class TestStreamApplicationDummy implements StreamApplication { + + @Override + public void describe(StreamAppDescriptor appDesc) { + + } + } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 079e7b36ad..759f087d72 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; @@ -41,6 +42,7 @@ import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.processor.StreamProcessor; +import org.apache.samza.runtime.LocalApplicationRunner.LocalJobConfigPlanner; import org.apache.samza.system.StreamSpec; import org.junit.Before; import org.junit.Test; @@ -49,19 +51,23 @@ import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; - -import static org.junit.Assert.*; +import org.powermock.reflect.Whitebox; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyLong; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import static org.powermock.api.mockito.PowerMockito.doReturn; @RunWith(PowerMockRunner.class) @@ -75,25 +81,28 @@ public class TestLocalApplicationRunner { "\"%s\":{" + "\"streamSpec\":{" + "\"id\":\"%s\"," + "\"systemName\":\"%s\"," + "\"physicalName\":\"%s\"," + "\"partitionCount\":2}," + "\"sourceJobs\":[\"test-app\"]," + "\"targetJobs\":[\"test-target-app\"]},"; + private Config config; + private ApplicationBase mockApp; private LocalApplicationRunner runner; + private LocalJobConfigPlanner localPlanner; @Before public void setUp() { - Map config = new HashMap<>(); - StreamApplication mockApp = mock(StreamApplication.class); - runner = spy(new LocalApplicationRunner(mockApp, new MapConfig(config))); + this.config = new MapConfig(config); + mockApp = mock(StreamApplication.class); + prepareTest(); } @Test public void testStreamCreation() throws Exception { StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(runner).buildAndStartStreamManager(); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); when(plan.getPlanAsJson()).thenReturn(""); - doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); + doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class); @@ -119,12 +128,12 @@ public void testStreamCreation() public void testStreamCreationWithCoordination() throws Exception { StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(runner).buildAndStartStreamManager(); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); when(plan.getPlanAsJson()).thenReturn(""); - doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); + doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); CoordinationUtils coordinationUtils = mock(CoordinationUtils.class); CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); @@ -157,11 +166,12 @@ public void testStreamCreationWithCoordination() @Test public void testRunStreamTask() throws Exception { - final Map config = new HashMap<>(); - config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); - config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); - Config samzaConfig = new MapConfig(config); - runner = spy(new LocalApplicationRunner(ApplicationClassUtils.fromConfig(samzaConfig), samzaConfig)); + final Map cfgs = new HashMap<>(); + cfgs.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); + cfgs.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); + config = new MapConfig(cfgs); + mockApp = ApplicationClassUtils.fromConfig(config); + prepareTest(); StreamProcessor sp = mock(StreamProcessor.class); @@ -169,12 +179,12 @@ public void testRunStreamTask() ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); doAnswer(i -> - { - ProcessorLifecycleListener listener = captor.getValue().apply(sp); - listener.afterStart(); - listener.afterStop(); - return null; - }).when(sp).start(); + { + ProcessorLifecycleListener listener = captor.getValue().apply(sp); + listener.afterStart(); + listener.afterStop(); + return null; + }).when(sp).start(); doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture()); doReturn(ApplicationStatus.SuccessfulFinish).when(runner).status(); @@ -187,22 +197,23 @@ public void testRunStreamTask() @Test public void testRunComplete() throws Exception { - Map config = new HashMap<>(); - config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); + Map cfgs = new HashMap<>(); + cfgs.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); + config = new MapConfig(cfgs); ProcessorLifecycleListenerFactory mockFactory = (pContext, cfg) -> mock(ProcessorLifecycleListener.class); - StreamApplication userApp = appDesc -> { + mockApp = (StreamApplication) appDesc -> { appDesc.withProcessorLifecycleListenerFactory(mockFactory); }; - runner = spy(new LocalApplicationRunner(userApp, new MapConfig(config))); + prepareTest(); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); - when(runner.buildAndStartStreamManager()).thenReturn(streamManager); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList()); when(plan.getPlanAsJson()).thenReturn(""); when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))); - doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); + doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); StreamProcessor sp = mock(StreamProcessor.class); ArgumentCaptor captor = @@ -228,22 +239,23 @@ public void testRunComplete() @Test public void testRunFailure() throws Exception { - Map config = new HashMap<>(); - config.put(ApplicationConfig.PROCESSOR_ID, "0"); + Map cfgs = new HashMap<>(); + cfgs.put(ApplicationConfig.PROCESSOR_ID, "0"); + config = new MapConfig(cfgs); ProcessorLifecycleListenerFactory mockFactory = (pContext, cfg) -> mock(ProcessorLifecycleListener.class); - StreamApplication userApp = appDesc -> { + mockApp = (StreamApplication) appDesc -> { appDesc.withProcessorLifecycleListenerFactory(mockFactory); }; - runner = spy(new LocalApplicationRunner(userApp, new MapConfig(config))); + prepareTest(); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); - when(runner.buildAndStartStreamManager()).thenReturn(streamManager); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList()); when(plan.getPlanAsJson()).thenReturn(""); when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))); - doReturn(plan).when(runner).getExecutionPlan(any(), eq(streamManager)); + doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); StreamProcessor sp = mock(StreamProcessor.class); ArgumentCaptor captor = @@ -343,6 +355,12 @@ public void testWaitForFinishTimesout() { assertFalse("Application finished before the timeout.", finished); } + private void prepareTest() { + runner = spy(new LocalApplicationRunner(mockApp, config)); + localPlanner = spy((LocalJobConfigPlanner) Whitebox.getInternalState(runner, "planner")); + Whitebox.setInternalState(runner, "planner", localPlanner); + } + private String getExecutionPlanId(List updatedStreamSpecs) { String intermediateStreamJson = updatedStreamSpecs.stream().map(this::streamSpecToJson).collect(Collectors.joining(",")); diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java index eb0c5e16c4..ae525fb84f 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java @@ -36,9 +36,11 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import static org.junit.Assert.*; -import static org.mockito.Matchers.*; -import static org.powermock.api.mockito.PowerMockito.*; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; /** diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 222e72ccdf..2ff5399bc0 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -27,8 +27,10 @@ import org.apache.samza.config.MapConfig; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; /** * Test methods to create {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} based on task class configuration diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java similarity index 98% rename from samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java rename to samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java index 55be9f852d..9fab5d50ee 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRuntime.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java @@ -31,7 +31,7 @@ import org.junit.Test; -public class TestSamzaSqlApplicationRuntime { +public class TestSamzaSqlApplicationRunner { @Test public void testComputeSamzaConfigs() { diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java index 2a9c06c6d4..148ce72772 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java @@ -95,7 +95,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { assertNotNull(filterSpec); assertEquals(filterSpec.getOpCode(), OperatorSpec.OpCode.FILTER); - // Verify that the bootstrap() method will establish the context for the filter function + // Verify that the describe() method will establish the context for the filter function Config mockConfig = mock(Config.class); TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null, new HashSet<>(), null, null, null, null, null, null); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java index 4a6cca95a8..587ac4e063 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java @@ -132,22 +132,22 @@ public void testTranslateStreamToTableJoin() throws IOException, ClassNotFoundEx when(mockRightInput.getRowType()).thenReturn(mockRightRowType); when(mockRightRowType.getFieldNames()).thenReturn(rightStreamFieldNames); - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamAppDescriptorImpl mockAppDesc = mock(StreamAppDescriptorImpl.class); OperatorSpec mockLeftInputOp = mock(OperatorSpec.class); - MessageStream mockLeftInputStream = new MessageStreamImpl<>(mockGraph, mockLeftInputOp); + MessageStream mockLeftInputStream = new MessageStreamImpl<>(mockAppDesc, mockLeftInputOp); when(mockContext.getMessageStream(eq(mockLeftInput.getId()))).thenReturn(mockLeftInputStream); OperatorSpec mockRightInputOp = mock(OperatorSpec.class); - MessageStream mockRightInputStream = new MessageStreamImpl<>(mockGraph, mockRightInputOp); + MessageStream mockRightInputStream = new MessageStreamImpl<>(mockAppDesc, mockRightInputOp); when(mockContext.getMessageStream(eq(mockRightInput.getId()))).thenReturn(mockRightInputStream); - when(mockContext.getStreamAppDescriptor()).thenReturn(mockGraph); + when(mockContext.getStreamAppDescriptor()).thenReturn(mockAppDesc); InputOperatorSpec mockInputOp = mock(InputOperatorSpec.class); OutputStreamImpl mockOutputStream = mock(OutputStreamImpl.class); when(mockInputOp.isKeyed()).thenReturn(true); when(mockOutputStream.isKeyed()).thenReturn(true); IntermediateMessageStreamImpl - mockPartitionedStream = new IntermediateMessageStreamImpl(mockGraph, mockInputOp, mockOutputStream); - when(mockGraph.getIntermediateStream(any(String.class), any(Serde.class))).thenReturn(mockPartitionedStream); + mockPartitionedStream = new IntermediateMessageStreamImpl(mockAppDesc, mockInputOp, mockOutputStream); + when(mockAppDesc.getIntermediateStream(any(String.class), any(Serde.class))).thenReturn(mockPartitionedStream); doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(3), any(MessageStream.class)); RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class); @@ -155,7 +155,7 @@ public void testTranslateStreamToTableJoin() throws IOException, ClassNotFoundEx Expression mockExpr = mock(Expression.class); when(mockCompiler.compile(any(), any())).thenReturn(mockExpr); - doAnswer(this.getRegisteredTableAnswer()).when(mockGraph).getTable(any(RocksDbTableDescriptor.class)); + doAnswer(this.getRegisteredTableAnswer()).when(mockAppDesc).getTable(any(RocksDbTableDescriptor.class)); when(mockJoin.getJoinType()).thenReturn(JoinRelType.INNER); SqlIOResolver mockResolver = mock(SqlIOResolver.class); SqlIOConfig mockIOConfig = mock(SqlIOConfig.class); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java index 24f963ecc0..5409a222a9 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java @@ -91,9 +91,9 @@ public void testTranslate() throws IOException, ClassNotFoundException { List> namedProjects = new ArrayList<>(); namedProjects.add(Pair.of(mockRexField, "test_field")); when(mockProject.getNamedProjects()).thenReturn(namedProjects); - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamAppDescriptorImpl mockAppDesc = mock(StreamAppDescriptorImpl.class); OperatorSpec mockInputOp = mock(OperatorSpec.class); - MessageStream mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp); + MessageStream mockStream = new MessageStreamImpl<>(mockAppDesc, mockInputOp); when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream); doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(2), any(MessageStream.class)); RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class); @@ -183,7 +183,7 @@ public void testTranslateWithFlatten() throws IOException, ClassNotFoundExceptio flattenProjects.add(mockFlattenProject); when(mockProject.getProjects()).thenReturn(flattenProjects); - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamAppDescriptorImpl mockAppDesc = mock(StreamAppDescriptorImpl.class); OperatorSpec mockInputOp = new OperatorSpec(OperatorSpec.OpCode.INPUT, "1") { @Override @@ -197,7 +197,7 @@ public TimerFunction getTimerFn() { } }; - MessageStream mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp); + MessageStream mockStream = new MessageStreamImpl<>(mockAppDesc, mockInputOp); when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream); doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(2), any(MessageStream.class)); RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class); @@ -248,7 +248,7 @@ public TimerFunction getTimerFn() { assertNotNull(projectSpec); assertEquals(projectSpec.getOpCode(), OperatorSpec.OpCode.MAP); - // Verify that the bootstrap() method will establish the context for the map function + // Verify that the describe() method will establish the context for the map function Config mockConfig = mock(Config.class); TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null, new HashSet<>(), null, null, null, null, null, null); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index e31e2f21a6..2cf5e6321e 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -43,8 +43,7 @@ import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; -import static org.mockito.Mockito.*; - +import static org.mockito.Mockito.spy; public class TestQueryTranslator { @@ -112,17 +111,17 @@ public void testTranslate() { validatePerTaskContextInit(appDesc, samzaConfig); } - private void validatePerTaskContextInit(StreamAppDescriptorImpl graphSpec, Config samzaConfig) { + private void validatePerTaskContextInit(StreamAppDescriptorImpl appDesc, Config samzaConfig) { // make sure that each task context would have a separate instance of cloned TranslatorContext TaskContextImpl testContext = new TaskContextImpl(new TaskName("Partition 1"), null, null, new HashSet<>(), null, null, null, null, null, null); // call ContextManager.bootstrap() to instantiate the per-task TranslatorContext - graphSpec.getContextManager().init(samzaConfig, testContext); + appDesc.getContextManager().init(samzaConfig, testContext); Assert.assertNotNull(testContext.getUserContext()); Assert.assertTrue(testContext.getUserContext() instanceof TranslatorContext); TranslatorContext contextPerTaskOne = (TranslatorContext) testContext.getUserContext(); // call ContextManager.bootstrap() second time to instantiate another clone of TranslatorContext - graphSpec.getContextManager().init(samzaConfig, testContext); + appDesc.getContextManager().init(samzaConfig, testContext); Assert.assertTrue(testContext.getUserContext() instanceof TranslatorContext); // validate the two copies of TranslatorContext are clones of each other validateClonedTranslatorContext(contextPerTaskOne, (TranslatorContext) testContext.getUserContext()); 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 b09fb26efe..4782b50aad 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 @@ -64,6 +64,7 @@ public void describe(StreamAppDescriptor appDesc) { Duration.ofMinutes(1), "join") .map(fulFilledOrder -> KV.of(fulFilledOrder.orderId, fulFilledOrder)) .sendTo(fulfilledOrders); + } static class MyJoinFunction implements JoinFunction { 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 0d4683a027..44df62acef 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 @@ -22,12 +22,14 @@ import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; -import org.apache.samza.runtime.ApplicationClassUtils; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.storage.kv.RocksDbTableDescriptor; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.task.MessageCollector; import org.apache.samza.task.StreamTask; import org.apache.samza.task.StreamTaskFactory; +import org.apache.samza.task.TaskCoordinator; import org.apache.samza.util.CommandLine; @@ -36,12 +38,19 @@ */ public class TaskApplicationExample implements TaskApplication { + public class MyStreamTask implements StreamTask { + + @Override + public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) + throws Exception { + + } + } + public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - //ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TaskApplicationExample(), config); - // ==> shouldn't be in user code example - ApplicationRunner runner = ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TaskApplicationExample(), config); runner.run(); runner.waitForFinish(); } @@ -54,8 +63,7 @@ public void describe(TaskAppDescriptor appDesc) { TableDescriptor td = new RocksDbTableDescriptor("mytable"); appDesc.addTable(td); // create the task factory based on configuration - appDesc.setTaskFactory((StreamTaskFactory) () -> (StreamTask) (envelope, collector, coordinator) -> { - }); + appDesc.setTaskFactory((StreamTaskFactory) () -> new MyStreamTask()); } -} +} \ No newline at end of file diff --git a/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java b/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java index beda1e2f1b..2e27a4c057 100644 --- a/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java +++ b/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemConsumer.java @@ -62,7 +62,7 @@ public class MockSystemConsumer extends BlockingEnvelopeMap { * The number of messages to add to the BlockingEnvelopeMap before * sleeping. * @param threadCount - * How many threads to start. + * How many threads to run. * @param brokerSleepMs * How long each thread should sleep between batch writes. */ diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index cad3af3968..214446308f 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -20,6 +20,7 @@ package org.apache.samza.test.integration; import joptsimple.OptionSet; +import org.apache.samza.application.ApplicationBase; import org.apache.samza.runtime.ApplicationClassUtils; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; @@ -45,15 +46,15 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationRunner - appRuntime = ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); + ApplicationBase app = ApplicationClassUtils.fromConfig(config); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config); try { - LOGGER.info("Launching stream application: {} to start.", appRuntime); - appRuntime.run(); - appRuntime.waitForFinish(); + LOGGER.info("Launching stream application: {} to run.", app); + runner.run(); + runner.waitForFinish(); } catch (Exception e) { - LOGGER.error("Exception occurred when running application: {}.", appRuntime, e); + LOGGER.error("Exception occurred when running application: {}.", app, e); } } } diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java index 72ebfcff40..5c2855315a 100644 --- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java +++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessor.java @@ -66,7 +66,7 @@ private void testStreamProcessor(String[] processorIds) { } - // start the processors in separate threads + // run the processors in separate threads Thread[] threads = new Thread[processorIds.length]; CountDownLatch[] stopLatches = new CountDownLatch[processorIds.length]; for (int i = 0; i < processorIds.length; i++) { @@ -213,14 +213,14 @@ private void testStreamProcessor(String[] processorIds) { waitUntilMessagesLeftN(totalEventsToGenerate - messageCount); CountDownLatch containerStopped2 = sp2.containerShutdownLatch; - // kill the first processor + // stop the first processor stopProcessor(stopLatch1); // wait until it's really down waitForProcessorToStartStop(waitStop1); - // processor2 will kill it container and start again. - // We wait for the container's kill to make sure we can count EXACTLY how many messages it reads. + // processor2 will stop it container and start again. + // We wait for the container's stop to make sure we can count EXACTLY how many messages it reads. LOG.info("containerStopped latch = " + containerStopped2); waitForProcessorToStartStop(containerStopped2); 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 f31a8cf782..5a0175fab9 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 @@ -48,7 +48,7 @@ /** * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input. - * It verifies the pipeline will kill and the number of output messages should equal to the input. + * It verifies the pipeline will stop and the number of output messages should equal to the input. */ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness { @@ -113,8 +113,6 @@ public void describe(StreamAppDescriptor appDesc) { assertEquals(received.size(), count * partitionCount); } - - public static final class Values { public static > MapFunction create() { return (M m) -> m.getValue(); 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 11427104c4..33f456c037 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 @@ -71,7 +71,7 @@ import org.junit.Test; import scala.collection.JavaConverters; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness { @@ -160,10 +160,11 @@ public void describe(StreamAppDescriptor appDesc) { final ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TestStreamApp(), new MapConfig(configs)); runner.run(); + // processors are only available when the app is running Map tasks = getTaskOperationGraphs((LocalApplicationRunner) runner); runner.waitForFinish(); - + // wait for the completion to ensure that all tasks are actually initialized and the OperatorImplGraph is initialized StreamOperatorTask task0 = tasks.get("Partition 0"); OperatorImplGraph graph = TestStreamOperatorTask.getOperatorImplGraph(task0); OperatorImpl pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY); 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 0fcdde619c..3ba4d5d797 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 @@ -35,8 +35,7 @@ import org.junit.Assert; import org.junit.Test; -import static org.apache.samza.test.controlmessages.TestData.*; - +import static org.apache.samza.test.controlmessages.TestData.PageView; public class StreamApplicationIntegrationTest { diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java index fe18e65d38..e7a1534ade 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTestHarness.java @@ -254,11 +254,11 @@ protected RunApplicationContext runApplication(StreamApplication streamApplicati } Config config = new MapConfig(configMap); - ApplicationRunner appRuntime = ApplicationRunners.getApplicationRunner(streamApplication, config); - appRuntime.run(); + ApplicationRunner runner = ApplicationRunners.getApplicationRunner(streamApplication, config); + runner.run(); MessageStreamAssert.waitForComplete(); - return new RunApplicationContext(appRuntime, config); + return new RunApplicationContext(runner, config); } /** @@ -277,16 +277,16 @@ public void tearDown() { * runApplication in order to do verification. */ protected static class RunApplicationContext { - private final ApplicationRunner appRuntime; + private final ApplicationRunner runner; private final Config config; - private RunApplicationContext(ApplicationRunner appRuntime, Config config) { + private RunApplicationContext(ApplicationRunner runner, Config config) { this.config = config; - this.appRuntime = appRuntime; + this.runner = runner; } - public ApplicationRunner getAppRuntime() { - return this.appRuntime; + public ApplicationRunner getRunner() { + return this.runner; } public Config getConfig() { 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 1c04a118e5..a4ddcd00cc 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 @@ -51,10 +51,6 @@ public class RepartitionJoinWindowApp implements StreamApplication { private final List intermediateStreamIds = new ArrayList<>(); - List getIntermediateStreamIds() { - return intermediateStreamIds; - } - @Override public void describe(StreamAppDescriptor appDesc) { String inputTopicName1 = appDesc.getConfig().get(INPUT_TOPIC_NAME_1_PROP); @@ -93,9 +89,15 @@ public void describe(StreamAppDescriptor appDesc) { messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue())); }); + intermediateStreamIds.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamId()); intermediateStreamIds.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamId()); intermediateStreamIds.add(((IntermediateMessageStreamImpl) userPageAdClicksByUserId).getStreamId()); + + } + + List getIntermediateStreamIds() { + return intermediateStreamIds; } private static class UserPageViewAdClicksJoiner implements JoinFunction { 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 c1af59b193..85d8fb7321 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 @@ -19,7 +19,6 @@ package org.apache.samza.test.operator; -import java.io.IOException; import java.time.Duration; import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.application.StreamApplication; @@ -39,6 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + /** * A {@link StreamApplication} that demonstrates a filter followed by a tumbling window. */ @@ -49,10 +49,11 @@ public class TumblingWindowApp implements StreamApplication { private static final Logger LOG = LoggerFactory.getLogger(TumblingWindowApp.class); private static final String FILTER_KEY = "badKey"; - public static void main(String[] args) throws IOException { + public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new TumblingWindowApp(), config); + runner.run(); runner.waitForFinish(); } diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 9791a73ad0..1090db0019 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -71,8 +71,9 @@ /** * Integration tests for {@link org.apache.samza.runtime.LocalApplicationRunner} with {@link ZkJobCoordinatorFactory}. * - * Brings up embedded ZooKeeper, Kafka broker and launches multiple {@link org.apache.samza.runtime.LocalApplicationRunner}s - * in an application to verify the guarantees made in stand alone execution environment. + * Brings up embedded ZooKeeper, Kafka broker and launches multiple {@link org.apache.samza.application.StreamApplication} + * through {@link org.apache.samza.runtime.LocalApplicationRunner} to verify the guarantees made in stand alone execution + * environment. */ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarness { @@ -301,6 +302,7 @@ public void shouldStopNewProcessorsJoiningGroupWhenNumContainersIsGreaterThanNum * A) JobModel generated before and after the addition of streamApp2 should not be equal. * B) Second stream application(streamApp2) should join the group and process all the messages. */ + @Test public void shouldUpdateJobModelWhenNewProcessorJoiningGroupUsingAllSspToSingleTaskGrouperFactory() throws InterruptedException { // Set up kafka topics. 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 b9417bd8cb..7915c55a42 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 @@ -67,11 +67,15 @@ import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyList; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** 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 2e34d33044..66c4624ac5 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 @@ -120,6 +120,16 @@ private void runTest(String systemName, StreamApplication app, List> table = appDesc.getTable(getTableDescriptor()); + + appDesc.getInputStream(PAGEVIEW_STREAM, new NoOpSerde()) + .partitionBy(TestTableData.PageView::getMemberId, v -> v, "partition-page-view") + .join(table, new TestLocalTable.PageViewToProfileJoinFunction()) + .sendTo(appDesc.getOutputStream(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<>())); + } + protected TableDescriptor getTableDescriptor() { return new InMemoryTableDescriptor(PROFILE_TABLE) .withSerde(KVSerde.of(new IntegerSerde(), new TestTableData.ProfileJsonSerde())) @@ -131,16 +141,6 @@ static class PageViewProfileJoin implements StreamApplication { return ImmutableList.of(new Entry<>(key, profile)); }); } - - @Override - public void describe(StreamAppDescriptor appDesc) { - Table> table = appDesc.getTable(getTableDescriptor()); - - appDesc.getInputStream(PAGEVIEW_STREAM, new NoOpSerde()) - .partitionBy(TestTableData.PageView::getMemberId, v -> v, "partition-page-view") - .join(table, new TestLocalTable.PageViewToProfileJoinFunction()) - .sendTo(appDesc.getOutputStream(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<>())); - } } static class DurablePageViewProfileJoin extends PageViewProfileJoin { diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java index 0cb355cfbf..a66e69dc06 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java @@ -62,7 +62,8 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; public class TestRemoteTable extends AbstractIntegrationTestHarness { diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala index 7c02f37648..864d2e57ff 100644 --- a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala +++ b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala @@ -194,7 +194,7 @@ object StreamTaskTestUtil { } /* This class implement the base utility to implement an integration test for StreamTask - * It implements helper functions to start/kill the job, send messages to a task, and read all messages from a topic + * It implements helper functions to start/stop the job, send messages to a task, and read all messages from a topic */ class StreamTaskTestUtil { import StreamTaskTestUtil._ diff --git a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java index df68f089a2..bfd217c33f 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/SamzaSqlConsole.java @@ -107,8 +107,8 @@ public static void main(String[] args) { public static void executeSql(List sqlStmts) { Map staticConfigs = fetchSamzaSqlConfig(); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); - SamzaSqlApplicationRunner appRunnable = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); - appRunnable.runAndWaitForFinish(); + SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); + runner.runAndWaitForFinish(); } public static Map fetchSamzaSqlConfig() { diff --git a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java index 6b8fb74d1f..e68f644af7 100644 --- a/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java +++ b/samza-tools/src/main/java/org/apache/samza/tools/benchmark/SystemConsumerWithSamzaBench.java @@ -70,8 +70,8 @@ public void addMoreSystemConfigs(Properties props) { public void start() throws IOException, InterruptedException { super.start(); MessageConsumer consumeFn = new MessageConsumer(); - StreamApplication app = spec -> { - MessageStream stream = spec.getInputStream(streamId); + StreamApplication app = appDesc -> { + MessageStream stream = appDesc.getInputStream(streamId); stream.map(consumeFn); }; ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, new MapConfig()); From 6e446fe6d201ab8b1e30ceeeb546a990ae2b77b2 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Wed, 22 Aug 2018 18:36:33 -0700 Subject: [PATCH 12/38] SAMZA-1789: address Cameron's review comments. --- .../application/ApplicationDescriptor.java | 20 ++-- .../application/StreamAppDescriptor.java | 2 +- .../samza/application/StreamApplication.java | 12 ++- .../samza/application/TaskAppDescriptor.java | 15 +-- .../samza/application/TaskApplication.java | 14 ++- .../samza/runtime/ProcessorContext.java | 11 +- .../ProcessorLifecycleListenerFactory.java | 3 +- ...Runner.java => MockApplicationRunner.java} | 4 +- .../samza/runtime/TestApplicationRunners.java | 4 +- .../samza/application/AppDescriptorImpl.java | 5 +- .../application/ApplicationDescriptors.java | 13 +++ .../application/StreamAppDescriptorImpl.java | 2 +- .../application/TaskAppDescriptorImpl.java | 7 +- .../samza/processor/StreamProcessor.java | 6 +- .../runtime/AbstractApplicationRunner.java | 44 +++++--- .../samza/runtime/ApplicationClassUtils.java | 9 +- .../samza/runtime/LocalApplicationRunner.java | 46 ++++++-- .../samza/runtime/LocalContainerRunner.java | 19 +--- .../runtime/RemoteApplicationRunner.java | 38 +++++-- .../apache/samza/task/TaskFactoryUtil.java | 31 ++---- .../samza/job/local/ThreadJobFactory.scala | 45 ++++---- ...lication.java => MockTaskApplication.java} | 2 +- .../TestStreamAppDescriptorImpl.java | 3 +- .../runtime/TestApplicationClassUtils.java | 8 +- .../runtime/TestApplicationRunnerMain.java | 3 +- .../runtime/TestLocalApplicationRunner.java | 25 +++-- .../samza/sql/translator/QueryTranslator.java | 2 +- .../sql/translator/TranslatorContext.java | 7 +- .../sql/translator/TestQueryTranslator.java | 102 +++++++++--------- .../src/test/resources/test-config.prop | 26 ----- 30 files changed, 275 insertions(+), 253 deletions(-) rename samza-api/src/test/java/org/apache/samza/runtime/{TestApplicationRunner.java => MockApplicationRunner.java} (93%) rename samza-core/src/test/java/org/apache/samza/application/{TestTaskApplication.java => MockTaskApplication.java} (94%) delete mode 100644 samza-test/src/test/resources/test-config.prop diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index f3e5dcae43..14fe256fa0 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -30,14 +30,14 @@ * Sub-classes {@link StreamAppDescriptor} and {@link TaskAppDescriptor} are specific interfaces for applications written * in high-level DAG and low-level task APIs, respectively. * - * @param type of user application this descriptor describes. It has to be either {@link StreamApplication} or - * {@link TaskApplication} + * @param sub-class of user application descriptor. It has to be either {@link StreamAppDescriptor} or + * {@link TaskAppDescriptor} */ @InterfaceStability.Evolving -public interface ApplicationDescriptor { +public interface ApplicationDescriptor { /** - * Get the user defined {@link Config} + * Get the initial {@link Config} supplied to the user application * @return config object */ Config getConfig(); @@ -45,23 +45,27 @@ public interface ApplicationDescriptor { /** * Sets the {@link ContextManager} for this application. *

    - * The provided {@link ContextManager} can be used to setup shared context between the operator functions - * within a task instance + * Setting the {@link ContextManager} is optional. The provided {@link ContextManager} can be used to build the shared + * context between the operator functions within a task instance * * TODO: this should be replaced by the shared context factory when SAMZA-1714 is fixed. * @param contextManager the {@link ContextManager} to use for the application * @return the {@link ApplicationDescriptor} with {@code contextManager} set as its {@link ContextManager} */ - ApplicationDescriptor withContextManager(ContextManager contextManager); + S withContextManager(ContextManager contextManager); /** * Sets the {@link ProcessorLifecycleListenerFactory} for this application. * + *

    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. + * * @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 the {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory} */ - ApplicationDescriptor withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); + S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java index 51bb36325a..d0a3fd5cfd 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java @@ -31,7 +31,7 @@ * The interface class to describe a user application in high-level DAG in Samza. */ @InterfaceStability.Evolving -public interface StreamAppDescriptor extends ApplicationDescriptor { +public interface StreamAppDescriptor extends ApplicationDescriptor { /** * Sets the default {@link Serde} to use for (de)serializing messages. 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 37e7394ce2..cab4430c05 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,7 +21,9 @@ import org.apache.samza.annotation.InterfaceStability; /** - * Describes and initializes the transforms for processing message streams and generating results. + * Describes and initializes the transforms for processing message streams and generating results in high-level API. + *

    + * This is a marker interface that users will implement for a high-level application. *

    * The following example removes page views older than 1 hour from the input stream: *

    {@code
    @@ -57,12 +59,12 @@
      * {@link StreamApplication} when planning the execution. All user-defined transformation functions and descriptors for
      * data entities used in the transformations (e.g. {@link org.apache.samza.operators.TableDescriptor}) are required to
      * be serializable. 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. The user-defined application
    - * class that implements {@link StreamApplication} also needs to be a class with proper fully-qualified class name to ensure
    - * successful instantiation in both local and remote environments. Execution is synchronous and thread-safe within each
    - * {@link org.apache.samza.task.StreamTask}.
    + * {@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}.
      *
      * 

    + * The user-implemented {@link StreamApplication} 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. * 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} diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java index 60edc002cf..dc68574a37 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java @@ -26,14 +26,15 @@ /** * The interface class to describe a user application as low-level task in Samza. */ -@InterfaceStability.Evolving -public interface TaskAppDescriptor extends ApplicationDescriptor { +@InterfaceStability.Unstable +public interface TaskAppDescriptor 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. * - * @param factory the user implemented {@link TaskFactory} including the low-level task processing logic + * @param factory the user implemented {@link TaskFactory} including the low-level task processing logic. The only + * allowed task factor classes are {@link org.apache.samza.task.StreamTaskFactory} and {@link org.apache.samza.task.AsyncStreamTaskFactory}. */ void setTaskFactory(TaskFactory factory); @@ -45,14 +46,6 @@ public interface TaskAppDescriptor extends ApplicationDescriptor + * This is a marker interface that users will implement for a low-level application. *

    * The following example removes page views older than 1 hour from the input stream: *

    {@code
    @@ -63,13 +65,15 @@
      * }
    * *

    - * Implementation Notes: TaskApplication allow users to instantiate {@link org.apache.samza.task.StreamTask} or + * 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 TaskAppDescriptor} * instance will be created and described by the user-defined {@link TaskApplication} when planning the execution. * {@link org.apache.samza.task.TaskFactory} and descriptors for data entities used in the task (e.g. - * {@link org.apache.samza.operators.TableDescriptor}) are required to be serializable. The user-defined application - * class that implements {@link TaskApplication} needs to be a class with proper fully-qualified class name to ensure - * successful instantiation in both local and remote environments. + * {@link org.apache.samza.operators.TableDescriptor}) are 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 ApplicationBase { diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java index 82c76ee6f3..4e9f5ba348 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorContext.java @@ -22,15 +22,10 @@ /** - * The context for a StreamProcessor. Currently, only defines a method to report the processorId + * The context for a StreamProcessor. A placeholder class for the general context for Samza application. * + * TODO: pending change with SAMZA-1714 */ -@InterfaceStability.Evolving +@InterfaceStability.Unstable public interface ProcessorContext { - /** - * Get the unique processorID for the corresponding StreamProcessor - * - * @return the processorId - */ - String getProcessorId(); } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java index b52161b7fb..eec7b70c70 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ProcessorLifecycleListenerFactory.java @@ -31,7 +31,8 @@ public interface ProcessorLifecycleListenerFactory extends Serializable { /** * Create an instance of {@link ProcessorLifecycleListener} for the StreamProcessor * - * @param pContext the context of the corresponding StreamProcessor + * @param pContext the context of the corresponding StreamProcessor. Note that {@link ProcessorContext} is just a + * placeholder before we have a proper implementation of general context in SAMZA-1714 * @param config the configuration of the corresponding StreamProcessor * @return the {@link ProcessorLifecycleListener} callback object for the StreamProcessor */ diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java b/samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java similarity index 93% rename from samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java rename to samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java index c38999b4bd..ca54353c2c 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunner.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java @@ -29,11 +29,11 @@ /** * Test class for {@link org.apache.samza.runtime.ApplicationRunners} unit test */ -public class TestApplicationRunner implements ApplicationRunner { +public class MockApplicationRunner implements ApplicationRunner { private final ApplicationBase userApp; private final Config config; - public TestApplicationRunner(ApplicationBase userApp, Config config) { + public MockApplicationRunner(ApplicationBase userApp, Config config) { this.userApp = userApp; this.config = config; } diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java index 780c23ae87..edd64ea01e 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java @@ -37,10 +37,10 @@ public class TestApplicationRunners { @Test public void testGetAppRunner() { Map configMap = new HashMap<>(); - configMap.put("app.runner.class", TestApplicationRunner.class.getName()); + configMap.put("app.runner.class", MockApplicationRunner.class.getName()); Config config = new MapConfig(configMap); StreamApplication app = mock(StreamApplication.class); ApplicationRunner appRunner = ApplicationRunners.getApplicationRunner(app, config); - assertTrue(appRunner instanceof TestApplicationRunner); + assertTrue(appRunner instanceof MockApplicationRunner); } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java index 2e62030384..6146d9d92c 100644 --- a/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java @@ -31,12 +31,11 @@ * This base class contains the common objects that are used by both high-level and low-level API applications, such as * {@link Config}, {@link ContextManager}, and {@link ProcessorLifecycleListenerFactory}. * - * @param the type of user application * @param the type of {@link ApplicationDescriptor} interface this implements. It has to be either * {@link org.apache.samza.application.StreamAppDescriptor} or {@link org.apache.samza.application.TaskAppDescriptor} */ -public abstract class AppDescriptorImpl> - implements ApplicationDescriptor { +public abstract class AppDescriptorImpl + implements ApplicationDescriptor { final Config config; final Class appClass; diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java index a9cb1bcd0f..28117adb81 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java @@ -18,6 +18,7 @@ */ package org.apache.samza.application; +import java.util.function.Function; import org.apache.samza.config.Config; @@ -46,4 +47,16 @@ public static AppDescriptorImpl getAppDescriptor(ApplicationBase userApp, Config throw new IllegalArgumentException(String.format("User application class %s is not supported. Only StreamApplication " + "and TaskApplication are supported.", userApp.getClass().getName())); } + + public static T forType(Function forTaskApp, Function forStreamApp, + AppDescriptorImpl desc) { + if (desc instanceof TaskAppDescriptorImpl) { + return forTaskApp.apply((TaskAppDescriptorImpl) desc); + } else if (desc instanceof StreamAppDescriptorImpl) { + return forStreamApp.apply((StreamAppDescriptorImpl) desc); + } + + throw new IllegalArgumentException(String.format("AppDescriptorImpl has to be either TaskAppDescriptorImpl or StreamAppDescriptorImpl." + + " class %s is not supported", desc.getClass().getName())); + } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java index 9343f1be09..4b4019bc38 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java @@ -57,7 +57,7 @@ * and {@link Table} to create the DAG of transforms. * 2) a builder that creates a serializable {@link OperatorSpecGraph} from user-defined DAG */ -public class StreamAppDescriptorImpl extends AppDescriptorImpl +public class StreamAppDescriptorImpl extends AppDescriptorImpl implements StreamAppDescriptor { private static final Logger LOGGER = LoggerFactory.getLogger(StreamAppDescriptorImpl.class); private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java index 8d07a6e970..35b43666f9 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java @@ -33,7 +33,7 @@ * the low-level {@link TaskFactory} that creates user-defined task instances, the lists of input/broadcast/output streams, * and the list of {@link TableDescriptor}s used in the application. */ -public class TaskAppDescriptorImpl extends AppDescriptorImpl +public class TaskAppDescriptorImpl extends AppDescriptorImpl implements TaskAppDescriptor { TaskFactory taskFactory; @@ -58,11 +58,6 @@ public void addInputStream(String inputStream) { this.inputStreams.add(inputStream); } - @Override - public void addBroadcastStream(String broadcastStream) { - this.broadcastStreams.add(broadcastStream); - } - @Override public void addOutputStream(String outputStream) { this.outputStreams.add(outputStream); 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 c4a4276b04..6c0ccff865 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 @@ -31,7 +31,6 @@ import java.util.function.Function; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.TaskConfigJava; import org.apache.samza.container.IllegalContainerStateException; @@ -297,8 +296,7 @@ public void stop() { * @return the {@link ProcessorContext} object */ public ProcessorContext getProcessorContext() { - JobConfig jobConfig = new JobConfig(config); - return () -> String.format("%s-%s-%s", jobConfig.getName(), jobConfig.getJobId(), processorId); + return new ProcessorContext() { }; } /** @@ -425,7 +423,7 @@ class ContainerListener implements SamzaContainerListener { @Override public void beforeStart() { - // processorListener.beforeStart() is invoked at the StreamProcessorLifecycleListener + // processorListener.beforeStart() is invoked in StreamProcessor.start() } @Override diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 8fa4d26dc1..48ea21c7d9 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; +import org.apache.samza.SamzaException; import org.apache.samza.application.AppDescriptorImpl; import org.apache.samza.application.ApplicationBase; import org.apache.samza.application.ApplicationDescriptors; @@ -67,19 +68,36 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { * * TODO: Fix SAMZA-1811 to consolidate the planning into {@link ExecutionPlanner} */ - abstract class JobConfigPlanner { + static abstract class JobPlanner { - abstract List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throws Exception; + protected final AppDescriptorImpl appDesc; + protected final Config config; - List createJobConfigs() throws Exception { - if (appDesc instanceof TaskAppDescriptorImpl) { - // low-level task application only needs a simple single job configuration - return Collections.singletonList(getTaskJobConfig((TaskAppDescriptorImpl) appDesc)); - } else if (appDesc instanceof StreamAppDescriptorImpl) { - return getStreamJobConfigs((StreamAppDescriptorImpl) appDesc); - } + JobPlanner(AppDescriptorImpl descriptor) { + this.appDesc = descriptor; + this.config = descriptor.getConfig(); + } - throw new IllegalArgumentException("ApplicationDescriptor class " + appDesc.getClass().getName() + " is not supported"); + abstract List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception; + + List prepareJobs() throws Exception { + String appId = new ApplicationConfig(appDesc.getConfig()).getGlobalAppId(); + return ApplicationDescriptors.forType( + taskAppDesc -> { + try { + return Collections.singletonList(JobPlanner.this.prepareTaskJob(taskAppDesc)); + } catch (Exception e) { + throw new SamzaException("Failed to generate JobConfig for TaskApplication " + appId, e); + } + }, + streamAppDesc -> { + try { + return JobPlanner.this.prepareStreamJobs(streamAppDesc); + } catch (Exception e) { + throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); + } + }, + appDesc); } StreamManager buildAndStartStreamManager() { @@ -136,7 +154,7 @@ final void writePlanJsonFile(String planJson) { } // helper method to generate a single node job configuration for low level task applications - private JobConfig getTaskJobConfig(TaskAppDescriptorImpl taskAppDesc) { + private JobConfig prepareTaskJob(TaskAppDescriptorImpl taskAppDesc) { Map cfg = new HashMap<>(taskAppDesc.getConfig()); //TODO: add stream and system descriptor to configuration conversion here when SAMZA-1804 is fixed. // adding table configuration @@ -158,8 +176,8 @@ private void validateAppClassCfg(Map cfg, Class appClass = (Class) Class.forName(appConfig.getAppClass()); + Class appClass = (Class) Class.forName(appClassName); if (StreamApplication.class.isAssignableFrom(appClass) || TaskApplication.class.isAssignableFrom(appClass)) { return appClass.newInstance(); } } catch (IllegalAccessException | InstantiationException | ClassNotFoundException e) { throw new ConfigException(String.format("Loading app.class %s failed. The user application has to implement " - + "StreamApplication or TaskApplication.", appConfig.getAppClass()), e); + + "StreamApplication or TaskApplication.", appClassName), e); } } // no app.class defined. It has to be a legacy application with task.class configuration diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 3b8dd91bf5..6032e631a4 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -33,6 +33,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.AppDescriptorImpl; import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; @@ -60,11 +61,11 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private static final String APPLICATION_RUNNER_PATH_SUFFIX = "/ApplicationRunnerData"; private final String uid; + private final LocalJobPlanner planner; private final Set processors = ConcurrentHashMap.newKeySet(); private final CountDownLatch shutdownLatch = new CountDownLatch(1); private final AtomicInteger numProcessorsToStart = new AtomicInteger(); private final AtomicReference failure = new AtomicReference<>(); - private final LocalJobConfigPlanner planner; private ApplicationStatus appStatus = ApplicationStatus.New; @@ -139,15 +140,22 @@ private void shutdownAndNotify() { } /** - * Defines a {@link JobConfigPlanner} with specific implementation of {@link JobConfigPlanner#getStreamJobConfigs(StreamAppDescriptorImpl)} + * Defines a {@link JobPlanner} with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} * for standalone Samza processors. * * TODO: we need to consolidate all planning logic into {@link org.apache.samza.execution.ExecutionPlanner} after SAMZA-1811. */ @VisibleForTesting - class LocalJobConfigPlanner extends JobConfigPlanner { + static class LocalJobPlanner extends JobPlanner { + private final String uid; + + LocalJobPlanner(AppDescriptorImpl descriptor, String uid) { + super(descriptor); + this.uid = uid; + } + @Override - List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { // for high-level DAG, generating the plan and job configs StreamManager streamManager = null; try { @@ -163,7 +171,7 @@ List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throw // 2. create the necessary streams // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 String planId = String.valueOf(executionPlanJson.hashCode()); - createStreams(planId, plan.getIntermediateStreams(), streamManager); + createStreams(planId, plan.getIntermediateStreams(), streamManager, config, uid); return plan.getJobConfigs(); } finally { @@ -174,16 +182,33 @@ List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throw } } + /** + * Default constructor that is required by any implementation of {@link ApplicationRunner} + * + * @param userApp user application + * @param config user configuration + */ public LocalApplicationRunner(ApplicationBase userApp, Config config) { - super(userApp, config); + super(ApplicationDescriptors.getAppDescriptor(userApp, config)); + this.uid = UUID.randomUUID().toString(); + this.planner = new LocalJobPlanner(appDesc, uid); + } + + /** + * Constructor only used in unit test to allow injection of {@link LocalJobPlanner} + * + */ + @VisibleForTesting + LocalApplicationRunner(AppDescriptorImpl appDesc, LocalJobPlanner planner) { + super(appDesc); this.uid = UUID.randomUUID().toString(); - this.planner = new LocalJobConfigPlanner(); + this.planner = planner; } @Override public void run() { try { - List jobConfigs = planner.createJobConfigs(); + List jobConfigs = planner.prepareJobs(); // 3. create the StreamProcessors if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs to run."); @@ -251,9 +276,8 @@ public boolean waitForFinish(Duration timeout) { * @param intStreams list of intermediate {@link StreamSpec}s * @throws TimeoutException exception for latch timeout */ - private void createStreams(String planId, - List intStreams, - StreamManager streamManager) throws TimeoutException { + private static void createStreams(String planId, List intStreams, StreamManager streamManager, + Config config, String uid) throws TimeoutException { if (intStreams.isEmpty()) { LOG.info("Set of intermediate streams is empty. Nothing to create."); return; diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 7ec7baf45a..33660c902e 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -23,12 +23,8 @@ import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; -import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -80,18 +76,12 @@ public static void main(String[] args) throws Exception { MDC.put("jobName", jobName); MDC.put("jobId", jobId); - AppDescriptorImpl appDesc = getAppDesc(config); + AppDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(ApplicationClassUtils.fromConfig(config), config); run(appDesc, containerId, jobModel, config); System.exit(0); } - private static AppDescriptorImpl getAppDesc(Config config) { - ApplicationBase userApp = ApplicationClassUtils.fromConfig(config); - return userApp instanceof StreamApplication ? new StreamAppDescriptorImpl((StreamApplication) userApp, config) : - new TaskAppDescriptorImpl((TaskApplication) userApp, config); - } - private static void run(AppDescriptorImpl appDesc, String containerId, JobModel jobModel, Config config) { TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); SamzaContainer container = SamzaContainer$.MODULE$.apply( @@ -101,9 +91,8 @@ private static void run(AppDescriptorImpl appDesc, String containerId, JobModel ScalaJavaUtil.toScalaMap(new HashMap<>()), taskFactory); - JobConfig jobConfig = new JobConfig(config); - ProcessorContext pContext = () -> String.format("%s-%s-%s", jobConfig.getName(), jobConfig.getJobId(), containerId); - ProcessorLifecycleListener pListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(pContext, config); + ProcessorLifecycleListener pListener = appDesc.getProcessorLifecycleListenerFactory() + .createInstance(new ProcessorContext() { }, config); container.setContainerListener( new SamzaContainerListener() { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 19b20be7d1..02905fee07 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -19,11 +19,14 @@ package org.apache.samza.runtime; +import com.google.common.annotations.VisibleForTesting; import java.time.Duration; import java.util.List; import java.util.UUID; import org.apache.samza.SamzaException; +import org.apache.samza.application.AppDescriptorImpl; import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; @@ -47,17 +50,22 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; - private final RemoteJobConfigPlanner planner; + private final RemoteJobPlanner planner; /** - * Defines a {@link JobConfigPlanner} with specific implementation of {@link JobConfigPlanner#getStreamJobConfigs(StreamAppDescriptorImpl)} + * Defines a {@link JobPlanner} with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} * for remote-launched Samza processors (e.g. in YARN). * * TODO: we need to consolidate all planning logic into {@link org.apache.samza.execution.ExecutionPlanner} after SAMZA-1811. */ - private class RemoteJobConfigPlanner extends JobConfigPlanner { + class RemoteJobPlanner extends JobPlanner { + + RemoteJobPlanner(AppDescriptorImpl descriptor) { + super(descriptor); + } + @Override - List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { // for high-level DAG, generate the plan and job configs StreamManager streamManager = null; try { @@ -86,15 +94,31 @@ List getStreamJobConfigs(StreamAppDescriptorImpl streamAppDesc) throw } } + /** + * Default constructor that is required by any implementation of {@link ApplicationRunner} + * + * @param userApp user application + * @param config user configuration + */ RemoteApplicationRunner(ApplicationBase userApp, Config config) { - super(userApp, config); - this.planner = new RemoteJobConfigPlanner(); + super(ApplicationDescriptors.getAppDescriptor(userApp, config)); + this.planner = new RemoteJobPlanner(appDesc); + } + + /** + * Constructor only used in unit test to allow injection of {@link LocalApplicationRunner.LocalJobPlanner} + * + */ + @VisibleForTesting + RemoteApplicationRunner(AppDescriptorImpl appDesc, RemoteJobPlanner planner) { + super(appDesc); + this.planner = planner; } @Override public void run() { try { - List jobConfigs = planner.createJobConfigs(); + List jobConfigs = planner.prepareJobs(); if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs to run."); } diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index cc847dc3b5..f7d03bc026 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -20,13 +20,10 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.application.TaskAppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.TaskConfig; -import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.OperatorSpecGraph; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,28 +42,14 @@ public class TaskFactoryUtil { * Creates a {@link TaskFactory} based on {@link AppDescriptorImpl} * * @param appDesc {@link AppDescriptorImpl} for this application - * @return {@link TaskFactory} object + * @return {@link TaskFactory} object defined by {@code appDesc} */ public static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { - if (appDesc instanceof StreamAppDescriptorImpl) { - StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; - return (StreamTaskFactory) () -> new StreamOperatorTask(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()); - } else if (appDesc instanceof TaskAppDescriptorImpl) { - TaskAppDescriptorImpl taskAppDescriptor = (TaskAppDescriptorImpl) appDesc; - return taskAppDescriptor.getTaskFactory(); - } - throw new IllegalArgumentException("Invalid ApplicationDescriptor class " + appDesc.getClass().getName()); - } - - /** - * Creates a {@link TaskFactory} based on the {@link OperatorSpecGraph} and {@link ContextManager} - * - * @param specGraph the {@link OperatorSpecGraph} - * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph} - * @return a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} - */ - public static TaskFactory createTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) { - return (StreamTaskFactory) () -> new StreamOperatorTask(specGraph, contextManager); + return ApplicationDescriptors.forType( + taskAppDesc -> taskAppDesc.getTaskFactory(), + streamAppDesc -> (StreamTaskFactory) () -> new StreamOperatorTask(streamAppDesc.getOperatorSpecGraph(), + streamAppDesc.getContextManager()), + appDesc); } /** diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index a6b2d282a9..ce22a7146a 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,17 +19,19 @@ package org.apache.samza.job.local +import java.util.concurrent.CountDownLatch + import org.apache.samza.application.ApplicationDescriptors import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ -import org.apache.samza.container.TaskName -import org.apache.samza.container.{SamzaContainer, SamzaContainerListener} -import org.apache.samza.coordinator.JobModelManager +import org.apache.samza.container.{SamzaContainer, SamzaContainerListener, TaskName} +import org.apache.samza.coordinator.{JobCoordinator, JobModelManager} import org.apache.samza.coordinator.stream.CoordinatorStreamManager import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} -import org.apache.samza.runtime.ApplicationClassUtils +import org.apache.samza.processor.StreamProcessor +import org.apache.samza.runtime.{ApplicationClassUtils, ProcessorContext, ProcessorLifecycleListener} import org.apache.samza.storage.ChangelogStreamManager import org.apache.samza.task.TaskFactory import org.apache.samza.task.TaskFactoryUtil @@ -74,8 +76,8 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val containerId = "0" val jmxServer = new JmxServer - val taskFactory : TaskFactory[_] = TaskFactoryUtil.getTaskFactory(ApplicationDescriptors.getAppDescriptor( - ApplicationClassUtils.fromConfig(config), config)) + val appDesc = ApplicationDescriptors.getAppDescriptor(ApplicationClassUtils.fromConfig(config), config) + val taskFactory : TaskFactory[_] = TaskFactoryUtil.getTaskFactory(appDesc) // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job. config.getTaskOpts match { @@ -84,23 +86,24 @@ class ThreadJobFactory extends StreamJobFactory with Logging { case _ => None } - val containerListener = new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { - error("Container failed.", t) - throw t - } + val pListener = { + val userListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(new ProcessorContext() { }, config) + new SamzaContainerListener { + override def onContainerFailed(t: Throwable): Unit = { + userListener.afterFailure(t) + } - override def onContainerStop(): Unit = { - } + override def onContainerStart(): Unit = { + userListener.afterStart() + } - override def onContainerStart(): Unit = { - - } + override def onContainerStop(): Unit = { + userListener.afterStop() + } - /** - * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started - */ - override def beforeStart(): Unit = { + override def beforeStart(): Unit = { + userListener.beforeStart() + } } } @@ -113,7 +116,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging { config, Map[String, MetricsReporter](), taskFactory) - container.setContainerListener(containerListener) + container.setContainerListener(pListener) val threadJob = new ThreadJob(container) threadJob diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java b/samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java similarity index 94% rename from samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java rename to samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java index 87381628e6..bb145a0e23 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplication.java +++ b/samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java @@ -21,7 +21,7 @@ /** * Test class of {@link TaskApplication} for unit tests */ -public class TestTaskApplication implements TaskApplication { +public class MockTaskApplication implements TaskApplication { @Override public void describe(TaskAppDescriptor appSpec) { diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java index 982ff7e29c..c56fc36d05 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java @@ -47,7 +47,6 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -62,7 +61,7 @@ public void testConstructor() { StreamApplication mockApp = mock(StreamApplication.class); Config mockConfig = mock(Config.class); StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(mockApp, mockConfig); - verify(mockApp, times(1)).describe(appDesc); + verify(mockApp).describe(appDesc); assertEquals(mockConfig, appDesc.config); } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java index 07fb895b9d..03cd26b658 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java @@ -24,7 +24,7 @@ import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.application.TaskApplication; import org.apache.samza.application.TestStreamApplication; -import org.apache.samza.application.TestTaskApplication; +import org.apache.samza.application.MockTaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; @@ -56,13 +56,13 @@ public void testStreamAppClass() { @Test public void testTaskAppClass() { Map configMap = new HashMap<>(); - configMap.put(ApplicationConfig.APP_CLASS, TestTaskApplication.class.getName()); + configMap.put(ApplicationConfig.APP_CLASS, MockTaskApplication.class.getName()); ApplicationBase app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); - assertTrue(app instanceof TestTaskApplication); + assertTrue(app instanceof MockTaskApplication); configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); - assertTrue(app instanceof TestTaskApplication); + assertTrue(app instanceof MockTaskApplication); } @Test diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index a590dd2fdf..2adbc682b9 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -20,6 +20,7 @@ import java.time.Duration; import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamAppDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; @@ -85,7 +86,7 @@ public static class TestApplicationRunnerInvocationCounts extends AbstractApplic protected static int statusCount = 0; public TestApplicationRunnerInvocationCounts(ApplicationBase userApp, Config config) { - super(userApp, config); + super(ApplicationDescriptors.getAppDescriptor(userApp, config)); } @Override diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 759f087d72..bc3f77d6a9 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -27,7 +27,10 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.AppDescriptorImpl; import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; @@ -42,7 +45,7 @@ import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.processor.StreamProcessor; -import org.apache.samza.runtime.LocalApplicationRunner.LocalJobConfigPlanner; +import org.apache.samza.runtime.LocalApplicationRunner.LocalJobPlanner; import org.apache.samza.system.StreamSpec; import org.junit.Before; import org.junit.Test; @@ -84,13 +87,13 @@ public class TestLocalApplicationRunner { private Config config; private ApplicationBase mockApp; private LocalApplicationRunner runner; - private LocalJobConfigPlanner localPlanner; + private LocalJobPlanner localPlanner; @Before public void setUp() { - this.config = new MapConfig(config); + config = new MapConfig(); mockApp = mock(StreamApplication.class); - prepareTest(); + prepareTest(mockApp, config); } @Test @@ -171,7 +174,7 @@ public void testRunStreamTask() cfgs.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); config = new MapConfig(cfgs); mockApp = ApplicationClassUtils.fromConfig(config); - prepareTest(); + prepareTest(mockApp, config); StreamProcessor sp = mock(StreamProcessor.class); @@ -204,7 +207,7 @@ public void testRunComplete() mockApp = (StreamApplication) appDesc -> { appDesc.withProcessorLifecycleListenerFactory(mockFactory); }; - prepareTest(); + prepareTest(mockApp, config); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -246,7 +249,7 @@ public void testRunFailure() mockApp = (StreamApplication) appDesc -> { appDesc.withProcessorLifecycleListenerFactory(mockFactory); }; - prepareTest(); + prepareTest(mockApp, config); // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); @@ -355,10 +358,10 @@ public void testWaitForFinishTimesout() { assertFalse("Application finished before the timeout.", finished); } - private void prepareTest() { - runner = spy(new LocalApplicationRunner(mockApp, config)); - localPlanner = spy((LocalJobConfigPlanner) Whitebox.getInternalState(runner, "planner")); - Whitebox.setInternalState(runner, "planner", localPlanner); + private void prepareTest(ApplicationBase userApp, Config config) { + AppDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); + localPlanner = spy(new LocalApplicationRunner.LocalJobPlanner(appDesc, "test-planner")); + runner = spy(new LocalApplicationRunner(appDesc, localPlanner)); } private String getExecutionPlanId(List updatedStreamSpecs) { diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index 77a6bf0daf..f25bdd0419 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -98,7 +98,7 @@ public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamAppDescript sqlConfig.getUdfMetadata()); final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig); final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery()); - final TranslatorContext context = new TranslatorContext((StreamAppDescriptorImpl)appDesc, relRoot, executionContext, this.converters); + final TranslatorContext context = new TranslatorContext(appDesc, relRoot, executionContext, this.converters); final RelNode node = relRoot.project(); final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver(); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java index 88c93b544d..cd48e39436 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java @@ -32,7 +32,6 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.SchemaPlus; import org.apache.samza.application.StreamAppDescriptor; -import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.operators.MessageStream; import org.apache.samza.sql.data.RexToJavaCompiler; import org.apache.samza.sql.data.SamzaSqlExecutionContext; @@ -46,7 +45,7 @@ public class TranslatorContext implements Cloneable { /** * The internal variables that are shared among all cloned {@link TranslatorContext} */ - private final StreamAppDescriptorImpl streamAppDesc; + private final StreamAppDescriptor streamAppDesc; private final RexToJavaCompiler compiler; private final Map relSamzaConverters; private final Map messsageStreams; @@ -123,12 +122,12 @@ private TranslatorContext(TranslatorContext other) { /** * Create the instance of TranslatorContext - * @param stramAppDesc Samza's streamGraph that is populated during the translation. + * @param stramAppDesc Samza's streamAppDesc that is populated during the translation. * @param relRoot Root of the relational graph from calcite. * @param executionContext the execution context * @param converters the map of schema to RelData converters */ - TranslatorContext(StreamAppDescriptorImpl stramAppDesc, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map converters) { + TranslatorContext(StreamAppDescriptor stramAppDesc, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map converters) { this.streamAppDesc = stramAppDesc; this.compiler = createExpressionCompiler(relRoot); this.executionContext = executionContext; diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index 2cf5e6321e..23170e5108 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -139,10 +139,10 @@ public void testTranslateComplex() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); + translator.translate(queryInfo, streamAppDesc); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -159,7 +159,7 @@ public void testTranslateComplex() { Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("COMPLEX1", inputPhysicalName); - validatePerTaskContextInit(mockSpec, samzaConfig); + validatePerTaskContextInit(streamAppDesc, samzaConfig); } @Test @@ -171,10 +171,10 @@ public void testTranslateSubQuery() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); + translator.translate(queryInfo, streamAppDesc); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String inputStreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -191,7 +191,7 @@ public void testTranslateSubQuery() { Assert.assertEquals("testavro", inputSystem); Assert.assertEquals("COMPLEX1", inputPhysicalName); - validatePerTaskContextInit(mockSpec, samzaConfig); + validatePerTaskContextInit(streamAppDesc, samzaConfig); } @Test (expected = SamzaException.class) @@ -207,9 +207,9 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -226,9 +226,9 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = IllegalStateException.class) @@ -245,9 +245,9 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -264,8 +264,8 @@ public void testTranslateStreamTableJoinWithThetaCondition() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -280,8 +280,8 @@ public void testTranslateStreamTableCrossJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -298,8 +298,8 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -317,8 +317,8 @@ public void testTranslateStreamTableJoinWithSubQuery() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -335,8 +335,8 @@ public void testTranslateTableTableJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -353,8 +353,8 @@ public void testTranslateStreamStreamJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -371,8 +371,8 @@ public void testTranslateJoinWithIncorrectLeftJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -389,8 +389,8 @@ public void testTranslateJoinWithIncorrectRightJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -411,8 +411,8 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test (expected = SamzaException.class) @@ -429,8 +429,8 @@ public void testTranslateStreamTableInnerJoinWithUdf() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } @Test @@ -447,10 +447,10 @@ public void testTranslateStreamTableInnerJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); + translator.translate(queryInfo, streamAppDesc); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -483,7 +483,7 @@ public void testTranslateStreamTableInnerJoin() { Assert.assertEquals("kafka", input3System); Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); - validatePerTaskContextInit(mockSpec, samzaConfig); + validatePerTaskContextInit(streamAppDesc, samzaConfig); } @Test @@ -500,11 +500,11 @@ public void testTranslateStreamTableLeftJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); + translator.translate(queryInfo, streamAppDesc); - OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -537,7 +537,7 @@ public void testTranslateStreamTableLeftJoin() { Assert.assertEquals("kafka", input3System); Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); - validatePerTaskContextInit(mockSpec, samzaConfig); + validatePerTaskContextInit(streamAppDesc, samzaConfig); } @Test @@ -554,10 +554,10 @@ public void testTranslateStreamTableRightJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); - OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); StreamConfig streamConfig = new StreamConfig(samzaConfig); String input1StreamId = specGraph.getInputOperators().keySet().stream().findFirst().get(); @@ -590,7 +590,7 @@ public void testTranslateStreamTableRightJoin() { Assert.assertEquals("kafka", input3System); Assert.assertEquals("sql-job-1-partition_by-stream_1", input3PhysicalName); - validatePerTaskContextInit(mockSpec, samzaConfig); + validatePerTaskContextInit(streamAppDesc, samzaConfig); } @Test @@ -607,10 +607,10 @@ public void testTranslateGroupBy() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); - translator.translate(queryInfo, mockSpec); - OperatorSpecGraph specGraph = mockSpec.getOperatorSpecGraph(); + translator.translate(queryInfo, streamAppDesc); + OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); Assert.assertEquals(1, specGraph.getInputOperators().size()); Assert.assertEquals(1, specGraph.getOutputStreams().size()); @@ -631,7 +631,7 @@ public void testTranslateGroupByWithSumAggregator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl mockSpec = spy(new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig)); - translator.translate(queryInfo, mockSpec); + StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + translator.translate(queryInfo, streamAppDesc); } } diff --git a/samza-test/src/test/resources/test-config.prop b/samza-test/src/test/resources/test-config.prop deleted file mode 100644 index be16c865ba..0000000000 --- a/samza-test/src/test/resources/test-config.prop +++ /dev/null @@ -1,26 +0,0 @@ -# -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# -# - -job.factory.class=org.apache.samza.job.MockJobFactory -job.name=test-job -foo=bar -systems.coordinator.samza.factory=org.apache.samza.coordinator.stream.MockCoordinatorStreamSystemFactory -job.coordinator.system=coordinator \ No newline at end of file From 1621c4d00a185cf0905a7500420955cefa56938c Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Wed, 22 Aug 2018 21:55:01 -0700 Subject: [PATCH 13/38] SAMZA-1789: a few more fixes to address Cameron's reviews --- .../runtime/AbstractApplicationRunner.java | 2 +- .../samza/runtime/LocalApplicationRunner.java | 97 +++++++++---------- .../samza/task/TestTaskFactoryUtil.java | 37 +++++++ 3 files changed, 86 insertions(+), 50 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 48ea21c7d9..e176969062 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -64,7 +64,7 @@ public abstract class AbstractApplicationRunner implements ApplicationRunner { /** * This is a temporary helper class to include all common logic to generate {@link JobConfig}s for high- and low-level - * applications in {@link LocalApplicationRunner} and {@link RemoteApplicationRunner} before we fix SAMZA-1811. + * applications in {@link LocalApplicationRunner} and {@link RemoteApplicationRunner}. * * TODO: Fix SAMZA-1811 to consolidate the planning into {@link ExecutionPlanner} */ diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 6032e631a4..780becde28 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -171,7 +171,7 @@ List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws // 2. create the necessary streams // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 String planId = String.valueOf(executionPlanJson.hashCode()); - createStreams(planId, plan.getIntermediateStreams(), streamManager, config, uid); + createStreams(planId, plan.getIntermediateStreams(), streamManager); return plan.getJobConfigs(); } finally { @@ -180,6 +180,53 @@ List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws } } } + + /** + * Create intermediate streams using {@link org.apache.samza.execution.StreamManager}. + * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader + * will create the streams. All the runner processes will wait on the latch that is released after the leader finishes + * stream creation. + * @param planId a unique identifier representing the plan used for coordination purpose + * @param intStreams list of intermediate {@link StreamSpec}s + * @throws TimeoutException exception for latch timeout + */ + private void createStreams(String planId, List intStreams, StreamManager streamManager) throws TimeoutException { + if (intStreams.isEmpty()) { + LOG.info("Set of intermediate streams is empty. Nothing to create."); + return; + } + LOG.info("A single processor must create the intermediate streams. Processor {} will attempt to acquire the lock.", uid); + // Move the scope of coordination utils within stream creation to address long idle connection problem. + // Refer SAMZA-1385 for more details + JobCoordinatorConfig jcConfig = new JobCoordinatorConfig(config); + String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_PATH_SUFFIX; + CoordinationUtils coordinationUtils = + jcConfig.getCoordinationUtilsFactory().getCoordinationUtils(coordinationId, uid, config); + if (coordinationUtils == null) { + LOG.warn("Processor {} failed to create utils. Each processor will attempt to create streams.", uid); + // each application process will try creating the streams, which + // requires stream creation to be idempotent + streamManager.createStreams(intStreams); + return; + } + + DistributedLockWithState lockWithState = coordinationUtils.getLockWithState(planId); + try { + // check if the processor needs to go through leader election and stream creation + if (lockWithState.lockIfNotSet(1000, TimeUnit.MILLISECONDS)) { + LOG.info("lock acquired for streams creation by " + uid); + streamManager.createStreams(intStreams); + lockWithState.unlockAndSet(); + } else { + LOG.info("Processor {} did not obtain the lock for streams creation. They must've been created by another processor.", uid); + } + } catch (TimeoutException e) { + String msg = String.format("Processor {} failed to get the lock for stream initialization", uid); + throw new SamzaException(msg, e); + } finally { + coordinationUtils.close(); + } + } } /** @@ -267,54 +314,6 @@ public boolean waitForFinish(Duration timeout) { return finished; } - /** - * Create intermediate streams using {@link org.apache.samza.execution.StreamManager}. - * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader - * will create the streams. All the runner processes will wait on the latch that is released after the leader finishes - * stream creation. - * @param planId a unique identifier representing the plan used for coordination purpose - * @param intStreams list of intermediate {@link StreamSpec}s - * @throws TimeoutException exception for latch timeout - */ - private static void createStreams(String planId, List intStreams, StreamManager streamManager, - Config config, String uid) throws TimeoutException { - if (intStreams.isEmpty()) { - LOG.info("Set of intermediate streams is empty. Nothing to create."); - return; - } - LOG.info("A single processor must create the intermediate streams. Processor {} will attempt to acquire the lock.", uid); - // Move the scope of coordination utils within stream creation to address long idle connection problem. - // Refer SAMZA-1385 for more details - JobCoordinatorConfig jcConfig = new JobCoordinatorConfig(config); - String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_PATH_SUFFIX; - CoordinationUtils coordinationUtils = - jcConfig.getCoordinationUtilsFactory().getCoordinationUtils(coordinationId, uid, config); - if (coordinationUtils == null) { - LOG.warn("Processor {} failed to create utils. Each processor will attempt to create streams.", uid); - // each application process will try creating the streams, which - // requires stream creation to be idempotent - streamManager.createStreams(intStreams); - return; - } - - DistributedLockWithState lockWithState = coordinationUtils.getLockWithState(planId); - try { - // check if the processor needs to go through leader election and stream creation - if (lockWithState.lockIfNotSet(1000, TimeUnit.MILLISECONDS)) { - LOG.info("lock acquired for streams creation by " + uid); - streamManager.createStreams(intStreams); - lockWithState.unlockAndSet(); - } else { - LOG.info("Processor {} did not obtain the lock for streams creation. They must've been created by another processor.", uid); - } - } catch (TimeoutException e) { - String msg = String.format("Processor {} failed to get the lock for stream initialization", uid); - throw new SamzaException(msg, e); - } finally { - coordinationUtils.close(); - } - } - /** * Create {@link StreamProcessor} based on config, {@link AppDescriptorImpl}, and {@link StreamProcessorListenerSupplier} * @param config config diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 2ff5399bc0..68caf602a8 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -22,15 +22,22 @@ import java.util.HashMap; import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; +import org.apache.samza.operators.OperatorSpecGraph; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + /** * Test methods to create {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} based on task class configuration @@ -119,4 +126,34 @@ public void testFinalizeTaskFactory() throws NoSuchFieldException, IllegalAccess retFactory = TaskFactoryUtil.finalizeTaskFactory(mockAsyncStreamFactory, false, null); assertEquals(retFactory, mockAsyncStreamFactory); } + + // test getTaskFactory with StreamAppDescriptor + @Test + public void testGetTaskFactoryWithStreamAppDescriptor() { + StreamAppDescriptorImpl mockStreamApp = mock(StreamAppDescriptorImpl.class); + OperatorSpecGraph mockSpecGraph = mock(OperatorSpecGraph.class); + when(mockStreamApp.getOperatorSpecGraph()).thenReturn(mockSpecGraph); + TaskFactory streamTaskFactory = TaskFactoryUtil.getTaskFactory(mockStreamApp); + assertTrue(streamTaskFactory instanceof StreamTaskFactory); + StreamTask streamTask = ((StreamTaskFactory) streamTaskFactory).createInstance(); + assertTrue(streamTask instanceof StreamOperatorTask); + verify(mockSpecGraph).clone(); + } + + // test getTaskFactory with TaskAppDescriptor + @Test + public void testGetTaskFactoryWithTaskAppDescriptor() { + TaskAppDescriptorImpl mockTaskApp = mock(TaskAppDescriptorImpl.class); + TaskFactory mockTaskFactory = mock(TaskFactory.class); + when(mockTaskApp.getTaskFactory()).thenReturn(mockTaskFactory); + TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(mockTaskApp); + assertEquals(mockTaskFactory, taskFactory); + } + + // test getTaskFactory with invalid AppDescriptorImpl + @Test(expected = IllegalArgumentException.class) + public void testGetTaskFactoryWithInvalidAddDescriptorImpl() { + AppDescriptorImpl mockInvalidApp = mock(AppDescriptorImpl.class); + TaskFactoryUtil.getTaskFactory(mockInvalidApp); + } } From 055bd91e475770a785b8a1e1d4aedb188561b5d2 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Wed, 22 Aug 2018 22:41:06 -0700 Subject: [PATCH 14/38] SAMZA-1789: fix unit test with ThreadJobFactory --- .../main/scala/org/apache/samza/job/local/ThreadJobFactory.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index ce22a7146a..dee1defbca 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -91,6 +91,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging { new SamzaContainerListener { override def onContainerFailed(t: Throwable): Unit = { userListener.afterFailure(t) + throw t; } override def onContainerStart(): Unit = { From 8d4d3ffda2650fa41757d8ded94a7fb0d878feb0 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 24 Aug 2018 13:37:39 -0700 Subject: [PATCH 15/38] Merge with master --- .../org/apache/samza/table/TableProvider.java | 12 +- .../application/StreamAppDescriptorImpl.java | 30 ++-- .../org/apache/samza/execution/JobNode.java | 3 +- .../samza/operators/BaseTableDescriptor.java | 2 +- .../runtime/AbstractApplicationRunner.java | 4 +- .../samza/table/TableConfigGenerator.java | 11 +- .../table/caching/CachingTableDescriptor.java | 3 +- .../table/caching/CachingTableProvider.java | 56 ++----- .../guava/GuavaCacheTableDescriptor.java | 3 +- .../guava/GuavaCacheTableProvider.java | 44 +----- .../table/remote/RemoteReadableTable.java | 5 + .../table/remote/RemoteTableDescriptor.java | 3 +- .../table/remote/RemoteTableProvider.java | 43 +---- .../samza/table/utils/BaseTableProvider.java | 76 +++++++++ .../apache/samza/config/StorageConfig.scala | 1 + .../TestStreamAppDescriptorImpl.java | 11 ++ .../kv/inmemory/InMemoryTableDescriptor.java | 3 + .../kv/inmemory/InMemoryTableProvider.java | 21 ++- .../inmemory/TestInMemoryTableProvider.java | 9 +- .../storage/kv/RocksDbTableDescriptor.java | 7 +- .../storage/kv/RocksDbTableProvider.java | 21 ++- .../kv/TestRocksDbTableDescriptor.java | 15 ++ .../storage/kv/TestRocksDbTableProvider.java | 9 +- .../BaseLocalStoreBackedTableDescriptor.java | 68 ++++++++ .../kv/BaseLocalStoreBackedTableProvider.java | 55 ++++--- .../kv/LocalStoreBackedReadableTable.java | 1 + ...TestBaseLocalStoreBackedTableProvider.java | 149 ++++++++++++++++++ ...TestLocalBaseStoreBackedTableProvider.java | 85 ---------- .../sql/planner/SamzaSqlOperatorTable.java | 1 + .../sql/translator/TranslatorContext.java | 16 +- .../sql/testutil/TestIOResolverFactory.java | 15 +- .../test/samzasql/TestSamzaSqlEndToEnd.java | 2 +- .../table/TestTableDescriptorsProvider.java | 10 +- 33 files changed, 494 insertions(+), 300 deletions(-) create mode 100644 samza-core/src/main/java/org/apache/samza/table/utils/BaseTableProvider.java create mode 100644 samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseLocalStoreBackedTableProvider.java delete mode 100644 samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java diff --git a/samza-api/src/main/java/org/apache/samza/table/TableProvider.java b/samza-api/src/main/java/org/apache/samza/table/TableProvider.java index 8e60dad44d..99446e4873 100644 --- a/samza-api/src/main/java/org/apache/samza/table/TableProvider.java +++ b/samza-api/src/main/java/org/apache/samza/table/TableProvider.java @@ -21,10 +21,10 @@ import java.util.Map; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.config.Config; import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.task.TaskContext; - /** * A table provider provides the implementation for a table. It ensures a table is * properly constructed and also manages its lifecycle. @@ -47,11 +47,15 @@ public interface TableProvider { /** * Generate any configuration for this table, the generated configuration * is used by Samza container to construct this table and any components - * necessary. - * @param config the current configuration + * necessary. Instead of manipulating the input parameters, this method + * should return the generated configuration. + * + * @param jobConfig the job config + * @param generatedConfig config generated by earlier processing, but has + * not yet been merged to job config * @return configuration for this table */ - Map generateConfig(Map config); + Map generateConfig(Config jobConfig, Map generatedConfig); /** * Shutdown the underlying table diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java index 4b4019bc38..07ecec0676 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java @@ -60,7 +60,8 @@ public class StreamAppDescriptorImpl extends AppDescriptorImpl implements StreamAppDescriptor { private static final Logger LOGGER = LoggerFactory.getLogger(StreamAppDescriptorImpl.class); - private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); + public static final Pattern STREAM_ID_PATTERN = Pattern.compile("[\\d\\w-_.]+"); + public static final Pattern TABLE_ID_PATTERN = Pattern.compile("[\\d\\w-_]+"); // We use a LHM for deterministic order in initializing and closing operators. private final Map inputOperators = new LinkedHashMap<>(); @@ -92,8 +93,8 @@ public void setDefaultSerde(Serde serde) { @Override public MessageStream getInputStream(String streamId, Serde serde) { - Preconditions.checkState(isValidId(streamId), - "streamId must be non-empty and must not contain spaces or special characters: " + streamId); + Preconditions.checkState(isValidStreamId(streamId), String.format( + "streamId %s doesn't confirm to pattern %s", streamId, STREAM_ID_PATTERN)); Preconditions.checkNotNull(serde, "serde must not be null for an input stream."); Preconditions.checkState(!inputOperators.containsKey(streamId), "getInputStream must not be called multiple times with the same streamId: " + streamId); @@ -123,8 +124,8 @@ public MessageStream getInputStream(String streamId) { @Override public OutputStream getOutputStream(String streamId, Serde serde) { - Preconditions.checkState(isValidId(streamId), - "streamId must be non-empty and must not contain spaces or special characters: " + streamId); + Preconditions.checkState(isValidStreamId(streamId), String.format( + "streamId %s doesn't confirm to pattern %s", streamId, STREAM_ID_PATTERN)); Preconditions.checkNotNull(serde, "serde must not be null for an output stream."); Preconditions.checkState(!outputStreams.containsKey(streamId), "getOutputStream must not be called multiple times with the same streamId: " + streamId); @@ -151,12 +152,11 @@ public OutputStream getOutputStream(String streamId) { @Override public Table> getTable(TableDescriptor tableDesc) { + Preconditions.checkState(isValidTableId(tableDesc.getTableId()), String.format( + "tableId %s doesn't confirm to pattern %s", tableDesc.getTableId(), TABLE_ID_PATTERN.toString())); TableSpec tableSpec = ((BaseTableDescriptor) tableDesc).getTableSpec(); - if (tables.containsKey(tableSpec)) { - throw new IllegalStateException(String.format( - "getTable() invoked multiple times with the same tableId: %s", - tableDesc.getTableId())); - } + Preconditions.checkState(!tables.containsKey(tableSpec), String.format( + "getTable() invoked multiple times with the same tableId: %s", tableDesc.getTableId())); tables.put(tableSpec, new TableImpl(tableSpec)); return tables.get(tableSpec); } @@ -174,7 +174,7 @@ public OperatorSpecGraph getOperatorSpecGraph() { * @return the unique ID for the next operator in the graph */ public String getNextOpId(OpCode opCode, String userDefinedId) { - if (StringUtils.isNotBlank(userDefinedId) && !ID_PATTERN.matcher(userDefinedId).matches()) { + if (StringUtils.isNotBlank(userDefinedId) && !STREAM_ID_PATTERN.matcher(userDefinedId).matches()) { throw new SamzaException("Operator ID must not contain spaces or special characters: " + userDefinedId); } @@ -262,8 +262,12 @@ public Map getTables() { return Collections.unmodifiableMap(tables); } - private boolean isValidId(String id) { - return StringUtils.isNotBlank(id) && ID_PATTERN.matcher(id).matches(); + public static boolean isValidStreamId(String id) { + return StringUtils.isNotBlank(id) && STREAM_ID_PATTERN.matcher(id).matches(); + } + + public static boolean isValidTableId(String id) { + return StringUtils.isNotBlank(id) && TABLE_ID_PATTERN.matcher(id).matches(); } private KV getKVSerdes(String streamId, Serde serde) { diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index dba47e1ff8..2b279ef6f7 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -129,6 +129,7 @@ void addTable(TableSpec tableSpec) { public JobConfig generateConfig(String executionPlanJson) { Map configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), jobName); + configs.put(JobConfig.JOB_ID(), jobId); final List inputs = new ArrayList<>(); final List broadcasts = new ArrayList<>(); @@ -177,7 +178,7 @@ public JobConfig generateConfig(String executionPlanJson) { // write serialized serde instances and stream serde configs to configs addSerdeConfigs(configs); - configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(tables)); + configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); // Add side inputs to the inputs and mark the stream as bootstrap tables.forEach(tableSpec -> { diff --git a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java index b875c2e52a..f81f3b8717 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java +++ b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java @@ -45,7 +45,7 @@ abstract public class BaseTableDescriptor cfg = new HashMap<>(taskAppDesc.getConfig()); + Map cfg = new HashMap<>(config); //TODO: add stream and system descriptor to configuration conversion here when SAMZA-1804 is fixed. // adding table configuration List tableSpecs = taskAppDesc.getTables().stream() .map(td -> ((BaseTableDescriptor) td).getTableSpec()) .collect(Collectors.toList()); - cfg.putAll(TableConfigGenerator.generateConfigsForTableSpecs(tableSpecs)); + cfg.putAll(TableConfigGenerator.generateConfigsForTableSpecs(config, tableSpecs)); validateAppClassCfg(cfg, taskAppDesc.getAppClass()); return new JobConfig(new MapConfig(cfg)); } diff --git a/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java b/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java index 3b87eff238..085131c6cd 100644 --- a/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.UUID; +import org.apache.samza.config.Config; import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.SerializerConfig; import org.apache.samza.operators.BaseTableDescriptor; @@ -48,19 +49,21 @@ public class TableConfigGenerator { /** * Generate table configurations given a list of table descriptors + * @param config the job configuration * @param tableDescriptors the list of tableDescriptors * @return configuration for the tables */ - static public Map generateConfigsForTableDescs(List tableDescriptors) { - return generateConfigsForTableSpecs(getTableSpecs(tableDescriptors)); + static public Map generateConfigsForTableDescs(Config config, List tableDescriptors) { + return generateConfigsForTableSpecs(config, getTableSpecs(tableDescriptors)); } /** * Generate table configurations given a list of table specs + * @param config the job configuration * @param tableSpecs the list of tableSpecs * @return configuration for the tables */ - static public Map generateConfigsForTableSpecs(List tableSpecs) { + static public Map generateConfigsForTableSpecs(Config config, List tableSpecs) { Map tableConfigs = new HashMap<>(); tableConfigs.putAll(generateTableKVSerdeConfigs(tableSpecs)); @@ -74,7 +77,7 @@ static public Map generateConfigsForTableSpecs(List t TableProviderFactory tableProviderFactory = Util.getObj(tableSpec.getTableProviderFactoryClassName(), TableProviderFactory.class); TableProvider tableProvider = tableProviderFactory.getTableProvider(tableSpec); - tableConfigs.putAll(tableProvider.generateConfig(tableConfigs)); + tableConfigs.putAll(tableProvider.generateConfig(config, tableConfigs)); }); LOG.info("TableConfigGenerator has generated configs {}", tableConfigs); diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java index 21463c2269..a1accd86d3 100644 --- a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java +++ b/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java @@ -45,8 +45,7 @@ public class CachingTableDescriptor extends BaseTableDescriptor defaultCaches = new ArrayList<>(); - private SamzaContainerContext containerContext; - private TaskContext taskContext; - public CachingTableProvider(TableSpec tableSpec) { - this.cachingTableSpec = tableSpec; - } - - @Override - public void init(SamzaContainerContext containerContext, TaskContext taskContext) { - this.taskContext = taskContext; - this.containerContext = containerContext; + super(tableSpec); } @Override public Table getTable() { - String realTableId = cachingTableSpec.getConfig().get(REAL_TABLE_ID); + String realTableId = tableSpec.getConfig().get(REAL_TABLE_ID); ReadableTable table = (ReadableTable) taskContext.getTable(realTableId); - String cacheTableId = cachingTableSpec.getConfig().get(CACHE_TABLE_ID); + String cacheTableId = tableSpec.getConfig().get(CACHE_TABLE_ID); ReadWriteTable cache; if (cacheTableId != null) { @@ -85,36 +66,21 @@ public Table getTable() { defaultCaches.add(cache); } - boolean isWriteAround = Boolean.parseBoolean(cachingTableSpec.getConfig().get(WRITE_AROUND)); - CachingTable cachingTable = new CachingTable(cachingTableSpec.getId(), table, cache, isWriteAround); + boolean isWriteAround = Boolean.parseBoolean(tableSpec.getConfig().get(WRITE_AROUND)); + CachingTable cachingTable = new CachingTable(tableSpec.getId(), table, cache, isWriteAround); cachingTable.init(containerContext, taskContext); return cachingTable; } - @Override - public Map generateConfig(Map config) { - Map tableConfig = new HashMap<>(); - - // Insert table_id prefix to config entries - cachingTableSpec.getConfig().forEach((k, v) -> { - String realKey = String.format(JavaTableConfig.TABLE_ID_PREFIX, cachingTableSpec.getId()) + "." + k; - tableConfig.put(realKey, v); - }); - - LOG.info("Generated configuration for table " + cachingTableSpec.getId()); - - return tableConfig; - } - @Override public void close() { defaultCaches.forEach(c -> c.close()); } private ReadWriteTable createDefaultCacheTable(String tableId) { - long readTtlMs = Long.parseLong(cachingTableSpec.getConfig().getOrDefault(READ_TTL_MS, "-1")); - long writeTtlMs = Long.parseLong(cachingTableSpec.getConfig().getOrDefault(WRITE_TTL_MS, "-1")); - long cacheSize = Long.parseLong(cachingTableSpec.getConfig().getOrDefault(CACHE_SIZE, "-1")); + long readTtlMs = Long.parseLong(tableSpec.getConfig().getOrDefault(READ_TTL_MS, "-1")); + long writeTtlMs = Long.parseLong(tableSpec.getConfig().getOrDefault(WRITE_TTL_MS, "-1")); + long cacheSize = Long.parseLong(tableSpec.getConfig().getOrDefault(CACHE_SIZE, "-1")); CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); if (readTtlMs != -1) { @@ -127,7 +93,7 @@ private ReadWriteTable createDefaultCacheTable(String tableId) { cacheBuilder.maximumSize(cacheSize); } - LOG.info(String.format("Creating default cache with: readTtl=%d, writeTtl=%d, maxSize=%d", + logger.info(String.format("Creating default cache with: readTtl=%d, writeTtl=%d, maxSize=%d", readTtlMs, writeTtlMs, cacheSize)); GuavaCacheTable cacheTable = new GuavaCacheTable(tableId + "-def-cache", cacheBuilder.build()); diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableDescriptor.java index ce125c00ef..4a05013706 100644 --- a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableDescriptor.java +++ b/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableDescriptor.java @@ -39,8 +39,7 @@ public class GuavaCacheTableDescriptor extends BaseTableDescriptor cache; /** - * Constructs a table descriptor instance - * @param tableId Id of the table + * {@inheritDoc} */ public GuavaCacheTableDescriptor(String tableId) { super(tableId); diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProvider.java index 1ba26c760d..151324997b 100644 --- a/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProvider.java +++ b/samza-core/src/main/java/org/apache/samza/table/caching/guava/GuavaCacheTableProvider.java @@ -20,19 +20,12 @@ package org.apache.samza.table.caching.guava; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import org.apache.samza.config.JavaTableConfig; -import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.table.Table; -import org.apache.samza.table.TableProvider; import org.apache.samza.table.TableSpec; +import org.apache.samza.table.utils.BaseTableProvider; import org.apache.samza.table.utils.SerdeUtils; -import org.apache.samza.task.TaskContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.google.common.cache.Cache; @@ -40,52 +33,25 @@ /** * Table provider for {@link GuavaCacheTable}. */ -public class GuavaCacheTableProvider implements TableProvider { - private static final Logger LOG = LoggerFactory.getLogger(GuavaCacheTableProvider.class); +public class GuavaCacheTableProvider extends BaseTableProvider { public static final String GUAVA_CACHE = "guavaCache"; - private final TableSpec guavaCacheTableSpec; - - private SamzaContainerContext containerContext; - private TaskContext taskContext; - private List guavaTables = new ArrayList<>(); public GuavaCacheTableProvider(TableSpec tableSpec) { - this.guavaCacheTableSpec = tableSpec; - } - - @Override - public void init(SamzaContainerContext containerContext, TaskContext taskContext) { - this.taskContext = taskContext; - this.containerContext = containerContext; + super(tableSpec); } @Override public Table getTable() { - Cache guavaCache = SerdeUtils.deserialize(GUAVA_CACHE, guavaCacheTableSpec.getConfig().get(GUAVA_CACHE)); - GuavaCacheTable table = new GuavaCacheTable(guavaCacheTableSpec.getId(), guavaCache); + Cache guavaCache = SerdeUtils.deserialize(GUAVA_CACHE, tableSpec.getConfig().get(GUAVA_CACHE)); + GuavaCacheTable table = new GuavaCacheTable(tableSpec.getId(), guavaCache); table.init(containerContext, taskContext); guavaTables.add(table); return table; } - @Override - public Map generateConfig(Map config) { - Map tableConfig = new HashMap<>(); - - // Insert table_id prefix to config entries - guavaCacheTableSpec.getConfig().forEach((k, v) -> { - String realKey = String.format(JavaTableConfig.TABLE_ID_PREFIX, guavaCacheTableSpec.getId()) + "." + k; - tableConfig.put(realKey, v); - }); - - LOG.info("Generated configuration for table " + guavaCacheTableSpec.getId()); - - return tableConfig; - } - @Override public void close() { guavaTables.forEach(t -> t.close()); diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java index 24edbce574..3186fee94a 100644 --- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java @@ -169,6 +169,7 @@ public CompletableFuture> getAllAsync(List keys) { /** * Execute an async request given a table key + * @param rateLimiter helper for rate limiting * @param key key of the table record * @param method method to be executed * @param timer latency metric to be updated @@ -199,6 +200,7 @@ protected CompletableFuture execute(TableRateLimiter rateLimiter, /** * Execute an async request given a table record (key+value) + * @param rateLimiter helper for rate limiting * @param key key of the table record * @param value value of the table record * @param method method to be executed @@ -229,9 +231,11 @@ protected CompletableFuture execute(TableRateLimiter rateLimiter, /** * Execute an async request given a collection of table keys + * @param rateLimiter helper for rate limiting * @param keys collection of keys * @param method method to be executed * @param timer latency metric to be updated + * @param return type * @return CompletableFuture of the operation */ protected CompletableFuture execute(TableRateLimiter rateLimiter, @@ -258,6 +262,7 @@ protected CompletableFuture execute(TableRateLimiter rateLimiter, /** * Execute an async request given a collection of table records + * @param rateLimiter helper for rate limiting * @param records list of records * @param method method to be executed * @param timer latency metric to be updated diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java index e405096c89..a8d419d1f3 100644 --- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java @@ -75,8 +75,7 @@ public class RemoteTableDescriptor extends BaseTableDescriptor> tables = new ArrayList<>(); - private SamzaContainerContext containerContext; - private TaskContext taskContext; /** * Map of tableId -> executor service for async table IO and callbacks. The same executors @@ -70,19 +60,10 @@ public class RemoteTableProvider implements TableProvider { private static Map callbackExecutors = new ConcurrentHashMap<>(); public RemoteTableProvider(TableSpec tableSpec) { - this.tableSpec = tableSpec; + super(tableSpec); this.readOnly = !tableSpec.getConfig().containsKey(WRITE_FN); } - /** - * {@inheritDoc} - */ - @Override - public void init(SamzaContainerContext containerContext, TaskContext taskContext) { - this.containerContext = containerContext; - this.taskContext = taskContext; - } - /** * {@inheritDoc} */ @@ -144,24 +125,6 @@ public Table getTable() { return table; } - /** - * {@inheritDoc} - */ - @Override - public Map generateConfig(Map config) { - Map tableConfig = new HashMap<>(); - - // Insert table_id prefix to config entries - tableSpec.getConfig().forEach((k, v) -> { - String realKey = String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; - tableConfig.put(realKey, v); - }); - - LOG.info("Generated configuration for table " + tableSpec.getId()); - - return tableConfig; - } - /** * {@inheritDoc} */ diff --git a/samza-core/src/main/java/org/apache/samza/table/utils/BaseTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/utils/BaseTableProvider.java new file mode 100644 index 0000000000..960e2a4a59 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/utils/BaseTableProvider.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.table.utils; + +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.config.Config; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableSpec; +import org.apache.samza.task.TaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Base class for all table provider implementations. + */ +abstract public class BaseTableProvider implements TableProvider { + + final protected Logger logger = LoggerFactory.getLogger(getClass()); + + final protected TableSpec tableSpec; + + protected SamzaContainerContext containerContext; + protected TaskContext taskContext; + + public BaseTableProvider(TableSpec tableSpec) { + this.tableSpec = tableSpec; + } + + /** + * {@inheritDoc} + */ + @Override + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + this.containerContext = containerContext; + this.taskContext = taskContext; + } + + /** + * {@inheritDoc} + */ + @Override + public Map generateConfig(Config jobConfig, Map generatedConfig) { + Map tableConfig = new HashMap<>(); + + // Insert table_id prefix to config entries + tableSpec.getConfig().forEach((k, v) -> { + String realKey = String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; + tableConfig.put(realKey, v); + }); + + logger.info("Generated configuration for table " + tableSpec.getId()); + + return tableConfig; + } + +} diff --git a/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala index 42b6130c53..1577e62837 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala @@ -31,6 +31,7 @@ object StorageConfig { val MSG_SERDE = "stores.%s.msg.serde" val CHANGELOG_STREAM = "stores.%s.changelog" val CHANGELOG_SYSTEM = "job.changelog.system" + val CHANGELOG_REPLICATION_FACTOR = "stores.%s.changelog.replication.factor" val CHANGELOG_DELETE_RETENTION_MS = "stores.%s.changelog.delete.retention.ms" val DEFAULT_CHANGELOG_DELETE_RETENTION_MS = TimeUnit.DAYS.toMillis(1) val ACCESSLOG_STREAM_SUFFIX = "access-log" diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java index c56fc36d05..9f68032eff 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java @@ -526,6 +526,7 @@ public void testGetTable() throws Exception { BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); TableSpec testTableSpec = new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()); when(mockTableDescriptor.getTableSpec()).thenReturn(testTableSpec); + when(mockTableDescriptor.getTableId()).thenReturn(testTableSpec.getId()); StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { appDesc.getTable(mockTableDescriptor); }, mockConfig); @@ -547,4 +548,14 @@ public void testProcessorLifecycleListenerFactory() { StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, mock(Config.class)); assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); } + + @Test(expected = IllegalStateException.class) + public void testGetTableWithBadId() { + Config mockConfig = mock(Config.class); + new StreamAppDescriptorImpl(appDesc -> { + BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); + when(mockTableDescriptor.getTableId()).thenReturn("my.table"); + appDesc.getTable(mockTableDescriptor); + }, mockConfig); + } } diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java index 2a9532ba23..8328417cab 100644 --- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java +++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java @@ -33,6 +33,9 @@ */ public class InMemoryTableDescriptor extends BaseLocalStoreBackedTableDescriptor> { + /** + * {@inheritDoc} + */ public InMemoryTableDescriptor(String tableId) { super(tableId); } diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java index c1c2f1ca5c..46406e513f 100644 --- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java +++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableProvider.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.samza.config.Config; import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.StorageConfig; import org.apache.samza.storage.kv.BaseLocalStoreBackedTableProvider; @@ -37,7 +38,7 @@ public InMemoryTableProvider(TableSpec tableSpec) { } @Override - public Map generateConfig(Map config) { + public Map generateConfig(Config jobConfig, Map generatedConfig) { Map tableConfig = new HashMap<>(); @@ -47,15 +48,19 @@ public Map generateConfig(Map config) { InMemoryKeyValueStorageEngineFactory.class.getName()); // Common store configuration - tableConfig.putAll(generateCommonStoreConfig(config)); + tableConfig.putAll(generateCommonStoreConfig(jobConfig, generatedConfig)); // Rest of the configuration - tableSpec.getConfig().forEach((k, v) -> { - String realKey = k.startsWith("inmemory.") ? - String.format("stores.%s", tableSpec.getId()) + "." + k.substring("inmemory.".length()) - : String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; - tableConfig.put(realKey, v); - }); + tableSpec.getConfig().entrySet().stream() + .filter(e -> !e.getKey().startsWith("internal.")) + .forEach(e -> { + String k = e.getKey(); + String v = e.getValue(); + String realKey = k.startsWith("inmemory.") + ? String.format("stores.%s", tableSpec.getId()) + "." + k.substring("inmemory.".length()) + : String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; + tableConfig.put(realKey, v); + }); logger.info("Generated configuration for table " + tableSpec.getId()); diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java index 76b7a666ba..2145b686cd 100644 --- a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java +++ b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableProvider.java @@ -23,6 +23,7 @@ import java.util.Map; import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.MapConfig; import org.apache.samza.config.StorageConfig; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; @@ -45,12 +46,12 @@ public void testGenerateConfig() { TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()), "my-table-provider-factory", tableSpecConfig); - Map config = new HashMap<>(); - config.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); - config.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); + Map generatedConfig = new HashMap<>(); + generatedConfig.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); + generatedConfig.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); TableProvider tableProvider = new InMemoryTableProvider(tableSpec); - Map tableConfig = tableProvider.generateConfig(config); + Map tableConfig = tableProvider.generateConfig(new MapConfig(), generatedConfig); Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java index 33bfc84ecb..9b81605c1c 100644 --- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java @@ -44,8 +44,8 @@ public class RocksDbTableDescriptor extends BaseLocalStoreBackedTableDescr static final public String ROCKSDB_MAX_LOG_FILE_SIZE_BYTES = "rocksdb.max.log.file.size.bytes"; static final public String ROCKSDB_KEEP_LOG_FILE_NUM = "rocksdb.keep.log.file.num"; - protected Integer writeBatchSize; - protected Integer objectCacheSize; + private Integer writeBatchSize; + private Integer objectCacheSize; private Integer cacheSize; private Integer writeBufferSize; private Integer blockSize; @@ -56,6 +56,9 @@ public class RocksDbTableDescriptor extends BaseLocalStoreBackedTableDescr private String compressionType; private String compactionStyle; + /** + * {@inheritDoc} + */ public RocksDbTableDescriptor(String tableId) { super(tableId); } diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java index dce7cc0a86..df60a5a615 100644 --- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableProvider.java @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.samza.config.ClusterManagerConfig; +import org.apache.samza.config.Config; import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.StorageConfig; import org.apache.samza.table.TableSpec; @@ -37,7 +38,7 @@ public RocksDbTableProvider(TableSpec tableSpec) { } @Override - public Map generateConfig(Map config) { + public Map generateConfig(Config jobConfig, Map generatedConfig) { Map tableConfig = new HashMap<>(); @@ -47,15 +48,19 @@ public Map generateConfig(Map config) { RocksDbKeyValueStorageEngineFactory.class.getName()); // Common store configuration - tableConfig.putAll(generateCommonStoreConfig(config)); + tableConfig.putAll(generateCommonStoreConfig(jobConfig, generatedConfig)); // Rest of the configuration - tableSpec.getConfig().forEach((k, v) -> { - String realKey = k.startsWith("rocksdb.") ? - String.format("stores.%s", tableSpec.getId()) + "." + k.substring("rocksdb.".length()) - : String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; - tableConfig.put(realKey, v); - }); + tableSpec.getConfig().entrySet().stream() + .filter(e -> !e.getKey().startsWith("internal.")) + .forEach(e -> { + String k = e.getKey(); + String v = e.getValue(); + String realKey = k.startsWith("rocksdb.") + ? String.format("stores.%s", tableSpec.getId()) + "." + k.substring("rocksdb.".length()) + : String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; + tableConfig.put(realKey, v); + }); // Enable host affinity tableConfig.put(ClusterManagerConfig.CLUSTER_MANAGER_HOST_AFFINITY_ENABLED, "true"); diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java index 49fe6eb975..50f0920a5b 100644 --- a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java +++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java @@ -79,6 +79,21 @@ public void testTableSpec() { Assert.assertEquals("snappy", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPRESSION)); Assert.assertEquals("fifo", getConfig(tableSpec, RocksDbTableDescriptor.ROCKSDB_COMPACTION_STYLE)); Assert.assertEquals("xyz", getConfig(tableSpec, "abc")); + Assert.assertEquals("false", tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_ENABLE_CHANGELOG)); + } + + @Test + public void testTableSpecWithChangelogEnabled() { + + TableSpec tableSpec = new RocksDbTableDescriptor("1") + .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde())) + .withChangelogStream("changelog-$tream") + .withChangelogReplicationFactor(10) + .getTableSpec(); + + Assert.assertEquals("10", tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_CHANGELOG_REPLICATION_FACTOR)); + Assert.assertEquals("changelog-$tream", tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_CHANGELOG_STREAM)); + Assert.assertEquals("true", tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_ENABLE_CHANGELOG)); } private String getConfig(TableSpec tableSpec, String key) { diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java index beda5da49c..8ce061cdb3 100644 --- a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java +++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableProvider.java @@ -23,6 +23,7 @@ import java.util.Map; import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.MapConfig; import org.apache.samza.config.StorageConfig; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.KVSerde; @@ -46,12 +47,12 @@ public void testGenerateConfig() { TableSpec tableSpec = new TableSpec("t1", KVSerde.of(new IntegerSerde(), new IntegerSerde()), "my-table-provider-factory", tableSpecConfig); - Map config = new HashMap<>(); - config.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); - config.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); + Map generatedConfig = new HashMap<>(); + generatedConfig.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); + generatedConfig.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); TableProvider tableProvider = new RocksDbTableProvider(tableSpec); - Map tableConfig = tableProvider.generateConfig(config); + Map tableConfig = tableProvider.generateConfig(new MapConfig(), generatedConfig); Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java index 2d05f95633..c46f9e1d4e 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java @@ -36,11 +36,20 @@ */ abstract public class BaseLocalStoreBackedTableDescriptor> extends BaseTableDescriptor { + + static final public String INTERNAL_ENABLE_CHANGELOG = "internal.enable.changelog"; + static final public String INTERNAL_CHANGELOG_STREAM = "internal.changelog.stream"; + static final public String INTERNAL_CHANGELOG_REPLICATION_FACTOR = "internal.changelog.replication.factor"; + protected List sideInputs; protected SideInputsProcessor sideInputsProcessor; + protected boolean enableChangelog; + protected String changelogStream; + protected Integer changelogReplicationFactor; /** * Constructs a table descriptor instance + * * @param tableId Id of the table */ public BaseLocalStoreBackedTableDescriptor(String tableId) { @@ -49,6 +58,10 @@ public BaseLocalStoreBackedTableDescriptor(String tableId) { public D withSideInputs(List sideInputs) { this.sideInputs = sideInputs; + // Disable changelog + this.enableChangelog = false; + this.changelogStream = null; + this.changelogReplicationFactor = null; return (D) this; } @@ -57,9 +70,56 @@ public D withSideInputsProcessor(SideInputsProcessor sideInputsProcessor) { return (D) this; } + /** + * Enable changelog for this table, by default changelog is disabled. When the + * changelog stream name is not specified, it is automatically generated in + * the format { @literal [job-name]-[job-id]-table-[table-id] }. + * Refer to stores.store-name.changelog in Samza configuration guide + * + * @return this table descriptor instance + */ + public D withChangelogEnabled() { + this.enableChangelog = true; + return (D) this; + } + + /** + * Refer to stores.store-name.changelog in Samza configuration guide + * + * @param changelogStream changelog stream name + * @return this table descriptor instance + */ + public D withChangelogStream(String changelogStream) { + this.enableChangelog = true; + this.changelogStream = changelogStream; + return (D) this; + } + + /** + * Refer to stores.store-name.changelog.replication.factor in Samza configuration guide + * + * @param replicationFactor replication factor + * @return this table descriptor instance + */ + public D withChangelogReplicationFactor(int replicationFactor) { + this.enableChangelog = true; + this.changelogReplicationFactor = replicationFactor; + return (D) this; + } + @Override protected void generateTableSpecConfig(Map tableSpecConfig) { super.generateTableSpecConfig(tableSpecConfig); + + tableSpecConfig.put(INTERNAL_ENABLE_CHANGELOG, String.valueOf(enableChangelog)); + if (enableChangelog) { + if (changelogStream != null) { + tableSpecConfig.put(INTERNAL_CHANGELOG_STREAM, changelogStream); + } + if (changelogReplicationFactor != null) { + tableSpecConfig.put(INTERNAL_CHANGELOG_REPLICATION_FACTOR, String.valueOf(changelogReplicationFactor)); + } + } } /** @@ -72,6 +132,14 @@ protected void validate() { String.format("Invalid side input configuration for table: %s. " + "Both side inputs and the processor must be provided", tableId)); } + if (!enableChangelog) { + Preconditions.checkState(changelogStream == null, + String.format("Invalid changelog configuration for table: %s. Changelog " + + "must be enabled, when changelog stream name is provided", tableId)); + Preconditions.checkState(changelogReplicationFactor == null, + String.format("Invalid changelog configuration for table: %s. Changelog " + + "must be enabled, when changelog replication factor is provided", tableId)); + } } } diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java index cacfe9528d..e0c575aef3 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java @@ -22,20 +22,22 @@ import java.util.List; import java.util.Map; +import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.config.Config; import org.apache.samza.config.JavaStorageConfig; import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StorageConfig; import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.table.ReadableTable; import org.apache.samza.table.Table; -import org.apache.samza.table.TableProvider; import org.apache.samza.table.TableSpec; +import org.apache.samza.table.utils.BaseTableProvider; import org.apache.samza.table.utils.SerdeUtils; import org.apache.samza.task.TaskContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; @@ -47,26 +49,18 @@ * the table provider will not manage the lifecycle of the backing * stores. */ -abstract public class BaseLocalStoreBackedTableProvider implements TableProvider { - - protected final Logger logger = LoggerFactory.getLogger(getClass()); - - protected final TableSpec tableSpec; +abstract public class BaseLocalStoreBackedTableProvider extends BaseTableProvider { protected KeyValueStore kvStore; - protected SamzaContainerContext containerContext; - - protected TaskContext taskContext; - public BaseLocalStoreBackedTableProvider(TableSpec tableSpec) { - this.tableSpec = tableSpec; + super(tableSpec); } @Override public void init(SamzaContainerContext containerContext, TaskContext taskContext) { - this.containerContext = containerContext; - this.taskContext = taskContext; + + super.init(containerContext, taskContext); Preconditions.checkNotNull(this.taskContext, "Must specify task context for local tables."); @@ -90,14 +84,14 @@ public Table getTable() { return table; } - protected Map generateCommonStoreConfig(Map config) { + protected Map generateCommonStoreConfig(Config jobConfig, Map generatedConfig) { Map storeConfig = new HashMap<>(); // We assume the configuration for serde are already generated for this table, // so we simply carry them over to store configuration. // - JavaTableConfig tableConfig = new JavaTableConfig(new MapConfig(config)); + JavaTableConfig tableConfig = new JavaTableConfig(new MapConfig(generatedConfig)); String keySerde = tableConfig.getKeySerde(tableSpec.getId()); storeConfig.put(String.format(StorageConfig.KEY_SERDE(), tableSpec.getId()), keySerde); @@ -107,13 +101,38 @@ protected Map generateCommonStoreConfig(Map conf List sideInputs = tableSpec.getSideInputs(); if (sideInputs != null && !sideInputs.isEmpty()) { + sideInputs.forEach(si -> Preconditions.checkState(StreamAppDescriptorImpl.isValidStreamId(si), String.format( + "Side input stream %s doesn't confirm to pattern %s", si, StreamAppDescriptorImpl.STREAM_ID_PATTERN))); String formattedSideInputs = String.join(",", sideInputs); - storeConfig.put(String.format(JavaStorageConfig.SIDE_INPUTS, tableSpec.getId()), formattedSideInputs); storeConfig.put(String.format(JavaStorageConfig.SIDE_INPUTS_PROCESSOR_SERIALIZED_INSTANCE, tableSpec.getId()), SerdeUtils.serialize("Side Inputs Processor", tableSpec.getSideInputsProcessor())); } + // Changelog configuration + Boolean enableChangelog = Boolean.valueOf( + tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_ENABLE_CHANGELOG)); + if (enableChangelog) { + String changelogStream = tableSpec.getConfig().get(BaseLocalStoreBackedTableDescriptor.INTERNAL_CHANGELOG_STREAM); + if (StringUtils.isEmpty(changelogStream)) { + changelogStream = String.format("%s-%s-table-%s", + jobConfig.get(JobConfig.JOB_NAME()), + jobConfig.get(JobConfig.JOB_ID(), "1"), + tableSpec.getId()); + } + + Preconditions.checkState(StreamAppDescriptorImpl.isValidStreamId(changelogStream), String.format( + "Changelog stream %s doesn't confirm to pattern %s", changelogStream, StreamAppDescriptorImpl.STREAM_ID_PATTERN)); + storeConfig.put(String.format(StorageConfig.CHANGELOG_STREAM(), tableSpec.getId()), changelogStream); + + String changelogReplicationFactor = tableSpec.getConfig().get( + BaseLocalStoreBackedTableDescriptor.INTERNAL_CHANGELOG_REPLICATION_FACTOR); + if (changelogReplicationFactor != null) { + storeConfig.put(String.format(StorageConfig.CHANGELOG_REPLICATION_FACTOR(), tableSpec.getId()), + changelogReplicationFactor); + } + } + return storeConfig; } diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java index 1c59eb6811..d0629c47cf 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java @@ -44,6 +44,7 @@ public class LocalStoreBackedReadableTable implements ReadableTable /** * Constructs an instance of {@link LocalStoreBackedReadableTable} + * @param tableId the table Id * @param kvStore the backing store */ public LocalStoreBackedReadableTable(String tableId, KeyValueStore kvStore) { diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseLocalStoreBackedTableProvider.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseLocalStoreBackedTableProvider.java new file mode 100644 index 0000000000..2b0166c3de --- /dev/null +++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseLocalStoreBackedTableProvider.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.storage.kv; + +import java.util.HashMap; +import java.util.Map; +import junit.framework.Assert; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.storage.StorageEngine; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableSpec; +import org.apache.samza.task.TaskContext; +import org.apache.samza.util.NoOpMetricsRegistry; +import org.junit.Test; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + + +public class TestBaseLocalStoreBackedTableProvider { + + @Test + public void testInit() { + StorageEngine store = mock(KeyValueStorageEngine.class); + SamzaContainerContext containerContext = mock(SamzaContainerContext.class); + TaskContext taskContext = mock(TaskContext.class); + when(taskContext.getStore(any())).thenReturn(store); + when(taskContext.getMetricsRegistry()).thenReturn(new NoOpMetricsRegistry()); + + TableSpec tableSpec = mock(TableSpec.class); + when(tableSpec.getId()).thenReturn("t1"); + + TableProvider tableProvider = createTableProvider(tableSpec); + tableProvider.init(containerContext, taskContext); + Assert.assertNotNull(tableProvider.getTable()); + } + + @Test(expected = SamzaException.class) + public void testInitFail() { + TableSpec tableSpec = mock(TableSpec.class); + when(tableSpec.getId()).thenReturn("t1"); + TableProvider tableProvider = createTableProvider(tableSpec); + Assert.assertNotNull(tableProvider.getTable()); + } + + @Test + public void testGenerateCommonStoreConfig() { + Map generatedConfig = new HashMap<>(); + generatedConfig.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); + generatedConfig.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); + + TableSpec tableSpec = mock(TableSpec.class); + when(tableSpec.getId()).thenReturn("t1"); + + TableProvider tableProvider = createTableProvider(tableSpec); + Map tableConfig = tableProvider.generateConfig(new MapConfig(), generatedConfig); + Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); + Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); + } + + @Test + public void testChangelogDisabled() { + TableSpec tableSpec = createTableDescriptor("t1") + .getTableSpec(); + + TableProvider tableProvider = createTableProvider(tableSpec); + Map tableConfig = tableProvider.generateConfig(new MapConfig(), new MapConfig()); + Assert.assertEquals(2, tableConfig.size()); + Assert.assertFalse(tableConfig.containsKey(String.format(StorageConfig.CHANGELOG_STREAM(), "t1"))); + } + + @Test + public void testChangelogEnabled() { + TableSpec tableSpec = createTableDescriptor("t1") + .withChangelogEnabled() + .getTableSpec(); + + Map jobConfig = new HashMap<>(); + jobConfig.put(JobConfig.JOB_NAME(), "test-job"); + jobConfig.put(JobConfig.JOB_ID(), "10"); + + TableProvider tableProvider = createTableProvider(tableSpec); + Map tableConfig = tableProvider.generateConfig(new MapConfig(jobConfig), new MapConfig()); + Assert.assertEquals(3, tableConfig.size()); + Assert.assertEquals("test-job-10-table-t1", String.format( + tableConfig.get(String.format(StorageConfig.CHANGELOG_STREAM(), "t1")))); + } + + @Test + public void testChangelogEnabledWithCustomParameters() { + TableSpec tableSpec = createTableDescriptor("t1") + .withChangelogStream("my-stream") + .withChangelogReplicationFactor(100) + .getTableSpec(); + + TableProvider tableProvider = createTableProvider(tableSpec); + Map tableConfig = tableProvider.generateConfig(new MapConfig(), new MapConfig()); + Assert.assertEquals(4, tableConfig.size()); + Assert.assertEquals("my-stream", String.format( + tableConfig.get(String.format(StorageConfig.CHANGELOG_STREAM(), "t1")))); + Assert.assertEquals("100", String.format( + tableConfig.get(String.format(StorageConfig.CHANGELOG_REPLICATION_FACTOR(), "t1")))); + } + + private TableProvider createTableProvider(TableSpec tableSpec) { + return new BaseLocalStoreBackedTableProvider(tableSpec) { + @Override + public Map generateConfig(Config jobConfig, Map generatedConfig) { + return generateCommonStoreConfig(jobConfig, generatedConfig); + } + }; + } + + private BaseLocalStoreBackedTableDescriptor createTableDescriptor(String tableId) { + return new BaseLocalStoreBackedTableDescriptor(tableId) { + @Override + public TableSpec getTableSpec() { + validate(); + Map tableSpecConfig = new HashMap<>(); + generateTableSpecConfig(tableSpecConfig); + return new TableSpec(tableId, serde, null, tableSpecConfig, + sideInputs, sideInputsProcessor); + } + }; + } +} diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java deleted file mode 100644 index 56818b536c..0000000000 --- a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.samza.storage.kv; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.samza.SamzaException; -import org.apache.samza.config.JavaTableConfig; -import org.apache.samza.config.StorageConfig; -import org.apache.samza.container.SamzaContainerContext; -import org.apache.samza.storage.StorageEngine; -import org.apache.samza.table.TableSpec; -import org.apache.samza.task.TaskContext; -import org.apache.samza.util.NoOpMetricsRegistry; -import org.junit.Before; -import org.junit.Test; - -import junit.framework.Assert; - -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - - -public class TestLocalBaseStoreBackedTableProvider { - - private BaseLocalStoreBackedTableProvider tableProvider; - - @Before - public void prepare() { - TableSpec tableSpec = mock(TableSpec.class); - when(tableSpec.getId()).thenReturn("t1"); - tableProvider = new BaseLocalStoreBackedTableProvider(tableSpec) { - @Override - public Map generateConfig(Map config) { - return generateCommonStoreConfig(config); - } - }; - } - - @Test - public void testInit() { - StorageEngine store = mock(KeyValueStorageEngine.class); - SamzaContainerContext containerContext = mock(SamzaContainerContext.class); - TaskContext taskContext = mock(TaskContext.class); - when(taskContext.getStore(any())).thenReturn(store); - when(taskContext.getMetricsRegistry()).thenReturn(new NoOpMetricsRegistry()); - tableProvider.init(containerContext, taskContext); - Assert.assertNotNull(tableProvider.getTable()); - } - - @Test(expected = SamzaException.class) - public void testInitFail() { - Assert.assertNotNull(tableProvider.getTable()); - } - - @Test - public void testGenerateCommonStoreConfig() { - Map config = new HashMap<>(); - config.put(String.format(JavaTableConfig.TABLE_KEY_SERDE, "t1"), "ks1"); - config.put(String.format(JavaTableConfig.TABLE_VALUE_SERDE, "t1"), "vs1"); - - Map tableConfig = tableProvider.generateConfig(config); - Assert.assertEquals("ks1", tableConfig.get(String.format(StorageConfig.KEY_SERDE(), "t1"))); - Assert.assertEquals("vs1", tableConfig.get(String.format(StorageConfig.MSG_SERDE(), "t1"))); - } -} diff --git a/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlOperatorTable.java b/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlOperatorTable.java index b078f5bbe8..9c16de8f49 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlOperatorTable.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/planner/SamzaSqlOperatorTable.java @@ -77,6 +77,7 @@ public class SamzaSqlOperatorTable extends ReflectiveSqlOperatorTable { public static final SqlFunction MOD = SqlStdOperatorTable.MOD; public static final SqlFunction FLOOR = SqlStdOperatorTable.FLOOR; public static final SqlFunction CEIL = SqlStdOperatorTable.CEIL; + public static final SqlFunction EXTRACT = SqlStdOperatorTable.EXTRACT; public static final SqlFunction LOCALTIME = SqlStdOperatorTable.LOCALTIME; public static final SqlFunction LOCALTIMESTAMP = SqlStdOperatorTable.LOCALTIMESTAMP; public static final SqlFunction CURRENT_TIME = SqlStdOperatorTable.CURRENT_TIME; diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java index cd48e39436..a578bc8582 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.TimeZone; import org.apache.calcite.DataContext; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.linq4j.QueryProvider; @@ -76,11 +77,18 @@ public QueryProvider getQueryProvider() { @Override public Object get(String name) { - if (name.equals(Variable.CURRENT_TIMESTAMP.camelName)) { - return System.currentTimeMillis(); + TimeZone timeZone = TimeZone.getDefault(); + long timeMs = System.currentTimeMillis(); + long offsetMs = timeZone.getOffset(timeMs); + if (name.equals(Variable.LOCAL_TIMESTAMP.camelName)) { + return timeMs + offsetMs; + } else if (name.equals(Variable.UTC_TIMESTAMP.camelName) || name.equals(Variable.CURRENT_TIMESTAMP.camelName)) { + return timeMs; + } else if (name.equals(Variable.TIME_ZONE.camelName)) { + return timeZone; + } else { + throw new UnsupportedOperationException("Unsupported operation " + name); } - - return null; } } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java index 574076e03b..bd61afd846 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java @@ -27,7 +27,6 @@ import org.apache.commons.lang.NotImplementedException; import org.apache.samza.config.Config; -import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.serializers.JsonSerdeV2; @@ -44,7 +43,7 @@ import org.apache.samza.table.TableProvider; import org.apache.samza.table.TableProviderFactory; import org.apache.samza.table.TableSpec; -import org.apache.samza.task.TaskContext; +import org.apache.samza.table.utils.BaseTableProvider; public class TestIOResolverFactory implements SqlIOResolverFactory { @@ -156,9 +155,10 @@ public TableProvider getTableProvider(TableSpec tableSpec) { } } - static class TestTableProvider implements TableProvider { - @Override - public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + static class TestTableProvider extends BaseTableProvider { + + public TestTableProvider() { + super(null); } @Override @@ -167,7 +167,7 @@ public Table getTable() { } @Override - public Map generateConfig(Map config) { + public Map generateConfig(Config jobConfig, Map generatedConfig) { return new HashMap<>(); } @@ -201,7 +201,8 @@ private SqlIOConfig fetchIOInfo(String ioName, boolean isSink) { if (isSink) { tableDescriptor = new TestTableDescriptor(TEST_TABLE_ID + tableDescMap.size()); } else { - tableDescriptor = new RocksDbTableDescriptor("InputTable-" + ioName) + String tableId = "InputTable-" + ioName.replace(".", "-").replace("$", "-"); + tableDescriptor = new RocksDbTableDescriptor(tableId) .withSerde(KVSerde.of( new JsonSerdeV2<>(SamzaSqlCompositeKey.class), new JsonSerdeV2<>(SamzaSqlRelMessage.class))); diff --git a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java index 4ee42ae245..c211f0300e 100644 --- a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java +++ b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java @@ -75,7 +75,7 @@ public void testEndToEnd() throws Exception { TestAvroSystemFactory.messages.clear(); Map staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages); - String sql1 = "Insert into testavro.outputTopic select id, CURRENT_TIME as long_value from testavro.SIMPLE1"; + String sql1 = "Insert into testavro.outputTopic select id, TIMESTAMPDIFF(HOUR, CURRENT_TIMESTAMP, LOCALTIMESTAMP) + MONTH(CURRENT_DATE) as long_value from testavro.SIMPLE1"; List sqlStmts = Arrays.asList(sql1); staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs)); diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java b/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java index 38cc47c988..41b6509fc9 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java @@ -29,6 +29,7 @@ import org.apache.samza.config.ConfigRewriter; import org.apache.samza.config.JavaStorageConfig; import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.serializers.KVSerde; @@ -42,7 +43,6 @@ import org.apache.samza.table.remote.RemoteTableDescriptor; import org.apache.samza.table.remote.RemoteTableProviderFactory; import org.apache.samza.table.remote.TableReadFunction; - import org.apache.samza.util.RateLimiter; import org.apache.samza.util.Util; import org.junit.Assert; @@ -79,6 +79,8 @@ public void testWithNonTableDescriptorsProviderClass() throws Exception { public void testWithTableDescriptorsProviderClass() throws Exception { Map configs = new HashMap<>(); String tableRewriterName = "tableRewriter"; + String jobName = "test-job"; + configs.put(JobConfig.JOB_NAME(), jobName); configs.put("tables.descriptors.provider.class", MySampleTableDescriptorsProvider.class.getName()); Config resultConfig = new MySampleTableConfigRewriter().rewrite(tableRewriterName, new MapConfig(configs)); Assert.assertNotNull(resultConfig); @@ -95,8 +97,8 @@ public void testWithTableDescriptorsProviderClass() throws Exception { Assert.assertTrue(storageConfig.getStorageKeySerde(localTableId).startsWith("StringSerde")); Assert.assertTrue(storageConfig.getStorageMsgSerde(localTableId).startsWith("StringSerde")); Config storeConfig = resultConfig.subset("stores." + localTableId + ".", true); - Assert.assertTrue(storeConfig.size() == 4); - Assert.assertEquals(storeConfig.getInt("rocksdb.block.size.bytes"), 4096); + Assert.assertEquals(4, storeConfig.size()); + Assert.assertEquals(4096, storeConfig.getInt("rocksdb.block.size.bytes")); JavaTableConfig tableConfig = new JavaTableConfig(resultConfig); Assert.assertEquals(tableConfig.getTableProviderFactory(localTableId), @@ -155,7 +157,7 @@ public Config rewrite(String name, Config config) { TableDescriptorsProvider tableDescriptorsProvider = Util.getObj(tableDescriptorsProviderClassName, TableDescriptorsProvider.class); List tableDescs = tableDescriptorsProvider.getTableDescriptors(config); - return new MapConfig(Arrays.asList(config, TableConfigGenerator.generateConfigsForTableDescs(tableDescs))); + return new MapConfig(Arrays.asList(config, TableConfigGenerator.generateConfigsForTableDescs(config, tableDescs))); } catch (Exception e) { throw new ConfigException(String.format("Invalid configuration for TableDescriptorsProvider class: %s", tableDescriptorsProviderClassName), e); From 12c09af06430a9b59412f369a4e86a70ee686418 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Sun, 26 Aug 2018 23:09:11 -0700 Subject: [PATCH 16/38] SAMZA-1789: Fix a merging error (with SAMZA-1813) --- .../runtime/AbstractApplicationRunner.java | 18 ++++--- .../samza/runtime/LocalApplicationRunner.java | 45 ++++++++--------- .../runtime/RemoteApplicationRunner.java | 48 ++++++++++--------- .../runtime/TestLocalApplicationRunner.java | 19 ++++---- 4 files changed, 70 insertions(+), 60 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java index 7a90fe75d2..27e9782b01 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java @@ -101,18 +101,18 @@ List prepareJobs() throws Exception { appDesc); } - StreamManager buildAndStartStreamManager() { + StreamManager buildAndStartStreamManager(Config config) { StreamManager streamManager = new StreamManager(config); streamManager.start(); return streamManager; } - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, StreamManager streamManager) throws Exception { - return getExecutionPlan(specGraph, null, streamManager); + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) throws Exception { + return getExecutionPlan(specGraph, null); } /* package private */ - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, StreamManager streamManager) throws Exception { + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) throws Exception { // update application configs Map cfg = new HashMap<>(config); @@ -129,8 +129,14 @@ ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId, Stream validateAppClassCfg(cfg, appDesc.getAppClass()); // create the physical execution plan - ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager); - return planner.plan(specGraph); + Config generatedConfig = new MapConfig(cfg); + StreamManager streamManager = buildAndStartStreamManager(generatedConfig); + try { + ExecutionPlanner planner = new ExecutionPlanner(generatedConfig, streamManager); + return planner.plan(specGraph); + } finally { + streamManager.stop(); + } } /** diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 0af623cb20..c53203ccfb 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -157,28 +157,30 @@ static class LocalJobPlanner extends JobPlanner { @Override List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { // for high-level DAG, generating the plan and job configs - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), streamManager); + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph()); - String executionPlanJson = plan.getPlanAsJson(); - writePlanJsonFile(executionPlanJson); - LOG.info("Execution Plan: \n" + executionPlanJson); + String executionPlanJson = plan.getPlanAsJson(); + writePlanJsonFile(executionPlanJson); + LOG.info("Execution Plan: \n" + executionPlanJson); + String planId = String.valueOf(executionPlanJson.hashCode()); - // 2. create the necessary streams - // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 - String planId = String.valueOf(executionPlanJson.hashCode()); - createStreams(planId, plan.getIntermediateStreams(), streamManager); - - return plan.getJobConfigs(); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } + List jobConfigs = plan.getJobConfigs(); + jobConfigs.forEach(jobConfig -> { + // 2. create the necessary streams + // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 + // TODO: why are we creating all intermediate streams in a plan repeatedly in multiple jobs? + StreamManager streamManager = null; + try { + streamManager = buildAndStartStreamManager(jobConfig); + createStreams(planId, plan.getIntermediateStreams(), streamManager); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + }); + return jobConfigs; } /** @@ -189,9 +191,8 @@ List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws * @param planId a unique identifier representing the plan used for coordination purpose * @param intStreams list of intermediate {@link StreamSpec}s * @param streamManager the {@link StreamManager} used to create streams - * @throws TimeoutException exception for latch timeout */ - private void createStreams(String planId, List intStreams, StreamManager streamManager) throws TimeoutException { + private void createStreams(String planId, List intStreams, StreamManager streamManager) { if (intStreams.isEmpty()) { LOG.info("Set of intermediate streams is empty. Nothing to create."); return; diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 0bca912906..fe5dbeea6c 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -66,30 +66,32 @@ class RemoteJobPlanner extends JobPlanner { @Override List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { // for high-level DAG, generate the plan and job configs - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(); - // TODO: run.id needs to be set for standalone: SAMZA-1531 - // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision - String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The run id for this run is {}", runId); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId, streamManager); - writePlanJsonFile(plan.getPlanAsJson()); - - // 2. create the necessary streams - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - streamManager.createStreams(plan.getIntermediateStreams()); + // TODO: run.id needs to be set for standalone: SAMZA-1531 + // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); + LOG.info("The run id for this run is {}", runId); - return plan.getJobConfigs(); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId); + writePlanJsonFile(plan.getPlanAsJson()); + + List jobConfigs = plan.getJobConfigs(); + jobConfigs.forEach(jobConfig -> { + StreamManager streamManager = null; + try { + // 2. create the necessary streams + streamManager = buildAndStartStreamManager(jobConfig); + if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { + streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); + } + streamManager.createStreams(plan.getIntermediateStreams()); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + }); + return jobConfigs; } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index ece2b88c24..54b9634b76 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -63,7 +63,6 @@ import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -99,12 +98,13 @@ public void setUp() { public void testStreamCreation() throws Exception { StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); when(plan.getPlanAsJson()).thenReturn(""); - doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); + when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(config))); + doReturn(plan).when(localPlanner).getExecutionPlan(any()); CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class); @@ -130,12 +130,13 @@ public void testStreamCreation() public void testStreamCreationWithCoordination() throws Exception { StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); when(plan.getPlanAsJson()).thenReturn(""); - doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); + when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(config))); + doReturn(plan).when(localPlanner).getExecutionPlan(any()); CoordinationUtils coordinationUtils = mock(CoordinationUtils.class); CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); @@ -210,12 +211,12 @@ public void testRunComplete() // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList()); when(plan.getPlanAsJson()).thenReturn(""); when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))); - doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); + doReturn(plan).when(localPlanner).getExecutionPlan(any()); StreamProcessor sp = mock(StreamProcessor.class); ArgumentCaptor captor = @@ -252,12 +253,12 @@ public void testRunFailure() // buildAndStartStreamManager already includes start, so not going to verify it gets called StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); ExecutionPlan plan = mock(ExecutionPlan.class); when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList()); when(plan.getPlanAsJson()).thenReturn(""); when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))); - doReturn(plan).when(localPlanner).getExecutionPlan(any(), eq(streamManager)); + doReturn(plan).when(localPlanner).getExecutionPlan(any()); StreamProcessor sp = mock(StreamProcessor.class); ArgumentCaptor captor = From f04404cc2e13e2b41c0ddd656fca7bc4fec721d2 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Tue, 28 Aug 2018 20:51:20 -0700 Subject: [PATCH 17/38] SAMZA-1789: move createStreams out of the loop in prepareJobs --- .../samza/runtime/LocalApplicationRunner.java | 36 +++++++++-------- .../runtime/RemoteApplicationRunner.java | 39 +++++++++++-------- 2 files changed, 42 insertions(+), 33 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index c53203ccfb..eb89f2769a 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -165,22 +165,26 @@ List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws LOG.info("Execution Plan: \n" + executionPlanJson); String planId = String.valueOf(executionPlanJson.hashCode()); - List jobConfigs = plan.getJobConfigs(); - jobConfigs.forEach(jobConfig -> { - // 2. create the necessary streams - // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 - // TODO: why are we creating all intermediate streams in a plan repeatedly in multiple jobs? - StreamManager streamManager = null; - try { - streamManager = buildAndStartStreamManager(jobConfig); - createStreams(planId, plan.getIntermediateStreams(), streamManager); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - }); - return jobConfigs; + if (plan.getJobConfigs().isEmpty()) { + throw new SamzaException("No jobs in the plan."); + } + + // 2. create the necessary streams + // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 + // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig + // to be used for the whole application + JobConfig jobConfig = plan.getJobConfigs().get(0); + StreamManager streamManager = null; + try { + // create the StreamManager to create intermediate streams in the plan + streamManager = buildAndStartStreamManager(jobConfig); + createStreams(planId, plan.getIntermediateStreams(), streamManager); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + return plan.getJobConfigs(); } /** diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index fe5dbeea6c..ef90aa213a 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -75,23 +75,28 @@ List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId); writePlanJsonFile(plan.getPlanAsJson()); - List jobConfigs = plan.getJobConfigs(); - jobConfigs.forEach(jobConfig -> { - StreamManager streamManager = null; - try { - // 2. create the necessary streams - streamManager = buildAndStartStreamManager(jobConfig); - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - streamManager.createStreams(plan.getIntermediateStreams()); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - }); - return jobConfigs; + if (plan.getJobConfigs().isEmpty()) { + throw new SamzaException("No jobs in the plan."); + } + + // 2. create the necessary streams + // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig + // to be used for the whole application + JobConfig jobConfig = plan.getJobConfigs().get(0); + StreamManager streamManager = null; + try { + // create the StreamManager to create intermediate streams in the plan + streamManager = buildAndStartStreamManager(jobConfig); + if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { + streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); + } + streamManager.createStreams(plan.getIntermediateStreams()); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + return plan.getJobConfigs(); } } From f2969f8dfa1bbe19bf823836daa03c55cb064051 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Thu, 30 Aug 2018 18:56:17 -0700 Subject: [PATCH 18/38] SAMZA-1789: fixed ApplicationDescriptor to use InputDescriptor and OutputDescriptor; addressed Prateek's comments. --- .../application/ApplicationDescriptor.java | 35 +- ...icationBase.java => SamzaApplication.java} | 4 +- .../application/StreamAppDescriptor.java | 16 - .../samza/application/StreamApplication.java | 33 +- .../samza/application/TaskAppDescriptor.java | 12 +- .../samza/application/TaskApplication.java | 2 +- .../samza/metrics/MetricsReporterFactory.java | 5 +- .../apache/samza/operators/MessageStream.java | 2 +- .../samza/runtime/ApplicationRunner.java | 20 +- .../samza/runtime/ApplicationRunners.java | 48 ++- .../samza/runtime/MockApplicationRunner.java | 70 ---- .../samza/runtime/TestApplicationRunners.java | 43 +++ .../samza/application/AppDescriptorImpl.java | 149 ++++++++- .../application/ApplicationDescriptors.java | 8 +- .../application/LegacyTaskApplication.java | 2 +- .../application/StreamAppDescriptorImpl.java | 70 +--- .../application/TaskAppDescriptorImpl.java | 63 +--- .../container/SamzaContainerListener.java | 19 +- .../apache/samza/execution/JobPlanner.java | 189 +++++++++++ .../samza/execution/LocalJobPlanner.java | 133 ++++++++ .../samza/execution/RemoteJobPlanner.java | 95 ++++++ .../stream/IntermediateMessageStreamImpl.java | 2 +- .../samza/processor/StreamProcessor.java | 89 ++--- .../runtime/AbstractApplicationRunner.java | 209 ------------ .../samza/runtime/ApplicationClassUtils.java | 12 +- .../samza/runtime/ApplicationRunnerMain.java | 4 +- .../samza/runtime/LocalApplicationRunner.java | 314 +++++++----------- .../samza/runtime/LocalContainerRunner.java | 30 +- .../runtime/RemoteApplicationRunner.java | 85 +---- .../apache/samza/task/TaskFactoryUtil.java | 2 +- .../samza/container/SamzaContainer.scala | 10 +- .../samza/job/local/ThreadJobFactory.scala | 17 +- ...cation.java => MockStreamApplication.java} | 2 +- .../application/MockTaskApplication.java | 29 -- .../TestStreamAppDescriptorImpl.java | 10 +- .../TestTaskAppDescriptorImpl.java | 62 +++- .../samza/execution/TestLocalJobPlanner.java | 210 ++++++++++++ .../samza/execution/TestRemoteJobPlanner.java | 87 +++++ .../runtime/TestApplicationClassUtils.java | 28 +- .../runtime/TestApplicationRunnerMain.java | 30 +- .../runtime/TestLocalApplicationRunner.java | 221 +----------- .../samza/task/TestTaskFactoryUtil.java | 8 +- .../samza/container/TestSamzaContainer.scala | 30 +- .../sql/runner/SamzaSqlApplicationRunner.java | 4 - .../samza/example/KeyValueStoreExample.java | 2 +- .../samza/example/RepartitionExample.java | 2 +- .../samza/example/TaskApplicationExample.java | 16 +- .../samza/test/framework/TestRunner.java | 4 +- .../LocalApplicationRunnerMain.java | 6 +- .../test/processor/TestStreamApplication.java | 70 ++-- 50 files changed, 1392 insertions(+), 1221 deletions(-) rename samza-api/src/main/java/org/apache/samza/application/{ApplicationBase.java => SamzaApplication.java} (91%) delete mode 100644 samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java create mode 100644 samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java create mode 100644 samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java create mode 100644 samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java delete mode 100644 samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java rename samza-core/src/test/java/org/apache/samza/application/{TestStreamApplication.java => MockStreamApplication.java} (93%) delete mode 100644 samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java create mode 100644 samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java create mode 100644 samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index 14fe256fa0..90f9ebf4cc 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -18,9 +18,13 @@ */ package org.apache.samza.application; +import java.util.Map; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; @@ -37,7 +41,7 @@ public interface ApplicationDescriptor { /** - * Get the initial {@link Config} supplied to the user application + * Get {@link Config} * @return config object */ Config getConfig(); @@ -51,7 +55,7 @@ public interface ApplicationDescriptor { * TODO: this should be replaced by the shared context factory when SAMZA-1714 is fixed. * @param contextManager the {@link ContextManager} to use for the application - * @return the {@link ApplicationDescriptor} with {@code contextManager} set as its {@link ContextManager} + * @return type {@code S} of {@link ApplicationDescriptor} with {@code contextManager} set as its {@link ContextManager} */ S withContextManager(ContextManager contextManager); @@ -64,8 +68,33 @@ public interface ApplicationDescriptor { * * @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 the {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory} + * @return type {@code S} of {@link ApplicationDescriptor} with {@code listenerFactory} set as its {@link ProcessorLifecycleListenerFactory} */ S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); + /** + * Sets the default SystemDescriptor to use for intermediate streams. This is equivalent to setting + * {@code job.default.system} and its properties in configuration. + *

    + * If the default system descriptor is set, it must be set before creating any input/output/intermediate streams. + *

    + * If an input/output stream is created with a stream-level Serde, they will be used, else the serde specified + * for the {@code job.default.system} in configuration will be used. + *

    + * Providing an incompatible message type for the intermediate streams that use the default serde will result in + * {@link ClassCastException}s at runtime. + * + * @param defaultSystemDescriptor the default system descriptor to use + * @return type {@code S} of {@link ApplicationDescriptor} with {@code defaultSystemDescriptor} set as its default system + */ + S withDefaultSystem(SystemDescriptor defaultSystemDescriptor); + + /** + * Sets a set of customized {@link MetricsReporter}s in the application + * + * @param reporterFactories the map of customized {@link MetricsReporterFactory} objects to be used + * @return type {@code S} of {@link ApplicationDescriptor} with {@code reporterFactories} + */ + S withMetricsReporterFactories(Map reporterFactories); + } diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java b/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java similarity index 91% rename from samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java rename to samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java index 4ae7e2dfc9..7606be8097 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationBase.java +++ b/samza-api/src/main/java/org/apache/samza/application/SamzaApplication.java @@ -24,12 +24,12 @@ /** * The base interface for all user-implemented applications in Samza. *

    - * The main processing logic of the user application should be implemented in {@link ApplicationBase#describe(ApplicationDescriptor)} + * 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. */ @InterfaceStability.Evolving -public interface ApplicationBase { +public interface SamzaApplication { /** * Describes the user processing logic via {@link ApplicationDescriptor} diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java index c322b011e1..ff0c305232 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java @@ -35,22 +35,6 @@ @InterfaceStability.Evolving public interface StreamAppDescriptor extends ApplicationDescriptor { - /** - * Sets the default SystemDescriptor to use for intermediate streams. This is equivalent to setting - * {@code job.default.system} and its properties in configuration. - *

    - * If the default system descriptor is set, it must be set before creating any intermediate streams. - *

    - * If the intermediate stream is created with a stream-level Serde, they will be used, else the serde specified - * for the {@code job.default.system} in configuration will be used. - *

    - * Providing an incompatible message type for the intermediate streams that use the default serde will result in - * {@link ClassCastException}s at runtime. - * - * @param defaultSystemDescriptor the default system descriptor to use - */ - void setDefaultSystem(SystemDescriptor defaultSystemDescriptor); - /** * Gets the input {@link MessageStream} corresponding to the {@code inputDescriptor}. *

    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 cab4430c05..c8796de7d8 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,26 @@ import org.apache.samza.annotation.InterfaceStability; /** - * Describes and initializes the transforms for processing message streams and generating results in high-level API. - *

    - * This is a marker interface that users will implement for a high-level application. + * Describes and initializes the transforms for processing message streams and generating results in high-level API. Your + * application is expected to implement this interface. *

    * The following example removes page views older than 1 hour from the input stream: *

    {@code
      * public class PageViewCounter implements StreamApplication {
      *   public void describe(StreamAppDescriptor appDesc) {
    - *     MessageStream pageViewEvents =
    - *       appDesc.getInputStream("pageViewEvents", (k, m) -> (PageViewEvent) m);
    - *     OutputStream recentPageViewEvents =
    - *       appDesc.getOutputStream("recentPageViewEvents", m -> m.memberId, m -> m);
    + *     KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking");
    + *     KafkaInputDescriptor inputStreamDescriptor =
    + *         trackingSystem.getInputDescriptor("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class));
    + *
    + *     KafkaOutputDescriptor> outputStreamDescriptor =
    + *         trackingSystem.getOutputDescriptor("recentPageViewEvent", new JsonSerdeV2<>(PageViewEvent.class)));
    + *
    + *     MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor);
    + *     OutputStream recentPageViewEvents = appDesc.getOutputStream(outputStreamDescriptor);
      *
      *     pageViewEvents
      *       .filter(m -> m.getCreationTime() > System.currentTimeMillis() - Duration.ofHours(1).toMillis())
    - *       .sendTo(filteredPageViewEvents);
    + *       .sendTo(recentPageViewEvents);
      *   }
      * }
      * }
    @@ -54,13 +58,10 @@ * }
    * *

    - * Implementation Notes: Currently StreamApplications are wrapped in a {@link org.apache.samza.task.StreamTask} during - * execution. A new {@link StreamAppDescriptor} instance will be created and described by the user-defined - * {@link StreamApplication} when planning the execution. All user-defined transformation functions and descriptors for - * data entities used in the transformations (e.g. {@link org.apache.samza.operators.TableDescriptor}) are required to - * be serializable. 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}. + * Implementation Notes: Currently {@link StreamApplication}s are wrapped in a {@link org.apache.samza.task.StreamTask} + * during execution. All user-defined transformation functions are required to be serializable. 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}. * *

    * The user-implemented {@link StreamApplication} class must be a class with proper fully-qualified class name and @@ -71,5 +72,5 @@ * instance is closed. See {@link org.apache.samza.operators.functions.InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ @InterfaceStability.Evolving -public interface StreamApplication extends ApplicationBase { +public interface StreamApplication extends SamzaApplication { } diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java index dc68574a37..d89eca72b7 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java @@ -20,6 +20,8 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.task.TaskFactory; @@ -41,18 +43,16 @@ public interface TaskAppDescriptor extends ApplicationDescriptor { +public interface TaskApplication extends SamzaApplication { } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java index 7807222787..b9934e52d1 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java @@ -19,11 +19,12 @@ package org.apache.samza.metrics; +import java.io.Serializable; import org.apache.samza.config.Config; /** * Build a {@link org.apache.samza.metrics.MetricsReporter} */ -public interface MetricsReporterFactory { - MetricsReporter getMetricsReporter(String name, String containerName, Config config); +public interface MetricsReporterFactory extends Serializable { + MetricsReporter getMetricsReporter(String name, String processorId, Config config); } diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index ea551c788f..21a200efd2 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -214,7 +214,7 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index a3180fcd32..7c1ad7f648 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -19,35 +19,34 @@ package org.apache.samza.runtime; import java.time.Duration; -import java.util.Map; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; /** - * The primary means of managing execution of the {@link org.apache.samza.application.ApplicationBase} at runtime. + * The primary means of managing execution of the {@link SamzaApplication} at runtime. */ @InterfaceStability.Evolving public interface ApplicationRunner { /** - * Deploy and run the Samza jobs to execute {@link org.apache.samza.application.ApplicationBase}. + * Deploy and run the Samza jobs to execute {@link SamzaApplication}. * It is non-blocking so it doesn't wait for the application running. */ void run(); /** - * Kill the Samza jobs represented by {@link org.apache.samza.application.ApplicationBase} + * Kill the Samza jobs represented by {@link SamzaApplication} * It is non-blocking so it doesn't wait for the application stopping. */ void kill(); /** - * Get the collective status of the Samza jobs represented by {@link org.apache.samza.application.ApplicationBase}. + * Get the collective status of the Samza jobs represented by {@link SamzaApplication}. * Returns {@link ApplicationStatus} object. * - * @return the current status of an instance of {@link org.apache.samza.application.ApplicationBase} + * @return the current status of an instance of {@link SamzaApplication} */ ApplicationStatus status(); @@ -65,11 +64,4 @@ public interface ApplicationRunner { */ boolean waitForFinish(Duration timeout); - /** - * Add a set of customized {@link MetricsReporter}s in the application - * - * @param metricsReporters the map of customized {@link MetricsReporter}s objects to be used - */ - void addMetricsReporters(Map metricsReporters); - } diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java index 53601a1a50..cd1d06b23f 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunners.java @@ -19,13 +19,13 @@ package org.apache.samza.runtime; import java.lang.reflect.Constructor; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; /** - * Creates {@link ApplicationRunner} instances based on configuration and user-implemented {@link ApplicationBase} + * Creates {@link ApplicationRunner} instances based on configuration and user-implemented {@link SamzaApplication} * *

    This class is usually used in main() function to create an instance of {@link ApplicationRunner}, as in the example * below: @@ -34,7 +34,7 @@ * CommandLine cmdLine = new CommandLine(); * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); * PageViewCounter app = new PageViewCounter(); - * ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PageViewCounter(), config); + * ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config); * runner.run(); * runner.waitForFinish(); * } @@ -42,6 +42,9 @@ */ public class ApplicationRunners { + private static final String APP_RUNNER_CFG = "app.runner.class"; + private static final String DEFAULT_APP_RUNNER = "org.apache.samza.runtime.RemoteApplicationRunner"; + private ApplicationRunners() { } @@ -53,36 +56,27 @@ private ApplicationRunners() { * @param config the configuration for this application * @return the {@link ApplicationRunner} object that will run the {@code userApp} */ - public static final ApplicationRunner getApplicationRunner(ApplicationBase userApp, Config config) { - AppRunnerConfig runnerConfig = new AppRunnerConfig(config); + public static final ApplicationRunner getApplicationRunner(SamzaApplication userApp, Config config) { + String appRunnerClassName = getAppRunnerClass(config); try { - Class runnerClass = Class.forName(runnerConfig.getAppRunnerClass()); - if (ApplicationRunner.class.isAssignableFrom(runnerClass)) { - Constructor constructor = runnerClass.getConstructor(ApplicationBase.class, Config.class); // *sigh* - return (ApplicationRunner) constructor.newInstance(userApp, config); + Class runnerClass = Class.forName(appRunnerClassName); + if (!ApplicationRunner.class.isAssignableFrom(runnerClass)) { + throw new ConfigException( + String.format("Class %s does not extend ApplicationRunner properly", appRunnerClassName)); } + Constructor constructor = runnerClass.getConstructor(SamzaApplication.class, Config.class); // *sigh* + return (ApplicationRunner) constructor.newInstance(userApp, config); + } catch (ConfigException ce) { + // this is thrown due to invalid app.runner.class configuration + throw ce; } catch (Exception e) { - throw new ConfigException(String.format("Problem in loading ApplicationRunner class %s", - runnerConfig.getAppRunnerClass()), e); + // other types of exception during class loading and construction of new instance + throw new ConfigException(String.format("Could not load ApplicationRunner class %s", appRunnerClassName), e); } - throw new ConfigException(String.format("Class %s does not extend ApplicationRunner properly", - runnerConfig.getAppRunnerClass())); } - static class AppRunnerConfig { - private static final String APP_RUNNER_CFG = "app.runner.class"; - private static final String DEFAULT_APP_RUNNER = "org.apache.samza.runtime.RemoteApplicationRunner"; - - private final Config config; - - AppRunnerConfig(Config config) { - this.config = config; - } - - String getAppRunnerClass() { - return this.config.getOrDefault(APP_RUNNER_CFG, DEFAULT_APP_RUNNER); - } - + private static String getAppRunnerClass(Config config) { + return config.getOrDefault(APP_RUNNER_CFG, DEFAULT_APP_RUNNER); } } diff --git a/samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java b/samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java deleted file mode 100644 index ca54353c2c..0000000000 --- a/samza-api/src/test/java/org/apache/samza/runtime/MockApplicationRunner.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.runtime; - -import java.time.Duration; -import java.util.Map; -import org.apache.samza.application.ApplicationBase; -import org.apache.samza.config.Config; -import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; - - -/** - * Test class for {@link org.apache.samza.runtime.ApplicationRunners} unit test - */ -public class MockApplicationRunner implements ApplicationRunner { - private final ApplicationBase userApp; - private final Config config; - - public MockApplicationRunner(ApplicationBase userApp, Config config) { - this.userApp = userApp; - this.config = config; - } - - @Override - public void run() { - - } - - @Override - public void kill() { - - } - - @Override - public ApplicationStatus status() { - return null; - } - - @Override - public void waitForFinish() { - - } - - @Override - public boolean waitForFinish(Duration timeout) { - return false; - } - - @Override - public void addMetricsReporters(Map metricsReporters) { - - } -} \ No newline at end of file diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java index edd64ea01e..34e5ecce9f 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java @@ -18,11 +18,15 @@ */ package org.apache.samza.runtime; +import java.time.Duration; import java.util.HashMap; import java.util.Map; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; +import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; import org.junit.Test; import static org.junit.Assert.assertTrue; @@ -43,4 +47,43 @@ public void testGetAppRunner() { ApplicationRunner appRunner = ApplicationRunners.getApplicationRunner(app, config); assertTrue(appRunner instanceof MockApplicationRunner); } + + /** + * Test class for {@link ApplicationRunners} unit test + */ + public static class MockApplicationRunner implements ApplicationRunner { + private final SamzaApplication userApp; + private final Config config; + + public MockApplicationRunner(SamzaApplication userApp, Config config) { + this.userApp = userApp; + this.config = config; + } + + @Override + public void run() { + + } + + @Override + public void kill() { + + } + + @Override + public ApplicationStatus status() { + return null; + } + + @Override + public void waitForFinish() { + + } + + @Override + public boolean waitForFinish(Duration timeout) { + return false; + } + + } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java index 6146d9d92c..bbc6b0f58a 100644 --- a/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java @@ -18,8 +18,20 @@ */ package org.apache.samza.application; +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; import org.apache.samza.config.Config; +import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; import org.apache.samza.task.TaskContext; @@ -38,7 +50,16 @@ public abstract class AppDescriptorImpl implements ApplicationDescriptor { final Config config; - final Class appClass; + final Class appClass; + + private final Map inputDescriptors = new LinkedHashMap<>(); + private final Map outputDescriptors = new LinkedHashMap<>(); + private final Map systemDescriptors = new LinkedHashMap<>(); + private final Set broadcastStreams = new HashSet<>(); + private final Map tableDescriptors = new LinkedHashMap<>(); + private final Map reporterFactories = new LinkedHashMap<>(); + + private Optional defaultSystemDescriptorOptional = Optional.empty(); // Default to no-op functions in ContextManager // TODO: this should be replaced by shared context factory defined in SAMZA-1714 @@ -55,7 +76,7 @@ public void close() { // Default to no-op ProcessorLifecycleListenerFactory ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { }; - AppDescriptorImpl(ApplicationBase userApp, Config config) { + AppDescriptorImpl(SamzaApplication userApp, Config config) { this.config = config; this.appClass = userApp.getClass(); } @@ -77,10 +98,26 @@ public S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory return (S) this; } + @Override + public S withDefaultSystem(SystemDescriptor defaultSystemDescriptor) { + Preconditions.checkNotNull(defaultSystemDescriptor, "Provided defaultSystemDescriptor must not be null."); + Preconditions.checkState(noInputOutputStreams(), + "Default system must be set before creating any input or output streams."); + addSystemDescriptor(defaultSystemDescriptor); + this.defaultSystemDescriptorOptional = Optional.of(defaultSystemDescriptor); + return (S) this; + } + + @Override + public S withMetricsReporterFactories(Map reporterFactories) { + this.reporterFactories.clear(); + this.reporterFactories.putAll(reporterFactories); + return (S) this; + } /** * Get the user application class */ - public Class getAppClass() { + public Class getAppClass() { return appClass; } @@ -102,4 +139,110 @@ public ProcessorLifecycleListenerFactory getProcessorLifecycleListenerFactory() return listenerFactory; } + /** + * Get all the {@link InputDescriptor}s to this application + * + * @return an immutable map of streamId to {@link InputDescriptor} + */ + public Map getInputDescriptors() { + return Collections.unmodifiableMap(inputDescriptors); + } + + /** + * Get all the {@link OutputDescriptor}s from this application + * + * @return an immutable map of streamId to {@link OutputDescriptor} + */ + public Map getOutputDescriptors() { + return Collections.unmodifiableMap(outputDescriptors); + } + + /** + * Get all the broadcast streamIds from this application + * + * @return an immutable set of streamIds + */ + public Set getBroadcastStreams() { + return Collections.unmodifiableSet(broadcastStreams); + } + + /** + * Get all the {@link TableDescriptor}s in this application + * + * @return an immutable set of {@link TableDescriptor}s + */ + public Set getTableDescriptors() { + return Collections.unmodifiableSet(new HashSet<>(tableDescriptors.values())); + } + + /** + * Get all the unique {@link SystemDescriptor}s in this application + * + * @return an immutable set of {@link SystemDescriptor}s + */ + public Set getSystemDescriptors() { + // We enforce that users must not use different system descriptor instances for the same system name + // when getting an input/output stream or setting the default system descriptor + return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); + } + + /** + * Get the default {@link SystemDescriptor} in this application + * + * @return the default {@link SystemDescriptor} + */ + public Optional getDefaultSystemDescriptor() { + return this.defaultSystemDescriptorOptional; + } + + /** + * Get the {@link MetricsReporterFactory}s used in the application + * + * @return the map of {@link MetricsReporterFactory}s + */ + public Map getMetricsReporterFactories() { + return Collections.unmodifiableMap(reporterFactories); + } + + // TODO: this should go away when partitionBy() and broadcast() will also generate InputDescriptor/OutputDescriptor as well + // helper method to determine that there is no input/output streams added in the application yet + protected abstract boolean noInputOutputStreams(); + + // internal method to add {@link TableDescriptor} to this application + void addTableDescriptor(TableDescriptor tableDescriptor) { + Preconditions.checkState(!tableDescriptors.containsKey(tableDescriptor.getTableId()), + String.format("add table descriptors multiple times with the same tableId: %s", tableDescriptor.getTableId())); + tableDescriptors.put(tableDescriptor.getTableId(), tableDescriptor); + } + + // internal method to add {@link InputDescriptor} to this application + void addInputDescriptor(InputDescriptor isd) { + // TODO: need to add to the broadcast streams if isd is a broadcast stream + Preconditions.checkState(!inputDescriptors.containsKey(isd.getStreamId()), + String.format("add input descriptors multiple times with the same streamId: %s", isd.getStreamId())); + inputDescriptors.put(isd.getStreamId(), isd); + addSystemDescriptor(isd.getSystemDescriptor()); + } + + // internal method to add {@link OutputDescriptor} to this application + void addOutputDescriptor(OutputDescriptor osd) { + // TODO: need to add to the broadcast streams if osd is a broadcast stream + Preconditions.checkState(!outputDescriptors.containsKey(osd.getStreamId()), + String.format("add output descriptors multiple times with the same streamId: %s", osd.getStreamId())); + outputDescriptors.put(osd.getStreamId(), osd); + addSystemDescriptor(osd.getSystemDescriptor()); + } + + // internal method to add a unique {@link SystemDescriptor} to this application + void addSystemDescriptor(SystemDescriptor systemDescriptor) { + Preconditions.checkState(!systemDescriptors.containsKey(systemDescriptor.getSystemName()) + || systemDescriptors.get(systemDescriptor.getSystemName()) == systemDescriptor, + "Must not use different system descriptor instances for the same system name: " + systemDescriptor.getSystemName()); + systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); + } + + // TODO: this should be completely internal to addInputDescriptor()/addOutputDescriptor after we add broadcast automatically + void addBroadcastStream(String streamId) { + this.broadcastStreams.add(streamId); + } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java index 28117adb81..261ff881f5 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java @@ -23,7 +23,7 @@ /** - * Util class to help creating {@link AppDescriptorImpl} instance from {@link ApplicationBase} and {@link Config} + * Util class to help creating {@link AppDescriptorImpl} instance from {@link SamzaApplication} and {@link Config} */ public class ApplicationDescriptors { private ApplicationDescriptors() { @@ -31,13 +31,13 @@ private ApplicationDescriptors() { } /** - * Create a new instance of {@link AppDescriptorImpl} based on {@link ApplicationBase} and {@link Config} + * Create a new instance of {@link AppDescriptorImpl} based on {@link SamzaApplication} and {@link Config} * - * @param userApp the user-implemented {@link ApplicationBase}. The {@code userApp} has to have a proper fully-qualified class name. + * @param userApp the user-implemented {@link SamzaApplication}. The {@code userApp} has to have a proper fully-qualified class name. * @param config the user-supplied {@link Config} for the the application * @return the {@link AppDescriptorImpl} instance containing the user processing logic and the config */ - public static AppDescriptorImpl getAppDescriptor(ApplicationBase userApp, Config config) { + public static AppDescriptorImpl getAppDescriptor(SamzaApplication userApp, Config config) { if (userApp instanceof StreamApplication) { return new StreamAppDescriptorImpl((StreamApplication) userApp, config); } 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 575ccae854..36579c2225 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 @@ -46,6 +46,6 @@ private Config validate(Config config) { @Override public void describe(TaskAppDescriptor appDesc) { - appDesc.setTaskFactory(TaskFactoryUtil.createTaskFactory(config)); + appDesc.setTaskFactory(TaskFactoryUtil.getTaskFactoryFromConfig(config)); } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java index 7c51793d8e..5512fa5637 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java @@ -71,10 +71,6 @@ public class StreamAppDescriptorImpl extends AppDescriptorImpl inputOperators = new LinkedHashMap<>(); private final Map outputStreams = new LinkedHashMap<>(); - private final Map inputDescriptors = new LinkedHashMap<>(); - private final Map outputDescriptors = new LinkedHashMap<>(); - private final Map systemDescriptors = new LinkedHashMap<>(); - private final Set broadcastStreams = new HashSet<>(); private final Map tables = new LinkedHashMap<>(); /** @@ -84,24 +80,12 @@ public class StreamAppDescriptorImpl extends AppDescriptorImpl operatorIds = new HashSet<>(); - private Optional defaultSystemDescriptorOptional = Optional.empty(); public StreamAppDescriptorImpl(StreamApplication userApp, Config config) { super(userApp, config); userApp.describe(this); } - @Override - public void setDefaultSystem(SystemDescriptor defaultSystemDescriptor) { - Preconditions.checkNotNull(defaultSystemDescriptor, "Provided defaultSystemDescriptor must not be null."); - String defaultSystemName = defaultSystemDescriptor.getSystemName(); - Preconditions.checkState(inputOperators.isEmpty() && outputStreams.isEmpty(), - "Default system must be set before creating any input or output streams."); - checkSystemDescriptorUniqueness(defaultSystemDescriptor, defaultSystemName); - systemDescriptors.put(defaultSystemName, defaultSystemDescriptor); - this.defaultSystemDescriptorOptional = Optional.of(defaultSystemDescriptor); - } - @Override public MessageStream getInputStream(InputDescriptor inputDescriptor) { SystemDescriptor systemDescriptor = inputDescriptor.getSystemDescriptor(); @@ -110,13 +94,10 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor return expander.get().apply(this, inputDescriptor); } + super.addInputDescriptor(inputDescriptor); String streamId = inputDescriptor.getStreamId(); Preconditions.checkState(!inputOperators.containsKey(streamId), "getInputStream must not be called multiple times with the same streamId: " + streamId); - Preconditions.checkState(!inputDescriptors.containsKey(streamId), - "getInputStream must not be called multiple times with the same input descriptor: " + streamId); - String systemName = systemDescriptor.getSystemName(); - checkSystemDescriptorUniqueness(systemDescriptor, systemName); Serde serde = inputDescriptor.getSerde(); KV kvSerdes = getKVSerdes(streamId, serde); @@ -135,21 +116,15 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor OperatorSpecs.createInputOperatorSpec(streamId, kvSerdes.getKey(), kvSerdes.getValue(), transformer, isKeyed, this.getNextOpId(OpCode.INPUT, null)); inputOperators.put(streamId, inputOperatorSpec); - inputDescriptors.put(streamId, inputDescriptor); - systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); return new MessageStreamImpl(this, inputOperators.get(streamId)); } @Override public OutputStream getOutputStream(OutputDescriptor outputDescriptor) { + super.addOutputDescriptor(outputDescriptor); String streamId = outputDescriptor.getStreamId(); Preconditions.checkState(!outputStreams.containsKey(streamId), "getOutputStream must not be called multiple times with the same streamId: " + streamId); - Preconditions.checkState(!outputDescriptors.containsKey(streamId), - "getOutputStream must not be called multiple times with the same output descriptor: " + streamId); - SystemDescriptor systemDescriptor = outputDescriptor.getSystemDescriptor(); - String systemName = systemDescriptor.getSystemName(); - checkSystemDescriptorUniqueness(systemDescriptor, systemName); Serde serde = outputDescriptor.getSerde(); KV kvSerdes = getKVSerdes(streamId, serde); @@ -164,8 +139,6 @@ public OutputStream getOutputStream(OutputDescriptor outputDescript boolean isKeyed = serde instanceof KVSerde; outputStreams.put(streamId, new OutputStreamImpl<>(streamId, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed)); - outputDescriptors.put(streamId, outputDescriptor); - systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); return outputStreams.get(streamId); } @@ -174,6 +147,7 @@ public Table> getTable(TableDescriptor tableDescriptor) String tableId = tableDescriptor.getTableId(); Preconditions.checkState(StringUtils.isNotBlank(tableId) && ID_PATTERN.matcher(tableId).matches(), String.format("tableId: %s must confirm to pattern: %s", tableId, ID_PATTERN.toString())); + super.addTableDescriptor(tableDescriptor); TableSpec tableSpec = ((BaseTableDescriptor) tableDescriptor).getTableSpec(); if (tables.containsKey(tableSpec)) { throw new IllegalStateException( @@ -245,7 +219,8 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI ". Key and message serdes configured for the job.default.system will be used."); } - if (isBroadcast) broadcastStreams.add(streamId); + if (isBroadcast) + super.addBroadcastStream(streamId); boolean isKeyed; KV kvSerdes; @@ -257,7 +232,7 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI kvSerdes = getKVSerdes(streamId, serde); } - InputTransformer transformer = (InputTransformer) defaultSystemDescriptorOptional + InputTransformer transformer = (InputTransformer) getDefaultSystemDescriptor() .flatMap(SystemDescriptor::getTransformer).orElse(null); InputOperatorSpec inputOperatorSpec = @@ -276,38 +251,10 @@ public Map getOutputStreams() { return Collections.unmodifiableMap(outputStreams); } - public Set getBroadcastStreams() { - return Collections.unmodifiableSet(broadcastStreams); - } - public Map getTables() { return Collections.unmodifiableMap(tables); } - public Map getInputDescriptors() { - return Collections.unmodifiableMap(inputDescriptors); - } - - public Map getOutputDescriptors() { - return Collections.unmodifiableMap(outputDescriptors); - } - - public Set getSystemDescriptors() { - // We enforce that users must not use different system descriptor instances for the same system name - // when getting an input/output stream or setting the default system descriptor - return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); - } - - public Optional getDefaultSystemDescriptor() { - return this.defaultSystemDescriptorOptional; - } - - private void checkSystemDescriptorUniqueness(SystemDescriptor systemDescriptor, String systemName) { - Preconditions.checkState(!systemDescriptors.containsKey(systemName) - || systemDescriptors.get(systemName) == systemDescriptor, - "Must not use different system descriptor instances for the same system name: " + systemName); - } - private KV getKVSerdes(String streamId, Serde serde) { Serde keySerde, valueSerde; @@ -330,4 +277,9 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } + + @Override + protected boolean noInputOutputStreams() { + return inputOperators.isEmpty() && outputStreams.isEmpty(); + } } diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java index 35b43666f9..d7a1ea264d 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java @@ -18,11 +18,10 @@ */ package org.apache.samza.application; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.task.TaskFactory; @@ -37,11 +36,6 @@ public class TaskAppDescriptorImpl extends AppDescriptorImpl implements TaskAppDescriptor { TaskFactory taskFactory; - //TODO: need to replace with InputStreamDescriptor and OutputStreamDescriptor when SAMZA-1804 is implemented - final List inputStreams = new ArrayList<>(); - final List outputStreams = new ArrayList<>(); - final List broadcastStreams = new ArrayList<>(); - final List tables = new ArrayList<>(); public TaskAppDescriptorImpl(TaskApplication userApp, Config config) { super(userApp, config); @@ -54,18 +48,18 @@ public void setTaskFactory(TaskFactory factory) { } @Override - public void addInputStream(String inputStream) { - this.inputStreams.add(inputStream); + public void addInputStream(InputDescriptor isd) { + super.addInputDescriptor(isd); } @Override - public void addOutputStream(String outputStream) { - this.outputStreams.add(outputStream); + public void addOutputStream(OutputDescriptor osd) { + super.addOutputDescriptor(osd); } @Override - public void addTable(TableDescriptor table) { - this.tables.add(table); + public void addTable(TableDescriptor tableDescriptor) { + this.addTableDescriptor(tableDescriptor); } /** @@ -76,43 +70,8 @@ public TaskFactory getTaskFactory() { return taskFactory; } - /** - * Get the input streams to this application - * - * TODO: need to change to InputStreamDescriptors after SAMZA-1804 - * - * @return the list of input streamIds - */ - public List getInputStreams() { - return Collections.unmodifiableList(this.inputStreams); - } - - /** - * Get the broadcast streams to this application - * - * @return the list of broadcast streamIds - */ - public List getBroadcastStreams() { - return Collections.unmodifiableList(this.broadcastStreams); - } - - /** - * Get the output streams to this application - * - * TODO: need to change to OutputStreamDescriptors after SAMZA-1804 - * - * @return the list of output streamIds - */ - public List getOutputStreams() { - return Collections.unmodifiableList(this.outputStreams); - } - - /** - * Get the {@link TableDescriptor}s used in this application - * - * @return the list of {@link TableDescriptor}s - */ - public List getTables() { - return Collections.unmodifiableList(this.tables); + @Override + protected boolean noInputOutputStreams() { + return getInputDescriptors().isEmpty() && getOutputDescriptors().isEmpty(); } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java index e9f3558a93..a6437fa92b 100644 --- a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java +++ b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java @@ -19,39 +19,40 @@ package org.apache.samza.container; /** - * A Listener for {@link org.apache.samza.container.SamzaContainer} lifecycle events. + * A Listener for {@link SamzaContainer} lifecycle events. */ public interface SamzaContainerListener { /** - * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started + * Method invoked when the {@link SamzaContainer} state is {@link org.apache.samza.SamzaContainerStatus#NOT_STARTED} + * and is about to transit to {@link org.apache.samza.SamzaContainerStatus#STARTING} to start the initialization sequence. */ void beforeStart(); /** - * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has successfully transitioned to + * Method invoked after the {@link SamzaContainer} has successfully transitioned to * the {@link org.apache.samza.SamzaContainerStatus#STARTED} state and is about to start the * {@link org.apache.samza.container.RunLoop} */ - void onContainerStart(); + void afterStart(); /** - * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has successfully transitioned to + * Method invoked after the {@link SamzaContainer} has successfully transitioned to * {@link org.apache.samza.SamzaContainerStatus#STOPPED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    * Note: This will be the last call after completely shutting down the SamzaContainer without any * exceptions/errors. */ - void onContainerStop(); + void afterStop(); /** - * Method invoked when the {@link org.apache.samza.container.SamzaContainer} has transitioned to + * Method invoked after the {@link SamzaContainer} has transitioned to * {@link org.apache.samza.SamzaContainerStatus#FAILED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    - * Note: {@link #onContainerFailed(Throwable)} is mutually exclusive to {@link #onContainerStop()}. + * Note: {@link #afterFailed(Throwable)} is mutually exclusive to {@link #afterStop()}. * @param t Throwable that caused the container failure. */ - void onContainerFailed(Throwable t); + void afterFailed(Throwable t); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java new file mode 100644 index 0000000000..556dff5725 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -0,0 +1,189 @@ +/* + * 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.execution; + +import java.io.File; +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.SamzaException; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.TaskAppDescriptorImpl; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.ShellCommandConfig; +import org.apache.samza.config.StreamConfig; +import org.apache.samza.operators.BaseTableDescriptor; +import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; +import org.apache.samza.table.TableConfigGenerator; +import org.apache.samza.table.TableSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is a temporary helper class to include all common logic to generate {@link JobConfig}s for high- and low-level + * applications in {@link org.apache.samza.runtime.LocalApplicationRunner} and {@link org.apache.samza.runtime.RemoteApplicationRunner}. + * + * TODO: Fix SAMZA-1811 to consolidate this class with {@link ExecutionPlanner} + */ +public abstract class JobPlanner { + private static final Logger LOG = LoggerFactory.getLogger(JobPlanner.class); + + protected final AppDescriptorImpl appDesc; + protected final Config config; + + JobPlanner(AppDescriptorImpl descriptor) { + this.appDesc = descriptor; + this.config = descriptor.getConfig(); + } + + public List prepareJobs() { + String appId = new ApplicationConfig(appDesc.getConfig()).getGlobalAppId(); + return ApplicationDescriptors.forType( + taskAppDesc -> { + try { + return Collections.singletonList(JobPlanner.this.prepareTaskJob(taskAppDesc)); + } catch (Exception e) { + throw new SamzaException("Failed to generate JobConfig for TaskApplication " + appId, e); + } + }, + streamAppDesc -> { + try { + return JobPlanner.this.prepareStreamJobs(streamAppDesc); + } catch (Exception e) { + throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); + } + }, + appDesc); + } + + abstract List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception; + + StreamManager buildAndStartStreamManager(Config config) { + StreamManager streamManager = new StreamManager(config); + streamManager.start(); + return streamManager; + } + + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) throws Exception { + return getExecutionPlan(specGraph, null); + } + + /* package private */ + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) throws Exception { + + // update application configs + Map cfg = new HashMap<>(config); + if (StringUtils.isNoneEmpty(runId)) { + cfg.put(ApplicationConfig.APP_RUN_ID, runId); + } + + StreamConfig streamConfig = new StreamConfig(config); + Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); + inputStreams.removeAll(specGraph.getOutputStreams().keySet()); + ApplicationConfig.ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) + ? ApplicationConfig.ApplicationMode.BATCH : ApplicationConfig.ApplicationMode.STREAM; + cfg.put(ApplicationConfig.APP_MODE, mode.name()); + + // merge user-provided configuration with input/output descriptor generated configuration + // descriptor generated configuration has higher priority + Map systemStreamConfigs = expandSystemStreamConfigs(appDesc); + cfg.putAll(systemStreamConfigs); + + // adding app.class in the configuration + cfg.put(ApplicationConfig.APP_CLASS, appDesc.getAppClass().getName()); + + // create the physical execution plan + Config generatedConfig = new MapConfig(cfg); + // creating the StreamManager to get all input/output streams' metadata for planning + StreamManager streamManager = buildAndStartStreamManager(generatedConfig); + try { + ExecutionPlanner planner = new ExecutionPlanner(generatedConfig, streamManager); + return planner.plan(specGraph); + } finally { + streamManager.stop(); + } + } + + /** + * Write the execution plan JSON to a file + * @param planJson JSON representation of the plan + */ + final void writePlanJsonFile(String planJson) { + try { + String content = "plan='" + planJson + "'"; + String planPath = System.getenv(ShellCommandConfig.EXECUTION_PLAN_DIR()); + if (planPath != null && !planPath.isEmpty()) { + // Write the plan json to plan path + File file = new File(planPath + "/plan.json"); + file.setReadable(true, false); + PrintWriter writer = new PrintWriter(file, "UTF-8"); + writer.println(content); + writer.close(); + } + } catch (Exception e) { + LOG.warn("Failed to write execution plan json to file", e); + } + } + + // helper method to generate a single node job configuration for low level task applications + private JobConfig prepareTaskJob(TaskAppDescriptorImpl taskAppDesc) { + // copy original configure + Map cfg = new HashMap<>(config); + // expand system and streams configure + Map systemStreamConfigs = expandSystemStreamConfigs(taskAppDesc); + cfg.putAll(systemStreamConfigs); + // expand table configure + cfg.putAll(expandTableConfigs(cfg, taskAppDesc)); + // adding app.class in the configuration + cfg.put(ApplicationConfig.APP_CLASS, appDesc.getAppClass().getName()); + return new JobConfig(new MapConfig(cfg)); + } + + private Map expandSystemStreamConfigs(AppDescriptorImpl appDesc) { + Map systemStreamConfigs = new HashMap<>(); + appDesc.getInputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(((InputDescriptor) value).toConfig())); + appDesc.getOutputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(((OutputDescriptor) value).toConfig())); + appDesc.getSystemDescriptors().forEach(sd -> systemStreamConfigs.putAll(((SystemDescriptor) sd).toConfig())); + appDesc.getDefaultSystemDescriptor().ifPresent(dsd -> + systemStreamConfigs.put(JobConfig.JOB_DEFAULT_SYSTEM(), ((SystemDescriptor) dsd).getSystemName())); + return systemStreamConfigs; + } + + private Map expandTableConfigs(Map originConfig, AppDescriptorImpl appDesc) { + List tableSpecs = new ArrayList<>(); + appDesc.getTableDescriptors().stream().map(td -> ((BaseTableDescriptor) td).getTableSpec()) + .forEach(spec -> tableSpecs.add((TableSpec) spec)); + return TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(originConfig), tableSpecs); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java new file mode 100644 index 0000000000..6c604b6ecc --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -0,0 +1,133 @@ +/* + * 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.execution; + +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.samza.SamzaException; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.coordinator.CoordinationUtils; +import org.apache.samza.coordinator.DistributedLockWithState; +import org.apache.samza.system.StreamSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Temporarily helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} + * for standalone Samza processors. + * + * TODO: we need to consolidate this with {@link ExecutionPlanner} after SAMZA-1811. + */ +public class LocalJobPlanner extends JobPlanner { + private static final Logger LOG = LoggerFactory.getLogger(LocalJobPlanner.class); + private static final String APPLICATION_RUNNER_PATH_SUFFIX = "/ApplicationRunnerData"; + + private final String uid = UUID.randomUUID().toString();; + + public LocalJobPlanner(AppDescriptorImpl descriptor) { + super(descriptor); + } + + @Override + List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + // for high-level DAG, generating the plan and job configs + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph()); + + String executionPlanJson = plan.getPlanAsJson(); + writePlanJsonFile(executionPlanJson); + LOG.info("Execution Plan: \n" + executionPlanJson); + String planId = String.valueOf(executionPlanJson.hashCode()); + + if (plan.getJobConfigs().isEmpty()) { + throw new SamzaException("No jobs in the plan."); + } + + // 2. create the necessary streams + // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 + // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig + // to be used for the whole application + JobConfig jobConfig = plan.getJobConfigs().get(0); + StreamManager streamManager = null; + try { + // create the StreamManager to create intermediate streams in the plan + streamManager = buildAndStartStreamManager(jobConfig); + createStreams(planId, plan.getIntermediateStreams(), streamManager); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + return plan.getJobConfigs(); + } + + /** + * Create intermediate streams using {@link StreamManager}. + * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader + * will create the streams. All the runner processes will wait on the latch that is released after the leader finishes + * stream creation. + * @param planId a unique identifier representing the plan used for coordination purpose + * @param intStreams list of intermediate {@link StreamSpec}s + * @param streamManager the {@link StreamManager} used to create streams + */ + private void createStreams(String planId, List intStreams, StreamManager streamManager) { + if (intStreams.isEmpty()) { + LOG.info("Set of intermediate streams is empty. Nothing to create."); + return; + } + LOG.info("A single processor must create the intermediate streams. Processor {} will attempt to acquire the lock.", uid); + // Move the scope of coordination utils within stream creation to address long idle connection problem. + // Refer SAMZA-1385 for more details + JobCoordinatorConfig jcConfig = new JobCoordinatorConfig(config); + String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_PATH_SUFFIX; + CoordinationUtils coordinationUtils = + jcConfig.getCoordinationUtilsFactory().getCoordinationUtils(coordinationId, uid, config); + if (coordinationUtils == null) { + LOG.warn("Processor {} failed to create utils. Each processor will attempt to create streams.", uid); + // each application process will try creating the streams, which + // requires stream creation to be idempotent + streamManager.createStreams(intStreams); + return; + } + + DistributedLockWithState lockWithState = coordinationUtils.getLockWithState(planId); + try { + // check if the processor needs to go through leader election and stream creation + if (lockWithState.lockIfNotSet(1000, TimeUnit.MILLISECONDS)) { + LOG.info("lock acquired for streams creation by " + uid); + streamManager.createStreams(intStreams); + lockWithState.unlockAndSet(); + } else { + LOG.info("Processor {} did not obtain the lock for streams creation. They must've been created by another processor.", uid); + } + } catch (TimeoutException e) { + String msg = String.format("Processor {} failed to get the lock for stream initialization", uid); + throw new SamzaException(msg, e); + } finally { + coordinationUtils.close(); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java new file mode 100644 index 0000000000..accf490108 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.execution; + +import java.util.List; +import java.util.UUID; +import org.apache.samza.SamzaException; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.coordinator.stream.CoordinatorStreamSystemConsumer; +import org.apache.samza.metrics.MetricsRegistryMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Temporary helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} + * for remote-launched Samza processors (e.g. in YARN). + * + * TODO: we need to consolidate this class with {@link ExecutionPlanner} after SAMZA-1811. + */ +public class RemoteJobPlanner extends JobPlanner { + private static final Logger LOG = LoggerFactory.getLogger(RemoteJobPlanner.class); + + public RemoteJobPlanner(AppDescriptorImpl descriptor) { + super(descriptor); + } + + @Override + List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + // for high-level DAG, generate the plan and job configs + // TODO: run.id needs to be set for standalone: SAMZA-1531 + // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision + String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); + LOG.info("The run id for this run is {}", runId); + + // 1. initialize and plan + ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId); + writePlanJsonFile(plan.getPlanAsJson()); + + if (plan.getJobConfigs().isEmpty()) { + throw new SamzaException("No jobs in the plan."); + } + + // 2. create the necessary streams + // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig + // to be used for the whole application + JobConfig jobConfig = plan.getJobConfigs().get(0); + StreamManager streamManager = null; + try { + // create the StreamManager to create intermediate streams in the plan + streamManager = buildAndStartStreamManager(jobConfig); + if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { + streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); + } + streamManager.createStreams(plan.getIntermediateStreams()); + } finally { + if (streamManager != null) { + streamManager.stop(); + } + } + return plan.getJobConfigs(); + } + + private Config getConfigFromPrevRun() { + CoordinatorStreamSystemConsumer consumer = new CoordinatorStreamSystemConsumer(config, new MetricsRegistryMap()); + consumer.register(); + consumer.start(); + consumer.bootstrap(); + consumer.stop(); + + Config cfg = consumer.getConfig(); + LOG.info("Previous config is: " + cfg.toString()); + return cfg; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java index 42e3c8a6a7..05e2ddcef0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java @@ -44,7 +44,7 @@ public class IntermediateMessageStreamImpl extends MessageStreamImpl imple private final OutputStreamImpl outputStream; private final boolean isKeyed; - public IntermediateMessageStreamImpl(StreamAppDescriptorImpl appDesc, InputOperatorSpec inputOperatorSpec, + public IntermediateMessageStreamImpl(StreamAppDescriptorImpl appDesc, InputOperatorSpec inputOperatorSpec, OutputStreamImpl outputStream) { super(appDesc, (OperatorSpec) inputOperatorSpec); this.outputStream = outputStream; 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 332696e419..82657891ab 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 @@ -28,7 +28,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.function.Function; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; import org.apache.samza.config.JobCoordinatorConfig; @@ -41,7 +40,6 @@ import org.apache.samza.coordinator.JobCoordinatorListener; import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.runtime.ProcessorContext; import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.task.TaskFactory; import org.apache.samza.util.ScalaJavaUtil; @@ -129,14 +127,6 @@ public String toString() { } } - /** - * Supplier interface to allow creation of {@link ProcessorLifecycleListener} for a {@link StreamProcessor} - */ - public interface StreamProcessorListenerSupplier extends Function { - - ProcessorLifecycleListener apply(StreamProcessor sp); - } - /** * @return the current state of StreamProcessor. */ @@ -163,10 +153,10 @@ public State getState() { * * Note: Lifecycle of the ExecutorService is fully managed by the StreamProcessor. * - * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer}. + * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer}. * @param customMetricsReporters metricReporter instances that will be used by SamzaContainer and JobCoordinator to report metrics. - * @param taskFactory The {@link TaskFactory} to be used for creating task instances. - * @param processorListener listener to the StreamProcessor life cycle. + * @param taskFactory the {@link TaskFactory} to be used for creating task instances. + * @param processorListener listener to the StreamProcessor life cycle. */ public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, ProcessorLifecycleListener processorListener) { @@ -176,10 +166,10 @@ public StreamProcessor(Config config, Map customMetrics /** * Same as {@link #StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener)}, except the * {@link JobCoordinator} is given for this {@link StreamProcessor}. - * @param config - config + * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer} * @param customMetricsReporters metric Reporter * @param taskFactory task factory to instantiate the Task - * @param processorListener listener to the StreamProcessor life cycle + * @param processorListener listener to the StreamProcessor life cycle * @param jobCoordinator the instance of {@link JobCoordinator} */ public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, @@ -188,40 +178,15 @@ public StreamProcessor(Config config, Map customMetrics this.taskFactory = taskFactory; this.config = config; this.taskShutdownMs = new TaskConfigJava(config).getShutdownMs(); - this.customMetricsReporter = customMetricsReporters; this.processorListener = processorListener; - this.jobCoordinator = (jobCoordinator != null) ? jobCoordinator : getJobCoordinator(); - this.jobCoordinatorListener = createJobCoordinatorListener(); - this.jobCoordinator.setListener(jobCoordinatorListener); - ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(CONTAINER_THREAD_NAME_FORMAT).setDaemon(true).build(); - this.executorService = Executors.newSingleThreadExecutor(threadFactory); - this.processorId = this.jobCoordinator.getProcessorId(); - } - - /** - * Same as {@link StreamProcessor#StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener, JobCoordinator)}, - * except user supplies a {@link StreamProcessorListenerSupplier} to create the listener for this {@link StreamProcessor}. - * @param config - config - * @param customMetricsReporters metric Reporter - * @param taskFactory task factory to instantiate the Task - * @param listenerSupplier a {@link Function} to create the {@link ProcessorLifecycleListener} for this {@link StreamProcessor} - * @param jobCoordinator the instance of {@link JobCoordinator} - */ - public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, - StreamProcessorListenerSupplier listenerSupplier, JobCoordinator jobCoordinator) { - Preconditions.checkNotNull(listenerSupplier, "listenerSupplier cannot be null."); - this.taskFactory = taskFactory; - this.config = config; - this.taskShutdownMs = new TaskConfigJava(config).getShutdownMs(); this.customMetricsReporter = customMetricsReporters; - this.jobCoordinator = (jobCoordinator != null) ? jobCoordinator : getJobCoordinator(); + this.jobCoordinator = (jobCoordinator != null) ? jobCoordinator : createJobCoordinator(); this.jobCoordinatorListener = createJobCoordinatorListener(); this.jobCoordinator.setListener(jobCoordinatorListener); ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(CONTAINER_THREAD_NAME_FORMAT).setDaemon(true).build(); this.executorService = Executors.newSingleThreadExecutor(threadFactory); + // TODO: remove the dependency on jobCoordinator for processorId after fixing SAMZA-1835 this.processorId = this.jobCoordinator.getProcessorId(); - // create and attach the listener to this StreamProcessor - this.processorListener = listenerSupplier.apply(this); } /** @@ -261,9 +226,9 @@ public void start() { *
    * If container is running, *

      - *
    1. container is shutdown cleanly and {@link SamzaContainerListener#onContainerStop()} will trigger + *
    2. container is shutdown cleanly and {@link SamzaContainerListener#afterStop()} will trigger * {@link JobCoordinator#stop()}
    3. - *
    4. container fails to shutdown cleanly and {@link SamzaContainerListener#onContainerFailed(Throwable)} will + *
    5. container fails to shutdown cleanly and {@link SamzaContainerListener#afterFailed(Throwable)} will * trigger {@link JobCoordinator#stop()}
    6. *
    * If container is not running, then this method will simply shutdown the {@link JobCoordinator}. @@ -291,33 +256,22 @@ public void stop() { } } - /** - * Get the {@link ProcessorContext} of this {@link StreamProcessor} - * @return the {@link ProcessorContext} object - */ - public ProcessorContext getProcessorContext() { - return new ProcessorContext() { }; - } - - /** - * Get the {@code config} of this {@link StreamProcessor} - * - * @return {@code config} object - */ - public Config getConfig() { - return config; - } - @VisibleForTesting JobCoordinator getCurrentJobCoordinator() { return jobCoordinator; } + /* package private for testing */ + @VisibleForTesting + SamzaContainer getContainer() { + return container; + } + SamzaContainer createSamzaContainer(String processorId, JobModel jobModel) { return SamzaContainer.apply(processorId, jobModel, config, ScalaJavaUtil.toScalaMap(customMetricsReporter), taskFactory); } - private JobCoordinator getJobCoordinator() { + private JobCoordinator createJobCoordinator() { String jobCoordinatorFactoryClassName = new JobCoordinatorConfig(config).getJobCoordinatorFactoryClassName(); return Util.getObj(jobCoordinatorFactoryClassName, JobCoordinatorFactory.class).getJobCoordinator(config); } @@ -414,11 +368,6 @@ public void onCoordinatorFailure(Throwable throwable) { }; } - /* package private for testing */ - SamzaContainer getContainer() { - return container; - } - class ContainerListener implements SamzaContainerListener { @Override @@ -427,7 +376,7 @@ public void beforeStart() { } @Override - public void onContainerStart() { + public void afterStart() { LOGGER.warn("Received container start notification for container: {} in stream processor: {}.", container, processorId); if (!processorOnStartCalled) { processorListener.afterStart(); @@ -437,7 +386,7 @@ public void onContainerStart() { } @Override - public void onContainerStop() { + public void afterStop() { containerShutdownLatch.countDown(); synchronized (lock) { if (state == State.IN_REBALANCE) { @@ -451,7 +400,7 @@ public void onContainerStop() { } @Override - public void onContainerFailed(Throwable t) { + public void afterFailed(Throwable t) { containerShutdownLatch.countDown(); synchronized (lock) { LOGGER.error(String.format("Container: %s failed with an exception. Stopping the stream processor: %s. Original exception:", container, processorId), containerException); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java deleted file mode 100644 index 46f10e22f2..0000000000 --- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java +++ /dev/null @@ -1,209 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.runtime; - -import java.io.File; -import java.io.PrintWriter; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; -import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.ApplicationBase; -import org.apache.samza.application.ApplicationDescriptors; -import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.application.TaskAppDescriptorImpl; -import org.apache.samza.config.ApplicationConfig; -import org.apache.samza.config.ApplicationConfig.ApplicationMode; -import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; -import org.apache.samza.config.ShellCommandConfig; -import org.apache.samza.config.StreamConfig; -import org.apache.samza.execution.ExecutionPlan; -import org.apache.samza.execution.ExecutionPlanner; -import org.apache.samza.execution.StreamManager; -import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.operators.BaseTableDescriptor; -import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.table.TableConfigGenerator; -import org.apache.samza.table.TableSpec; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * Defines common, core behavior for implementations of the {@link ApplicationRunner} API. - */ -public abstract class AbstractApplicationRunner implements ApplicationRunner { - private static final Logger log = LoggerFactory.getLogger(AbstractApplicationRunner.class); - - protected final AppDescriptorImpl appDesc; - protected final Config config; - protected final Map metricsReporters = new HashMap<>(); - - /** - * This is a temporary helper class to include all common logic to generate {@link JobConfig}s for high- and low-level - * applications in {@link LocalApplicationRunner} and {@link RemoteApplicationRunner}. - * - * TODO: Fix SAMZA-1811 to consolidate the planning into {@link ExecutionPlanner} - */ - static abstract class JobPlanner { - - protected final AppDescriptorImpl appDesc; - protected final Config config; - - JobPlanner(AppDescriptorImpl descriptor) { - this.appDesc = descriptor; - this.config = descriptor.getConfig(); - } - - abstract List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception; - - List prepareJobs() throws Exception { - String appId = new ApplicationConfig(appDesc.getConfig()).getGlobalAppId(); - return ApplicationDescriptors.forType( - taskAppDesc -> { - try { - return Collections.singletonList(JobPlanner.this.prepareTaskJob(taskAppDesc)); - } catch (Exception e) { - throw new SamzaException("Failed to generate JobConfig for TaskApplication " + appId, e); - } - }, - streamAppDesc -> { - try { - return JobPlanner.this.prepareStreamJobs(streamAppDesc); - } catch (Exception e) { - throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); - } - }, - appDesc); - } - - StreamManager buildAndStartStreamManager(Config config) { - StreamManager streamManager = new StreamManager(config); - streamManager.start(); - return streamManager; - } - - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) throws Exception { - return getExecutionPlan(specGraph, null); - } - - /* package private */ - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) throws Exception { - - // update application configs - Map cfg = new HashMap<>(config); - if (StringUtils.isNoneEmpty(runId)) { - cfg.put(ApplicationConfig.APP_RUN_ID, runId); - } - - StreamConfig streamConfig = new StreamConfig(config); - Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); - inputStreams.removeAll(specGraph.getOutputStreams().keySet()); - ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) - ? ApplicationMode.BATCH : ApplicationMode.STREAM; - cfg.put(ApplicationConfig.APP_MODE, mode.name()); - validateAppClassCfg(cfg, appDesc.getAppClass()); - - // merge user-provided configuration with input/output descriptor generated configuration - // descriptor generated configuration has higher priority - Map systemStreamConfigs = new HashMap<>(); - StreamAppDescriptorImpl streamAppDesc = (StreamAppDescriptorImpl) appDesc; - streamAppDesc.getInputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); - streamAppDesc.getOutputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); - streamAppDesc.getSystemDescriptors().forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); - streamAppDesc.getDefaultSystemDescriptor().ifPresent(dsd -> - systemStreamConfigs.put(JobConfig.JOB_DEFAULT_SYSTEM(), dsd.getSystemName())); - Map appConfigs = new HashMap<>(cfg); - appConfigs.putAll(systemStreamConfigs); - - // create the physical execution plan - Config generatedConfig = new MapConfig(cfg); - // creating the StreamManager to get all input/output streams' metadata for planning - StreamManager streamManager = buildAndStartStreamManager(generatedConfig); - try { - ExecutionPlanner planner = new ExecutionPlanner(generatedConfig, streamManager); - return planner.plan(specGraph); - } finally { - streamManager.stop(); - } - } - - /** - * Write the execution plan JSON to a file - * @param planJson JSON representation of the plan - */ - final void writePlanJsonFile(String planJson) { - try { - String content = "plan='" + planJson + "'"; - String planPath = System.getenv(ShellCommandConfig.EXECUTION_PLAN_DIR()); - if (planPath != null && !planPath.isEmpty()) { - // Write the plan json to plan path - File file = new File(planPath + "/plan.json"); - file.setReadable(true, false); - PrintWriter writer = new PrintWriter(file, "UTF-8"); - writer.println(content); - writer.close(); - } - } catch (Exception e) { - log.warn("Failed to write execution plan json to file", e); - } - } - - // helper method to generate a single node job configuration for low level task applications - private JobConfig prepareTaskJob(TaskAppDescriptorImpl taskAppDesc) { - Map cfg = new HashMap<>(config); - //TODO: add stream and system descriptor to configuration conversion here when SAMZA-1804 is fixed. - // adding table configuration - List tableSpecs = taskAppDesc.getTables().stream() - .map(td -> ((BaseTableDescriptor) td).getTableSpec()) - .collect(Collectors.toList()); - cfg.putAll(TableConfigGenerator.generateConfigsForTableSpecs(config, tableSpecs)); - validateAppClassCfg(cfg, taskAppDesc.getAppClass()); - return new JobConfig(new MapConfig(cfg)); - } - - private void validateAppClassCfg(Map cfg, Class appClass) { - if (StringUtils.isNotBlank(cfg.get(ApplicationConfig.APP_CLASS))) { - // app.class is already set - return; - } - // adding app.class in the configuration - cfg.put(ApplicationConfig.APP_CLASS, appClass.getCanonicalName()); - } - } - - AbstractApplicationRunner(AppDescriptorImpl appDesc) { - this.appDesc = appDesc; - this.config = appDesc.getConfig(); - } - - @Override - public final void addMetricsReporters(Map metricsReporters) { - this.metricsReporters.putAll(metricsReporters); - } - -} diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java index 665ab285df..f2fc3c0546 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java @@ -19,7 +19,7 @@ package org.apache.samza.runtime; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; @@ -29,22 +29,22 @@ /** - * Util class to create {@link ApplicationBase} from the configuration. + * Util class to create {@link SamzaApplication} from the configuration. */ public class ApplicationClassUtils { /** - * Creates the {@link ApplicationBase} object from the {@code config} + * Creates the {@link SamzaApplication} object from the {@code config} * * @param config the configuration of the application - * @return the {@link ApplicationBase} object + * @return the {@link SamzaApplication} object */ - public static ApplicationBase fromConfig(Config config) { + public static SamzaApplication fromConfig(Config config) { String appClassName = new ApplicationConfig(config).getAppClass(); if (StringUtils.isNotBlank(appClassName)) { // app.class is configured try { - Class appClass = (Class) Class.forName(appClassName); + Class appClass = (Class) Class.forName(appClassName); if (StreamApplication.class.isAssignableFrom(appClass) || TaskApplication.class.isAssignableFrom(appClass)) { return appClass.newInstance(); } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 3e14412a46..7e15c16cef 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -53,8 +53,8 @@ public static void main(String[] args) throws Exception { Config config = Util.rewriteConfig(orgConfig); ApplicationRunnerOperation op = cmdLine.getOperation(options); - ApplicationRunner - appRunner = ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); + ApplicationRunner appRunner = + ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); switch (op) { case RUN: diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index eb89f2769a..ceaff05076 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -21,32 +21,27 @@ import com.google.common.annotations.VisibleForTesting; import java.time.Duration; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; -import org.apache.samza.application.StreamAppDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.config.JobCoordinatorConfig; -import org.apache.samza.coordinator.CoordinationUtils; -import org.apache.samza.coordinator.DistributedLockWithState; -import org.apache.samza.execution.ExecutionPlan; -import org.apache.samza.execution.StreamManager; +import org.apache.samza.execution.LocalJobPlanner; import org.apache.samza.job.ApplicationStatus; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.processor.StreamProcessor; -import org.apache.samza.processor.StreamProcessor.StreamProcessorListenerSupplier; -import org.apache.samza.system.StreamSpec; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.slf4j.Logger; @@ -55,12 +50,11 @@ /** * This class implements the {@link ApplicationRunner} that runs the applications in standalone environment */ -public class LocalApplicationRunner extends AbstractApplicationRunner { +public class LocalApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(LocalApplicationRunner.class); - private static final String APPLICATION_RUNNER_PATH_SUFFIX = "/ApplicationRunnerData"; - private final String uid; + private final AppDescriptorImpl appDesc; private final LocalJobPlanner planner; private final Set processors = ConcurrentHashMap.newKeySet(); private final CountDownLatch shutdownLatch = new CountDownLatch(1); @@ -69,182 +63,15 @@ public class LocalApplicationRunner extends AbstractApplicationRunner { private ApplicationStatus appStatus = ApplicationStatus.New; - /** - * Defines a specific implementation of {@link ProcessorLifecycleListener} for local {@link StreamProcessor}s. - */ - private final class LocalStreamProcessorLifecycleListener implements ProcessorLifecycleListener { - private final StreamProcessor processor; - private final ProcessorLifecycleListener processorLifecycleListener; - - @Override - public void beforeStart() { - processorLifecycleListener.beforeStart(); - } - - @Override - public void afterStart() { - processorLifecycleListener.afterStart(); - if (numProcessorsToStart.decrementAndGet() == 0) { - appStatus = ApplicationStatus.Running; - } - } - - @Override - public void afterStop() { - processors.remove(processor); - - processorLifecycleListener.afterStop(); - if (processors.isEmpty()) { - // successful shutdown - shutdownAndNotify(); - } - } - - @Override - public void afterFailure(Throwable t) { - processors.remove(processor); - - processorLifecycleListener.afterFailure(t); - // the processor stopped with failure - if (failure.compareAndSet(null, t)) { - // shutdown the other processors - processors.forEach(StreamProcessor::stop); - } - - if (processors.isEmpty()) { - shutdownAndNotify(); - } - } - - LocalStreamProcessorLifecycleListener(StreamProcessor processor) { - this.processor = processor; - this.processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory(). - createInstance(processor.getProcessorContext(), processor.getConfig()); - } - - private void shutdownAndNotify() { - if (failure.get() != null) { - appStatus = ApplicationStatus.unsuccessfulFinish(failure.get()); - } else { - if (appStatus == ApplicationStatus.Running) { - appStatus = ApplicationStatus.SuccessfulFinish; - } else if (appStatus == ApplicationStatus.New) { - // the processor is shutdown before started - appStatus = ApplicationStatus.UnsuccessfulFinish; - } - } - - shutdownLatch.countDown(); - } - - } - - /** - * Defines a {@link JobPlanner} with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} - * for standalone Samza processors. - * - * TODO: we need to consolidate all planning logic into {@link org.apache.samza.execution.ExecutionPlanner} after SAMZA-1811. - */ - @VisibleForTesting - static class LocalJobPlanner extends JobPlanner { - private final String uid; - - LocalJobPlanner(AppDescriptorImpl descriptor, String uid) { - super(descriptor); - this.uid = uid; - } - - @Override - List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { - // for high-level DAG, generating the plan and job configs - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph()); - - String executionPlanJson = plan.getPlanAsJson(); - writePlanJsonFile(executionPlanJson); - LOG.info("Execution Plan: \n" + executionPlanJson); - String planId = String.valueOf(executionPlanJson.hashCode()); - - if (plan.getJobConfigs().isEmpty()) { - throw new SamzaException("No jobs in the plan."); - } - - // 2. create the necessary streams - // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 - // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig - // to be used for the whole application - JobConfig jobConfig = plan.getJobConfigs().get(0); - StreamManager streamManager = null; - try { - // create the StreamManager to create intermediate streams in the plan - streamManager = buildAndStartStreamManager(jobConfig); - createStreams(planId, plan.getIntermediateStreams(), streamManager); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - return plan.getJobConfigs(); - } - - /** - * Create intermediate streams using {@link org.apache.samza.execution.StreamManager}. - * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader - * will create the streams. All the runner processes will wait on the latch that is released after the leader finishes - * stream creation. - * @param planId a unique identifier representing the plan used for coordination purpose - * @param intStreams list of intermediate {@link StreamSpec}s - * @param streamManager the {@link StreamManager} used to create streams - */ - private void createStreams(String planId, List intStreams, StreamManager streamManager) { - if (intStreams.isEmpty()) { - LOG.info("Set of intermediate streams is empty. Nothing to create."); - return; - } - LOG.info("A single processor must create the intermediate streams. Processor {} will attempt to acquire the lock.", uid); - // Move the scope of coordination utils within stream creation to address long idle connection problem. - // Refer SAMZA-1385 for more details - JobCoordinatorConfig jcConfig = new JobCoordinatorConfig(config); - String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_PATH_SUFFIX; - CoordinationUtils coordinationUtils = - jcConfig.getCoordinationUtilsFactory().getCoordinationUtils(coordinationId, uid, config); - if (coordinationUtils == null) { - LOG.warn("Processor {} failed to create utils. Each processor will attempt to create streams.", uid); - // each application process will try creating the streams, which - // requires stream creation to be idempotent - streamManager.createStreams(intStreams); - return; - } - - DistributedLockWithState lockWithState = coordinationUtils.getLockWithState(planId); - try { - // check if the processor needs to go through leader election and stream creation - if (lockWithState.lockIfNotSet(1000, TimeUnit.MILLISECONDS)) { - LOG.info("lock acquired for streams creation by " + uid); - streamManager.createStreams(intStreams); - lockWithState.unlockAndSet(); - } else { - LOG.info("Processor {} did not obtain the lock for streams creation. They must've been created by another processor.", uid); - } - } catch (TimeoutException e) { - String msg = String.format("Processor {} failed to get the lock for stream initialization", uid); - throw new SamzaException(msg, e); - } finally { - coordinationUtils.close(); - } - } - } - /** * Default constructor that is required by any implementation of {@link ApplicationRunner} * * @param userApp user application * @param config user configuration */ - public LocalApplicationRunner(ApplicationBase userApp, Config config) { - super(ApplicationDescriptors.getAppDescriptor(userApp, config)); - this.uid = UUID.randomUUID().toString(); - this.planner = new LocalJobPlanner(appDesc, uid); + public LocalApplicationRunner(SamzaApplication userApp, Config config) { + this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); + this.planner = new LocalJobPlanner(appDesc); } /** @@ -253,8 +80,7 @@ public LocalApplicationRunner(ApplicationBase userApp, Config config) { */ @VisibleForTesting LocalApplicationRunner(AppDescriptorImpl appDesc, LocalJobPlanner planner) { - super(appDesc); - this.uid = UUID.randomUUID().toString(); + this.appDesc = appDesc; this.planner = planner; } @@ -262,18 +88,21 @@ public LocalApplicationRunner(ApplicationBase userApp, Config config) { public void run() { try { List jobConfigs = planner.prepareJobs(); - // 3. create the StreamProcessors + + // create the StreamProcessors if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs to run."); } jobConfigs.forEach(jobConfig -> { LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); - StreamProcessor processor = createStreamProcessor(jobConfig, appDesc, sp -> new LocalStreamProcessorLifecycleListener(sp)); + LocalStreamProcessorLifecycleListener localListener = new LocalStreamProcessorLifecycleListener(jobConfig); + StreamProcessor processor = createStreamProcessor(jobConfig, appDesc, localListener); + localListener.setProcessor(processor); processors.add(processor); }); numProcessorsToStart.set(processors.size()); - // 4. start the StreamProcessors + // start the StreamProcessors processors.forEach(StreamProcessor::start); } catch (Throwable throwable) { appStatus = ApplicationStatus.unsuccessfulFinish(throwable); @@ -321,26 +150,105 @@ public boolean waitForFinish(Duration timeout) { return finished; } - /** - * Create {@link StreamProcessor} based on config, {@link AppDescriptorImpl}, and {@link StreamProcessorListenerSupplier} - * @param config config - * @param appDesc {@link AppDescriptorImpl} - * @param listenerSupplier {@link StreamProcessorListenerSupplier} to create {@link ProcessorLifecycleListener} - * @return {@link StreamProcessor]} - */ /* package private */ - StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, StreamProcessorListenerSupplier listenerSupplier) { + StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, + LocalStreamProcessorLifecycleListener listener) { TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); - return new StreamProcessor(config, this.metricsReporters, taskFactory, listenerSupplier, null); - } - - /* package private for testing */ - Set getProcessors() { - return processors; + Map reporters = new HashMap<>(); + // TODO: the null processorId has to be fixed after SAMZA-1835 + ((Map) appDesc.getMetricsReporterFactories()) + .forEach((name, factory) -> reporters.put(name, factory.getMetricsReporter(name, null, config))); + return new StreamProcessor(config, reporters, taskFactory, listener, null); } @VisibleForTesting CountDownLatch getShutdownLatch() { return shutdownLatch; } + + /** + * Defines a specific implementation of {@link ProcessorLifecycleListener} for local {@link StreamProcessor}s. + */ + final class LocalStreamProcessorLifecycleListener implements ProcessorLifecycleListener { + private StreamProcessor processor; + private ProcessorLifecycleListener processorLifecycleListener; + + @Override + public void beforeStart() { + processorLifecycleListener.beforeStart(); + } + + @Override + public void afterStart() { + if (numProcessorsToStart.decrementAndGet() == 0) { + appStatus = ApplicationStatus.Running; + } + processorLifecycleListener.afterStart(); + } + + @Override + public void afterStop() { + processors.remove(processor); + processor = null; + + // successful shutdown + handleProcessorShutdown(null); + } + + @Override + public void afterFailure(Throwable t) { + processors.remove(processor); + processor = null; + + // the processor stopped with failure, this is logging the first processor's failure as the cause of + // the whole application failure + if (failure.compareAndSet(null, t)) { + // shutdown the other processors + processors.forEach(StreamProcessor::stop); + } + + // handle the current processor's shutdown failure. + handleProcessorShutdown(t); + } + + LocalStreamProcessorLifecycleListener(Config jobConfig) { + this.processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(new ProcessorContext() { + }, jobConfig); + } + + private void setProcessor(StreamProcessor processor) { + this.processor = processor; + } + + private void handleProcessorShutdown(Throwable error) { + if (processors.isEmpty()) { + // all processors are shutdown, setting the application final status + setApplicationFinalStatus(); + } + if (error != null) { + // current processor shutdown with a failure + processorLifecycleListener.afterFailure(error); + } else { + // current processor shutdown successfully + processorLifecycleListener.afterStop(); + } + if (processors.isEmpty()) { + // no processor is still running. Notify callers waiting on waitForFinish() + shutdownLatch.countDown(); + } + } + + private void setApplicationFinalStatus() { + if (failure.get() != null) { + appStatus = ApplicationStatus.unsuccessfulFinish(failure.get()); + } else { + if (appStatus == ApplicationStatus.Running) { + appStatus = ApplicationStatus.SuccessfulFinish; + } else if (appStatus == ApplicationStatus.New) { + // the processor is shutdown before started + appStatus = ApplicationStatus.UnsuccessfulFinish; + } + } + } + } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 33660c902e..d6afd99954 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -20,6 +20,7 @@ package org.apache.samza.runtime; import java.util.HashMap; +import java.util.Map; import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; @@ -34,6 +35,8 @@ import org.apache.samza.container.SamzaContainer$; import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.model.JobModel; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.SamzaUncaughtExceptionHandler; @@ -88,10 +91,10 @@ private static void run(AppDescriptorImpl appDesc, String containerId, JobModel containerId, jobModel, config, - ScalaJavaUtil.toScalaMap(new HashMap<>()), + ScalaJavaUtil.toScalaMap(loadMetricsReporters(appDesc, containerId, config)), taskFactory); - ProcessorLifecycleListener pListener = appDesc.getProcessorLifecycleListenerFactory() + ProcessorLifecycleListener listener = appDesc.getProcessorLifecycleListenerFactory() .createInstance(new ProcessorContext() { }, config); container.setContainerListener( @@ -99,26 +102,26 @@ private static void run(AppDescriptorImpl appDesc, String containerId, JobModel @Override public void beforeStart() { log.info("Before starting the container."); - pListener.beforeStart(); + listener.beforeStart(); } @Override - public void onContainerStart() { + public void afterStart() { log.info("Container Started"); - pListener.afterStart(); + listener.afterStart(); } @Override - public void onContainerStop() { + public void afterStop() { log.info("Container Stopped"); - pListener.afterStop(); + listener.afterStop(); } @Override - public void onContainerFailed(Throwable t) { + public void afterFailed(Throwable t) { log.info("Container Failed"); containerRunnerException = t; - pListener.afterFailure(t); + listener.afterFailure(t); } }); @@ -139,6 +142,15 @@ public void onContainerFailed(Throwable t) { } } + // TODO: this is going away when SAMZA-1168 is done and the initialization of metrics reporters are done via + // LocalApplicationRunner#createStreamProcessor() + private static Map loadMetricsReporters(AppDescriptorImpl appDesc, String containerId, Config config) { + Map reporters = new HashMap<>(); + ((Map) appDesc.getMetricsReporterFactories()) + .forEach((name, factory) -> reporters.put(name, factory.getMetricsReporter(name, containerId, config))); + return reporters; + } + /** * Creates a new container heartbeat monitor if possible. * @param container the container to monitor diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index ef90aa213a..2c061efbb8 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -21,21 +21,15 @@ import java.time.Duration; import java.util.List; -import java.util.UUID; import org.apache.samza.SamzaException; import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; -import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.coordinator.stream.CoordinatorStreamSystemConsumer; -import org.apache.samza.execution.ExecutionPlan; -import org.apache.samza.execution.StreamManager; +import org.apache.samza.execution.RemoteJobPlanner; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.job.JobRunner; -import org.apache.samza.metrics.MetricsRegistryMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,60 +39,13 @@ /** * This class implements the {@link ApplicationRunner} that runs the applications in a remote cluster */ -public class RemoteApplicationRunner extends AbstractApplicationRunner { +public class RemoteApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; - private final RemoteJobPlanner planner; - - /** - * Defines a {@link JobPlanner} with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} - * for remote-launched Samza processors (e.g. in YARN). - * - * TODO: we need to consolidate all planning logic into {@link org.apache.samza.execution.ExecutionPlanner} after SAMZA-1811. - */ - class RemoteJobPlanner extends JobPlanner { - - RemoteJobPlanner(AppDescriptorImpl descriptor) { - super(descriptor); - } - - @Override - List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { - // for high-level DAG, generate the plan and job configs - // TODO: run.id needs to be set for standalone: SAMZA-1531 - // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision - String runId = String.valueOf(System.currentTimeMillis()) + "-" + UUID.randomUUID().toString().substring(0, 8); - LOG.info("The run id for this run is {}", runId); - - // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId); - writePlanJsonFile(plan.getPlanAsJson()); - - if (plan.getJobConfigs().isEmpty()) { - throw new SamzaException("No jobs in the plan."); - } - // 2. create the necessary streams - // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig - // to be used for the whole application - JobConfig jobConfig = plan.getJobConfigs().get(0); - StreamManager streamManager = null; - try { - // create the StreamManager to create intermediate streams in the plan - streamManager = buildAndStartStreamManager(jobConfig); - if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) { - streamManager.clearStreamsFromPreviousRun(getConfigFromPrevRun()); - } - streamManager.createStreams(plan.getIntermediateStreams()); - } finally { - if (streamManager != null) { - streamManager.stop(); - } - } - return plan.getJobConfigs(); - } - } + private final AppDescriptorImpl appDesc; + private final RemoteJobPlanner planner; /** * Default constructor that is required by any implementation of {@link ApplicationRunner} @@ -106,8 +53,8 @@ List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws * @param userApp user application * @param config user configuration */ - RemoteApplicationRunner(ApplicationBase userApp, Config config) { - super(ApplicationDescriptors.getAppDescriptor(userApp, config)); + RemoteApplicationRunner(SamzaApplication userApp, Config config) { + this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); this.planner = new RemoteJobPlanner(appDesc); } @@ -135,7 +82,7 @@ public void kill() { // since currently we only support single actual remote job, we can get its status without // building the execution plan. try { - JobConfig jc = new JobConfig(config); + JobConfig jc = new JobConfig(appDesc.getConfig()); LOG.info("Killing job {}", jc.getName()); JobRunner runner = new JobRunner(jc); runner.kill(); @@ -149,7 +96,7 @@ public ApplicationStatus status() { // since currently we only support single actual remote job, we can get its status without // building the execution plan try { - JobConfig jc = new JobConfig(config); + JobConfig jc = new JobConfig(appDesc.getConfig()); return getApplicationStatus(jc); } catch (Throwable t) { throw new SamzaException("Failed to get status for application", t); @@ -163,7 +110,7 @@ public void waitForFinish() { @Override public boolean waitForFinish(Duration timeout) { - JobConfig jobConfig = new JobConfig(config); + JobConfig jobConfig = new JobConfig(appDesc.getConfig()); boolean finished = true; long timeoutInMs = timeout.toMillis(); long startTimeInMs = System.currentTimeMillis(); @@ -203,16 +150,4 @@ public boolean waitForFinish(Duration timeout) { LOG.debug("Status is {} for job {}", new Object[]{status, jobConfig.getName()}); return status; } - - private Config getConfigFromPrevRun() { - CoordinatorStreamSystemConsumer consumer = new CoordinatorStreamSystemConsumer(config, new MetricsRegistryMap()); - consumer.register(); - consumer.start(); - consumer.bootstrap(); - consumer.stop(); - - Config cfg = consumer.getConfig(); - LOG.info("Previous config is: " + cfg.toString()); - return cfg; - } } diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index f7d03bc026..32fa9d1339 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -60,7 +60,7 @@ public static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { * @param config the {@link Config} for this job * @return a {@link TaskFactory} object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} */ - public static TaskFactory createTaskFactory(Config config) { + public static TaskFactory getTaskFactoryFromConfig(Config config) { // if there is configuration to set the job w/ a specific type of task, instantiate the corresponding task factory String taskClassName = new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( () -> { diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index b3372c6fca..a7d2c03ad0 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -819,7 +819,7 @@ class SamzaContainer( info("Entering run loop.") status = SamzaContainerStatus.STARTED if (containerListener != null) { - containerListener.onContainerStart() + containerListener.afterStart() } metrics.containerStartupTime.update(System.nanoTime() - startTime) runLoop.run @@ -870,11 +870,11 @@ class SamzaContainer( status match { case SamzaContainerStatus.STOPPED => if (containerListener != null) { - containerListener.onContainerStop() + containerListener.afterStop() } case SamzaContainerStatus.FAILED => if (containerListener != null) { - containerListener.onContainerFailed(exceptionSeen) + containerListener.afterFailed(exceptionSeen) } } } @@ -886,8 +886,8 @@ class SamzaContainer( *
    * Implementation: Stops the [[RunLoop]], which will eventually transition the container from * [[SamzaContainerStatus.STARTED]] to either [[SamzaContainerStatus.STOPPED]] or [[SamzaContainerStatus.FAILED]]]. - * Based on the final `status`, [[SamzaContainerListener#onContainerStop()]] or - * [[SamzaContainerListener#onContainerFailed(Throwable]] will be invoked respectively. + * Based on the final `status`, [[SamzaContainerListener#afterStop()]] or + * [[SamzaContainerListener#afterFailed(Throwable]] will be invoked respectively. * * @throws SamzaException, Thrown when the container has already been stopped or failed */ diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index dee1defbca..d97684971d 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,19 +19,16 @@ package org.apache.samza.job.local -import java.util.concurrent.CountDownLatch - import org.apache.samza.application.ApplicationDescriptors import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ import org.apache.samza.container.{SamzaContainer, SamzaContainerListener, TaskName} -import org.apache.samza.coordinator.{JobCoordinator, JobModelManager} +import org.apache.samza.coordinator.JobModelManager import org.apache.samza.coordinator.stream.CoordinatorStreamManager import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} -import org.apache.samza.processor.StreamProcessor -import org.apache.samza.runtime.{ApplicationClassUtils, ProcessorContext, ProcessorLifecycleListener} +import org.apache.samza.runtime.{ApplicationClassUtils, ProcessorContext} import org.apache.samza.storage.ChangelogStreamManager import org.apache.samza.task.TaskFactory import org.apache.samza.task.TaskFactoryUtil @@ -86,19 +83,19 @@ class ThreadJobFactory extends StreamJobFactory with Logging { case _ => None } - val pListener = { + val listener = { val userListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(new ProcessorContext() { }, config) new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { + override def afterFailed(t: Throwable): Unit = { userListener.afterFailure(t) throw t; } - override def onContainerStart(): Unit = { + override def afterStart(): Unit = { userListener.afterStart() } - override def onContainerStop(): Unit = { + override def afterStop(): Unit = { userListener.afterStop() } @@ -117,7 +114,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging { config, Map[String, MetricsReporter](), taskFactory) - container.setContainerListener(pListener) + container.setContainerListener(listener) val threadJob = new ThreadJob(container) threadJob diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java b/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java similarity index 93% rename from samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java rename to samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java index 8beb76f0db..3bab1acf15 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamApplication.java +++ b/samza-core/src/test/java/org/apache/samza/application/MockStreamApplication.java @@ -21,7 +21,7 @@ /** * Test class of {@link StreamApplication} for unit tests */ -public class TestStreamApplication implements StreamApplication { +public class MockStreamApplication implements StreamApplication { @Override public void describe(StreamAppDescriptor appSpec) { diff --git a/samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java b/samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java deleted file mode 100644 index bb145a0e23..0000000000 --- a/samza-core/src/test/java/org/apache/samza/application/MockTaskApplication.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.application; - -/** - * Test class of {@link TaskApplication} for unit tests - */ -public class MockTaskApplication implements TaskApplication { - @Override - public void describe(TaskAppDescriptor appSpec) { - - } -} \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java index bffc0296b8..cb1f0d6a0d 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java @@ -250,7 +250,7 @@ public void testMultipleSystemDescriptorForSameSystemName() { }, mock(Config.class)); new StreamAppDescriptorImpl(appDesc -> { - appDesc.setDefaultSystem(sd2); + appDesc.withDefaultSystem(sd2); try { appDesc.getInputStream(isd1); fail("Adding input stream with the same system name as the default system but different SystemDescriptor should have failed"); @@ -316,7 +316,7 @@ public void testSetDefaultSystemDescriptorAfterGettingInputStream() { new StreamAppDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); - appDesc.setDefaultSystem(sd); // should throw exception + appDesc.withDefaultSystem(sd); // should throw exception }, mock(Config.class)); } @@ -327,7 +327,7 @@ public void testSetDefaultSystemDescriptorAfterGettingOutputStream() { GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mock(Serde.class)); new StreamAppDescriptorImpl(appDesc -> { appDesc.getOutputStream(osd); - appDesc.setDefaultSystem(sd); // should throw exception + appDesc.withDefaultSystem(sd); // should throw exception }, mock(Config.class)); } @@ -336,7 +336,7 @@ public void testSetDefaultSystemDescriptorAfterGettingIntermediateStream() { String streamId = "test-stream-1"; StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false); - streamAppDesc.setDefaultSystem(mock(SystemDescriptor.class)); // should throw exception + streamAppDesc.withDefaultSystem(mock(SystemDescriptor.class)); // should throw exception } @Test(expected = IllegalStateException.class) @@ -398,7 +398,7 @@ public void testGetIntermediateStreamWithDefaultSystemDescriptor() { StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); GenericSystemDescriptor sd = new GenericSystemDescriptor("mock-system", "mock-system-factory"); - streamAppDesc.setDefaultSystem(sd); + streamAppDesc.withDefaultSystem(sd); IntermediateMessageStreamImpl intermediateStreamImpl = streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false); diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java index 33bec07c5c..cd30d26889 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java @@ -19,16 +19,24 @@ package org.apache.samza.application; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.TableDescriptor; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; import org.apache.samza.task.TaskFactory; +import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** @@ -37,43 +45,73 @@ public class TestTaskAppDescriptorImpl { private Config config = mock(Config.class); + private String defaultSystemName = "test-system"; + private SystemDescriptor defaultSystemDescriptor = mock(SystemDescriptor.class); + private List mockInputs = new ArrayList() { { + InputDescriptor mock1 = mock(InputDescriptor.class); + InputDescriptor mock2 = mock(InputDescriptor.class); + when(mock1.getStreamId()).thenReturn("test-input1"); + when(mock2.getStreamId()).thenReturn("test-input2"); + this.add(mock1); + this.add(mock2); + } }; + private List mockOutputs = new ArrayList() { { + OutputDescriptor mock1 = mock(OutputDescriptor.class); + OutputDescriptor mock2 = mock(OutputDescriptor.class); + when(mock1.getStreamId()).thenReturn("test-output1"); + when(mock2.getStreamId()).thenReturn("test-output2"); + this.add(mock1); + this.add(mock2); + } }; + private Set mockTables = new HashSet() { { + TableDescriptor mock1 = mock(TableDescriptor.class); + TableDescriptor mock2 = mock(TableDescriptor.class); + when(mock1.getTableId()).thenReturn("test-table1"); + when(mock2.getTableId()).thenReturn("test-table2"); + this.add(mock1); + this.add(mock2); + } }; + + @Before + public void setUp() { + when(defaultSystemDescriptor.getSystemName()).thenReturn(defaultSystemName); + mockInputs.forEach(isd -> when(isd.getSystemDescriptor()).thenReturn(defaultSystemDescriptor)); + mockOutputs.forEach(osd -> when(osd.getSystemDescriptor()).thenReturn(defaultSystemDescriptor)); + } @Test public void testConstructor() { TaskApplication mockApp = mock(TaskApplication.class); TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(mockApp, config); - verify(mockApp, times(1)).describe(appDesc); + verify(mockApp).describe(appDesc); assertEquals(config, appDesc.config); } @Test public void testAddInputStreams() { - List testInputs = new ArrayList() { { this.add("myinput1"); this.add("myinput2"); } }; TaskApplication testApp = appDesc -> { - testInputs.forEach(input -> appDesc.addInputStream(input)); + mockInputs.forEach(appDesc::addInputStream); }; TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); - assertEquals(appDesc.getInputStreams(), testInputs); + assertEquals(mockInputs.toArray(), appDesc.getInputDescriptors().values().toArray()); } @Test public void testAddOutputStreams() { - List testOutputs = new ArrayList() { { this.add("myoutput1"); this.add("myoutput2"); } }; TaskApplication testApp = appDesc -> { - testOutputs.forEach(output -> appDesc.addOutputStream(output)); + mockOutputs.forEach(appDesc::addOutputStream); }; TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); - assertEquals(appDesc.getOutputStreams(), testOutputs); + assertEquals(mockOutputs.toArray(), appDesc.getOutputDescriptors().values().toArray()); } @Test public void testAddTables() { - List testTables = new ArrayList() { { this.add(mock(TableDescriptor.class)); } }; TaskApplication testApp = appDesc -> { - testTables.forEach(table -> appDesc.addTable(table)); + mockTables.forEach(appDesc::addTable); }; TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); - assertEquals(appDesc.getTables(), testTables); + assertEquals(mockTables, appDesc.getTableDescriptors()); } @Test diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java new file mode 100644 index 0000000000..24faedded5 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java @@ -0,0 +1,210 @@ +/* + * 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.execution; + +import com.google.common.collect.ImmutableList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.coordinator.CoordinationUtils; +import org.apache.samza.coordinator.CoordinationUtilsFactory; +import org.apache.samza.coordinator.DistributedLockWithState; +import org.apache.samza.system.StreamSpec; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +/** + * Unit tests for {@link LocalJobPlanner} + * + * TODO: consolidate this with unit tests for ExecutionPlanner after SAMZA-1811 + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({LocalJobPlanner.class, JobCoordinatorConfig.class}) +public class TestLocalJobPlanner { + + private static final String PLAN_JSON = + "{" + "\"jobs\":[{" + "\"jobName\":\"test-application\"," + "\"jobId\":\"1\"," + "\"operatorGraph\":{" + + "\"intermediateStreams\":{%s}," + "\"applicationName\":\"test-application\",\"applicationId\":\"1\"}"; + private static final String STREAM_SPEC_JSON_FORMAT = + "\"%s\":{" + "\"streamSpec\":{" + "\"id\":\"%s\"," + "\"systemName\":\"%s\"," + "\"physicalName\":\"%s\"," + + "\"partitionCount\":2}," + "\"sourceJobs\":[\"test-app\"]," + "\"targetJobs\":[\"test-target-app\"]},"; + + private LocalJobPlanner localPlanner; + + @Test + public void testStreamCreation() + throws Exception { + localPlanner = createLocalJobPlanner(mock(StreamAppDescriptorImpl.class)); + StreamManager streamManager = mock(StreamManager.class); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); + + ExecutionPlan plan = mock(ExecutionPlan.class); + when(plan.getIntermediateStreams()).thenReturn( + Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); + when(plan.getPlanAsJson()).thenReturn(""); + when(plan.getJobConfigs()).thenReturn(Collections.singletonList(mock(JobConfig.class))); + doReturn(plan).when(localPlanner).getExecutionPlan(any()); + + CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); + JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class); + when(mockJcConfig.getCoordinationUtilsFactory()).thenReturn(coordinationUtilsFactory); + PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig); + + localPlanner.prepareJobs(); + + ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); + verify(streamManager).createStreams(captor.capture()); + List streamSpecs = captor.getValue(); + assertEquals(streamSpecs.size(), 1); + assertEquals(streamSpecs.get(0).getId(), "test-stream"); + verify(streamManager).stop(); + } + + @Test + public void testStreamCreationWithCoordination() + throws Exception { + localPlanner = createLocalJobPlanner(mock(StreamAppDescriptorImpl.class)); + StreamManager streamManager = mock(StreamManager.class); + doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); + + ExecutionPlan plan = mock(ExecutionPlan.class); + when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); + when(plan.getPlanAsJson()).thenReturn(""); + when(plan.getJobConfigs()).thenReturn(Collections.singletonList(mock(JobConfig.class))); + doReturn(plan).when(localPlanner).getExecutionPlan(any()); + + CoordinationUtils coordinationUtils = mock(CoordinationUtils.class); + CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); + JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class); + when(mockJcConfig.getCoordinationUtilsFactory()).thenReturn(coordinationUtilsFactory); + PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig); + + DistributedLockWithState lock = mock(DistributedLockWithState.class); + when(lock.lockIfNotSet(anyLong(), anyObject())).thenReturn(true); + when(coordinationUtils.getLockWithState(anyString())).thenReturn(lock); + when(coordinationUtilsFactory.getCoordinationUtils(anyString(), anyString(), anyObject())) + .thenReturn(coordinationUtils); + + localPlanner.prepareJobs(); + + ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); + verify(streamManager).createStreams(captor.capture()); + + List streamSpecs = captor.getValue(); + assertEquals(streamSpecs.size(), 1); + assertEquals(streamSpecs.get(0).getId(), "test-stream"); + verify(streamManager).stop(); + } + + /** + * A test case to verify if the plan results in different hash if there is change in topological sort order. + * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases. + */ + @Test + public void testPlanIdWithShuffledStreamSpecs() { + List streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), + new StreamSpec("test-stream-2", "stream-2", "testStream"), + new StreamSpec("test-stream-3", "stream-3", "testStream")); + String planIdBeforeShuffle = getExecutionPlanId(streamSpecs); + + List shuffledStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-2", "stream-2", "testStream"), + new StreamSpec("test-stream-1", "stream-1", "testStream"), + new StreamSpec("test-stream-3", "stream-3", "testStream")); + + + assertFalse("Expected both of the latch ids to be different", + planIdBeforeShuffle.equals(getExecutionPlanId(shuffledStreamSpecs))); + } + + /** + * A test case to verify if the plan results in same hash in case of same plan. + * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases. + */ + @Test + public void testGeneratePlanIdWithSameStreamSpecs() { + List streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), + new StreamSpec("test-stream-2", "stream-2", "testStream"), + new StreamSpec("test-stream-3", "stream-3", "testStream")); + String planIdForFirstAttempt = getExecutionPlanId(streamSpecs); + String planIdForSecondAttempt = getExecutionPlanId(streamSpecs); + + assertEquals("Expected latch ids to match!", "1447946713", planIdForFirstAttempt); + assertEquals("Expected latch ids to match for the second attempt!", planIdForFirstAttempt, planIdForSecondAttempt); + } + + /** + * A test case to verify plan results in different hash in case of different intermediate stream. + * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases. + */ + @Test + public void testGeneratePlanIdWithDifferentStreamSpecs() { + List streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), + new StreamSpec("test-stream-2", "stream-2", "testStream"), + new StreamSpec("test-stream-3", "stream-3", "testStream")); + String planIdBeforeShuffle = getExecutionPlanId(streamSpecs); + + List updatedStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), + new StreamSpec("test-stream-4", "stream-4", "testStream"), + new StreamSpec("test-stream-3", "stream-3", "testStream")); + + + assertFalse("Expected both of the latch ids to be different", + planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs))); + } + + private LocalJobPlanner createLocalJobPlanner(AppDescriptorImpl appDesc) { + return spy(new LocalJobPlanner(appDesc)); + } + + private String getExecutionPlanId(List updatedStreamSpecs) { + String intermediateStreamJson = + updatedStreamSpecs.stream().map(this::streamSpecToJson).collect(Collectors.joining(",")); + + int planId = String.format(PLAN_JSON, intermediateStreamJson).hashCode(); + + return String.valueOf(planId); + } + + private String streamSpecToJson(StreamSpec streamSpec) { + return String.format(STREAM_SPEC_JSON_FORMAT, streamSpec.getId(), streamSpec.getId(), streamSpec.getSystemName(), + streamSpec.getPhysicalName()); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java new file mode 100644 index 0000000000..5b96c4ee97 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.execution; + +import java.util.Collections; +import java.util.List; +import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.system.StreamSpec; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +/** + * Unit tests for {@link RemoteJobPlanner} + * + * TODO: consolidate this with unit tests for ExecutionPlanner after SAMZA-1811 + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(RemoteJobPlanner.class) +public class TestRemoteJobPlanner { + + private RemoteJobPlanner remotePlanner; + + @Test + public void testStreamCreation() + throws Exception { + remotePlanner = createRemoteJobPlanner(mock(StreamAppDescriptorImpl.class)); + StreamManager streamManager = mock(StreamManager.class); + doReturn(streamManager).when(remotePlanner).buildAndStartStreamManager(any(Config.class)); + + ExecutionPlan plan = mock(ExecutionPlan.class); + when(plan.getIntermediateStreams()).thenReturn( + Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); + when(plan.getPlanAsJson()).thenReturn(""); + when(plan.getJobConfigs()).thenReturn(Collections.singletonList(mock(JobConfig.class))); + ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); + when(mockAppConfig.getAppMode()).thenReturn(ApplicationConfig.ApplicationMode.STREAM); + when(plan.getApplicationConfig()).thenReturn(mockAppConfig); + doReturn(plan).when(remotePlanner).getExecutionPlan(any(), any()); + + remotePlanner.prepareJobs(); + + verify(streamManager, times(0)).clearStreamsFromPreviousRun(any()); + ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); + verify(streamManager).createStreams(captor.capture()); + List streamSpecs = captor.getValue(); + assertEquals(streamSpecs.size(), 1); + assertEquals(streamSpecs.get(0).getId(), "test-stream"); + verify(streamManager).stop(); + } + + private RemoteJobPlanner createRemoteJobPlanner(AppDescriptorImpl appDesc) { + return spy(new RemoteJobPlanner(appDesc)); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java index 03cd26b658..868fbb794d 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java @@ -20,11 +20,11 @@ import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; +import org.apache.samza.application.TaskAppDescriptor; import org.apache.samza.application.TaskAppDescriptorImpl; import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TestStreamApplication; -import org.apache.samza.application.MockTaskApplication; +import org.apache.samza.application.MockStreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; @@ -44,20 +44,20 @@ public class TestApplicationClassUtils { @Test public void testStreamAppClass() { Map configMap = new HashMap<>(); - configMap.put(ApplicationConfig.APP_CLASS, TestStreamApplication.class.getName()); - ApplicationBase app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); - assertTrue(app instanceof TestStreamApplication); + configMap.put(ApplicationConfig.APP_CLASS, MockStreamApplication.class.getName()); + SamzaApplication app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + assertTrue(app instanceof MockStreamApplication); configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); - assertTrue(app instanceof TestStreamApplication); + assertTrue(app instanceof MockStreamApplication); } @Test public void testTaskAppClass() { Map configMap = new HashMap<>(); configMap.put(ApplicationConfig.APP_CLASS, MockTaskApplication.class.getName()); - ApplicationBase app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + SamzaApplication app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockTaskApplication); configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); @@ -70,7 +70,7 @@ public void testTaskClassOnly() { Map configMap = new HashMap<>(); configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); Config config = new MapConfig(configMap); - ApplicationBase app = ApplicationClassUtils.fromConfig(config); + SamzaApplication app = ApplicationClassUtils.fromConfig(config); assertTrue(app instanceof TaskApplication); TaskAppDescriptorImpl appSpec = new TaskAppDescriptorImpl((TaskApplication) app, config); assertTrue(appSpec.getTaskFactory().createInstance() instanceof TestStreamTask); @@ -81,4 +81,14 @@ public void testNoAppClassNoTaskClass() { Map configMap = new HashMap<>(); ApplicationClassUtils.fromConfig(new MapConfig(configMap)); } + + /** + * Test class of {@link TaskApplication} for unit tests + */ + public static class MockTaskApplication implements TaskApplication { + @Override + public void describe(TaskAppDescriptor appSpec) { + + } + } } \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index 2adbc682b9..a27cfc7b6c 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -19,10 +19,9 @@ package org.apache.samza.runtime; import java.time.Duration; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; -import org.apache.samza.application.StreamAppDescriptor; -import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.MockStreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.job.ApplicationStatus; @@ -41,8 +40,8 @@ public void TestRunOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationConfig.APP_CLASS + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", - "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts" + "-config", String.format("%s=%s", ApplicationConfig.APP_CLASS, MockStreamApplication.class.getName()), + "-config", String.format("app.runner.class=%s", TestApplicationRunnerInvocationCounts.class.getName()), }); assertEquals(1, TestApplicationRunnerInvocationCounts.runCount); @@ -56,8 +55,8 @@ public void TestKillOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationConfig.APP_CLASS + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", - "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts", + "-config", String.format("%s=%s", ApplicationConfig.APP_CLASS, MockStreamApplication.class.getName()), + "-config", String.format("app.runner.class=%s", TestApplicationRunnerInvocationCounts.class.getName()), "--operation=kill" }); @@ -72,21 +71,20 @@ public void TestStatusOperation() throws Exception { "org.apache.samza.config.factories.PropertiesConfigFactory", "--config-path", getClass().getResource("/test.properties").getPath(), - "-config", ApplicationConfig.APP_CLASS + "=org.apache.samza.runtime.TestApplicationRunnerMain$TestStreamApplicationDummy", - "-config", "app.runner.class=org.apache.samza.runtime.TestApplicationRunnerMain$TestApplicationRunnerInvocationCounts", + "-config", String.format("%s=%s", ApplicationConfig.APP_CLASS, MockStreamApplication.class.getName()), + "-config", String.format("app.runner.class=%s", TestApplicationRunnerInvocationCounts.class.getName()), "--operation=status" }); assertEquals(1, TestApplicationRunnerInvocationCounts.statusCount); } - public static class TestApplicationRunnerInvocationCounts extends AbstractApplicationRunner { + public static class TestApplicationRunnerInvocationCounts implements ApplicationRunner { protected static int runCount = 0; protected static int killCount = 0; protected static int statusCount = 0; - public TestApplicationRunnerInvocationCounts(ApplicationBase userApp, Config config) { - super(ApplicationDescriptors.getAppDescriptor(userApp, config)); + public TestApplicationRunnerInvocationCounts(SamzaApplication userApp, Config config) { } @Override @@ -116,12 +114,4 @@ public boolean waitForFinish(Duration timeout) { } } - - public static class TestStreamApplicationDummy implements StreamApplication { - - @Override - public void describe(StreamAppDescriptor appDesc) { - - } - } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 54b9634b76..6a61b5ec1c 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -19,71 +19,41 @@ package org.apache.samza.runtime; -import com.google.common.collect.ImmutableList; - import java.time.Duration; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; -import org.apache.samza.coordinator.CoordinationUtils; -import org.apache.samza.coordinator.CoordinationUtilsFactory; -import org.apache.samza.coordinator.DistributedLockWithState; -import org.apache.samza.execution.ExecutionPlan; -import org.apache.samza.execution.StreamManager; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.processor.StreamProcessor; -import org.apache.samza.runtime.LocalApplicationRunner.LocalJobPlanner; -import org.apache.samza.system.StreamSpec; +import org.apache.samza.execution.LocalJobPlanner; import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyObject; -import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -@RunWith(PowerMockRunner.class) -@PrepareForTest(LocalApplicationRunner.class) public class TestLocalApplicationRunner { - private static final String PLAN_JSON = - "{" + "\"jobs\":[{" + "\"jobName\":\"test-application\"," + "\"jobId\":\"1\"," + "\"operatorGraph\":{" - + "\"intermediateStreams\":{%s}," + "\"applicationName\":\"test-application\",\"applicationId\":\"1\"}"; - private static final String STREAM_SPEC_JSON_FORMAT = - "\"%s\":{" + "\"streamSpec\":{" + "\"id\":\"%s\"," + "\"systemName\":\"%s\"," + "\"physicalName\":\"%s\"," - + "\"partitionCount\":2}," + "\"sourceJobs\":[\"test-app\"]," + "\"targetJobs\":[\"test-target-app\"]},"; - private Config config; - private ApplicationBase mockApp; + private SamzaApplication mockApp; private LocalApplicationRunner runner; private LocalJobPlanner localPlanner; @@ -94,78 +64,6 @@ public void setUp() { prepareTest(); } - @Test - public void testStreamCreation() - throws Exception { - StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); - - ExecutionPlan plan = mock(ExecutionPlan.class); - when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); - when(plan.getPlanAsJson()).thenReturn(""); - when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(config))); - doReturn(plan).when(localPlanner).getExecutionPlan(any()); - - CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); - JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class); - when(mockJcConfig.getCoordinationUtilsFactory()).thenReturn(coordinationUtilsFactory); - PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig); - - try { - runner.run(); - runner.waitForFinish(); - } catch (Throwable t) { - assertNotNull(t); //no jobs exception - } - - ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); - verify(streamManager).createStreams(captor.capture()); - List streamSpecs = captor.getValue(); - assertEquals(streamSpecs.size(), 1); - assertEquals(streamSpecs.get(0).getId(), "test-stream"); - verify(streamManager).stop(); - } - - @Test - public void testStreamCreationWithCoordination() - throws Exception { - StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); - - ExecutionPlan plan = mock(ExecutionPlan.class); - when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"))); - when(plan.getPlanAsJson()).thenReturn(""); - when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(config))); - doReturn(plan).when(localPlanner).getExecutionPlan(any()); - - CoordinationUtils coordinationUtils = mock(CoordinationUtils.class); - CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class); - JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class); - when(mockJcConfig.getCoordinationUtilsFactory()).thenReturn(coordinationUtilsFactory); - PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig); - - DistributedLockWithState lock = mock(DistributedLockWithState.class); - when(lock.lockIfNotSet(anyLong(), anyObject())).thenReturn(true); - when(coordinationUtils.getLockWithState(anyString())).thenReturn(lock); - when(coordinationUtilsFactory.getCoordinationUtils(anyString(), anyString(), anyObject())) - .thenReturn(coordinationUtils); - - try { - runner.run(); - runner.waitForFinish(); - } catch (Throwable t) { - assertNotNull(t); //no jobs exception - } - - ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); - verify(streamManager).createStreams(captor.capture()); - - List streamSpecs = captor.getValue(); - assertEquals(streamSpecs.size(), 1); - assertEquals(streamSpecs.get(0).getId(), "test-stream"); - verify(streamManager).stop(); - } - @Test public void testRunStreamTask() throws Exception { @@ -178,12 +76,12 @@ public void testRunStreamTask() StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(LocalApplicationRunner.LocalStreamProcessorLifecycleListener.class); doAnswer(i -> { - ProcessorLifecycleListener listener = captor.getValue().apply(sp); + ProcessorLifecycleListener listener = captor.getValue(); listener.afterStart(); listener.afterStop(); return null; @@ -209,22 +107,16 @@ public void testRunComplete() }; prepareTest(); - // buildAndStartStreamManager already includes start, so not going to verify it gets called - StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); - ExecutionPlan plan = mock(ExecutionPlan.class); - when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList()); - when(plan.getPlanAsJson()).thenReturn(""); - when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))); - doReturn(plan).when(localPlanner).getExecutionPlan(any()); + // return the jobConfigs from the planner + doReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))).when(localPlanner).prepareJobs(); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(LocalApplicationRunner.LocalStreamProcessorLifecycleListener.class); doAnswer(i -> { - ProcessorLifecycleListener listener = captor.getValue().apply(sp); + ProcessorLifecycleListener listener = captor.getValue(); listener.afterStart(); listener.afterStop(); return null; @@ -236,7 +128,6 @@ public void testRunComplete() runner.waitForFinish(); assertEquals(runner.status(), ApplicationStatus.SuccessfulFinish); - verify(streamManager).stop(); } @Test @@ -251,18 +142,12 @@ public void testRunFailure() }; prepareTest(); - // buildAndStartStreamManager already includes start, so not going to verify it gets called - StreamManager streamManager = mock(StreamManager.class); - doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); - ExecutionPlan plan = mock(ExecutionPlan.class); - when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList()); - when(plan.getPlanAsJson()).thenReturn(""); - when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))); - doReturn(plan).when(localPlanner).getExecutionPlan(any()); + // return the jobConfigs from the planner + doReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))).when(localPlanner).prepareJobs(); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerSupplier.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(LocalApplicationRunner.LocalStreamProcessorLifecycleListener.class); doAnswer(i -> { @@ -279,67 +164,6 @@ public void testRunFailure() } assertEquals(runner.status(), ApplicationStatus.UnsuccessfulFinish); - verify(streamManager).stop(); - } - - public static Set getProcessors(LocalApplicationRunner runner) { - return runner.getProcessors(); - } - - /** - * A test case to verify if the plan results in different hash if there is change in topological sort order. - * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases. - */ - @Test - public void testPlanIdWithShuffledStreamSpecs() { - List streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), - new StreamSpec("test-stream-2", "stream-2", "testStream"), - new StreamSpec("test-stream-3", "stream-3", "testStream")); - String planIdBeforeShuffle = getExecutionPlanId(streamSpecs); - - List shuffledStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-2", "stream-2", "testStream"), - new StreamSpec("test-stream-1", "stream-1", "testStream"), - new StreamSpec("test-stream-3", "stream-3", "testStream")); - - - assertFalse("Expected both of the latch ids to be different", - planIdBeforeShuffle.equals(getExecutionPlanId(shuffledStreamSpecs))); - } - - /** - * A test case to verify if the plan results in same hash in case of same plan. - * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases. - */ - @Test - public void testGeneratePlanIdWithSameStreamSpecs() { - List streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), - new StreamSpec("test-stream-2", "stream-2", "testStream"), - new StreamSpec("test-stream-3", "stream-3", "testStream")); - String planIdForFirstAttempt = getExecutionPlanId(streamSpecs); - String planIdForSecondAttempt = getExecutionPlanId(streamSpecs); - - assertEquals("Expected latch ids to match!", "1447946713", planIdForFirstAttempt); - assertEquals("Expected latch ids to match for the second attempt!", planIdForFirstAttempt, planIdForSecondAttempt); - } - - /** - * A test case to verify plan results in different hash in case of different intermediate stream. - * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases. - */ - @Test - public void testGeneratePlanIdWithDifferentStreamSpecs() { - List streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), - new StreamSpec("test-stream-2", "stream-2", "testStream"), - new StreamSpec("test-stream-3", "stream-3", "testStream")); - String planIdBeforeShuffle = getExecutionPlanId(streamSpecs); - - List updatedStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"), - new StreamSpec("test-stream-4", "stream-4", "testStream"), - new StreamSpec("test-stream-3", "stream-3", "testStream")); - - - assertFalse("Expected both of the latch ids to be different", - planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs))); } @Test @@ -360,21 +184,8 @@ public void testWaitForFinishTimesout() { private void prepareTest() { AppDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(mockApp, config); - localPlanner = spy(new LocalApplicationRunner.LocalJobPlanner(appDesc, "test-planner")); + localPlanner = spy(new LocalJobPlanner(appDesc)); runner = spy(new LocalApplicationRunner(appDesc, localPlanner)); } - private String getExecutionPlanId(List updatedStreamSpecs) { - String intermediateStreamJson = - updatedStreamSpecs.stream().map(this::streamSpecToJson).collect(Collectors.joining(",")); - - int planId = String.format(PLAN_JSON, intermediateStreamJson).hashCode(); - - return String.valueOf(planId); - } - - private String streamSpecToJson(StreamSpec streamSpec) { - return String.format(STREAM_SPEC_JSON_FORMAT, streamSpec.getId(), streamSpec.getId(), streamSpec.getSystemName(), - streamSpec.getPhysicalName()); - } } diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 68caf602a8..fa0ee06dd1 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -51,7 +51,7 @@ public void testStreamTaskClass() { this.put("task.class", TestStreamTask.class.getName()); } }); - Object retFactory = TaskFactoryUtil.createTaskFactory(config); + Object retFactory = TaskFactoryUtil.getTaskFactoryFromConfig(config); assertTrue(retFactory instanceof StreamTaskFactory); assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof TestStreamTask); @@ -61,7 +61,7 @@ public void testStreamTaskClass() { } }); try { - TaskFactoryUtil.createTaskFactory(config); + TaskFactoryUtil.getTaskFactoryFromConfig(config); fail("Should have failed w/ no.such.class"); } catch (ConfigException cfe) { // expected @@ -75,7 +75,7 @@ public void testAsyncStreamTask() { this.put("task.class", TestAsyncStreamTask.class.getName()); } }); - Object retFactory = TaskFactoryUtil.createTaskFactory(config); + Object retFactory = TaskFactoryUtil.getTaskFactoryFromConfig(config); assertTrue(retFactory instanceof AsyncStreamTaskFactory); assertTrue(((AsyncStreamTaskFactory) retFactory).createInstance() instanceof TestAsyncStreamTask); @@ -85,7 +85,7 @@ public void testAsyncStreamTask() { } }); try { - TaskFactoryUtil.createTaskFactory(config); + TaskFactoryUtil.getTaskFactoryFromConfig(config); fail("Should have failed w/ no.such.class"); } catch (ConfigException cfe) { // expected diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala index 09f410f9da..c586766122 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala @@ -199,16 +199,16 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { + override def afterFailed(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } - override def onContainerStop(): Unit = { + override def afterStop(): Unit = { onContainerStopCalled = true } - override def onContainerStart(): Unit = { + override def afterStart(): Unit = { onContainerStartCalled = true } @@ -287,16 +287,16 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { producerMultiplexer = producerMultiplexer, metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { + override def afterFailed(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } - override def onContainerStop(): Unit = { + override def afterStop(): Unit = { onContainerStopCalled = true } - override def onContainerStart(): Unit = { + override def afterStart(): Unit = { onContainerStartCalled = true } @@ -379,16 +379,16 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { producerMultiplexer = producerMultiplexer, metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { + override def afterFailed(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } - override def onContainerStop(): Unit = { + override def afterStop(): Unit = { onContainerStopCalled = true } - override def onContainerStart(): Unit = { + override def afterStart(): Unit = { onContainerStartCalled = true } @@ -471,16 +471,16 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { producerMultiplexer = producerMultiplexer, metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { + override def afterFailed(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } - override def onContainerStop(): Unit = { + override def afterStop(): Unit = { onContainerStopCalled = true } - override def onContainerStart(): Unit = { + override def afterStart(): Unit = { onContainerStartCalled = true } @@ -559,16 +559,16 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def onContainerFailed(t: Throwable): Unit = { + override def afterFailed(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } - override def onContainerStop(): Unit = { + override def afterStop(): Unit = { onContainerStopCalled = true } - override def onContainerStart(): Unit = { + override def afterStart(): Unit = { onContainerStartCalled = true } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java index af67b4b22a..027fd23169 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java @@ -130,8 +130,4 @@ public boolean waitForFinish(Duration timeout) { return runner.waitForFinish(timeout); } - @Override - public void addMetricsReporters(Map metricsReporters) { - runner.addMetricsReporters(metricsReporters); - } } 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 58c84dd979..b304eaacb7 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 @@ -68,7 +68,7 @@ public void describe(StreamAppDescriptor appDesc) { trackingSystem.getOutputDescriptor("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class))); - appDesc.setDefaultSystem(trackingSystem); + appDesc.withDefaultSystem(trackingSystem); MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor); OutputStream> pageViewEventPerMember = appDesc.getOutputStream(outputStreamDescriptor); 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 4c7b8043b7..3d2bfade38 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 @@ -64,7 +64,7 @@ public void describe(StreamAppDescriptor appDesc) { trackingSystem.getOutputDescriptor("pageViewEventPerMember", KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class))); - appDesc.setDefaultSystem(trackingSystem); + appDesc.withDefaultSystem(trackingSystem); MessageStream pageViewEvents = appDesc.getInputStream(inputStreamDescriptor); OutputStream> pageViewEventPerMember = appDesc.getOutputStream(outputStreamDescriptor); 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 44df62acef..b81c5b1574 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 @@ -24,8 +24,14 @@ import org.apache.samza.operators.TableDescriptor; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunners; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.NoOpSerde; +import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.RocksDbTableDescriptor; import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.kafka.KafkaInputDescriptor; +import org.apache.samza.system.kafka.KafkaOutputDescriptor; +import org.apache.samza.system.kafka.KafkaSystemDescriptor; import org.apache.samza.task.MessageCollector; import org.apache.samza.task.StreamTask; import org.apache.samza.task.StreamTaskFactory; @@ -43,7 +49,7 @@ public class MyStreamTask implements StreamTask { @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { - + // processing logic here } } @@ -58,9 +64,13 @@ public static void main(String[] args) { @Override public void describe(TaskAppDescriptor appDesc) { // add input and output streams - appDesc.addInputStream("myinput"); - appDesc.addOutputStream("myoutput"); + 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); // create the task factory based on configuration appDesc.setTaskFactory((StreamTaskFactory) () -> new MyStreamTask()); diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index 5f2466a050..477d5b890e 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -31,7 +31,7 @@ import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.samza.SamzaException; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; @@ -290,7 +290,7 @@ public void run(Duration timeout) { Preconditions.checkState((app == null && taskClass != null) || (app != null && taskClass == null), "TestRunner should run for Low Level Task api or High Level Application Api"); Preconditions.checkState(!timeout.isZero() || !timeout.isNegative(), "Timeouts should be positive"); - ApplicationBase testApp = app == null ? (TaskApplication) appDesc -> appDesc.setTaskFactory(createTaskFactory()) : app; + SamzaApplication testApp = app == null ? (TaskApplication) appDesc -> appDesc.setTaskFactory(createTaskFactory()) : app; final LocalApplicationRunner runner = new LocalApplicationRunner(testApp, new MapConfig(configs)); runner.run(); boolean timedOut = !runner.waitForFinish(timeout); diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index 214446308f..43bd62ed20 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -20,7 +20,7 @@ package org.apache.samza.test.integration; import joptsimple.OptionSet; -import org.apache.samza.application.ApplicationBase; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.runtime.ApplicationClassUtils; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; @@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory; /** - * {@link ApplicationRunnerMain} was designed for deploying {@link org.apache.samza.application.ApplicationBase} in yarn + * {@link ApplicationRunnerMain} was designed for deploying {@link SamzaApplication} in yarn * and doesn't work for in standalone. * * This runner class is built for standalone failure tests and not recommended for general use. @@ -46,7 +46,7 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - ApplicationBase app = ApplicationClassUtils.fromConfig(config); + SamzaApplication app = ApplicationClassUtils.fromConfig(config); ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config); try { 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 b9e43d109b..4379361d1a 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 @@ -41,7 +41,7 @@ /** * Test class to create an {@link StreamApplication} instance */ -public class TestStreamApplication implements StreamApplication, Serializable { +public class TestStreamApplication implements StreamApplication { private final String systemName; private final String inputTopic; @@ -65,41 +65,51 @@ public void describe(StreamAppDescriptor streamAppDesc) { KafkaOutputDescriptor osd = ksd.getOutputDescriptor(outputTopic, new StringSerde()); MessageStream inputStream = streamAppDesc.getInputStream(isd); OutputStream outputStream = streamAppDesc.getOutputStream(osd); - inputStream.map(new MapFunction() { - transient CountDownLatch latch1; - transient CountDownLatch latch2; - transient StreamApplicationCallback callback; - - @Override - public String apply(String message) { - TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message); - if (callback != null) { - callback.onMessage(incomingMessage); - } - if (latch1 != null) { - latch1.countDown(); - } - if (latch2 != null) { - latch2.countDown(); - } - return incomingMessage.toString(); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - SharedContextFactories.SharedContextFactory contextFactory = - SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName); - this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch"); - this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch"); - this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback"); - } - }).sendTo(outputStream); + inputStream.map(new TestMapFunction(appName, processorName)).sendTo(outputStream); } public interface StreamApplicationCallback { void onMessage(TestKafkaEvent m); } + public static class TestMapFunction implements MapFunction { + private final String appName; + private final String processorName; + + private transient CountDownLatch latch1; + private transient CountDownLatch latch2; + private transient StreamApplicationCallback callback; + + TestMapFunction(String appName, String processorName) { + this.appName = appName; + this.processorName = processorName; + } + + @Override + public String apply(String message) { + TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message); + if (callback != null) { + callback.onMessage(incomingMessage); + } + if (latch1 != null) { + latch1.countDown(); + } + if (latch2 != null) { + latch2.countDown(); + } + return incomingMessage.toString(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + SharedContextFactories.SharedContextFactory contextFactory = + SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName); + this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch"); + this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch"); + this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback"); + } + } + public static class TestKafkaEvent implements Serializable { // Actual content of the event. From f4b3d43a478ab9300d4b3e15257288323ff62348 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Thu, 30 Aug 2018 23:05:11 -0700 Subject: [PATCH 19/38] SAMZA-1789: Fxing TaskApplication examples and some checkstyle errors --- .../application/ApplicationDescriptor.java | 3 +- .../application/StreamAppDescriptor.java | 1 - .../samza/application/StreamApplication.java | 6 ++-- .../samza/application/TaskApplication.java | 30 +++++++++++-------- .../samza/runtime/ApplicationRunner.java | 11 ++++--- .../samza/runtime/LocalApplicationRunner.java | 16 ++++++---- .../TestTaskAppDescriptorImpl.java | 8 ++++- .../runtime/TestApplicationClassUtils.java | 10 +++---- ...estStreamTask.java => MockStreamTask.java} | 2 +- .../samza/task/TestTaskFactoryUtil.java | 4 +-- .../example/AppWithGlobalConfigExample.java | 3 +- .../samza/example/TaskApplicationExample.java | 1 + .../WatermarkIntegrationTest.java | 26 ++++++++++++---- 13 files changed, 76 insertions(+), 45 deletions(-) rename samza-core/src/test/java/org/apache/samza/task/{TestStreamTask.java => MockStreamTask.java} (95%) diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index 90f9ebf4cc..e878b2e65e 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -21,7 +21,6 @@ import java.util.Map; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; -import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; @@ -90,7 +89,7 @@ public interface ApplicationDescriptor { S withDefaultSystem(SystemDescriptor defaultSystemDescriptor); /** - * Sets a set of customized {@link MetricsReporter}s in the application + * Sets a set of customized {@link MetricsReporterFactory}s in the application * * @param reporterFactories the map of customized {@link MetricsReporterFactory} objects to be used * @return type {@code S} of {@link ApplicationDescriptor} with {@code reporterFactories} diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java index ff0c305232..a9018b1b83 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java @@ -25,7 +25,6 @@ import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; -import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.table.Table; 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 c8796de7d8..f0be7b5157 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 @@ -26,7 +26,7 @@ *

    * The following example removes page views older than 1 hour from the input stream: *

    {@code
    - * public class PageViewCounter implements StreamApplication {
    + * public class PageViewFilter implements StreamApplication {
      *   public void describe(StreamAppDescriptor appDesc) {
      *     KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking");
      *     KafkaInputDescriptor inputStreamDescriptor =
    @@ -50,8 +50,8 @@
      *   public static void main(String[] args) {
      *     CommandLine cmdLine = new CommandLine();
      *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
    - *     PageViewCounter app = new PageViewCounter();
    - *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PageViewCounter(), config);
    + *     PageViewFilter app = new PageViewFilter();
    + *     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config);
      *     runner.run();
      *     runner.waitForFinish();
      *   }
    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 bea291e82e..e5f484fd1b 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
    @@ -22,16 +22,22 @@
     
     
     /**
    - * Describes and initializes the transforms for processing message streams and generating results in low-level API.
    - * 

    - * This is a marker interface that users will implement for a low-level application. + * Describes and initializes the transforms for processing message streams and generating results in low-level API. Your + * application is expected to implement this interface. *

    * The following example removes page views older than 1 hour from the input stream: *

    {@code
    - * public class PageViewCounter implements TaskApplication {
    + * public class PageViewFilter implements TaskApplication {
      *   public void describe(TaskAppDescriptor appDesc) {
    - *     appDesc.addInputStream(PageViewTask.TASK_INPUT);
    - *     appDesc.addOutputStream(PageViewTask.TASK_OUTPUT);
    + *     KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor(PageViewTask.SYSTEM);
    + *     KafkaInputDescriptor inputStreamDescriptor =
    + *         trackingSystem.getInputDescriptor(PageViewTask.TASK_INPUT, new JsonSerdeV2<>(PageViewEvent.class));
    + *
    + *     KafkaOutputDescriptor> outputStreamDescriptor =
    + *         trackingSystem.getOutputDescriptor(PageViewTask.TASK_OUTPUT, new JsonSerdeV2<>(PageViewEvent.class)));
    + *
    + *     appDesc.addInputStream(inputStreamDescriptor);
    + *     appDesc.addOutputStream(outputStreamDescriptor);
      *     appDesc.setTaskFactory((StreamTaskFactory) () -> new PageViewTask());
      *   }
      * }
    @@ -39,17 +45,16 @@
      * public class PageViewTask implements StreamTask {
      *   final static String TASK_INPUT = "pageViewEvents";
      *   final static String TASK_OUTPUT = "recentPageViewEvents";
    - *   final static String OUTPUT_SYSTEM = "kafka";
    + *   final static String SYSTEM = "kafka";
      *
      *   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(OUTPUT_SYSTEM, TASK_OUTPUT),
    + *       collector.send(new OutgoingMessageEnvelope(new SystemStream(SYSTEM, TASK_OUTPUT),
      *           message.getKey(), message.getKey(), m));
      *     }
      *   }
    - * }
      * }
    *

    * The example above can be run using an ApplicationRunner: @@ -57,8 +62,8 @@ * public static void main(String[] args) { * CommandLine cmdLine = new CommandLine(); * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); - * PageViewCounter app = new PageViewCounter(); - * ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new PageViewCounter(), config); + * PageViewFilter app = new PageViewFilter(); + * ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config); * runner.run(); * runner.waitForFinish(); * } @@ -68,8 +73,7 @@ * 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 TaskAppDescriptor} * instance will be created and described by the user-defined {@link TaskApplication} when planning the execution. - * {@link org.apache.samza.task.TaskFactory} and descriptors for data entities used in the task (e.g. - * {@link org.apache.samza.operators.TableDescriptor}) are required to be serializable. + * {@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 diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index 7c1ad7f648..de4cb60b2b 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -20,33 +20,32 @@ import java.time.Duration; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.SamzaApplication; import org.apache.samza.job.ApplicationStatus; /** - * The primary means of managing execution of the {@link SamzaApplication} at runtime. + * The primary means of managing execution of the {@link org.apache.samza.application.SamzaApplication} at runtime. */ @InterfaceStability.Evolving public interface ApplicationRunner { /** - * Deploy and run the Samza jobs to execute {@link SamzaApplication}. + * Deploy and run the Samza jobs to execute {@link org.apache.samza.application.SamzaApplication}. * It is non-blocking so it doesn't wait for the application running. */ void run(); /** - * Kill the Samza jobs represented by {@link SamzaApplication} + * Kill the Samza jobs represented by {@link org.apache.samza.application.SamzaApplication} * It is non-blocking so it doesn't wait for the application stopping. */ void kill(); /** - * Get the collective status of the Samza jobs represented by {@link SamzaApplication}. + * Get the collective status of the Samza jobs represented by {@link org.apache.samza.application.SamzaApplication}. * Returns {@link ApplicationStatus} object. * - * @return the current status of an instance of {@link SamzaApplication} + * @return the current status of an instance of {@link org.apache.samza.application.SamzaApplication} */ ApplicationStatus status(); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index ceaff05076..54e1966837 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import java.time.Duration; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -150,6 +151,16 @@ public boolean waitForFinish(Duration timeout) { return finished; } + @VisibleForTesting + protected Set getProcessors() { + return Collections.unmodifiableSet(processors); + } + + @VisibleForTesting + CountDownLatch getShutdownLatch() { + return shutdownLatch; + } + /* package private */ StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, LocalStreamProcessorLifecycleListener listener) { @@ -161,11 +172,6 @@ StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, return new StreamProcessor(config, reporters, taskFactory, listener, null); } - @VisibleForTesting - CountDownLatch getShutdownLatch() { - return shutdownLatch; - } - /** * Defines a specific implementation of {@link ProcessorLifecycleListener} for local {@link StreamProcessor}s. */ diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java index cd30d26889..207cfee060 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java @@ -22,18 +22,25 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.samza.SamzaException; import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; +import org.apache.samza.serializers.SerializableSerde; +import org.apache.samza.task.MockStreamTask; +import org.apache.samza.task.StreamTask; +import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.task.TaskFactory; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -141,5 +148,4 @@ public void testProcessorLifecycleListener() { TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); } - } \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java index 868fbb794d..fbe33a6c0d 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java @@ -30,7 +30,7 @@ import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; -import org.apache.samza.task.TestStreamTask; +import org.apache.samza.task.MockStreamTask; import org.junit.Test; import static org.junit.Assert.assertTrue; @@ -48,7 +48,7 @@ public void testStreamAppClass() { SamzaApplication app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockStreamApplication); - configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); + configMap.put(TaskConfig.TASK_CLASS(), MockStreamTask.class.getName()); app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockStreamApplication); } @@ -60,7 +60,7 @@ public void testTaskAppClass() { SamzaApplication app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockTaskApplication); - configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); + configMap.put(TaskConfig.TASK_CLASS(), MockStreamTask.class.getName()); app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockTaskApplication); } @@ -68,12 +68,12 @@ public void testTaskAppClass() { @Test public void testTaskClassOnly() { Map configMap = new HashMap<>(); - configMap.put(TaskConfig.TASK_CLASS(), TestStreamTask.class.getName()); + configMap.put(TaskConfig.TASK_CLASS(), MockStreamTask.class.getName()); Config config = new MapConfig(configMap); SamzaApplication app = ApplicationClassUtils.fromConfig(config); assertTrue(app instanceof TaskApplication); TaskAppDescriptorImpl appSpec = new TaskAppDescriptorImpl((TaskApplication) app, config); - assertTrue(appSpec.getTaskFactory().createInstance() instanceof TestStreamTask); + assertTrue(appSpec.getTaskFactory().createInstance() instanceof MockStreamTask); } @Test(expected = ConfigException.class) diff --git a/samza-core/src/test/java/org/apache/samza/task/TestStreamTask.java b/samza-core/src/test/java/org/apache/samza/task/MockStreamTask.java similarity index 95% rename from samza-core/src/test/java/org/apache/samza/task/TestStreamTask.java rename to samza-core/src/test/java/org/apache/samza/task/MockStreamTask.java index 1ad0398a4c..d089c4be5f 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestStreamTask.java +++ b/samza-core/src/test/java/org/apache/samza/task/MockStreamTask.java @@ -23,7 +23,7 @@ /** * Test implementation class for {@link StreamTask} */ -public class TestStreamTask implements StreamTask { +public class MockStreamTask implements StreamTask { @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index fa0ee06dd1..927bbaf925 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -48,12 +48,12 @@ public class TestTaskFactoryUtil { public void testStreamTaskClass() { Config config = new MapConfig(new HashMap() { { - this.put("task.class", TestStreamTask.class.getName()); + this.put("task.class", MockStreamTask.class.getName()); } }); Object retFactory = TaskFactoryUtil.getTaskFactoryFromConfig(config); assertTrue(retFactory instanceof StreamTaskFactory); - assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof TestStreamTask); + assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof MockStreamTask); config = new MapConfig(new HashMap() { { 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 822546f265..fbea9dfdef 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 @@ -49,7 +49,6 @@ public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); Config config = cmdLine.loadConfig(cmdLine.parser().parse(args)); ApplicationRunner runner = ApplicationRunners.getApplicationRunner(new AppWithGlobalConfigExample(), config); - runner.addMetricsReporters(new HashMap<>()); runner.run(); runner.waitForFinish(); @@ -73,6 +72,8 @@ public void describe(StreamAppDescriptor appDesc) { .setAccumulationMode(AccumulationMode.DISCARDING), "window1") .map(m -> KV.of(m.getKey().getKey(), new PageViewCount(m))) .sendTo(appDesc.getOutputStream(outputStreamDescriptor)); + + appDesc.withMetricsReporterFactories(new HashMap<>()); } class PageViewEvent { 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 b81c5b1574..b12ca327c0 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 @@ -18,6 +18,7 @@ */ package org.apache.samza.example; +import java.io.Serializable; import org.apache.samza.application.TaskAppDescriptor; import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; 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 994e12cdfd..ddea2f1941 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 @@ -19,6 +19,7 @@ package org.apache.samza.test.controlmessages; +import org.apache.samza.application.SamzaApplication; import scala.collection.JavaConverters; import java.lang.reflect.Field; @@ -53,7 +54,6 @@ import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunners; import org.apache.samza.runtime.LocalApplicationRunner; -import org.apache.samza.runtime.TestLocalApplicationRunner; import org.apache.samza.serializers.IntegerSerdeFactory; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; @@ -126,7 +126,7 @@ private static IncomingMessageEnvelope createIncomingMessage(Object message, Sys @Test public void testWatermark() throws Exception { Map configs = new HashMap<>(); - configs.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner"); + configs.put("app.runner.class", MockLocalApplicationRunner.class.getName()); configs.put("systems.test.samza.factory", TestSystemFactory.class.getName()); configs.put("streams.PageView.samza.system", "test"); configs.put("streams.PageView.partitionCount", String.valueOf(PARTITION_COUNT)); @@ -170,7 +170,7 @@ public void describe(StreamAppDescriptor appDesc) { runner.run(); // processors are only available when the app is running - Map tasks = getTaskOperationGraphs((LocalApplicationRunner) runner); + Map tasks = getTaskOperationGraphs((MockLocalApplicationRunner) runner); runner.waitForFinish(); // wait for the completion to ensure that all tasks are actually initialized and the OperatorImplGraph is initialized @@ -193,8 +193,8 @@ public void describe(StreamAppDescriptor appDesc) { assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3); } - Map getTaskOperationGraphs(LocalApplicationRunner runner) throws Exception { - StreamProcessor processor = TestLocalApplicationRunner.getProcessors(runner).iterator().next(); + Map getTaskOperationGraphs(MockLocalApplicationRunner runner) throws Exception { + StreamProcessor processor = runner.getProcessors().iterator().next(); SamzaContainer container = TestStreamProcessorUtil.getContainer(processor); Map taskInstances = JavaConverters.mapAsJavaMapConverter(container.getTaskInstances()).asJava(); Map tasks = new HashMap<>(); @@ -222,4 +222,20 @@ OperatorImpl getOperator(OperatorImplGraph graph, OperatorSpec.OpCode opCode) { } return null; } + + class MockLocalApplicationRunner extends LocalApplicationRunner { + + /** + * Default constructor that is required by any implementation of {@link ApplicationRunner} + * @param userApp user application + * @param config user configuration + */ + public MockLocalApplicationRunner(SamzaApplication userApp, Config config) { + super(userApp, config); + } + + protected Set getProcessors() { + return super.getProcessors(); + } + } } From 9997b98bbbc051746eb01524ef4dfdd8da3596d9 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Thu, 30 Aug 2018 23:48:02 -0700 Subject: [PATCH 20/38] SAMZA-1789: renamed all ApplicationDescriptor classes with full-spelling of Application --- .../application/ApplicationDescriptor.java | 6 +- .../samza/application/StreamApplication.java | 2 +- ....java => StreamApplicationDescriptor.java} | 2 +- .../samza/application/TaskApplication.java | 6 +- ...or.java => TaskApplicationDescriptor.java} | 4 +- .../apache/samza/operators/MessageStream.java | 5 +- .../operators/functions/ClosableFunction.java | 4 +- .../operators/functions/InitableFunction.java | 3 +- .../operators/functions/StreamExpander.java | 16 ++--- ...pl.java => ApplicationDescriptorImpl.java} | 8 ++- .../application/ApplicationDescriptors.java | 24 +++---- .../application/LegacyTaskApplication.java | 2 +- ...a => StreamApplicationDescriptorImpl.java} | 10 +-- ...ava => TaskApplicationDescriptorImpl.java} | 10 +-- .../apache/samza/execution/JobPlanner.java | 30 +++++---- .../samza/execution/LocalJobPlanner.java | 11 ++-- .../samza/execution/RemoteJobPlanner.java | 11 ++-- .../samza/operators/MessageStreamImpl.java | 10 +-- .../samza/operators/OperatorSpecGraph.java | 16 ++--- .../samza/operators/spec/OperatorSpec.java | 3 +- .../stream/IntermediateMessageStreamImpl.java | 4 +- .../samza/runtime/LocalApplicationRunner.java | 14 ++-- .../samza/runtime/LocalContainerRunner.java | 16 +++-- .../runtime/RemoteApplicationRunner.java | 5 +- .../apache/samza/task/StreamOperatorTask.java | 8 ++- .../apache/samza/task/TaskFactoryUtil.java | 17 ++--- .../application/MockStreamApplication.java | 2 +- ... TestStreamApplicationDescriptorImpl.java} | 66 +++++++++---------- ...=> TestTaskApplicationDescriptorImpl.java} | 24 +++---- .../samza/execution/TestExecutionPlanner.java | 36 +++++----- .../execution/TestJobGraphJsonGenerator.java | 6 +- .../apache/samza/execution/TestJobNode.java | 6 +- .../samza/execution/TestLocalJobPlanner.java | 11 ++-- .../samza/execution/TestRemoteJobPlanner.java | 9 +-- .../samza/operators/TestJoinOperator.java | 32 ++++----- .../operators/TestMessageStreamImpl.java | 30 ++++----- .../operators/TestOperatorSpecGraph.java | 6 +- .../operators/impl/TestOperatorImplGraph.java | 18 ++--- .../operators/impl/TestWindowOperator.java | 18 ++--- .../spec/TestPartitionByOperatorSpec.java | 16 ++--- .../runtime/TestApplicationClassUtils.java | 8 +-- .../runtime/TestLocalApplicationRunner.java | 11 ++-- .../samza/task/TestTaskFactoryUtil.java | 18 ++--- .../samza/sql/runner/SamzaSqlApplication.java | 4 +- .../samza/sql/translator/QueryTranslator.java | 8 +-- .../samza/sql/translator/ScanTranslator.java | 4 +- .../sql/translator/TranslatorContext.java | 10 +-- .../sql/translator/TestFilterTranslator.java | 4 +- .../sql/translator/TestJoinTranslator.java | 4 +- .../sql/translator/TestProjectTranslator.java | 6 +- .../sql/translator/TestQueryTranslator.java | 46 ++++++------- .../example/AppWithGlobalConfigExample.java | 4 +- .../samza/example/BroadcastExample.java | 4 +- .../samza/example/KeyValueStoreExample.java | 4 +- .../apache/samza/example/MergeExample.java | 4 +- .../example/OrderShipmentJoinExample.java | 4 +- .../samza/example/PageViewCounterExample.java | 4 +- .../samza/example/RepartitionExample.java | 4 +- .../samza/example/TaskApplicationExample.java | 7 +- .../apache/samza/example/WindowExample.java | 4 +- .../TestStandaloneIntegrationApplication.java | 4 +- .../EndOfStreamIntegrationTest.java | 4 +- .../WatermarkIntegrationTest.java | 4 +- .../test/framework/BroadcastAssertApp.java | 4 +- .../samza/test/framework/TestTimerApp.java | 4 +- .../operator/RepartitionJoinWindowApp.java | 4 +- .../test/operator/RepartitionWindowApp.java | 4 +- .../samza/test/operator/SessionWindowApp.java | 4 +- .../test/operator/TumblingWindowApp.java | 4 +- .../test/processor/TestStreamApplication.java | 4 +- .../table/TestLocalTableWithSideInputs.java | 4 +- .../samza/test/table/TestRemoteTable.java | 4 +- 72 files changed, 377 insertions(+), 360 deletions(-) rename samza-api/src/main/java/org/apache/samza/application/{StreamAppDescriptor.java => StreamApplicationDescriptor.java} (97%) rename samza-api/src/main/java/org/apache/samza/application/{TaskAppDescriptor.java => TaskApplicationDescriptor.java} (94%) rename samza-core/src/main/java/org/apache/samza/application/{AppDescriptorImpl.java => ApplicationDescriptorImpl.java} (96%) rename samza-core/src/main/java/org/apache/samza/application/{StreamAppDescriptorImpl.java => StreamApplicationDescriptorImpl.java} (96%) rename samza-core/src/main/java/org/apache/samza/application/{TaskAppDescriptorImpl.java => TaskApplicationDescriptorImpl.java} (86%) rename samza-core/src/test/java/org/apache/samza/application/{TestStreamAppDescriptorImpl.java => TestStreamApplicationDescriptorImpl.java} (89%) rename samza-core/src/test/java/org/apache/samza/application/{TestTaskAppDescriptorImpl.java => TestTaskApplicationDescriptorImpl.java} (84%) diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index e878b2e65e..e426522144 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -30,11 +30,11 @@ /** * The base interface class to describe a user application in Samza. *

    - * Sub-classes {@link StreamAppDescriptor} and {@link TaskAppDescriptor} are specific interfaces for applications written + * Sub-classes {@link StreamApplicationDescriptor} and {@link TaskApplicationDescriptor} are specific interfaces for applications written * in high-level DAG and low-level task APIs, respectively. * - * @param sub-class of user application descriptor. It has to be either {@link StreamAppDescriptor} or - * {@link TaskAppDescriptor} + * @param sub-class of user application descriptor. It has to be either {@link StreamApplicationDescriptor} or + * {@link TaskApplicationDescriptor} */ @InterfaceStability.Evolving public interface ApplicationDescriptor { 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 f0be7b5157..111f7a4e73 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 @@ -72,5 +72,5 @@ * instance is closed. See {@link org.apache.samza.operators.functions.InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ @InterfaceStability.Evolving -public interface StreamApplication extends SamzaApplication { +public interface StreamApplication extends SamzaApplication { } diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java similarity index 97% rename from samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java rename to samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java index a9018b1b83..654b5d6d1a 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java @@ -32,7 +32,7 @@ * The interface class to describe a user application in high-level DAG in Samza. */ @InterfaceStability.Evolving -public interface StreamAppDescriptor extends ApplicationDescriptor { +public interface StreamApplicationDescriptor extends ApplicationDescriptor { /** * Gets the input {@link MessageStream} corresponding to the {@code inputDescriptor}. 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 e5f484fd1b..424634df5d 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 @@ -55,7 +55,9 @@ * message.getKey(), message.getKey(), m)); * } * } + * } * }

    + * *

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

    {@code
    @@ -71,7 +73,7 @@
      *
      * 

    * 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 TaskAppDescriptor} + * {@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. * @@ -80,5 +82,5 @@ * a default constructor with no parameters to ensure successful instantiation in both local and remote environments. */ @InterfaceStability.Evolving -public interface TaskApplication extends SamzaApplication { +public interface TaskApplication extends SamzaApplication { } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java similarity index 94% rename from samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java rename to samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java index d89eca72b7..d60df283ec 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskAppDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java @@ -28,8 +28,8 @@ /** * The interface class to describe a user application as low-level task in Samza. */ -@InterfaceStability.Unstable -public interface TaskAppDescriptor extends ApplicationDescriptor { +@InterfaceStability.Evolving +public interface TaskApplicationDescriptor extends ApplicationDescriptor { /** * Sets the {@link TaskFactory} for the user application. The {@link TaskFactory#createInstance()} creates task instance diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index 21a200efd2..26fc91af26 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -23,6 +23,7 @@ import java.util.Collection; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; @@ -40,7 +41,7 @@ * A stream of messages that can be transformed into another {@link MessageStream}. *

    * A {@link MessageStream} corresponding to an input stream can be obtained using - * {@link org.apache.samza.application.StreamAppDescriptor#getInputStream}. + * {@link StreamApplicationDescriptor#getInputStream}. * * @param the type of messages in this stream */ @@ -214,7 +215,7 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java index f29d4767d4..089f4e5938 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java @@ -20,6 +20,8 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationDescriptor; + /** * A function that can be closed after its execution. @@ -27,7 +29,7 @@ *

    Implement {@link #close()} to free resources used during the execution of the function, clean up state etc. * *

    Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the - * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B + * {@link StreamApplicationDescriptor}. For any two operators A and B in the graph, if operator B * consumes results from operator A, then operator B is guaranteed to be closed before operator A. * */ diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index 4f82b9fbae..f0c5d239c9 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -20,6 +20,7 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.task.TaskContext; @@ -27,7 +28,7 @@ * A function that can be initialized before execution. * *

    Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the - * {@link org.apache.samza.application.StreamAppDescriptor}. For any two operators A and B in the graph, if operator B + * {@link StreamApplicationDescriptor}. For any two operators A and B in the graph, if operator B * consumes results from operator A, then operator A is guaranteed to be initialized before operator B. * */ diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/StreamExpander.java b/samza-api/src/main/java/org/apache/samza/operators/functions/StreamExpander.java index 5764f70cd2..7bbf601c9c 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/StreamExpander.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/StreamExpander.java @@ -19,13 +19,13 @@ package org.apache.samza.operators.functions; import java.io.Serializable; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; /** - * Expands the provided {@link InputDescriptor} to a sub-DAG of one or more operators on the {@link StreamAppDescriptor}, - * and returns a new {@link MessageStream} with the combined results. Called when {@link StreamAppDescriptor#getInputStream} + * Expands the provided {@link InputDescriptor} to a sub-DAG of one or more operators on the {@link StreamApplicationDescriptor}, + * and returns a new {@link MessageStream} with the combined results. Called when {@link StreamApplicationDescriptor#getInputStream} * is being used to get a {@link MessageStream} using an {@link InputDescriptor} from an expanding system descriptor. *

    * This is provided by default by expanding system descriptor implementations and can not be overridden @@ -36,23 +36,23 @@ public interface StreamExpander extends Serializable { /** - * Expands the provided {@link InputDescriptor} to a sub-DAG of one or more operators on the {@link StreamAppDescriptor}, + * Expands the provided {@link InputDescriptor} to a sub-DAG of one or more operators on the {@link StreamApplicationDescriptor}, * and returns a new {@link MessageStream} with the combined results. Called when the {@link InputDescriptor} - * is being used to get an {@link MessageStream} using {@link StreamAppDescriptor#getInputStream}. + * is being used to get an {@link MessageStream} using {@link StreamApplicationDescriptor#getInputStream}. *

    * Notes for system implementers: *

    * Take care to avoid infinite recursion in the implementation; e.g., by ensuring that it doesn't call - * {@link StreamAppDescriptor#getInputStream} with an {@link InputDescriptor} from an expanding system descriptor + * {@link StreamApplicationDescriptor#getInputStream} with an {@link InputDescriptor} from an expanding system descriptor * (like this one) again. *

    * It's the {@link StreamExpander}'s responsibility to propagate any properties, including serde, from the * user-provided {@link InputDescriptor} to the expanded input descriptors. * - * @param streamAppDesc the {@link StreamAppDescriptor} to register the expanded sub-DAG of operators on + * @param streamAppDesc the {@link StreamApplicationDescriptor} to register the expanded sub-DAG of operators on * @param inputDescriptor the {@link InputDescriptor} to be expanded * @return the {@link MessageStream} containing the combined results of the sub-DAG of operators */ - MessageStream apply(StreamAppDescriptor streamAppDesc, InputDescriptor inputDescriptor); + MessageStream apply(StreamApplicationDescriptor streamAppDesc, InputDescriptor inputDescriptor); } diff --git a/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java similarity index 96% rename from samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java rename to samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java index bbc6b0f58a..ef97704690 100644 --- a/samza-core/src/main/java/org/apache/samza/application/AppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java @@ -44,9 +44,9 @@ * {@link Config}, {@link ContextManager}, and {@link ProcessorLifecycleListenerFactory}. * * @param the type of {@link ApplicationDescriptor} interface this implements. It has to be either - * {@link org.apache.samza.application.StreamAppDescriptor} or {@link org.apache.samza.application.TaskAppDescriptor} + * {@link StreamApplicationDescriptor} or {@link TaskApplicationDescriptor} */ -public abstract class AppDescriptorImpl +public abstract class ApplicationDescriptorImpl implements ApplicationDescriptor { final Config config; @@ -76,7 +76,7 @@ public void close() { // Default to no-op ProcessorLifecycleListenerFactory ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { }; - AppDescriptorImpl(SamzaApplication userApp, Config config) { + ApplicationDescriptorImpl(SamzaApplication userApp, Config config) { this.config = config; this.appClass = userApp.getClass(); } @@ -116,6 +116,8 @@ public S withMetricsReporterFactories(Map report } /** * Get the user application class + * + * @return user implemented {@link SamzaApplication} class */ public Class getAppClass() { return appClass; diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java index 261ff881f5..de15b80e2c 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java @@ -23,7 +23,7 @@ /** - * Util class to help creating {@link AppDescriptorImpl} instance from {@link SamzaApplication} and {@link Config} + * Util class to help creating {@link ApplicationDescriptorImpl} instance from {@link SamzaApplication} and {@link Config} */ public class ApplicationDescriptors { private ApplicationDescriptors() { @@ -31,29 +31,29 @@ private ApplicationDescriptors() { } /** - * Create a new instance of {@link AppDescriptorImpl} based on {@link SamzaApplication} and {@link Config} + * Create a new instance of {@link ApplicationDescriptorImpl} based on {@link SamzaApplication} and {@link Config} * * @param userApp the user-implemented {@link SamzaApplication}. The {@code userApp} has to have a proper fully-qualified class name. * @param config the user-supplied {@link Config} for the the application - * @return the {@link AppDescriptorImpl} instance containing the user processing logic and the config + * @return the {@link ApplicationDescriptorImpl} instance containing the user processing logic and the config */ - public static AppDescriptorImpl getAppDescriptor(SamzaApplication userApp, Config config) { + public static ApplicationDescriptorImpl getAppDescriptor(SamzaApplication userApp, Config config) { if (userApp instanceof StreamApplication) { - return new StreamAppDescriptorImpl((StreamApplication) userApp, config); + return new StreamApplicationDescriptorImpl((StreamApplication) userApp, config); } if (userApp instanceof TaskApplication) { - return new TaskAppDescriptorImpl((TaskApplication) userApp, config); + return new TaskApplicationDescriptorImpl((TaskApplication) userApp, config); } throw new IllegalArgumentException(String.format("User application class %s is not supported. Only StreamApplication " + "and TaskApplication are supported.", userApp.getClass().getName())); } - public static T forType(Function forTaskApp, Function forStreamApp, - AppDescriptorImpl desc) { - if (desc instanceof TaskAppDescriptorImpl) { - return forTaskApp.apply((TaskAppDescriptorImpl) desc); - } else if (desc instanceof StreamAppDescriptorImpl) { - return forStreamApp.apply((StreamAppDescriptorImpl) desc); + public static T forType(Function forTaskApp, Function forStreamApp, + ApplicationDescriptorImpl desc) { + if (desc instanceof TaskApplicationDescriptorImpl) { + return forTaskApp.apply((TaskApplicationDescriptorImpl) desc); + } else if (desc instanceof StreamApplicationDescriptorImpl) { + return forStreamApp.apply((StreamApplicationDescriptorImpl) desc); } throw new IllegalArgumentException(String.format("AppDescriptorImpl has to be either TaskAppDescriptorImpl or StreamAppDescriptorImpl." 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 36579c2225..370306855a 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 @@ -45,7 +45,7 @@ private Config validate(Config config) { } @Override - public void describe(TaskAppDescriptor appDesc) { + public void describe(TaskApplicationDescriptor appDesc) { appDesc.setTaskFactory(TaskFactoryUtil.getTaskFactoryFromConfig(config)); } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java similarity index 96% rename from samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java rename to samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java index 5512fa5637..92cdbbb539 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java @@ -59,13 +59,13 @@ /** * This class defines: - * 1) an implementation of {@link StreamAppDescriptor} that provides APIs to access {@link MessageStream}, {@link OutputStream}, + * 1) an implementation of {@link StreamApplicationDescriptor} that provides APIs to access {@link MessageStream}, {@link OutputStream}, * and {@link Table} to create the DAG of transforms. * 2) a builder that creates a serializable {@link OperatorSpecGraph} from user-defined DAG */ -public class StreamAppDescriptorImpl extends AppDescriptorImpl - implements StreamAppDescriptor { - private static final Logger LOGGER = LoggerFactory.getLogger(StreamAppDescriptorImpl.class); +public class StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl + implements StreamApplicationDescriptor { + private static final Logger LOGGER = LoggerFactory.getLogger(StreamApplicationDescriptorImpl.class); private static final Pattern ID_PATTERN = Pattern.compile("[\\d\\w-_]+"); // We use a LHM for deterministic order in initializing and closing operators. @@ -81,7 +81,7 @@ public class StreamAppDescriptorImpl extends AppDescriptorImpl operatorIds = new HashSet<>(); - public StreamAppDescriptorImpl(StreamApplication userApp, Config config) { + public StreamApplicationDescriptorImpl(StreamApplication userApp, Config config) { super(userApp, config); userApp.describe(this); } diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java similarity index 86% rename from samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java rename to samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java index d7a1ea264d..6d9256b01a 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskAppDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java @@ -26,18 +26,18 @@ /** - * This class implements interface {@link TaskAppDescriptor}. + * This class implements interface {@link TaskApplicationDescriptor}. *

    - * In addition to the common objects for an application defined in {@link AppDescriptorImpl}, this class also includes + * In addition to the common objects for an application defined in {@link ApplicationDescriptorImpl}, this class also includes * the low-level {@link TaskFactory} that creates user-defined task instances, the lists of input/broadcast/output streams, * and the list of {@link TableDescriptor}s used in the application. */ -public class TaskAppDescriptorImpl extends AppDescriptorImpl - implements TaskAppDescriptor { +public class TaskApplicationDescriptorImpl extends ApplicationDescriptorImpl + implements TaskApplicationDescriptor { TaskFactory taskFactory; - public TaskAppDescriptorImpl(TaskApplication userApp, Config config) { + public TaskApplicationDescriptorImpl(TaskApplication userApp, Config config) { super(userApp, config); userApp.describe(this); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index 556dff5725..e1604e37a0 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -29,10 +29,11 @@ import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.ApplicationDescriptors; -import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.application.TaskAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -59,10 +60,10 @@ public abstract class JobPlanner { private static final Logger LOG = LoggerFactory.getLogger(JobPlanner.class); - protected final AppDescriptorImpl appDesc; + protected final ApplicationDescriptorImpl appDesc; protected final Config config; - JobPlanner(AppDescriptorImpl descriptor) { + JobPlanner(ApplicationDescriptorImpl descriptor) { this.appDesc = descriptor; this.config = descriptor.getConfig(); } @@ -87,7 +88,7 @@ public List prepareJobs() { appDesc); } - abstract List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception; + abstract List prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception; StreamManager buildAndStartStreamManager(Config config) { StreamManager streamManager = new StreamManager(config); @@ -157,7 +158,7 @@ final void writePlanJsonFile(String planJson) { } // helper method to generate a single node job configuration for low level task applications - private JobConfig prepareTaskJob(TaskAppDescriptorImpl taskAppDesc) { + private JobConfig prepareTaskJob(TaskApplicationDescriptorImpl taskAppDesc) { // copy original configure Map cfg = new HashMap<>(config); // expand system and streams configure @@ -170,20 +171,21 @@ private JobConfig prepareTaskJob(TaskAppDescriptorImpl taskAppDesc) { return new JobConfig(new MapConfig(cfg)); } - private Map expandSystemStreamConfigs(AppDescriptorImpl appDesc) { + private Map expandSystemStreamConfigs(ApplicationDescriptorImpl appDesc) { Map systemStreamConfigs = new HashMap<>(); - appDesc.getInputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(((InputDescriptor) value).toConfig())); - appDesc.getOutputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(((OutputDescriptor) value).toConfig())); - appDesc.getSystemDescriptors().forEach(sd -> systemStreamConfigs.putAll(((SystemDescriptor) sd).toConfig())); + appDesc.getInputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); + appDesc.getOutputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); + appDesc.getSystemDescriptors().forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); appDesc.getDefaultSystemDescriptor().ifPresent(dsd -> - systemStreamConfigs.put(JobConfig.JOB_DEFAULT_SYSTEM(), ((SystemDescriptor) dsd).getSystemName())); + systemStreamConfigs.put(JobConfig.JOB_DEFAULT_SYSTEM(), dsd.getSystemName())); return systemStreamConfigs; } - private Map expandTableConfigs(Map originConfig, AppDescriptorImpl appDesc) { + private Map expandTableConfigs(Map originConfig, + ApplicationDescriptorImpl appDesc) { List tableSpecs = new ArrayList<>(); appDesc.getTableDescriptors().stream().map(td -> ((BaseTableDescriptor) td).getTableSpec()) - .forEach(spec -> tableSpecs.add((TableSpec) spec)); + .forEach(spec -> tableSpecs.add(spec)); return TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(originConfig), tableSpecs); } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java index 6c604b6ecc..7996d6bb27 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -23,8 +23,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -36,7 +37,7 @@ /** - * Temporarily helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} + * Temporarily helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamApplicationDescriptorImpl)} * for standalone Samza processors. * * TODO: we need to consolidate this with {@link ExecutionPlanner} after SAMZA-1811. @@ -47,12 +48,12 @@ public class LocalJobPlanner extends JobPlanner { private final String uid = UUID.randomUUID().toString();; - public LocalJobPlanner(AppDescriptorImpl descriptor) { + public LocalJobPlanner(ApplicationDescriptorImpl descriptor) { super(descriptor); } @Override - List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + List prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception { // for high-level DAG, generating the plan and job configs // 1. initialize and plan ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph()); diff --git a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java index accf490108..254ff97c51 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java @@ -21,8 +21,9 @@ import java.util.List; import java.util.UUID; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -33,7 +34,7 @@ /** - * Temporary helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamAppDescriptorImpl)} + * Temporary helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamApplicationDescriptorImpl)} * for remote-launched Samza processors (e.g. in YARN). * * TODO: we need to consolidate this class with {@link ExecutionPlanner} after SAMZA-1811. @@ -41,12 +42,12 @@ public class RemoteJobPlanner extends JobPlanner { private static final Logger LOG = LoggerFactory.getLogger(RemoteJobPlanner.class); - public RemoteJobPlanner(AppDescriptorImpl descriptor) { + public RemoteJobPlanner(ApplicationDescriptorImpl descriptor) { super(descriptor); } @Override - List prepareStreamJobs(StreamAppDescriptorImpl streamAppDesc) throws Exception { + List prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception { // for high-level DAG, generate the plan and job configs // TODO: run.id needs to be set for standalone: SAMZA-1531 // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index 30346a508b..5411af336c 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -24,7 +24,7 @@ import java.util.Collection; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; @@ -55,7 +55,7 @@ /** * The {@link MessageStream} implementation that lets users describe their logical DAG. - * Users can obtain an instance by calling {@link StreamAppDescriptorImpl#getInputStream}. + * Users can obtain an instance by calling {@link StreamApplicationDescriptorImpl#getInputStream}. *

    * Each {@link MessageStreamImpl} is associated with a single {@link OperatorSpec} in the DAG and allows * users to chain further operators on its {@link OperatorSpec}. In other words, a {@link MessageStreamImpl} @@ -65,16 +65,16 @@ */ public class MessageStreamImpl implements MessageStream { /** - * The {@link StreamAppDescriptorImpl} that contains this {@link MessageStreamImpl} + * The {@link StreamApplicationDescriptorImpl} that contains this {@link MessageStreamImpl} */ - private final StreamAppDescriptorImpl streamAppDesc; + private final StreamApplicationDescriptorImpl streamAppDesc; /** * The {@link OperatorSpec} associated with this {@link MessageStreamImpl} */ private final OperatorSpec operatorSpec; - public MessageStreamImpl(StreamAppDescriptorImpl streamAppDesc, OperatorSpec operatorSpec) { + public MessageStreamImpl(StreamApplicationDescriptorImpl streamAppDesc, OperatorSpec operatorSpec) { this.streamAppDesc = streamAppDesc; this.operatorSpec = operatorSpec; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java index f1eec24b4c..b75b1e8538 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java @@ -25,7 +25,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; @@ -34,9 +34,9 @@ /** - * Defines the serialized format of the operator graph in {@link StreamAppDescriptorImpl}. This class encapsulates all - * getter methods to get the {@link OperatorSpec} initialized in the {@link StreamAppDescriptorImpl} and constructs the - * corresponding serialized instances of {@link OperatorSpec}. The {@link StreamAppDescriptorImpl} and {@link OperatorSpec} + * Defines the serialized format of the operator graph in {@link StreamApplicationDescriptorImpl}. This class encapsulates all + * getter methods to get the {@link OperatorSpec} initialized in the {@link StreamApplicationDescriptorImpl} and constructs the + * corresponding serialized instances of {@link OperatorSpec}. The {@link StreamApplicationDescriptorImpl} and {@link OperatorSpec} * instances included in this class are considered as immutable and read-only. The instance of {@link OperatorSpecGraph} * should only be used in runtime to construct {@link org.apache.samza.task.StreamOperatorTask}. */ @@ -53,7 +53,7 @@ public class OperatorSpecGraph implements Serializable { private transient final SerializableSerde opSpecGraphSerde = new SerializableSerde<>(); private transient final byte[] serializedOpSpecGraph; - public OperatorSpecGraph(StreamAppDescriptorImpl streamAppDesc) { + public OperatorSpecGraph(StreamApplicationDescriptorImpl streamAppDesc) { this.inputOperators = streamAppDesc.getInputOperators(); this.outputStreams = streamAppDesc.getOutputStreams(); this.broadcastStreams = streamAppDesc.getBroadcastStreams(); @@ -80,7 +80,7 @@ public Map getTables() { } /** - * Get all {@link OperatorSpec}s available in this {@link StreamAppDescriptorImpl} + * Get all {@link OperatorSpec}s available in this {@link StreamApplicationDescriptorImpl} * * @return all available {@link OperatorSpec}s */ @@ -89,9 +89,9 @@ public Collection getAllOperatorSpecs() { } /** - * Returns true iff this {@link StreamAppDescriptorImpl} contains a join or a window operator + * Returns true iff this {@link StreamApplicationDescriptorImpl} contains a join or a window operator * - * @return true iff this {@link StreamAppDescriptorImpl} contains a join or a window operator + * @return true iff this {@link StreamApplicationDescriptorImpl} contains a join or a window operator */ public boolean hasWindowOrJoins() { return hasWindowOrJoins; diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index 019046a171..fcf343ae43 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -23,6 +23,7 @@ import java.util.LinkedHashSet; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.functions.TimerFunction; @@ -103,7 +104,7 @@ public final OpCode getOpCode() { } /** - * Get the unique ID of this operator in the {@link org.apache.samza.application.StreamAppDescriptorImpl}. + * Get the unique ID of this operator in the {@link StreamApplicationDescriptorImpl}. * @return the unique operator ID */ public final String getOpId() { diff --git a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java index 05e2ddcef0..4db8e6097c 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java @@ -18,7 +18,7 @@ */ package org.apache.samza.operators.stream; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -44,7 +44,7 @@ public class IntermediateMessageStreamImpl extends MessageStreamImpl imple private final OutputStreamImpl outputStream; private final boolean isKeyed; - public IntermediateMessageStreamImpl(StreamAppDescriptorImpl appDesc, InputOperatorSpec inputOperatorSpec, + public IntermediateMessageStreamImpl(StreamApplicationDescriptorImpl appDesc, InputOperatorSpec inputOperatorSpec, OutputStreamImpl outputStream) { super(appDesc, (OperatorSpec) inputOperatorSpec); this.outputStream = outputStream; diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 54e1966837..3ccce3d6ef 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -32,7 +32,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.config.ApplicationConfig; @@ -41,7 +42,6 @@ import org.apache.samza.execution.LocalJobPlanner; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; @@ -55,7 +55,7 @@ public class LocalApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(LocalApplicationRunner.class); - private final AppDescriptorImpl appDesc; + private final ApplicationDescriptorImpl appDesc; private final LocalJobPlanner planner; private final Set processors = ConcurrentHashMap.newKeySet(); private final CountDownLatch shutdownLatch = new CountDownLatch(1); @@ -80,7 +80,7 @@ public LocalApplicationRunner(SamzaApplication userApp, Config config) { * */ @VisibleForTesting - LocalApplicationRunner(AppDescriptorImpl appDesc, LocalJobPlanner planner) { + LocalApplicationRunner(ApplicationDescriptorImpl appDesc, LocalJobPlanner planner) { this.appDesc = appDesc; this.planner = planner; } @@ -162,13 +162,13 @@ CountDownLatch getShutdownLatch() { } /* package private */ - StreamProcessor createStreamProcessor(Config config, AppDescriptorImpl appDesc, + StreamProcessor createStreamProcessor(Config config, ApplicationDescriptorImpl appDesc, LocalStreamProcessorLifecycleListener listener) { TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); Map reporters = new HashMap<>(); // TODO: the null processorId has to be fixed after SAMZA-1835 - ((Map) appDesc.getMetricsReporterFactories()) - .forEach((name, factory) -> reporters.put(name, factory.getMetricsReporter(name, null, config))); + appDesc.getMetricsReporterFactories().forEach((name, factory) -> + reporters.put(name, factory.getMetricsReporter(name, null, config))); return new StreamProcessor(config, reporters, taskFactory, listener, null); } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index d6afd99954..45529f966b 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -24,8 +24,9 @@ import java.util.Random; import org.apache.log4j.MDC; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptors; -import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.ShellCommandConfig; @@ -79,13 +80,15 @@ public static void main(String[] args) throws Exception { MDC.put("jobName", jobName); MDC.put("jobId", jobId); - AppDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(ApplicationClassUtils.fromConfig(config), config); + ApplicationDescriptorImpl appDesc = + ApplicationDescriptors.getAppDescriptor(ApplicationClassUtils.fromConfig(config), config); run(appDesc, containerId, jobModel, config); System.exit(0); } - private static void run(AppDescriptorImpl appDesc, String containerId, JobModel jobModel, Config config) { + private static void run(ApplicationDescriptorImpl appDesc, String containerId, + JobModel jobModel, Config config) { TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); SamzaContainer container = SamzaContainer$.MODULE$.apply( containerId, @@ -144,10 +147,11 @@ public void afterFailed(Throwable t) { // TODO: this is going away when SAMZA-1168 is done and the initialization of metrics reporters are done via // LocalApplicationRunner#createStreamProcessor() - private static Map loadMetricsReporters(AppDescriptorImpl appDesc, String containerId, Config config) { + private static Map loadMetricsReporters( + ApplicationDescriptorImpl appDesc, String containerId, Config config) { Map reporters = new HashMap<>(); - ((Map) appDesc.getMetricsReporterFactories()) - .forEach((name, factory) -> reporters.put(name, factory.getMetricsReporter(name, containerId, config))); + appDesc.getMetricsReporterFactories().forEach((name, factory) -> + reporters.put(name, factory.getMetricsReporter(name, containerId, config))); return reporters; } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 2c061efbb8..d9a26e9218 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -22,7 +22,8 @@ import java.time.Duration; import java.util.List; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.config.Config; @@ -44,7 +45,7 @@ public class RemoteApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; - private final AppDescriptorImpl appDesc; + private final ApplicationDescriptorImpl appDesc; private final RemoteJobPlanner planner; /** diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java index f18e0750f4..032074dbd9 100644 --- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java +++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java @@ -18,6 +18,8 @@ */ package org.apache.samza.task; +import org.apache.samza.application.StreamApplicationDescriptor; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.system.EndOfStreamMessage; @@ -69,10 +71,10 @@ public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextMan * Initializes this task during startup. *

    * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}. - * The {@link org.apache.samza.application.StreamAppDescriptorImpl} sets the input and output streams and the task-wide - * context manager using the {@link org.apache.samza.application.StreamAppDescriptor} APIs, + * The {@link StreamApplicationDescriptorImpl} sets the input and output streams and the task-wide + * context manager using the {@link StreamApplicationDescriptor} APIs, * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. After the - * {@link org.apache.samza.application.StreamAppDescriptorImpl} is initialized once by the application, it then creates + * {@link StreamApplicationDescriptorImpl} is initialized once by the application, it then creates * an immutable {@link OperatorSpecGraph} accordingly, which is passed in to this class to create the {@link OperatorImplGraph} * corresponding to the logical DAG. * diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index 32fa9d1339..c350907fc5 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -19,8 +19,10 @@ package org.apache.samza.task; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.TaskConfig; @@ -39,16 +41,15 @@ public class TaskFactoryUtil { private static final Logger log = LoggerFactory.getLogger(TaskFactoryUtil.class); /** - * Creates a {@link TaskFactory} based on {@link AppDescriptorImpl} + * Creates a {@link TaskFactory} based on {@link ApplicationDescriptorImpl} * - * @param appDesc {@link AppDescriptorImpl} for this application + * @param appDesc {@link ApplicationDescriptorImpl} for this application * @return {@link TaskFactory} object defined by {@code appDesc} */ - public static TaskFactory getTaskFactory(AppDescriptorImpl appDesc) { - return ApplicationDescriptors.forType( - taskAppDesc -> taskAppDesc.getTaskFactory(), - streamAppDesc -> (StreamTaskFactory) () -> new StreamOperatorTask(streamAppDesc.getOperatorSpecGraph(), - streamAppDesc.getContextManager()), + public static TaskFactory getTaskFactory(ApplicationDescriptorImpl appDesc) { + return ApplicationDescriptors.forType(TaskApplicationDescriptorImpl::getTaskFactory, + streamAppDesc -> (StreamTaskFactory) () -> + new StreamOperatorTask(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()), appDesc); } 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 3bab1acf15..ccd88b8e74 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 @@ -23,7 +23,7 @@ */ public class MockStreamApplication implements StreamApplication { @Override - public void describe(StreamAppDescriptor appSpec) { + public void describe(StreamApplicationDescriptor appSpec) { } } diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java similarity index 89% rename from samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java rename to samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java index cb1f0d6a0d..db85e3335e 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java @@ -66,15 +66,15 @@ import static org.mockito.Mockito.when; /** - * Unit test for {@link StreamAppDescriptorImpl} + * Unit test for {@link StreamApplicationDescriptorImpl} */ -public class TestStreamAppDescriptorImpl { +public class TestStreamApplicationDescriptorImpl { @Test public void testConstructor() { StreamApplication mockApp = mock(StreamApplication.class); Config mockConfig = mock(Config.class); - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(mockApp, mockConfig); + StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(mockApp, mockConfig); verify(mockApp).describe(appDesc); assertEquals(mockConfig, appDesc.config); } @@ -86,7 +86,7 @@ public void testGetInputStreamWithValueSerde() { Serde mockValueSerde = mock(Serde.class); GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mockValueSerde); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); }, mock(Config.class)); @@ -109,7 +109,7 @@ public void testGetInputStreamWithKeyValueSerde() { doReturn(mockValueSerde).when(mockKVSerde).getValueSerde(); GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mockKVSerde); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); }, mock(Config.class)); @@ -125,7 +125,7 @@ public void testGetInputStreamWithKeyValueSerde() { public void testGetInputStreamWithNullSerde() { GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericInputDescriptor isd = sd.getInputDescriptor("mockStreamId", null); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); }, mock(Config.class)); } @@ -137,7 +137,7 @@ public void testGetInputStreamWithTransformFunction() { InputTransformer transformer = ime -> ime; MockTransformingSystemDescriptor sd = new MockTransformingSystemDescriptor("mockSystem", transformer); MockInputDescriptor isd = sd.getInputDescriptor(streamId, mockValueSerde); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); }, mock(Config.class)); @@ -163,7 +163,7 @@ public void testGetInputStreamWithExpandingSystem() { }; MockExpandingSystemDescriptor sd = new MockExpandingSystemDescriptor("mock-system", expander); MockInputDescriptor isd = sd.getInputDescriptor(streamId, new IntegerSerde()); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); }, mock(Config.class)); @@ -181,7 +181,7 @@ public void testGetInputStreamWithRelaxedTypes() { String streamId = "test-stream-1"; GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mock(Serde.class)); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); }, mock(Config.class)); @@ -199,7 +199,7 @@ public void testMultipleGetInputStreams() { GenericInputDescriptor isd1 = sd.getInputDescriptor(streamId1, mock(Serde.class)); GenericInputDescriptor isd2 = sd.getInputDescriptor(streamId2, mock(Serde.class)); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd1); appDesc.getInputStream(isd2); }, mock(Config.class)); @@ -221,7 +221,7 @@ public void testGetSameInputStreamTwice() { GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericInputDescriptor isd1 = sd.getInputDescriptor(streamId, mock(Serde.class)); GenericInputDescriptor isd2 = sd.getInputDescriptor(streamId, mock(Serde.class)); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd1); // should throw exception appDesc.getInputStream(isd2); @@ -236,7 +236,7 @@ public void testMultipleSystemDescriptorForSameSystemName() { GenericInputDescriptor isd2 = sd2.getInputDescriptor("test-stream-2", mock(Serde.class)); GenericOutputDescriptor osd1 = sd2.getOutputDescriptor("test-stream-3", mock(Serde.class)); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd1); try { appDesc.getInputStream(isd2); @@ -249,7 +249,7 @@ public void testMultipleSystemDescriptorForSameSystemName() { } catch (IllegalStateException e) { } }, mock(Config.class)); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.withDefaultSystem(sd2); try { appDesc.getInputStream(isd1); @@ -269,7 +269,7 @@ public void testGetOutputStreamWithKeyValueSerde() { GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mockKVSerde); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getOutputStream(osd); }, mock(Config.class)); @@ -285,7 +285,7 @@ public void testGetOutputStreamWithNullSerde() { String streamId = "test-stream-1"; GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, null); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getOutputStream(osd); }, mock(Config.class)); } @@ -297,7 +297,7 @@ public void testGetOutputStreamWithValueSerde() { GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mockValueSerde); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getOutputStream(osd); }, mock(Config.class)); @@ -314,7 +314,7 @@ public void testSetDefaultSystemDescriptorAfterGettingInputStream() { GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericInputDescriptor isd = sd.getInputDescriptor(streamId, mock(Serde.class)); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(isd); appDesc.withDefaultSystem(sd); // should throw exception }, mock(Config.class)); @@ -325,7 +325,7 @@ public void testSetDefaultSystemDescriptorAfterGettingOutputStream() { String streamId = "test-stream-1"; GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericOutputDescriptor osd = sd.getOutputDescriptor(streamId, mock(Serde.class)); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getOutputStream(osd); appDesc.withDefaultSystem(sd); // should throw exception }, mock(Config.class)); @@ -334,7 +334,7 @@ public void testSetDefaultSystemDescriptorAfterGettingOutputStream() { @Test(expected = IllegalStateException.class) public void testSetDefaultSystemDescriptorAfterGettingIntermediateStream() { String streamId = "test-stream-1"; - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class)); streamAppDesc.getIntermediateStream(streamId, mock(Serde.class), false); streamAppDesc.withDefaultSystem(mock(SystemDescriptor.class)); // should throw exception } @@ -345,7 +345,7 @@ public void testGetSameOutputStreamTwice() { GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); GenericOutputDescriptor osd1 = sd.getOutputDescriptor(streamId, mock(Serde.class)); GenericOutputDescriptor osd2 = sd.getOutputDescriptor(streamId, mock(Serde.class)); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getOutputStream(osd1); appDesc.getOutputStream(osd2); // should throw exception }, mock(Config.class)); @@ -354,7 +354,7 @@ public void testGetSameOutputStreamTwice() { @Test public void testGetIntermediateStreamWithValueSerde() { String streamId = "stream-1"; - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class)); Serde mockValueSerde = mock(Serde.class); IntermediateMessageStreamImpl intermediateStreamImpl = @@ -372,7 +372,7 @@ public void testGetIntermediateStreamWithValueSerde() { @Test public void testGetIntermediateStreamWithKeyValueSerde() { String streamId = "streamId"; - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class)); KVSerde mockKVSerde = mock(KVSerde.class); Serde mockKeySerde = mock(Serde.class); @@ -396,7 +396,7 @@ public void testGetIntermediateStreamWithDefaultSystemDescriptor() { Config mockConfig = mock(Config.class); String streamId = "streamId"; - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig); GenericSystemDescriptor sd = new GenericSystemDescriptor("mock-system", "mock-system-factory"); streamAppDesc.withDefaultSystem(sd); IntermediateMessageStreamImpl intermediateStreamImpl = @@ -412,7 +412,7 @@ public void testGetIntermediateStreamWithNoSerde() { Config mockConfig = mock(Config.class); String streamId = "streamId"; - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig); IntermediateMessageStreamImpl intermediateStreamImpl = streamAppDesc.getIntermediateStream(streamId, null, false); @@ -427,7 +427,7 @@ public void testGetIntermediateStreamWithNoSerde() { @Test(expected = IllegalStateException.class) public void testGetSameIntermediateStreamTwice() { - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class)); streamAppDesc.getIntermediateStream("test-stream-1", mock(Serde.class), false); // should throw exception streamAppDesc.getIntermediateStream("test-stream-1", mock(Serde.class), false); @@ -439,7 +439,7 @@ public void testGetNextOpIdIncrementsId() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig); assertEquals("jobName-1234-merge-0", streamAppDesc.getNextOpId(OpCode.MERGE, null)); assertEquals("jobName-1234-join-customName", streamAppDesc.getNextOpId(OpCode.JOIN, "customName")); assertEquals("jobName-1234-map-2", streamAppDesc.getNextOpId(OpCode.MAP, null)); @@ -451,7 +451,7 @@ public void testGetNextOpIdRejectsDuplicates() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig); assertEquals("jobName-1234-join-customName", streamAppDesc.getNextOpId(OpCode.JOIN, "customName")); streamAppDesc.getNextOpId(OpCode.JOIN, "customName"); // should throw } @@ -462,7 +462,7 @@ public void testOpIdValidation() { when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234"); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, mockConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, mockConfig); // null and empty userDefinedIDs should fall back to autogenerated IDs. try { @@ -501,7 +501,7 @@ public void testGetInputStreamPreservesInsertionOrder() { String testStreamId3 = "test-stream-3"; GenericSystemDescriptor sd = new GenericSystemDescriptor("mockSystem", "mockSystemFactoryClass"); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getInputStream(sd.getInputDescriptor(testStreamId1, mock(Serde.class))); appDesc.getInputStream(sd.getInputDescriptor(testStreamId2, mock(Serde.class))); appDesc.getInputStream(sd.getInputDescriptor(testStreamId3, mock(Serde.class))); @@ -522,7 +522,7 @@ public void testGetTable() throws Exception { TableSpec testTableSpec = new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()); when(mockTableDescriptor.getTableSpec()).thenReturn(testTableSpec); when(mockTableDescriptor.getTableId()).thenReturn(testTableSpec.getId()); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getTable(mockTableDescriptor); }, mockConfig); assertNotNull(streamAppDesc.getTables().get(testTableSpec)); @@ -532,7 +532,7 @@ public void testGetTable() throws Exception { public void testContextManager() { ContextManager cntxMan = mock(ContextManager.class); StreamApplication testApp = appDesc -> appDesc.withContextManager(cntxMan); - StreamAppDescriptorImpl appSpec = new StreamAppDescriptorImpl(testApp, mock(Config.class)); + StreamApplicationDescriptorImpl appSpec = new StreamApplicationDescriptorImpl(testApp, mock(Config.class)); assertEquals(appSpec.getContextManager(), cntxMan); } @@ -540,14 +540,14 @@ public void testContextManager() { public void testProcessorLifecycleListenerFactory() { ProcessorLifecycleListenerFactory mockFactory = mock(ProcessorLifecycleListenerFactory.class); StreamApplication testApp = appSpec -> appSpec.withProcessorLifecycleListenerFactory(mockFactory); - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(testApp, mock(Config.class)); + StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(testApp, mock(Config.class)); assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); } @Test(expected = IllegalStateException.class) public void testGetTableWithBadId() { Config mockConfig = mock(Config.class); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); when(mockTableDescriptor.getTableId()).thenReturn("my.table"); appDesc.getTable(mockTableDescriptor); diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java similarity index 84% rename from samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java rename to samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java index 207cfee060..826b622638 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskAppDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java @@ -22,19 +22,13 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import org.apache.samza.SamzaException; import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; -import org.apache.samza.serializers.SerializableSerde; -import org.apache.samza.task.MockStreamTask; -import org.apache.samza.task.StreamTask; -import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.task.TaskFactory; import org.junit.Before; import org.junit.Test; @@ -47,9 +41,9 @@ /** - * Unit test for {@link TaskAppDescriptorImpl} + * Unit test for {@link TaskApplicationDescriptorImpl} */ -public class TestTaskAppDescriptorImpl { +public class TestTaskApplicationDescriptorImpl { private Config config = mock(Config.class); private String defaultSystemName = "test-system"; @@ -89,7 +83,7 @@ public void setUp() { @Test public void testConstructor() { TaskApplication mockApp = mock(TaskApplication.class); - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(mockApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(mockApp, config); verify(mockApp).describe(appDesc); assertEquals(config, appDesc.config); } @@ -99,7 +93,7 @@ public void testAddInputStreams() { TaskApplication testApp = appDesc -> { mockInputs.forEach(appDesc::addInputStream); }; - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config); assertEquals(mockInputs.toArray(), appDesc.getInputDescriptors().values().toArray()); } @@ -108,7 +102,7 @@ public void testAddOutputStreams() { TaskApplication testApp = appDesc -> { mockOutputs.forEach(appDesc::addOutputStream); }; - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config); assertEquals(mockOutputs.toArray(), appDesc.getOutputDescriptors().values().toArray()); } @@ -117,7 +111,7 @@ public void testAddTables() { TaskApplication testApp = appDesc -> { mockTables.forEach(appDesc::addTable); }; - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config); assertEquals(mockTables, appDesc.getTableDescriptors()); } @@ -125,7 +119,7 @@ public void testAddTables() { public void testSetTaskFactory() { TaskFactory mockTf = mock(TaskFactory.class); TaskApplication testApp = appDesc -> appDesc.setTaskFactory(mockTf); - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config); assertEquals(appDesc.getTaskFactory(), mockTf); } @@ -135,7 +129,7 @@ public void testContextManager() { TaskApplication testApp = appDesc -> { appDesc.withContextManager(cntxMan); }; - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config); assertEquals(appDesc.getContextManager(), cntxMan); } @@ -145,7 +139,7 @@ public void testProcessorLifecycleListener() { TaskApplication testApp = appDesc -> { appDesc.withProcessorLifecycleListenerFactory(mockFactory); }; - TaskAppDescriptorImpl appDesc = new TaskAppDescriptorImpl(testApp, config); + TaskApplicationDescriptorImpl appDesc = new TaskApplicationDescriptorImpl(testApp, config); assertEquals(appDesc.getProcessorLifecycleListenerFactory(), mockFactory); } } \ No newline at end of file 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 bc638c93d5..61cf6c52f1 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 @@ -28,7 +28,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -107,14 +107,14 @@ public Integer offsetComparator(String offset1, String offset2) { }; } - private StreamAppDescriptorImpl createSimpleGraph() { + private StreamApplicationDescriptorImpl createSimpleGraph() { /** * a simple graph of partitionBy and map * * input1 -> partitionBy -> map -> output1 * */ - return new StreamAppDescriptorImpl(appDesc-> { + return new StreamApplicationDescriptorImpl(appDesc-> { MessageStream> input1 = appDesc.getInputStream(input1Descriptor); OutputStream> output1 = appDesc.getOutputStream(output1Descriptor); input1 @@ -124,7 +124,7 @@ private StreamAppDescriptorImpl createSimpleGraph() { }, config); } - private StreamAppDescriptorImpl createStreamGraphWithJoin() { + private StreamApplicationDescriptorImpl createStreamGraphWithJoin() { /** * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value. @@ -136,7 +136,7 @@ private StreamAppDescriptorImpl createStreamGraphWithJoin() { * input3 (32) -> filter -> partitionBy ("64") -> map -> join -> output2 (16) * */ - return new StreamAppDescriptorImpl(appDesc -> { + return new StreamApplicationDescriptorImpl(appDesc -> { MessageStream> messageStream1 = appDesc.getInputStream(input1Descriptor) .map(m -> m); @@ -165,9 +165,9 @@ private StreamAppDescriptorImpl createStreamGraphWithJoin() { }, config); } - private StreamAppDescriptorImpl createStreamGraphWithJoinAndWindow() { + private StreamApplicationDescriptorImpl createStreamGraphWithJoinAndWindow() { - return new StreamAppDescriptorImpl(appDesc -> { + return new StreamApplicationDescriptorImpl(appDesc -> { MessageStream> messageStream1 = appDesc.getInputStream(input1Descriptor) .map(m -> m); @@ -262,7 +262,7 @@ public void setup() { @Test public void testCreateProcessorGraph() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); assertTrue(jobGraph.getSources().size() == 3); @@ -273,7 +273,7 @@ public void testCreateProcessorGraph() { @Test public void testFetchExistingStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); @@ -291,7 +291,7 @@ public void testFetchExistingStreamPartitions() { @Test public void testCalculateJoinInputPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); @@ -310,7 +310,7 @@ public void testDefaultPartitions() { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamAppDescriptorImpl graphSpec = createSimpleGraph(); + StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); planner.calculatePartitions(jobGraph); @@ -327,7 +327,7 @@ public void testTriggerIntervalForJoins() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoin(); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); for (JobConfig config : jobConfigs) { @@ -342,7 +342,7 @@ public void testTriggerIntervalForWindowsAndJoins() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -359,7 +359,7 @@ public void testTriggerIntervalWithInvalidWindowMs() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamAppDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -376,7 +376,7 @@ public void testTriggerIntervalForStatelessOperators() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamAppDescriptorImpl graphSpec = createSimpleGraph(); + StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -391,7 +391,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); - StreamAppDescriptorImpl graphSpec = createSimpleGraph(); + StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -401,7 +401,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { @Test public void testCalculateIntStreamPartitions() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamAppDescriptorImpl graphSpec = createSimpleGraph(); + StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph()); // the partitions should be the same as input1 @@ -434,7 +434,7 @@ public void testMaxPartitionLimit() throws Exception { int partitionLimit = ExecutionPlanner.MAX_INFERRED_PARTITIONS; ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { MessageStream> input1 = appDesc.getInputStream(input4Descriptor); OutputStream> output1 = appDesc.getOutputStream(output1Descriptor); input1.partitionBy(m -> m.key, m -> m.value, "p1").map(kv -> kv).sendTo(output1); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index bb3eb1ec58..ae6e25e5ee 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -22,7 +22,7 @@ import java.time.Duration; import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -102,7 +102,7 @@ public void test() throws Exception { when(systemAdmins.getSystemAdmin("system2")).thenReturn(systemAdmin2); StreamManager streamManager = new StreamManager(systemAdmins); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { KVSerde kvSerde = new KVSerde<>(new NoOpSerde(), new NoOpSerde()); String mockSystemFactoryClass = "factory.class.name"; GenericSystemDescriptor system1 = new GenericSystemDescriptor("system1", mockSystemFactoryClass); @@ -178,7 +178,7 @@ public void test2() throws Exception { when(systemAdmins.getSystemAdmin("kafka")).thenReturn(systemAdmin2); StreamManager streamManager = new StreamManager(systemAdmins); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { KVSerde pvSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewEvent.class)); GenericSystemDescriptor isd = new GenericSystemDescriptor("hdfs", "mockSystemFactoryClass"); GenericInputDescriptor> pageView = isd.getInputDescriptor("PageView", pvSerde); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index 781b8d594b..163b094960 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -24,7 +24,7 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -65,7 +65,7 @@ public void testAddSerdeConfigs() { when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { KVSerde serde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); GenericSystemDescriptor sd = new GenericSystemDescriptor("system1", "mockSystemFactoryClass"); GenericInputDescriptor> inputDescriptor1 = sd.getInputDescriptor("input1", serde); @@ -188,7 +188,7 @@ public void testAddSerdeConfigsForRepartitionWithNoDefaultSystem() { when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor sd = new GenericSystemDescriptor("system1", "mockSystemFactoryClassName"); GenericInputDescriptor> inputDescriptor1 = sd.getInputDescriptor("input", KVSerde.of(new StringSerde(), new JsonSerdeV2<>())); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java index 24faedded5..9ed57fae72 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestLocalJobPlanner.java @@ -22,8 +22,9 @@ import java.util.Collections; import java.util.List; import java.util.stream.Collectors; -import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -72,7 +73,7 @@ public class TestLocalJobPlanner { @Test public void testStreamCreation() throws Exception { - localPlanner = createLocalJobPlanner(mock(StreamAppDescriptorImpl.class)); + localPlanner = createLocalJobPlanner(mock(StreamApplicationDescriptorImpl.class)); StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); @@ -101,7 +102,7 @@ public void testStreamCreation() @Test public void testStreamCreationWithCoordination() throws Exception { - localPlanner = createLocalJobPlanner(mock(StreamAppDescriptorImpl.class)); + localPlanner = createLocalJobPlanner(mock(StreamApplicationDescriptorImpl.class)); StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(localPlanner).buildAndStartStreamManager(any(Config.class)); @@ -190,7 +191,7 @@ public void testGeneratePlanIdWithDifferentStreamSpecs() { planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs))); } - private LocalJobPlanner createLocalJobPlanner(AppDescriptorImpl appDesc) { + private LocalJobPlanner createLocalJobPlanner(ApplicationDescriptorImpl appDesc) { return spy(new LocalJobPlanner(appDesc)); } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java index 5b96c4ee97..988fb341f8 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java @@ -20,8 +20,9 @@ import java.util.Collections; import java.util.List; -import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -56,7 +57,7 @@ public class TestRemoteJobPlanner { @Test public void testStreamCreation() throws Exception { - remotePlanner = createRemoteJobPlanner(mock(StreamAppDescriptorImpl.class)); + remotePlanner = createRemoteJobPlanner(mock(StreamApplicationDescriptorImpl.class)); StreamManager streamManager = mock(StreamManager.class); doReturn(streamManager).when(remotePlanner).buildAndStartStreamManager(any(Config.class)); @@ -81,7 +82,7 @@ public void testStreamCreation() verify(streamManager).stop(); } - private RemoteJobPlanner createRemoteJobPlanner(AppDescriptorImpl appDesc) { + private RemoteJobPlanner createRemoteJobPlanner(ApplicationDescriptorImpl appDesc) { return spy(new RemoteJobPlanner(appDesc)); } } diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java index 9a868c2902..6fa9ed1505 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java @@ -22,7 +22,7 @@ import org.apache.samza.Partition; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.container.TaskContextImpl; @@ -73,7 +73,7 @@ public class TestJoinOperator { @Test public void join() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -95,7 +95,7 @@ public void joinWithSelfThrowsException() throws Exception { StreamTestUtils.addStreamConfigs(mapConfig, "inStream", "insystem", "instream"); Config config = new MapConfig(mapConfig); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { IntegerSerde integerSerde = new IntegerSerde(); KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); GenericSystemDescriptor sd = new GenericSystemDescriptor("insystem", "mockFactoryClassName"); @@ -112,7 +112,7 @@ public void joinWithSelfThrowsException() throws Exception { @Test public void joinFnInitAndClose() throws Exception { TestJoinFunction joinFn = new TestJoinFunction(); - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(joinFn); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(joinFn); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); MessageCollector messageCollector = mock(MessageCollector.class); @@ -131,7 +131,7 @@ public void joinFnInitAndClose() throws Exception { @Test public void joinReverse() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -147,7 +147,7 @@ public void joinReverse() throws Exception { @Test public void joinNoMatch() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -162,7 +162,7 @@ public void joinNoMatch() throws Exception { @Test public void joinNoMatchReverse() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -177,7 +177,7 @@ public void joinNoMatchReverse() throws Exception { @Test public void joinRetainsLatestMessageForKey() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -195,7 +195,7 @@ public void joinRetainsLatestMessageForKey() throws Exception { @Test public void joinRetainsLatestMessageForKeyReverse() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -213,7 +213,7 @@ public void joinRetainsLatestMessageForKeyReverse() throws Exception { @Test public void joinRetainsMatchedMessages() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -236,7 +236,7 @@ public void joinRetainsMatchedMessages() throws Exception { @Test public void joinRetainsMatchedMessagesReverse() throws Exception { - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -260,7 +260,7 @@ public void joinRetainsMatchedMessagesReverse() throws Exception { @Test public void joinRemovesExpiredMessages() throws Exception { TestClock testClock = new TestClock(); - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(testClock, streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -280,7 +280,7 @@ public void joinRemovesExpiredMessages() throws Exception { @Test public void joinRemovesExpiredMessagesReverse() throws Exception { TestClock testClock = new TestClock(); - StreamAppDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); + StreamApplicationDescriptorImpl streamAppDesc = this.getTestJoinStreamGraph(new TestJoinFunction()); StreamOperatorTask sot = createStreamOperatorTask(testClock, streamAppDesc); List output = new ArrayList<>(); MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage()); @@ -297,7 +297,7 @@ public void joinRemovesExpiredMessagesReverse() throws Exception { assertTrue(output.isEmpty()); } - private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamAppDescriptorImpl graphSpec) + private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamApplicationDescriptorImpl graphSpec) throws Exception { Map mapConfig = new HashMap<>(); mapConfig.put("job.name", "jobName"); @@ -323,7 +323,7 @@ private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamAppDescri return sot; } - private StreamAppDescriptorImpl getTestJoinStreamGraph(TestJoinFunction joinFn) throws IOException { + private StreamApplicationDescriptorImpl getTestJoinStreamGraph(TestJoinFunction joinFn) throws IOException { Map mapConfig = new HashMap<>(); mapConfig.put("job.name", "jobName"); mapConfig.put("job.id", "jobId"); @@ -331,7 +331,7 @@ private StreamAppDescriptorImpl getTestJoinStreamGraph(TestJoinFunction joinFn) StreamTestUtils.addStreamConfigs(mapConfig, "inStream2", "insystem", "instream2"); Config config = new MapConfig(mapConfig); - return new StreamAppDescriptorImpl(appDesc -> { + return new StreamApplicationDescriptorImpl(appDesc -> { IntegerSerde integerSerde = new IntegerSerde(); KVSerde kvSerde = KVSerde.of(integerSerde, integerSerde); GenericSystemDescriptor sd = new GenericSystemDescriptor("insystem", "mockFactoryClassName"); diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java index 2e262f24ee..566079bed5 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java @@ -24,7 +24,7 @@ import java.util.Collection; import java.util.Collections; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.data.TestMessageEnvelope; import org.apache.samza.operators.data.TestOutputMessageEnvelope; import org.apache.samza.operators.functions.FilterFunction; @@ -71,7 +71,7 @@ public class TestMessageStreamImpl { @Test public void testMap() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -96,7 +96,7 @@ public void testMap() { @Test public void testFlatMap() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -113,7 +113,7 @@ public void testFlatMap() { @Test public void testFlatMapWithRelaxedTypes() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -133,7 +133,7 @@ public void testFlatMapWithRelaxedTypes() { @Test public void testFilter() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -158,7 +158,7 @@ public void testFilter() { @Test public void testSink() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -175,7 +175,7 @@ public void testSink() { @Test public void testSendTo() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStreamImpl inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); OutputStreamImpl mockOutputStreamImpl = mock(OutputStreamImpl.class); @@ -201,7 +201,7 @@ public void testSendTo() { @Test public void testPartitionBy() throws IOException { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); String mockOpName = "mockName"; when(mockGraph.getNextOpId(anyObject(), anyObject())).thenReturn(mockOpName); @@ -232,7 +232,7 @@ public void testPartitionBy() throws IOException { @Test public void testRepartitionWithoutSerde() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); String mockOpName = "mockName"; when(mockGraph.getNextOpId(anyObject(), anyObject())).thenReturn(mockOpName); @@ -262,7 +262,7 @@ public void testRepartitionWithoutSerde() { @Test public void testWindowWithRelaxedTypes() throws Exception { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec = mock(OperatorSpec.class); MessageStream inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec); @@ -286,7 +286,7 @@ public void testWindowWithRelaxedTypes() throws Exception { @Test public void testJoin() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec leftInputOpSpec = mock(OperatorSpec.class); MessageStreamImpl source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec); OperatorSpec rightInputOpSpec = mock(OperatorSpec.class); @@ -318,7 +318,7 @@ public void testJoin() { @Test public void testSendToTable() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec inputOpSpec = mock(OperatorSpec.class); MessageStreamImpl source = new MessageStreamImpl<>(mockGraph, inputOpSpec); @@ -340,7 +340,7 @@ public void testSendToTable() { @Test public void testStreamTableJoin() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec leftInputOpSpec = mock(OperatorSpec.class); MessageStreamImpl> source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec); OperatorSpec rightInputOpSpec = mock(OperatorSpec.class); @@ -368,7 +368,7 @@ public void testStreamTableJoin() { @Test public void testMerge() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockOpSpec1 = mock(OperatorSpec.class); MessageStream inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec1); @@ -408,7 +408,7 @@ public void testMerge() { @Test public void testMergeWithRelaxedTypes() { - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); MessageStream inputStream = new MessageStreamImpl<>(mockGraph, mock(OperatorSpec.class)); // other streams have the same message type T as input stream message type M diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java index 5824027317..a5b15b8b1e 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java @@ -27,7 +27,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -58,14 +58,14 @@ @PrepareForTest(OperatorSpec.class) public class TestOperatorSpecGraph { - private StreamAppDescriptorImpl mockAppDesc; + private StreamApplicationDescriptorImpl mockAppDesc; private Map inputOpSpecMap; private Map outputStrmMap; private Set allOpSpecs; @Before public void setUp() { - this.mockAppDesc = mock(StreamAppDescriptorImpl.class); + this.mockAppDesc = mock(StreamApplicationDescriptorImpl.class); /** * Setup two linear transformation pipelines: diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 2be0867ba0..6f8a8bcb81 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -34,7 +34,7 @@ import java.util.function.BiFunction; import java.util.function.Function; import org.apache.samza.Partition; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -219,7 +219,7 @@ public void tearDown() { @Test public void testEmptyChain() { - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { }, mock(Config.class)); + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { }, mock(Config.class)); OperatorImplGraph opGraph = new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mock(TaskContextImpl.class), mock(Clock.class)); assertEquals(0, opGraph.getAllInputOperators().size()); @@ -243,7 +243,7 @@ public void testLinearChain() { StreamTestUtils.addStreamConfigs(configs, outputStreamId, outputSystem, outputPhysicalName); Config config = new MapConfig(configs); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor sd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor = sd.getInputDescriptor(inputStreamId, mock(Serde.class)); GenericOutputDescriptor outputDescriptor = sd.getOutputDescriptor(outputStreamId, mock(Serde.class)); @@ -297,7 +297,7 @@ public void testPartitionByChain() { StreamTestUtils.addStreamConfigs(configs, outputStreamId, outputSystem, outputPhysicalName); Config config = new MapConfig(configs); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor isd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericSystemDescriptor osd = new GenericSystemDescriptor(outputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor = isd.getInputDescriptor(inputStreamId, mock(Serde.class)); @@ -352,7 +352,7 @@ public void testBroadcastChain() { HashMap configMap = new HashMap<>(); StreamTestUtils.addStreamConfigs(configMap, inputStreamId, inputSystem, inputPhysicalName); Config config = new MapConfig(configMap); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor sd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor = sd.getInputDescriptor(inputStreamId, mock(Serde.class)); MessageStream inputStream = appDesc.getInputStream(inputDescriptor); @@ -381,7 +381,7 @@ public void testMergeChain() { HashMap configs = new HashMap<>(); StreamTestUtils.addStreamConfigs(configs, inputStreamId, inputSystem, inputPhysicalName); Config config = new MapConfig(configs); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor sd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor = sd.getInputDescriptor(inputStreamId, mock(Serde.class)); MessageStream inputStream = appDesc.getInputStream(inputDescriptor); @@ -429,7 +429,7 @@ public void testJoinChain() { JoinFunction testJoinFunction = new TestJoinFunction("jobName-jobId-join-j1", (BiFunction & Serializable) (m1, m2) -> KV.of(m1, m2), keyFn, keyFn); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor sd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor1 = sd.getInputDescriptor(inputStreamId1, mock(Serde.class)); GenericInputDescriptor inputDescriptor2 = sd.getInputDescriptor(inputStreamId2, mock(Serde.class)); @@ -495,7 +495,7 @@ public void testOperatorGraphInitAndClose() { TaskContextImpl mockContext = mock(TaskContextImpl.class); when(mockContext.getTaskName()).thenReturn(mockTaskName); when(mockContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap()); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor sd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor1 = sd.getInputDescriptor(inputStreamId1, mock(Serde.class)); GenericInputDescriptor inputDescriptor2 = sd.getInputDescriptor(inputStreamId2, mock(Serde.class)); @@ -591,7 +591,7 @@ public void testGetOutputToInputStreams() { StreamTestUtils.addStreamConfigs(configs, outputStreamId2, outputSystem, outputStreamId2); Config config = new MapConfig(configs); - StreamAppDescriptorImpl graphSpec = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { GenericSystemDescriptor isd = new GenericSystemDescriptor(inputSystem, "mockFactoryClass"); GenericInputDescriptor inputDescriptor1 = isd.getInputDescriptor(inputStreamId1, mock(Serde.class)); GenericInputDescriptor inputDescriptor2 = isd.getInputDescriptor(inputStreamId2, mock(Serde.class)); diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java index 1a9b5dc88b..7d468c94c3 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; import org.apache.samza.Partition; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -545,7 +545,7 @@ public void testEndOfStreamFlushesWithNoTriggerFirings() throws Exception { verify(taskCoordinator, times(1)).shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); } - private StreamAppDescriptorImpl getKeyedTumblingWindowStreamGraph(AccumulationMode mode, + private StreamApplicationDescriptorImpl getKeyedTumblingWindowStreamGraph(AccumulationMode mode, Duration duration, Trigger> earlyTrigger) throws IOException { StreamApplication userApp = appDesc -> { @@ -561,10 +561,10 @@ private StreamAppDescriptorImpl getKeyedTumblingWindowStreamGraph(AccumulationMo }); }; - return new StreamAppDescriptorImpl(userApp, config); + return new StreamApplicationDescriptorImpl(userApp, config); } - private StreamAppDescriptorImpl getTumblingWindowStreamGraph(AccumulationMode mode, + private StreamApplicationDescriptorImpl getTumblingWindowStreamGraph(AccumulationMode mode, Duration duration, Trigger> earlyTrigger) throws IOException { StreamApplication userApp = appDesc -> { KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); @@ -579,10 +579,10 @@ private StreamAppDescriptorImpl getTumblingWindowStreamGraph(AccumulationMode mo }); }; - return new StreamAppDescriptorImpl(userApp, config); + return new StreamApplicationDescriptorImpl(userApp, config); } - private StreamAppDescriptorImpl getKeyedSessionWindowStreamGraph(AccumulationMode mode, Duration duration) throws IOException { + private StreamApplicationDescriptorImpl getKeyedSessionWindowStreamGraph(AccumulationMode mode, Duration duration) throws IOException { StreamApplication userApp = appDesc -> { KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); GenericSystemDescriptor sd = new GenericSystemDescriptor("kafka", "mockFactoryClass"); @@ -596,10 +596,10 @@ private StreamAppDescriptorImpl getKeyedSessionWindowStreamGraph(AccumulationMod }); }; - return new StreamAppDescriptorImpl(userApp, config); + return new StreamApplicationDescriptorImpl(userApp, config); } - private StreamAppDescriptorImpl getAggregateTumblingWindowStreamGraph(AccumulationMode mode, Duration timeDuration, + private StreamApplicationDescriptorImpl getAggregateTumblingWindowStreamGraph(AccumulationMode mode, Duration timeDuration, Trigger earlyTrigger) throws IOException { StreamApplication userApp = appDesc -> { KVSerde kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde()); @@ -618,7 +618,7 @@ private StreamAppDescriptorImpl getAggregateTumblingWindowStreamGraph(Accumulati }); }; - return new StreamAppDescriptorImpl(userApp, config); + return new StreamApplicationDescriptorImpl(userApp, config); } private static class IntegerEnvelope extends IncomingMessageEnvelope { diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java index bcc740dc4e..db7079c8b3 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java @@ -20,7 +20,7 @@ import java.util.Collection; import java.util.Map; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.MessageStream; @@ -105,7 +105,7 @@ public void testPartitionBy() { MapFunction keyFn = m -> m.toString(); MapFunction valueFn = m -> m; KVSerde partitionBySerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(keyFn, valueFn, partitionBySerde, testRepartitionedStreamName); }, mockConfig); @@ -134,7 +134,7 @@ public void testPartitionBy() { public void testPartitionByWithNoSerde() { MapFunction keyFn = m -> m.toString(); MapFunction valueFn = m -> m; - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(keyFn, valueFn, testRepartitionedStreamName); }, mockConfig); @@ -159,7 +159,7 @@ public void testPartitionByWithNoSerde() { @Test public void testCopy() { - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(m -> m.toString(), m -> m, testRepartitionedStreamName); }, mockConfig); @@ -171,7 +171,7 @@ public void testCopy() { @Test(expected = IllegalArgumentException.class) public void testTimerFunctionAsKeyFn() { TimerMapFn keyFn = new TimerMapFn(); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(keyFn, m -> m, "parByKey"); }, mockConfig); @@ -180,7 +180,7 @@ public void testTimerFunctionAsKeyFn() { @Test(expected = IllegalArgumentException.class) public void testWatermarkFunctionAsKeyFn() { WatermarkMapFn keyFn = new WatermarkMapFn(); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(keyFn, m -> m, "parByKey"); }, mockConfig); @@ -189,7 +189,7 @@ public void testWatermarkFunctionAsKeyFn() { @Test(expected = IllegalArgumentException.class) public void testTimerFunctionAsValueFn() { TimerMapFn valueFn = new TimerMapFn(); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); }, mockConfig); @@ -198,7 +198,7 @@ public void testTimerFunctionAsValueFn() { @Test(expected = IllegalArgumentException.class) public void testWatermarkFunctionAsValueFn() { WatermarkMapFn valueFn = new WatermarkMapFn(); - new StreamAppDescriptorImpl(appDesc -> { + new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testinputDescriptor); inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey"); }, mockConfig); diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java index fbe33a6c0d..faba6d844b 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java @@ -21,8 +21,8 @@ import java.util.HashMap; import java.util.Map; import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.TaskAppDescriptor; -import org.apache.samza.application.TaskAppDescriptorImpl; +import org.apache.samza.application.TaskApplicationDescriptor; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.application.TaskApplication; import org.apache.samza.application.MockStreamApplication; import org.apache.samza.config.ApplicationConfig; @@ -72,7 +72,7 @@ public void testTaskClassOnly() { Config config = new MapConfig(configMap); SamzaApplication app = ApplicationClassUtils.fromConfig(config); assertTrue(app instanceof TaskApplication); - TaskAppDescriptorImpl appSpec = new TaskAppDescriptorImpl((TaskApplication) app, config); + TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl((TaskApplication) app, config); assertTrue(appSpec.getTaskFactory().createInstance() instanceof MockStreamTask); } @@ -87,7 +87,7 @@ public void testNoAppClassNoTaskClass() { */ public static class MockTaskApplication implements TaskApplication { @Override - public void describe(TaskAppDescriptor appSpec) { + public void describe(TaskApplicationDescriptor appSpec) { } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 6a61b5ec1c..3bb7847d1d 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -23,10 +23,12 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.AppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.TaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -35,6 +37,8 @@ import org.apache.samza.job.ApplicationStatus; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.execution.LocalJobPlanner; +import org.apache.samza.task.IdentityStreamTask; +import org.apache.samza.task.StreamTaskFactory; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -69,9 +73,8 @@ public void testRunStreamTask() throws Exception { final Map cfgs = new HashMap<>(); cfgs.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); - cfgs.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask"); config = new MapConfig(cfgs); - mockApp = ApplicationClassUtils.fromConfig(config); + mockApp = (TaskApplication) appDesc -> appDesc.setTaskFactory((StreamTaskFactory) () -> new IdentityStreamTask()) ; prepareTest(); StreamProcessor sp = mock(StreamProcessor.class); @@ -183,7 +186,7 @@ public void testWaitForFinishTimesout() { } private void prepareTest() { - AppDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(mockApp, config); + ApplicationDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(mockApp, config); localPlanner = spy(new LocalJobPlanner(appDesc)); runner = spy(new LocalApplicationRunner(appDesc, localPlanner)); } diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 927bbaf925..297016a948 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -22,9 +22,9 @@ import java.util.HashMap; import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; -import org.apache.samza.application.AppDescriptorImpl; -import org.apache.samza.application.StreamAppDescriptorImpl; -import org.apache.samza.application.TaskAppDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; import org.apache.samza.config.MapConfig; @@ -127,10 +127,10 @@ public void testFinalizeTaskFactory() throws NoSuchFieldException, IllegalAccess assertEquals(retFactory, mockAsyncStreamFactory); } - // test getTaskFactory with StreamAppDescriptor + // test getTaskFactory with StreamApplicationDescriptor @Test public void testGetTaskFactoryWithStreamAppDescriptor() { - StreamAppDescriptorImpl mockStreamApp = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockStreamApp = mock(StreamApplicationDescriptorImpl.class); OperatorSpecGraph mockSpecGraph = mock(OperatorSpecGraph.class); when(mockStreamApp.getOperatorSpecGraph()).thenReturn(mockSpecGraph); TaskFactory streamTaskFactory = TaskFactoryUtil.getTaskFactory(mockStreamApp); @@ -140,20 +140,20 @@ public void testGetTaskFactoryWithStreamAppDescriptor() { verify(mockSpecGraph).clone(); } - // test getTaskFactory with TaskAppDescriptor + // test getTaskFactory with TaskApplicationDescriptor @Test public void testGetTaskFactoryWithTaskAppDescriptor() { - TaskAppDescriptorImpl mockTaskApp = mock(TaskAppDescriptorImpl.class); + TaskApplicationDescriptorImpl mockTaskApp = mock(TaskApplicationDescriptorImpl.class); TaskFactory mockTaskFactory = mock(TaskFactory.class); when(mockTaskApp.getTaskFactory()).thenReturn(mockTaskFactory); TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(mockTaskApp); assertEquals(mockTaskFactory, taskFactory); } - // test getTaskFactory with invalid AppDescriptorImpl + // test getTaskFactory with invalid ApplicationDescriptorImpl @Test(expected = IllegalArgumentException.class) public void testGetTaskFactoryWithInvalidAddDescriptorImpl() { - AppDescriptorImpl mockInvalidApp = mock(AppDescriptorImpl.class); + ApplicationDescriptorImpl mockInvalidApp = mock(ApplicationDescriptorImpl.class); TaskFactoryUtil.getTaskFactory(mockInvalidApp); } } 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 36e5402876..9a871d765c 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 @@ -21,7 +21,7 @@ import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.sql.testutil.SamzaSqlQueryParser; import org.apache.samza.sql.translator.QueryTranslator; import org.slf4j.Logger; @@ -36,7 +36,7 @@ public class SamzaSqlApplication implements StreamApplication { private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlApplication.class); @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { try { SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(appDesc.getConfig()); QueryTranslator queryTranslator = new QueryTranslator(sqlConfig); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java index f729a664a6..fe4d8da7a2 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java @@ -30,7 +30,7 @@ import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalProject; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.KV; @@ -57,9 +57,9 @@ /** - * This class is used to populate the {@link StreamAppDescriptor} using the SQL queries. + * This class is used to populate the {@link StreamApplicationDescriptor} using the SQL queries. * This class contains the core of the SamzaSQL control code that converts the SQL statements to calcite relational graph. - * It then walks the relational graph and then populates the Samza's {@link StreamAppDescriptor} accordingly. + * It then walks the relational graph and then populates the Samza's {@link StreamApplicationDescriptor} accordingly. */ public class QueryTranslator { private static final Logger LOG = LoggerFactory.getLogger(QueryTranslator.class); @@ -95,7 +95,7 @@ public QueryTranslator(SamzaSqlApplicationConfig sqlConfig) { this.converters = sqlConfig.getSamzaRelConverters(); } - public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamAppDescriptor appDesc) { + public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplicationDescriptor appDesc) { QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getInputSystemStreamConfigBySource(), sqlConfig.getUdfMetadata()); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java index 6990410096..2dc28be710 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java @@ -23,7 +23,7 @@ import java.util.Map; import org.apache.calcite.rel.core.TableScan; import org.apache.commons.lang.Validate; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -73,7 +73,7 @@ public SamzaSqlRelMessage apply(KV message) { } void translate(final TableScan tableScan, final TranslatorContext context) { - StreamAppDescriptor streamAppDesc = context.getStreamAppDescriptor(); + StreamApplicationDescriptor streamAppDesc = context.getStreamAppDescriptor(); List tableNameParts = tableScan.getTable().getQualifiedName(); String sourceName = SqlIOConfig.getSourceFromSourceParts(tableNameParts); diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java index 5d31775278..a7ab66334a 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java @@ -32,7 +32,7 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.SchemaPlus; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor; import org.apache.samza.sql.data.RexToJavaCompiler; @@ -41,13 +41,13 @@ /** - * State that is maintained while translating the Calcite relational graph to Samza {@link StreamAppDescriptor}. + * State that is maintained while translating the Calcite relational graph to Samza {@link StreamApplicationDescriptor}. */ public class TranslatorContext implements Cloneable { /** * The internal variables that are shared among all cloned {@link TranslatorContext} */ - private final StreamAppDescriptor streamAppDesc; + private final StreamApplicationDescriptor streamAppDesc; private final RexToJavaCompiler compiler; private final Map relSamzaConverters; private final Map messageStreams; @@ -138,7 +138,7 @@ private TranslatorContext(TranslatorContext other) { * @param executionContext the execution context * @param converters the map of schema to RelData converters */ - TranslatorContext(StreamAppDescriptor stramAppDesc, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map converters) { + TranslatorContext(StreamApplicationDescriptor stramAppDesc, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map converters) { this.streamAppDesc = stramAppDesc; this.compiler = createExpressionCompiler(relRoot); this.executionContext = executionContext; @@ -154,7 +154,7 @@ private TranslatorContext(TranslatorContext other) { * * @return the stream graph */ - public StreamAppDescriptor getStreamAppDescriptor() { + public StreamApplicationDescriptor getStreamAppDescriptor() { return streamAppDesc; } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java index 148ce72772..e7c2195218 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java @@ -25,7 +25,7 @@ import org.apache.calcite.DataContext; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.logical.LogicalFilter; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; @@ -73,7 +73,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { when(mockFilter.getInput()).thenReturn(mockInput); when(mockInput.getId()).thenReturn(1); when(mockFilter.getId()).thenReturn(2); - StreamAppDescriptorImpl mockGraph = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockGraph = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockInputOp = mock(OperatorSpec.class); MessageStream mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp); when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java index d0e8bb76ef..f0a8a89fef 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java @@ -33,7 +33,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.TableDescriptor; @@ -132,7 +132,7 @@ public void testTranslateStreamToTableJoin() throws IOException, ClassNotFoundEx when(mockRightInput.getRowType()).thenReturn(mockRightRowType); when(mockRightRowType.getFieldNames()).thenReturn(rightStreamFieldNames); - StreamAppDescriptorImpl mockAppDesc = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockAppDesc = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockLeftInputOp = mock(OperatorSpec.class); MessageStream mockLeftInputStream = new MessageStreamImpl<>(mockAppDesc, mockLeftInputOp); when(mockContext.getMessageStream(eq(mockLeftInput.getId()))).thenReturn(mockLeftInputStream); diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java index 5409a222a9..1acfc4795c 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java @@ -32,7 +32,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.validate.SqlUserDefinedFunction; import org.apache.calcite.util.Pair; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.container.TaskName; @@ -91,7 +91,7 @@ public void testTranslate() throws IOException, ClassNotFoundException { List> namedProjects = new ArrayList<>(); namedProjects.add(Pair.of(mockRexField, "test_field")); when(mockProject.getNamedProjects()).thenReturn(namedProjects); - StreamAppDescriptorImpl mockAppDesc = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockAppDesc = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockInputOp = mock(OperatorSpec.class); MessageStream mockStream = new MessageStreamImpl<>(mockAppDesc, mockInputOp); when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream); @@ -183,7 +183,7 @@ public void testTranslateWithFlatten() throws IOException, ClassNotFoundExceptio flattenProjects.add(mockFlattenProject); when(mockProject.getProjects()).thenReturn(flattenProjects); - StreamAppDescriptorImpl mockAppDesc = mock(StreamAppDescriptorImpl.class); + StreamApplicationDescriptorImpl mockAppDesc = mock(StreamApplicationDescriptorImpl.class); OperatorSpec mockInputOp = new OperatorSpec(OperatorSpec.OpCode.INPUT, "1") { @Override diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java index e0a2e1e475..c9f59e6ad0 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java @@ -24,7 +24,7 @@ import java.util.HashSet; import java.util.Map; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; @@ -84,7 +84,7 @@ public void testTranslate() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl appDesc = new StreamAppDescriptorImpl(descriptor -> { },samzaConfig); + StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(descriptor -> { },samzaConfig); translator.translate(queryInfo, appDesc); OperatorSpecGraph specGraph = appDesc.getOperatorSpecGraph(); @@ -108,7 +108,7 @@ public void testTranslate() { validatePerTaskContextInit(appDesc, samzaConfig); } - private void validatePerTaskContextInit(StreamAppDescriptorImpl appDesc, Config samzaConfig) { + private void validatePerTaskContextInit(StreamApplicationDescriptorImpl appDesc, Config samzaConfig) { // make sure that each task context would have a separate instance of cloned TranslatorContext TaskContextImpl testContext = new TaskContextImpl(new TaskName("Partition 1"), null, null, new HashSet<>(), null, null, null, null, null, null); @@ -136,7 +136,7 @@ public void testTranslateComplex() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); @@ -168,7 +168,7 @@ public void testTranslateSubQuery() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); @@ -204,7 +204,7 @@ public void testTranslateStreamTableJoinWithoutJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -223,7 +223,7 @@ public void testTranslateStreamTableJoinWithFullJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -242,7 +242,7 @@ public void testTranslateStreamTableJoinWithSelfJoinOperator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -261,7 +261,7 @@ public void testTranslateStreamTableJoinWithThetaCondition() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -277,7 +277,7 @@ public void testTranslateStreamTableCrossJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -295,7 +295,7 @@ public void testTranslateStreamTableJoinWithAndLiteralCondition() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -314,7 +314,7 @@ public void testTranslateStreamTableJoinWithSubQuery() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -332,7 +332,7 @@ public void testTranslateTableTableJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -350,7 +350,7 @@ public void testTranslateStreamStreamJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -368,7 +368,7 @@ public void testTranslateJoinWithIncorrectLeftJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -386,7 +386,7 @@ public void testTranslateJoinWithIncorrectRightJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -408,7 +408,7 @@ public void testTranslateStreamTableInnerJoinWithMissingStream() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -426,7 +426,7 @@ public void testTranslateStreamTableInnerJoinWithUdf() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } @@ -444,7 +444,7 @@ public void testTranslateStreamTableInnerJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); @@ -497,7 +497,7 @@ public void testTranslateStreamTableLeftJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); @@ -551,7 +551,7 @@ public void testTranslateStreamTableRightJoin() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); @@ -604,7 +604,7 @@ public void testTranslateGroupBy() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph(); @@ -628,7 +628,7 @@ public void testTranslateGroupByWithSumAggregator() { SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config)); QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig); SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0); - StreamAppDescriptorImpl streamAppDesc = new StreamAppDescriptorImpl(appDesc -> { }, samzaConfig); + StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig); translator.translate(queryInfo, streamAppDesc); } } 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 fbea9dfdef..7d5e0d2482 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 @@ -21,7 +21,7 @@ import java.time.Duration; import java.util.HashMap; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.triggers.Triggers; @@ -55,7 +55,7 @@ public static void main(String[] args) { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = 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 7b5d70ef0d..4ef2402576 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 @@ -20,7 +20,7 @@ package org.apache.samza.example; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KVSerde serde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor> 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 b304eaacb7..dfc4b42b35 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 @@ -23,7 +23,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -58,7 +58,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = 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 b06a8bbc1a..fe018f3f63 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 @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.runtime.ApplicationRunner; @@ -49,7 +49,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KVSerde serde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)); KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); 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 626f814718..8d3812bcc6 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 @@ -20,7 +20,7 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.functions.JoinFunction; @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor orderStreamDescriptor = 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 c2942254e7..b540585323 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 @@ -19,7 +19,7 @@ package org.apache.samza.example; import java.time.Duration; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; @@ -59,7 +59,7 @@ public static void main(String[] args) { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = 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 3d2bfade38..8a0ca287a6 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 @@ -20,7 +20,7 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -54,7 +54,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = 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 b12ca327c0..73dc10a4a4 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 @@ -18,15 +18,12 @@ */ package org.apache.samza.example; -import java.io.Serializable; -import org.apache.samza.application.TaskAppDescriptor; +import org.apache.samza.application.TaskApplicationDescriptor; import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.runtime.ApplicationRunners; -import org.apache.samza.serializers.KVSerde; -import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.RocksDbTableDescriptor; import org.apache.samza.system.IncomingMessageEnvelope; @@ -63,7 +60,7 @@ public static void main(String[] args) { } @Override - public void describe(TaskAppDescriptor appDesc) { + public void describe(TaskApplicationDescriptor appDesc) { // add input and output streams KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor isd = ksd.getInputDescriptor("myinput", new StringSerde()); 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 155bf87805..2f4c19c028 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 @@ -21,7 +21,7 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; @@ -57,7 +57,7 @@ public static void main(String[] args) throws Exception { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { KafkaSystemDescriptor trackingSystem = new KafkaSystemDescriptor("tracking"); KafkaInputDescriptor inputStreamDescriptor = 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 a326b58733..1954cc3c89 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 @@ -19,7 +19,7 @@ package org.apache.samza.test.integration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.KV; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.NoOpSerde; @@ -37,7 +37,7 @@ public class TestStandaloneIntegrationApplication implements StreamApplication { private static final Logger LOGGER = LoggerFactory.getLogger(TestStandaloneIntegrationApplication.class); @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { String systemName = "testSystemName"; String inputStreamName = appDesc.getConfig().get("input.stream.name"); String outputStreamName = "standaloneIntegrationTestKafkaOutputTopic"; 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 372a5634d9..d2aab116f5 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 @@ -24,7 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Random; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; @@ -99,7 +99,7 @@ public void testPipeline() throws Exception { class PipelineApplication implements StreamApplication { @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { DelegatingSystemDescriptor sd = new DelegatingSystemDescriptor("test"); GenericInputDescriptor> isd = sd.getInputDescriptor("PageView", KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); 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 ddea2f1941..ea90c10120 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 @@ -29,7 +29,7 @@ import java.util.Map; import java.util.Set; import org.apache.samza.Partition; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -153,7 +153,7 @@ public void testWatermark() throws Exception { class TestStreamApp implements StreamApplication { @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { DelegatingSystemDescriptor sd = new DelegatingSystemDescriptor("test"); GenericInputDescriptor> isd = sd.getInputDescriptor("PageView", KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>())); 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 449ec34fe5..4caf2664c5 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 @@ -21,7 +21,7 @@ import java.util.Arrays; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.JsonSerdeV2; @@ -35,7 +35,7 @@ public class BroadcastAssertApp implements StreamApplication { @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { Config config = appDesc.getConfig(); String inputTopic = config.get(INPUT_TOPIC_NAME_PROP); diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java b/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java index 81feea4679..e72a965d26 100644 --- a/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/framework/TestTimerApp.java @@ -25,7 +25,7 @@ import java.util.Collections; import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.FlatMapFunction; @@ -39,7 +39,7 @@ public class TestTimerApp implements StreamApplication { public static final String PAGE_VIEWS = "page-views"; @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("kafka"); KafkaInputDescriptor isd = ksd.getInputDescriptor(PAGE_VIEWS, 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 a0c5fa9604..c63c11f299 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 @@ -23,7 +23,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; @@ -55,7 +55,7 @@ public class RepartitionJoinWindowApp implements StreamApplication { private final List intermediateStreamIds = new ArrayList<>(); @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { // 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(); 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 ff7bd8a6c2..79a25e77d0 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 @@ -21,7 +21,7 @@ import java.time.Duration; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.KV; import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.IntegerSerde; @@ -47,7 +47,7 @@ public class RepartitionWindowApp implements StreamApplication { @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { 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); 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 3cab1f4e25..f116f1d8a5 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 @@ -20,7 +20,7 @@ package org.apache.samza.test.operator; import java.time.Duration; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; @@ -57,7 +57,7 @@ public static void main(String[] args) { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { JsonSerdeV2 inputSerde = new JsonSerdeV2<>(PageView.class); KVSerde outputSerde = KVSerde.of(new StringSerde(), new IntegerSerde()); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); 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 a09a65fe07..018401337f 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 @@ -20,7 +20,7 @@ package org.apache.samza.test.operator; import java.time.Duration; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.Config; import org.apache.samza.operators.KV; @@ -59,7 +59,7 @@ public static void main(String[] args) { } @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { JsonSerdeV2 inputSerde = new JsonSerdeV2<>(PageView.class); KVSerde outputSerde = KVSerde.of(new StringSerde(), new IntegerSerde()); KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM); 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 4379361d1a..51f33b5425 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 @@ -23,7 +23,7 @@ import java.io.ObjectInputStream; import java.io.Serializable; import java.util.concurrent.CountDownLatch; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; @@ -59,7 +59,7 @@ private TestStreamApplication(String systemName, String inputTopic, String outpu } @Override - public void describe(StreamAppDescriptor streamAppDesc) { + public void describe(StreamApplicationDescriptor streamAppDesc) { KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(systemName); KafkaInputDescriptor isd = ksd.getInputDescriptor(inputTopic, new NoOpSerde<>()); KafkaOutputDescriptor osd = ksd.getOutputDescriptor(outputTopic, new StringSerde()); 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 b764302dfe..be1c02de33 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 @@ -27,7 +27,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -125,7 +125,7 @@ static class PageViewProfileJoin implements StreamApplication { static final String PROFILE_TABLE = "profile-table"; @Override - public void describe(StreamAppDescriptor appDesc) { + public void describe(StreamApplicationDescriptor appDesc) { Table> table = appDesc.getTable(getTableDescriptor()); KafkaSystemDescriptor sd = new KafkaSystemDescriptor(appDesc.getConfig().get(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), PAGEVIEW_STREAM))); diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java index 80db0a016a..d79683e698 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java @@ -34,7 +34,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import org.apache.samza.SamzaException; -import org.apache.samza.application.StreamAppDescriptor; +import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.application.StreamApplication; import org.apache.samza.config.MapConfig; import org.apache.samza.container.SamzaContainerContext; @@ -126,7 +126,7 @@ public CompletableFuture deleteAsync(Integer key) { } } - private Table> getCachingTable(Table> actualTable, boolean defaultCache, String id, StreamAppDescriptor appDesc) { + private Table> getCachingTable(Table> actualTable, boolean defaultCache, String id, StreamApplicationDescriptor appDesc) { CachingTableDescriptor cachingDesc = new CachingTableDescriptor<>("caching-table-" + id); if (defaultCache) { cachingDesc.withReadTtl(Duration.ofMinutes(5)); From 7a73992a50a5850f1924e7cba8b67382c5b33912 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 31 Aug 2018 00:14:29 -0700 Subject: [PATCH 21/38] SAMZA-1789: fixing checkstyle and javadoc errors --- .../java/org/apache/samza/operators/MessageStream.java | 5 ++--- .../apache/samza/operators/functions/ClosableFunction.java | 3 +-- .../apache/samza/operators/functions/InitableFunction.java | 3 +-- .../org/apache/samza/runtime/TestApplicationRunners.java | 1 - .../main/java/org/apache/samza/execution/JobPlanner.java | 3 --- .../java/org/apache/samza/operators/spec/OperatorSpec.java | 3 +-- .../org/apache/samza/runtime/LocalContainerRunner.java | 1 - .../java/org/apache/samza/task/StreamOperatorTask.java | 7 ++----- .../application/TestTaskApplicationDescriptorImpl.java | 1 - .../apache/samza/runtime/TestApplicationRunnerMain.java | 1 - .../apache/samza/runtime/TestLocalApplicationRunner.java | 3 +-- 11 files changed, 8 insertions(+), 23 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index 26fc91af26..a7935d330f 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -23,7 +23,6 @@ import java.util.Collection; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.JoinFunction; @@ -41,7 +40,7 @@ * A stream of messages that can be transformed into another {@link MessageStream}. *

    * A {@link MessageStream} corresponding to an input stream can be obtained using - * {@link StreamApplicationDescriptor#getInputStream}. + * {@link org.apache.samza.application.StreamApplicationDescriptor#getInputStream}. * * @param the type of messages in this stream */ @@ -215,7 +214,7 @@ static MessageStream mergeAll(Collection * Uses the provided {@link KVSerde} for serialization of keys and values. If the provided {@code serde} is null, diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java index 089f4e5938..12823cc7b9 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java @@ -20,7 +20,6 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationDescriptor; /** @@ -29,7 +28,7 @@ *

    Implement {@link #close()} to free resources used during the execution of the function, clean up state etc. * *

    Order of finalization: {@link ClosableFunction}s are closed in the reverse topological order of operators in the - * {@link StreamApplicationDescriptor}. For any two operators A and B in the graph, if operator B + * {@link org.apache.samza.application.StreamApplicationDescriptor}. For any two operators A and B in the graph, if operator B * consumes results from operator A, then operator B is guaranteed to be closed before operator A. * */ diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index f0c5d239c9..8a5d83be55 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -20,7 +20,6 @@ package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationDescriptor; import org.apache.samza.config.Config; import org.apache.samza.task.TaskContext; @@ -28,7 +27,7 @@ * A function that can be initialized before execution. * *

    Order of initialization: {@link InitableFunction}s are invoked in the topological order of operators in the - * {@link StreamApplicationDescriptor}. For any two operators A and B in the graph, if operator B + * {@link org.apache.samza.application.StreamApplicationDescriptor}. For any two operators A and B in the graph, if operator B * consumes results from operator A, then operator A is guaranteed to be initialized before operator B. * */ diff --git a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java index 34e5ecce9f..5829cf7b4d 100644 --- a/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java +++ b/samza-api/src/test/java/org/apache/samza/runtime/TestApplicationRunners.java @@ -26,7 +26,6 @@ import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.job.ApplicationStatus; -import org.apache.samza.metrics.MetricsReporter; import org.junit.Test; import static org.junit.Assert.assertTrue; diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index e1604e37a0..6eb2ac627a 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -42,9 +42,6 @@ import org.apache.samza.config.StreamConfig; import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; -import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; -import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.table.TableConfigGenerator; import org.apache.samza.table.TableSpec; import org.slf4j.Logger; diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index fcf343ae43..1e021f564c 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -23,7 +23,6 @@ import java.util.LinkedHashSet; import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.functions.TimerFunction; @@ -104,7 +103,7 @@ public final OpCode getOpCode() { } /** - * Get the unique ID of this operator in the {@link StreamApplicationDescriptorImpl}. + * Get the unique ID of this operator in the {@link org.apache.samza.application.StreamApplicationDescriptorImpl}. * @return the unique operator ID */ public final String getOpId() { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 45529f966b..8df9f892f1 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -37,7 +37,6 @@ import org.apache.samza.container.SamzaContainerListener; import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.MetricsReporter; -import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.task.TaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.SamzaUncaughtExceptionHandler; diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java index 032074dbd9..2ca4e81f0b 100644 --- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java +++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java @@ -18,8 +18,6 @@ */ package org.apache.samza.task; -import org.apache.samza.application.StreamApplicationDescriptor; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.system.EndOfStreamMessage; @@ -71,10 +69,9 @@ public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextMan * Initializes this task during startup. *

    * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}. - * The {@link StreamApplicationDescriptorImpl} sets the input and output streams and the task-wide - * context manager using the {@link StreamApplicationDescriptor} APIs, + * Users set the input and output streams and the task-wide context manager using {@link org.apache.samza.application.StreamApplicationDescriptor} APIs, * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. After the - * {@link StreamApplicationDescriptorImpl} is initialized once by the application, it then creates + * {@link org.apache.samza.application.StreamApplicationDescriptorImpl} is initialized once by the application, it then creates * an immutable {@link OperatorSpecGraph} accordingly, which is passed in to this class to create the {@link OperatorImplGraph} * corresponding to the logical DAG. * diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java index 826b622638..9418c1f056 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java @@ -34,7 +34,6 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java index a27cfc7b6c..cfa2680144 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java @@ -20,7 +20,6 @@ import java.time.Duration; import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.MockStreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 3bb7847d1d..81cce5698c 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -33,7 +33,6 @@ import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; -import org.apache.samza.config.TaskConfig; import org.apache.samza.job.ApplicationStatus; import org.apache.samza.processor.StreamProcessor; import org.apache.samza.execution.LocalJobPlanner; @@ -74,7 +73,7 @@ public void testRunStreamTask() final Map cfgs = new HashMap<>(); cfgs.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); config = new MapConfig(cfgs); - mockApp = (TaskApplication) appDesc -> appDesc.setTaskFactory((StreamTaskFactory) () -> new IdentityStreamTask()) ; + mockApp = (TaskApplication) appDesc -> appDesc.setTaskFactory((StreamTaskFactory) () -> new IdentityStreamTask()); prepareTest(); StreamProcessor sp = mock(StreamProcessor.class); From 222abf21f13b44727b7296fe26a02bfe15f77bd0 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 31 Aug 2018 01:08:31 -0700 Subject: [PATCH 22/38] SAMZA-1789: added a constructor to StreamProcessor to take a StreamProcessorListenerFactory --- .../samza/processor/StreamProcessor.java | 36 ++++++++++++++++ .../samza/runtime/LocalApplicationRunner.java | 24 ++++------- .../samza/processor/TestStreamProcessor.java | 42 +++++++++++++------ .../runtime/TestLocalApplicationRunner.java | 16 +++---- 4 files changed, 83 insertions(+), 35 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 82657891ab..9a5eadb53c 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 @@ -189,6 +189,34 @@ public StreamProcessor(Config config, Map customMetrics this.processorId = this.jobCoordinator.getProcessorId(); } + /** + * Same as {@link #StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener, JobCoordinator)}, except + * there is a {@link StreamProcessorListenerFactory} as input instead of {@link ProcessorLifecycleListener}. + * This is useful to create a {@link ProcessorLifecycleListener} with a reference to this {@link StreamProcessor} + * + * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer} + * @param customMetricsReporters metric Reporter + * @param taskFactory task factory to instantiate the Task + * @param listenerFactory listener to the StreamProcessor life cycle + * @param jobCoordinator the instance of {@link JobCoordinator} + */ + public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, + StreamProcessorListenerFactory listenerFactory, JobCoordinator jobCoordinator) { + Preconditions.checkNotNull(listenerFactory, "StreamProcessorListenerFactory cannot be null."); + this.taskFactory = taskFactory; + this.config = config; + this.taskShutdownMs = new TaskConfigJava(config).getShutdownMs(); + this.customMetricsReporter = customMetricsReporters; + this.jobCoordinator = (jobCoordinator != null) ? jobCoordinator : createJobCoordinator(); + this.jobCoordinatorListener = createJobCoordinatorListener(); + this.jobCoordinator.setListener(jobCoordinatorListener); + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(CONTAINER_THREAD_NAME_FORMAT).setDaemon(true).build(); + this.executorService = Executors.newSingleThreadExecutor(threadFactory); + // TODO: remove the dependency on jobCoordinator for processorId after fixing SAMZA-1835 + this.processorId = this.jobCoordinator.getProcessorId(); + this.processorListener = listenerFactory.createInstance(this); + } + /** * Asynchronously starts this {@link StreamProcessor}. *

    @@ -368,6 +396,14 @@ public void onCoordinatorFailure(Throwable throwable) { }; } + /** + * Interface to create a {@link ProcessorLifecycleListener} + */ + @FunctionalInterface + public interface StreamProcessorListenerFactory { + ProcessorLifecycleListener createInstance(StreamProcessor processor); + } + class ContainerListener implements SamzaContainerListener { @Override diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 3ccce3d6ef..8986e631f7 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -96,9 +96,8 @@ public void run() { } jobConfigs.forEach(jobConfig -> { LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig); - LocalStreamProcessorLifecycleListener localListener = new LocalStreamProcessorLifecycleListener(jobConfig); - StreamProcessor processor = createStreamProcessor(jobConfig, appDesc, localListener); - localListener.setProcessor(processor); + StreamProcessor processor = createStreamProcessor(jobConfig, appDesc, + sp -> new LocalStreamProcessorLifecycleListener(sp, jobConfig)); processors.add(processor); }); numProcessorsToStart.set(processors.size()); @@ -163,21 +162,21 @@ CountDownLatch getShutdownLatch() { /* package private */ StreamProcessor createStreamProcessor(Config config, ApplicationDescriptorImpl appDesc, - LocalStreamProcessorLifecycleListener listener) { + StreamProcessor.StreamProcessorListenerFactory listenerFactory) { TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); Map reporters = new HashMap<>(); // TODO: the null processorId has to be fixed after SAMZA-1835 appDesc.getMetricsReporterFactories().forEach((name, factory) -> reporters.put(name, factory.getMetricsReporter(name, null, config))); - return new StreamProcessor(config, reporters, taskFactory, listener, null); + return new StreamProcessor(config, reporters, taskFactory, listenerFactory, null); } /** * Defines a specific implementation of {@link ProcessorLifecycleListener} for local {@link StreamProcessor}s. */ final class LocalStreamProcessorLifecycleListener implements ProcessorLifecycleListener { - private StreamProcessor processor; - private ProcessorLifecycleListener processorLifecycleListener; + private final StreamProcessor processor; + private final ProcessorLifecycleListener processorLifecycleListener; @Override public void beforeStart() { @@ -195,7 +194,6 @@ public void afterStart() { @Override public void afterStop() { processors.remove(processor); - processor = null; // successful shutdown handleProcessorShutdown(null); @@ -204,7 +202,6 @@ public void afterStop() { @Override public void afterFailure(Throwable t) { processors.remove(processor); - processor = null; // the processor stopped with failure, this is logging the first processor's failure as the cause of // the whole application failure @@ -217,12 +214,9 @@ public void afterFailure(Throwable t) { handleProcessorShutdown(t); } - LocalStreamProcessorLifecycleListener(Config jobConfig) { - this.processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(new ProcessorContext() { - }, jobConfig); - } - - private void setProcessor(StreamProcessor processor) { + LocalStreamProcessorLifecycleListener(StreamProcessor processor, Config jobConfig) { + this.processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory() + .createInstance(new ProcessorContext() { }, jobConfig); this.processor = processor; } diff --git a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java index b407b5a2fc..673015aa56 100644 --- a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java +++ b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java @@ -40,6 +40,7 @@ import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.task.StreamTask; import org.apache.samza.task.StreamTaskFactory; +import org.apache.samza.task.TaskFactory; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -47,6 +48,7 @@ import org.mockito.Mockito; import org.powermock.api.mockito.PowerMockito; +import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -201,7 +203,7 @@ public void beforeStart() { processor.start(); processorListenerStart.await(); - Assert.assertEquals(SamzaContainerStatus.STARTED, processor.getContainerStatus()); + assertEquals(SamzaContainerStatus.STARTED, processor.getContainerStatus()); // This block is required for the mockRunloop is actually start. // Otherwise, processor.stop gets triggered before mockRunloop begins to block @@ -308,7 +310,7 @@ public void afterFailure(Throwable t) { Assert.assertTrue( "Container failed and processor listener failed was not invoked within timeout!", processorListenerFailed.await(30, TimeUnit.SECONDS)); - Assert.assertEquals(expectedThrowable, actualThrowable.get()); + assertEquals(expectedThrowable, actualThrowable.get()); Assert.assertTrue(processorListenerState.get(ListenerCallback.BEFORE_START)); Assert.assertTrue(processorListenerState.get(ListenerCallback.AFTER_START)); @@ -322,14 +324,14 @@ public void testStartOperationShouldBeIdempotent() { Mockito.doNothing().when(mockJobCoordinator).start(); ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); StreamProcessor streamProcessor = new StreamProcessor(new MapConfig(), new HashMap<>(), null, lifecycleListener, mockJobCoordinator); - Assert.assertEquals(State.NEW, streamProcessor.getState()); + assertEquals(State.NEW, streamProcessor.getState()); streamProcessor.start(); - Assert.assertEquals(State.STARTED, streamProcessor.getState()); + assertEquals(State.STARTED, streamProcessor.getState()); streamProcessor.start(); - Assert.assertEquals(State.STARTED, streamProcessor.getState()); + assertEquals(State.STARTED, streamProcessor.getState()); Mockito.verify(mockJobCoordinator, Mockito.times(1)).start(); } @@ -349,11 +351,11 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() { streamProcessor.start(); - Assert.assertEquals(State.STARTED, streamProcessor.getState()); + assertEquals(State.STARTED, streamProcessor.getState()); streamProcessor.jobCoordinatorListener.onJobModelExpired(); - Assert.assertEquals(State.IN_REBALANCE, streamProcessor.getState()); + assertEquals(State.IN_REBALANCE, streamProcessor.getState()); /** * When there's initialized SamzaContainer in StreamProcessor and the container shutdown @@ -372,7 +374,7 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() { streamProcessor.jobCoordinatorListener.onJobModelExpired(); - Assert.assertEquals(State.STOPPING, streamProcessor.getState()); + assertEquals(State.STOPPING, streamProcessor.getState()); Mockito.verify(mockSamzaContainer, Mockito.times(1)).shutdown(); Mockito.verify(mockJobCoordinator, Mockito.times(1)).stop(); @@ -381,7 +383,7 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() { streamProcessor.jobCoordinatorListener.onJobModelExpired(); - Assert.assertEquals(State.IN_REBALANCE, streamProcessor.state); + assertEquals(State.IN_REBALANCE, streamProcessor.state); } @Test @@ -420,7 +422,7 @@ public void testStopShouldBeIdempotent() { streamProcessor.stop(); - Assert.assertEquals(State.STOPPING, streamProcessor.state); + assertEquals(State.STOPPING, streamProcessor.state); } @Test @@ -440,7 +442,7 @@ public void testCoordinatorFailureShouldStopTheStreamProcessor() { Mockito.when(mockSamzaContainer.hasStopped()).thenReturn(false); - Assert.assertEquals(State.STOPPED, streamProcessor.state); + assertEquals(State.STOPPED, streamProcessor.state); Mockito.verify(lifecycleListener).afterFailure(failureException); Mockito.verify(mockSamzaContainer).shutdown(); } @@ -455,7 +457,23 @@ public void testCoordinatorStopShouldStopTheStreamProcessor() { streamProcessor.state = State.RUNNING; streamProcessor.jobCoordinatorListener.onCoordinatorStop(); - Assert.assertEquals(State.STOPPED, streamProcessor.state); + assertEquals(State.STOPPED, streamProcessor.state); Mockito.verify(lifecycleListener).afterStop(); } + + @Test + public void testStreamProcessorWithStreamProcessorListenerFactory() { + AtomicReference mockListener = new AtomicReference<>(); + StreamProcessor streamProcessor = new StreamProcessor(mock(Config.class), new HashMap<>(), mock(TaskFactory.class), + sp -> mockListener.updateAndGet(old -> new MockStreamProcessorLifecycleListener(sp)), mock(JobCoordinator.class)); + assertEquals(streamProcessor, mockListener.get().processor); + } + + class MockStreamProcessorLifecycleListener implements ProcessorLifecycleListener { + final StreamProcessor processor; + + MockStreamProcessorLifecycleListener(StreamProcessor processor) { + this.processor = processor; + } + } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 81cce5698c..ecbe33a026 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -78,12 +78,12 @@ public void testRunStreamTask() StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(LocalApplicationRunner.LocalStreamProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerFactory.class); doAnswer(i -> { - ProcessorLifecycleListener listener = captor.getValue(); + ProcessorLifecycleListener listener = captor.getValue().createInstance(sp); listener.afterStart(); listener.afterStop(); return null; @@ -113,12 +113,12 @@ public void testRunComplete() doReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))).when(localPlanner).prepareJobs(); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(LocalApplicationRunner.LocalStreamProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerFactory.class); doAnswer(i -> { - ProcessorLifecycleListener listener = captor.getValue(); + ProcessorLifecycleListener listener = captor.getValue().createInstance(sp); listener.afterStart(); listener.afterStop(); return null; @@ -148,8 +148,8 @@ public void testRunFailure() doReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))).when(localPlanner).prepareJobs(); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(LocalApplicationRunner.LocalStreamProcessorLifecycleListener.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerFactory.class); doAnswer(i -> { From 02076c850997292d6ef29b3d16d862d9339d5673 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 31 Aug 2018 02:12:12 -0700 Subject: [PATCH 23/38] SAMZA-1789: fixed the modifier for the mandatory constructor of ApplicationRunner; Disabled three tests due to wrong configure for test systems --- .../java/org/apache/samza/runtime/ApplicationRunner.java | 3 +++ .../org/apache/samza/runtime/LocalApplicationRunner.java | 2 +- .../org/apache/samza/runtime/RemoteApplicationRunner.java | 4 ++-- .../test/controlmessages/WatermarkIntegrationTest.java | 2 +- .../test/framework/StreamApplicationIntegrationTest.java | 3 ++- .../samza/test/table/TestLocalTableWithSideInputs.java | 6 ++++-- 6 files changed, 13 insertions(+), 7 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java index de4cb60b2b..409482f0ca 100644 --- a/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java +++ b/samza-api/src/main/java/org/apache/samza/runtime/ApplicationRunner.java @@ -25,6 +25,9 @@ /** * The primary means of managing execution of the {@link org.apache.samza.application.SamzaApplication} at runtime. + * + *

    + * Implementation of {@link ApplicationRunner} class must have a public default constructor #ApplicationRunner(SamzaApplication, Config) */ @InterfaceStability.Evolving public interface ApplicationRunner { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 8986e631f7..580f71a014 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -65,7 +65,7 @@ public class LocalApplicationRunner implements ApplicationRunner { private ApplicationStatus appStatus = ApplicationStatus.New; /** - * Default constructor that is required by any implementation of {@link ApplicationRunner} + * Default public constructor that is required by any implementation of {@link ApplicationRunner} * * @param userApp user application * @param config user configuration diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index d9a26e9218..e2cbfdcd34 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -49,12 +49,12 @@ public class RemoteApplicationRunner implements ApplicationRunner { private final RemoteJobPlanner planner; /** - * Default constructor that is required by any implementation of {@link ApplicationRunner} + * Default public constructor that is required by any implementation of {@link ApplicationRunner} * * @param userApp user application * @param config user configuration */ - RemoteApplicationRunner(SamzaApplication userApp, Config config) { + public RemoteApplicationRunner(SamzaApplication userApp, Config config) { this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); this.planner = new RemoteJobPlanner(appDesc); } 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 ea90c10120..05818e96b9 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 @@ -223,7 +223,7 @@ OperatorImpl getOperator(OperatorImplGraph graph, OperatorSpec.OpCode opCode) { return null; } - class MockLocalApplicationRunner extends LocalApplicationRunner { + public static class MockLocalApplicationRunner extends LocalApplicationRunner { /** * Default constructor that is required by any implementation of {@link ApplicationRunner} 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 1000f22e1b..fcd2b0f6f6 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 @@ -69,7 +69,8 @@ public class StreamApplicationIntegrationTest { private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"}; - @Test + // @Test + // TODO: fix after the system descriptor is using InMemorySystemFactory public void testHighLevelApi() throws Exception { Random random = new Random(); int count = 10; 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 be1c02de33..dd1b34b186 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 @@ -57,7 +57,8 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness private static final String PROFILE_STREAM = "profile"; private static final String ENRICHED_PAGEVIEW_STREAM = "enrichedpageview"; - @Test + // @Test + // TODO: re-enable after fixing the test system should use InMemorySystemFactory instead of Kafka public void testJoinWithSideInputsTable() { runTest( "side-input-join", @@ -66,7 +67,8 @@ public void testJoinWithSideInputsTable() { Arrays.asList(TestTableData.generateProfiles(10))); } - @Test + // @Test + // TODO: re-enable after fixing the test system should use InMemorySystemFactory instead of Kafka public void testJoinWithDurableSideInputTable() { runTest( "durable-side-input", From 34ffda8aeae346370906d6015523ac3063f7caa9 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 31 Aug 2018 02:23:56 -0700 Subject: [PATCH 24/38] SAMZA-1789: disabling tests due to SAMZA-1836 --- .../test/framework/StreamApplicationIntegrationTest.java | 2 +- .../samza/test/table/TestLocalTableWithSideInputs.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) 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 fcd2b0f6f6..30335983a7 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 @@ -70,7 +70,7 @@ public class StreamApplicationIntegrationTest { private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"}; // @Test - // TODO: fix after the system descriptor is using InMemorySystemFactory + // TODO: fix after the system descriptor is using InMemorySystemFactory (SAMZA-1836) public void testHighLevelApi() throws Exception { Random random = new Random(); int count = 10; 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 dd1b34b186..3b5c2d8615 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 @@ -45,7 +45,7 @@ import org.apache.samza.test.framework.TestRunner; import org.apache.samza.test.framework.stream.CollectionStream; import org.apache.samza.test.harness.AbstractIntegrationTestHarness; -import org.junit.Test; +//import org.junit.Test; import static org.apache.samza.test.table.TestTableData.*; import static org.junit.Assert.assertEquals; @@ -58,7 +58,7 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness private static final String ENRICHED_PAGEVIEW_STREAM = "enrichedpageview"; // @Test - // TODO: re-enable after fixing the test system should use InMemorySystemFactory instead of Kafka + // TODO: re-enable after fixing the test system should use InMemorySystemFactory instead of Kafka (SAMZA-1836) public void testJoinWithSideInputsTable() { runTest( "side-input-join", @@ -68,7 +68,7 @@ public void testJoinWithSideInputsTable() { } // @Test - // TODO: re-enable after fixing the test system should use InMemorySystemFactory instead of Kafka + // TODO: re-enable after fixing the test system should use InMemorySystemFactory instead of Kafka (SAMZA-1836) public void testJoinWithDurableSideInputTable() { runTest( "durable-side-input", From ec4bb1dca0c5da3d1f6b790f9321bf001da6db7a Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 31 Aug 2018 18:06:42 -0700 Subject: [PATCH 25/38] SAMZA-1789: merge with fix for SAMZA-1836 --- .../org/apache/samza/runtime/TestLocalApplicationRunner.java | 1 + 1 file changed, 1 insertion(+) diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index ecbe33a026..6a74b1945a 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -72,6 +72,7 @@ public void testRunStreamTask() throws Exception { final Map cfgs = new HashMap<>(); cfgs.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); + cfgs.put(JobConfig.JOB_NAME(), "test-task-job"); config = new MapConfig(cfgs); mockApp = (TaskApplication) appDesc -> appDesc.setTaskFactory((StreamTaskFactory) () -> new IdentityStreamTask()); prepareTest(); From 66af5b7063f9d98fdf5a7594d7367f1513aa55de Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Tue, 4 Sep 2018 23:28:05 -0700 Subject: [PATCH 26/38] SAMZA-1789: addressing Cameron's review comments. --- .../application/ApplicationDescriptorImpl.java | 16 ++++++++-------- .../StreamApplicationDescriptorImpl.java | 8 ++++---- .../TaskApplicationDescriptorImpl.java | 6 +++--- .../org/apache/samza/execution/JobPlanner.java | 4 ++-- .../samza/runtime/LocalApplicationRunner.java | 12 ++++++------ 5 files changed, 23 insertions(+), 23 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java index ef97704690..2573e67e5b 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java @@ -104,7 +104,7 @@ public S withDefaultSystem(SystemDescriptor defaultSystemDescriptor) { Preconditions.checkState(noInputOutputStreams(), "Default system must be set before creating any input or output streams."); addSystemDescriptor(defaultSystemDescriptor); - this.defaultSystemDescriptorOptional = Optional.of(defaultSystemDescriptor); + defaultSystemDescriptorOptional = Optional.of(defaultSystemDescriptor); return (S) this; } @@ -194,7 +194,7 @@ public Set getSystemDescriptors() { * @return the default {@link SystemDescriptor} */ public Optional getDefaultSystemDescriptor() { - return this.defaultSystemDescriptorOptional; + return defaultSystemDescriptorOptional; } /** @@ -235,16 +235,16 @@ void addOutputDescriptor(OutputDescriptor osd) { addSystemDescriptor(osd.getSystemDescriptor()); } + // TODO: this should be completely internal to addInputDescriptor()/addOutputDescriptor after we add broadcast automatically + void addBroadcastStream(String streamId) { + broadcastStreams.add(streamId); + } + // internal method to add a unique {@link SystemDescriptor} to this application - void addSystemDescriptor(SystemDescriptor systemDescriptor) { + private void addSystemDescriptor(SystemDescriptor systemDescriptor) { Preconditions.checkState(!systemDescriptors.containsKey(systemDescriptor.getSystemName()) || systemDescriptors.get(systemDescriptor.getSystemName()) == systemDescriptor, "Must not use different system descriptor instances for the same system name: " + systemDescriptor.getSystemName()); systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); } - - // TODO: this should be completely internal to addInputDescriptor()/addOutputDescriptor after we add broadcast automatically - void addBroadcastStream(String streamId) { - this.broadcastStreams.add(streamId); - } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java index 92cdbbb539..1220f724a6 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java @@ -94,7 +94,7 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor return expander.get().apply(this, inputDescriptor); } - super.addInputDescriptor(inputDescriptor); + addInputDescriptor(inputDescriptor); String streamId = inputDescriptor.getStreamId(); Preconditions.checkState(!inputOperators.containsKey(streamId), "getInputStream must not be called multiple times with the same streamId: " + streamId); @@ -121,7 +121,7 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor @Override public OutputStream getOutputStream(OutputDescriptor outputDescriptor) { - super.addOutputDescriptor(outputDescriptor); + addOutputDescriptor(outputDescriptor); String streamId = outputDescriptor.getStreamId(); Preconditions.checkState(!outputStreams.containsKey(streamId), "getOutputStream must not be called multiple times with the same streamId: " + streamId); @@ -147,7 +147,7 @@ public Table> getTable(TableDescriptor tableDescriptor) String tableId = tableDescriptor.getTableId(); Preconditions.checkState(StringUtils.isNotBlank(tableId) && ID_PATTERN.matcher(tableId).matches(), String.format("tableId: %s must confirm to pattern: %s", tableId, ID_PATTERN.toString())); - super.addTableDescriptor(tableDescriptor); + addTableDescriptor(tableDescriptor); TableSpec tableSpec = ((BaseTableDescriptor) tableDescriptor).getTableSpec(); if (tables.containsKey(tableSpec)) { throw new IllegalStateException( @@ -220,7 +220,7 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI } if (isBroadcast) - super.addBroadcastStream(streamId); + addBroadcastStream(streamId); boolean isKeyed; KV kvSerdes; diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java index 6d9256b01a..e7572b910b 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java @@ -49,17 +49,17 @@ public void setTaskFactory(TaskFactory factory) { @Override public void addInputStream(InputDescriptor isd) { - super.addInputDescriptor(isd); + addInputDescriptor(isd); } @Override public void addOutputStream(OutputDescriptor osd) { - super.addOutputDescriptor(osd); + addOutputDescriptor(osd); } @Override public void addTable(TableDescriptor tableDescriptor) { - this.addTableDescriptor(tableDescriptor); + addTableDescriptor(tableDescriptor); } /** diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index 1913939cb2..41b28a7b27 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -70,14 +70,14 @@ public List prepareJobs() { return ApplicationDescriptors.forType( taskAppDesc -> { try { - return Collections.singletonList(JobPlanner.this.prepareTaskJob(taskAppDesc)); + return Collections.singletonList(prepareTaskJob(taskAppDesc)); } catch (Exception e) { throw new SamzaException("Failed to generate JobConfig for TaskApplication " + appId, e); } }, streamAppDesc -> { try { - return JobPlanner.this.prepareStreamJobs(streamAppDesc); + return prepareStreamJobs(streamAppDesc); } catch (Exception e) { throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 580f71a014..88354d36f0 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -176,11 +176,11 @@ StreamProcessor createStreamProcessor(Config config, ApplicationDescriptorImpl Date: Thu, 6 Sep 2018 22:26:00 -0700 Subject: [PATCH 27/38] SAMZA-1814: WIP fixing the task application configuration generation in the planner --- .../application/ApplicationDescriptor.java | 33 +--- .../samza/application/StreamApplication.java | 14 +- .../StreamApplicationDescriptor.java | 22 ++- .../TaskApplicationDescriptor.java | 10 +- .../samza/runtime/ApplicationRunner.java | 3 +- .../org/apache/samza/task/TaskFactory.java | 2 +- .../ApplicationDescriptorImpl.java | 139 +++++------------ ...rs.java => ApplicationDescriptorUtil.java} | 35 ++--- .../ApplicationUtil.java} | 19 ++- .../application/LegacyTaskApplication.java | 22 +-- .../StreamApplicationDescriptorImpl.java | 139 ++++++++++++++--- .../TaskApplicationDescriptorImpl.java | 71 +++++++-- .../container/SamzaContainerListener.java | 6 +- .../samza/execution/ExecutionPlanner.java | 43 +++++- .../org/apache/samza/execution/JobGraph.java | 25 ++- .../org/apache/samza/execution/JobNode.java | 142 ++++++++++++++++++ .../apache/samza/execution/JobPlanner.java | 60 +++----- .../samza/processor/StreamProcessor.java | 37 ++--- .../samza/runtime/ApplicationRunnerMain.java | 3 +- .../samza/runtime/LocalApplicationRunner.java | 31 ++-- .../samza/runtime/LocalContainerRunner.java | 7 +- .../runtime/RemoteApplicationRunner.java | 12 +- .../apache/samza/task/TaskFactoryUtil.java | 31 ++-- .../samza/container/SamzaContainer.scala | 4 +- .../samza/job/local/ThreadJobFactory.scala | 24 +-- .../TestApplicationUtil.java} | 30 ++-- .../samza/execution/TestExecutionPlanner.java | 93 +++++++++++- .../runtime/TestLocalApplicationRunner.java | 16 +- ...reamTask.java => MockAsyncStreamTask.java} | 2 +- .../samza/task/TestTaskFactoryUtil.java | 33 +--- .../samza/container/TestSamzaContainer.scala | 10 +- .../LocalApplicationRunnerMain.java | 4 +- 32 files changed, 704 insertions(+), 418 deletions(-) rename samza-core/src/main/java/org/apache/samza/application/{ApplicationDescriptors.java => ApplicationDescriptorUtil.java} (53%) rename samza-core/src/main/java/org/apache/samza/{runtime/ApplicationClassUtils.java => application/ApplicationUtil.java} (76%) rename samza-core/src/test/java/org/apache/samza/{runtime/TestApplicationClassUtils.java => application/TestApplicationUtil.java} (76%) rename samza-core/src/test/java/org/apache/samza/task/{TestAsyncStreamTask.java => MockAsyncStreamTask.java} (94%) diff --git a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java index e426522144..178fdeea0e 100644 --- a/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/ApplicationDescriptor.java @@ -23,25 +23,23 @@ import org.apache.samza.config.Config; import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.operators.ContextManager; -import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; /** - * The base interface class to describe a user application in Samza. + * The interface class to describe the configuration, input and output streams, and processing logic in a {@link SamzaApplication}. *

    - * Sub-classes {@link StreamApplicationDescriptor} and {@link TaskApplicationDescriptor} are specific interfaces for applications written - * in high-level DAG and low-level task APIs, respectively. + * Sub-classes {@link StreamApplicationDescriptor} and {@link TaskApplicationDescriptor} are specific interfaces for applications + * written in high-level {@link StreamApplication} and low-level {@link TaskApplication} APIs, respectively. * - * @param sub-class of user application descriptor. It has to be either {@link StreamApplicationDescriptor} or - * {@link TaskApplicationDescriptor} + * @param sub-class of user application descriptor. */ @InterfaceStability.Evolving public interface ApplicationDescriptor { /** - * Get {@link Config} - * @return config object + * Get the {@link Config} of the application + * @return config of the application */ Config getConfig(); @@ -71,27 +69,10 @@ public interface ApplicationDescriptor { */ S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory listenerFactory); - /** - * Sets the default SystemDescriptor to use for intermediate streams. This is equivalent to setting - * {@code job.default.system} and its properties in configuration. - *

    - * If the default system descriptor is set, it must be set before creating any input/output/intermediate streams. - *

    - * If an input/output stream is created with a stream-level Serde, they will be used, else the serde specified - * for the {@code job.default.system} in configuration will be used. - *

    - * Providing an incompatible message type for the intermediate streams that use the default serde will result in - * {@link ClassCastException}s at runtime. - * - * @param defaultSystemDescriptor the default system descriptor to use - * @return type {@code S} of {@link ApplicationDescriptor} with {@code defaultSystemDescriptor} set as its default system - */ - S withDefaultSystem(SystemDescriptor defaultSystemDescriptor); - /** * Sets a set of customized {@link MetricsReporterFactory}s in the application * - * @param reporterFactories the map of customized {@link MetricsReporterFactory} objects to be used + * @param reporterFactories the map of customized {@link MetricsReporterFactory}s to be used * @return type {@code S} of {@link ApplicationDescriptor} with {@code reporterFactories} */ S withMetricsReporterFactories(Map reporterFactories); 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 111f7a4e73..a83cb378d4 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,8 +21,7 @@ import org.apache.samza.annotation.InterfaceStability; /** - * Describes and initializes the transforms for processing message streams and generating results in high-level API. Your - * application is expected to implement this interface. + * Describes and initializes the transforms for processing message streams and generating results in high-level API. *

    * The following example removes page views older than 1 hour from the input stream: *

    {@code
    @@ -59,17 +58,18 @@
      *
      * 

    * Implementation Notes: Currently {@link StreamApplication}s are wrapped in a {@link org.apache.samza.task.StreamTask} - * during execution. All user-defined transformation functions are required to be serializable. 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}. + * 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}. * *

    - * The user-implemented {@link StreamApplication} 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. + * 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}. */ @InterfaceStability.Evolving public interface StreamApplication extends SamzaApplication { diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java index 654b5d6d1a..5a3de49660 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplicationDescriptor.java @@ -25,15 +25,33 @@ import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.table.Table; /** - * The interface class to describe a user application in high-level DAG in Samza. + * The interface class to describe a {@link SamzaApplication} in high-level API in Samza. */ @InterfaceStability.Evolving public interface StreamApplicationDescriptor extends ApplicationDescriptor { + /** + * Sets the default SystemDescriptor to use for intermediate streams. This is equivalent to setting + * {@code job.default.system} and its properties in configuration. + *

    + * If the default system descriptor is set, it must be set before creating any input/output/intermediate streams. + *

    + * If an input/output stream is created with a stream-level Serde, they will be used, else the serde specified + * for the {@code job.default.system} in configuration will be used. + *

    + * Providing an incompatible message type for the intermediate streams that use the default serde will result in + * {@link ClassCastException}s at runtime. + * + * @param defaultSystemDescriptor the default system descriptor to use + * @return type {@code S} of {@link ApplicationDescriptor} with {@code defaultSystemDescriptor} set as its default system + */ + StreamApplicationDescriptor withDefaultSystem(SystemDescriptor defaultSystemDescriptor); + /** * Gets the input {@link MessageStream} corresponding to the {@code inputDescriptor}. *

    @@ -92,4 +110,4 @@ public interface StreamApplicationDescriptor extends ApplicationDescriptor Table> getTable(TableDescriptor tableDescriptor); -} +} \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java index d60df283ec..0226bb557f 100644 --- a/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java +++ b/samza-api/src/main/java/org/apache/samza/application/TaskApplicationDescriptor.java @@ -26,7 +26,7 @@ /** - * The interface class to describe a user application as low-level task in Samza. + * The interface to describe a {@link SamzaApplication} that uses low-level API task for processing. */ @InterfaceStability.Evolving public interface TaskApplicationDescriptor extends ApplicationDescriptor { @@ -35,20 +35,20 @@ public interface TaskApplicationDescriptor extends ApplicationDescriptor - * Implementation of {@link ApplicationRunner} class must have a public default constructor #ApplicationRunner(SamzaApplication, Config) + * Implementation Notes: implementation of {@link ApplicationRunner} must have a public default constructor + * #ApplicationRunner(SamzaApplication, Config) */ @InterfaceStability.Evolving public interface ApplicationRunner { 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 e3e372b386..8443d20698 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,7 +23,7 @@ /** - * The base interface class for all task factories (i.e. {@link StreamTaskFactory} and {@link AsyncStreamTaskFactory} + * The interface for all task factories (i.e. {@link StreamTaskFactory} and {@link AsyncStreamTaskFactory} * * @param the type of task instances */ diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java index 2573e67e5b..96791366f3 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java @@ -18,9 +18,7 @@ */ package org.apache.samza.application; -import com.google.common.base.Preconditions; import java.util.Collections; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; @@ -50,17 +48,9 @@ public abstract class ApplicationDescriptorImpl implements ApplicationDescriptor { final Config config; - final Class appClass; - - private final Map inputDescriptors = new LinkedHashMap<>(); - private final Map outputDescriptors = new LinkedHashMap<>(); - private final Map systemDescriptors = new LinkedHashMap<>(); - private final Set broadcastStreams = new HashSet<>(); - private final Map tableDescriptors = new LinkedHashMap<>(); + private final Class appClass; private final Map reporterFactories = new LinkedHashMap<>(); - private Optional defaultSystemDescriptorOptional = Optional.empty(); - // Default to no-op functions in ContextManager // TODO: this should be replaced by shared context factory defined in SAMZA-1714 ContextManager contextManager = new ContextManager() { @@ -76,9 +66,9 @@ public void close() { // Default to no-op ProcessorLifecycleListenerFactory ProcessorLifecycleListenerFactory listenerFactory = (pcontext, cfg) -> new ProcessorLifecycleListener() { }; - ApplicationDescriptorImpl(SamzaApplication userApp, Config config) { + ApplicationDescriptorImpl(SamzaApplication app, Config config) { this.config = config; - this.appClass = userApp.getClass(); + this.appClass = app.getClass(); } @Override @@ -98,153 +88,92 @@ public S withProcessorLifecycleListenerFactory(ProcessorLifecycleListenerFactory return (S) this; } - @Override - public S withDefaultSystem(SystemDescriptor defaultSystemDescriptor) { - Preconditions.checkNotNull(defaultSystemDescriptor, "Provided defaultSystemDescriptor must not be null."); - Preconditions.checkState(noInputOutputStreams(), - "Default system must be set before creating any input or output streams."); - addSystemDescriptor(defaultSystemDescriptor); - defaultSystemDescriptorOptional = Optional.of(defaultSystemDescriptor); - return (S) this; - } - @Override public S withMetricsReporterFactories(Map reporterFactories) { this.reporterFactories.clear(); this.reporterFactories.putAll(reporterFactories); return (S) this; } + /** - * Get the user application class + * Get the application class * - * @return user implemented {@link SamzaApplication} class + * @return an implementation of {@link SamzaApplication} */ public Class getAppClass() { return appClass; } /** - * Get the user-implemented {@link ContextManager} object associated with this application + * Get the {@link ContextManager} associated with this application * - * @return the {@link ContextManager} object + * @return the {@link ContextManager} for this application */ public ContextManager getContextManager() { return contextManager; } /** - * Get the user-implemented {@link ProcessorLifecycleListenerFactory} object associated with this application + * Get the {@link ProcessorLifecycleListenerFactory} associated with this application * - * @return the {@link ProcessorLifecycleListenerFactory} object + * @return the {@link ProcessorLifecycleListenerFactory} in this application */ public ProcessorLifecycleListenerFactory getProcessorLifecycleListenerFactory() { return listenerFactory; } /** - * Get all the {@link InputDescriptor}s to this application + * Get the {@link MetricsReporterFactory}s used in the application * - * @return an immutable map of streamId to {@link InputDescriptor} + * @return the map of {@link MetricsReporterFactory}s */ - public Map getInputDescriptors() { - return Collections.unmodifiableMap(inputDescriptors); + public Map getMetricsReporterFactories() { + return Collections.unmodifiableMap(reporterFactories); } /** - * Get all the {@link OutputDescriptor}s from this application + * Get the default {@link SystemDescriptor} in this application * - * @return an immutable map of streamId to {@link OutputDescriptor} + * @return the default {@link SystemDescriptor} */ - public Map getOutputDescriptors() { - return Collections.unmodifiableMap(outputDescriptors); + public Optional getDefaultSystemDescriptor() { + // default is not set + return Optional.empty(); } /** - * Get all the broadcast streamIds from this application + * Get all the {@link InputDescriptor}s to this application * - * @return an immutable set of streamIds + * @return an immutable map of streamId to {@link InputDescriptor} */ - public Set getBroadcastStreams() { - return Collections.unmodifiableSet(broadcastStreams); - } + public abstract Map getInputDescriptors(); /** - * Get all the {@link TableDescriptor}s in this application + * Get all the {@link OutputDescriptor}s from this application * - * @return an immutable set of {@link TableDescriptor}s + * @return an immutable map of streamId to {@link OutputDescriptor} */ - public Set getTableDescriptors() { - return Collections.unmodifiableSet(new HashSet<>(tableDescriptors.values())); - } + public abstract Map getOutputDescriptors(); /** - * Get all the unique {@link SystemDescriptor}s in this application + * Get all the broadcast streamIds from this application * - * @return an immutable set of {@link SystemDescriptor}s + * @return an immutable set of streamIds */ - public Set getSystemDescriptors() { - // We enforce that users must not use different system descriptor instances for the same system name - // when getting an input/output stream or setting the default system descriptor - return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); - } + public abstract Set getBroadcastStreams(); /** - * Get the default {@link SystemDescriptor} in this application + * Get all the {@link TableDescriptor}s in this application * - * @return the default {@link SystemDescriptor} + * @return an immutable set of {@link TableDescriptor}s */ - public Optional getDefaultSystemDescriptor() { - return defaultSystemDescriptorOptional; - } + public abstract Set getTableDescriptors(); /** - * Get the {@link MetricsReporterFactory}s used in the application + * Get all the unique {@link SystemDescriptor}s in this application * - * @return the map of {@link MetricsReporterFactory}s + * @return an immutable set of {@link SystemDescriptor}s */ - public Map getMetricsReporterFactories() { - return Collections.unmodifiableMap(reporterFactories); - } - - // TODO: this should go away when partitionBy() and broadcast() will also generate InputDescriptor/OutputDescriptor as well - // helper method to determine that there is no input/output streams added in the application yet - protected abstract boolean noInputOutputStreams(); - - // internal method to add {@link TableDescriptor} to this application - void addTableDescriptor(TableDescriptor tableDescriptor) { - Preconditions.checkState(!tableDescriptors.containsKey(tableDescriptor.getTableId()), - String.format("add table descriptors multiple times with the same tableId: %s", tableDescriptor.getTableId())); - tableDescriptors.put(tableDescriptor.getTableId(), tableDescriptor); - } + public abstract Set getSystemDescriptors(); - // internal method to add {@link InputDescriptor} to this application - void addInputDescriptor(InputDescriptor isd) { - // TODO: need to add to the broadcast streams if isd is a broadcast stream - Preconditions.checkState(!inputDescriptors.containsKey(isd.getStreamId()), - String.format("add input descriptors multiple times with the same streamId: %s", isd.getStreamId())); - inputDescriptors.put(isd.getStreamId(), isd); - addSystemDescriptor(isd.getSystemDescriptor()); - } - - // internal method to add {@link OutputDescriptor} to this application - void addOutputDescriptor(OutputDescriptor osd) { - // TODO: need to add to the broadcast streams if osd is a broadcast stream - Preconditions.checkState(!outputDescriptors.containsKey(osd.getStreamId()), - String.format("add output descriptors multiple times with the same streamId: %s", osd.getStreamId())); - outputDescriptors.put(osd.getStreamId(), osd); - addSystemDescriptor(osd.getSystemDescriptor()); - } - - // TODO: this should be completely internal to addInputDescriptor()/addOutputDescriptor after we add broadcast automatically - void addBroadcastStream(String streamId) { - broadcastStreams.add(streamId); - } - - // internal method to add a unique {@link SystemDescriptor} to this application - private void addSystemDescriptor(SystemDescriptor systemDescriptor) { - Preconditions.checkState(!systemDescriptors.containsKey(systemDescriptor.getSystemName()) - || systemDescriptors.get(systemDescriptor.getSystemName()) == systemDescriptor, - "Must not use different system descriptor instances for the same system name: " + systemDescriptor.getSystemName()); - systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); - } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java similarity index 53% rename from samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java rename to samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java index de15b80e2c..acf4bf7e9e 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptors.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorUtil.java @@ -18,45 +18,34 @@ */ package org.apache.samza.application; -import java.util.function.Function; import org.apache.samza.config.Config; /** * Util class to help creating {@link ApplicationDescriptorImpl} instance from {@link SamzaApplication} and {@link Config} */ -public class ApplicationDescriptors { - private ApplicationDescriptors() { +public class ApplicationDescriptorUtil { + + private ApplicationDescriptorUtil() { } /** * Create a new instance of {@link ApplicationDescriptorImpl} based on {@link SamzaApplication} and {@link Config} * - * @param userApp the user-implemented {@link SamzaApplication}. The {@code userApp} has to have a proper fully-qualified class name. - * @param config the user-supplied {@link Config} for the the application - * @return the {@link ApplicationDescriptorImpl} instance containing the user processing logic and the config + * @param app an implementation of {@link SamzaApplication}. The {@code app} has to have a proper fully-qualified class name. + * @param config the {@link Config} for the application + * @return the {@link ApplicationDescriptorImpl} instance containing the processing logic and the config */ - public static ApplicationDescriptorImpl getAppDescriptor(SamzaApplication userApp, Config config) { - if (userApp instanceof StreamApplication) { - return new StreamApplicationDescriptorImpl((StreamApplication) userApp, config); + public static ApplicationDescriptorImpl getAppDescriptor(SamzaApplication app, Config config) { + if (app instanceof StreamApplication) { + return new StreamApplicationDescriptorImpl((StreamApplication) app, config); } - if (userApp instanceof TaskApplication) { - return new TaskApplicationDescriptorImpl((TaskApplication) userApp, config); + if (app instanceof TaskApplication) { + return new TaskApplicationDescriptorImpl((TaskApplication) app, config); } throw new IllegalArgumentException(String.format("User application class %s is not supported. Only StreamApplication " - + "and TaskApplication are supported.", userApp.getClass().getName())); + + "and TaskApplication are supported.", app.getClass().getName())); } - public static T forType(Function forTaskApp, Function forStreamApp, - ApplicationDescriptorImpl desc) { - if (desc instanceof TaskApplicationDescriptorImpl) { - return forTaskApp.apply((TaskApplicationDescriptorImpl) desc); - } else if (desc instanceof StreamApplicationDescriptorImpl) { - return forStreamApp.apply((StreamApplicationDescriptorImpl) desc); - } - - throw new IllegalArgumentException(String.format("AppDescriptorImpl has to be either TaskAppDescriptorImpl or StreamAppDescriptorImpl." - + " class %s is not supported", desc.getClass().getName())); - } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java similarity index 76% rename from samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java rename to samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java index f2fc3c0546..b39ad3c377 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationClassUtils.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java @@ -16,25 +16,23 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime; +package org.apache.samza.application; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.LegacyTaskApplication; -import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.TaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; +import org.apache.samza.config.TaskConfig; +import scala.Option; /** * Util class to create {@link SamzaApplication} from the configuration. */ -public class ApplicationClassUtils { +public class ApplicationUtil { /** - * Creates the {@link SamzaApplication} object from the {@code config} + * Creates the {@link SamzaApplication} object from the task or application class name specified in {@code config} * * @param config the configuration of the application * @return the {@link SamzaApplication} object @@ -54,7 +52,12 @@ public static SamzaApplication fromConfig(Config config) { } } // no app.class defined. It has to be a legacy application with task.class configuration - return new LegacyTaskApplication(config); + Option taskClassOption = new TaskConfig(config).getTaskClass(); + if (!taskClassOption.isDefined() || !StringUtils.isNotBlank(taskClassOption.getOrElse(null))) { + // no task.class defined either. This is wrong. + throw new ConfigException("Legacy task applications must set a non-empty task.class in configuration."); + } + return new LegacyTaskApplication(taskClassOption.get()); } } \ No newline at end of file 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 370306855a..e9e2635e10 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 @@ -18,34 +18,20 @@ */ package org.apache.samza.application; -import org.apache.samza.config.Config; -import org.apache.samza.config.ConfigException; -import org.apache.samza.config.TaskConfig; import org.apache.samza.task.TaskFactoryUtil; -import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; - - /** * Default {@link TaskApplication} for legacy applications w/ only task.class implemented */ public final class LegacyTaskApplication implements TaskApplication { - private final Config config; - - public LegacyTaskApplication(Config config) { - this.config = validate(config); - } + private final String taskClassName; - private Config validate(Config config) { - new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( - () -> { - throw new ConfigException("No task class defined in the configuration."); - })); - return config; + public LegacyTaskApplication(String taskClassName) { + this.taskClassName = taskClassName; } @Override public void describe(TaskApplicationDescriptor appDesc) { - appDesc.setTaskFactory(TaskFactoryUtil.getTaskFactoryFromConfig(config)); + appDesc.setTaskFactory(TaskFactoryUtil.getTaskFactory(taskClassName)); } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java index 1220f724a6..df96d85c2a 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java @@ -68,10 +68,18 @@ public class StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl inputDescriptors = new LinkedHashMap<>(); + private final Map outputDescriptors = new LinkedHashMap<>(); + private final Set broadcastStreams = new HashSet<>(); + private final Map tableDescriptors = new LinkedHashMap<>(); + private final Map systemDescriptors = new LinkedHashMap<>(); // We use a LHM for deterministic order in initializing and closing operators. private final Map inputOperators = new LinkedHashMap<>(); private final Map outputStreams = new LinkedHashMap<>(); private final Map tables = new LinkedHashMap<>(); + private final Set operatorIds = new HashSet<>(); + + private Optional defaultSystemDescriptorOptional = Optional.empty(); /** * The 0-based position of the next operator in the graph. @@ -79,13 +87,23 @@ public class StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl operatorIds = new HashSet<>(); public StreamApplicationDescriptorImpl(StreamApplication userApp, Config config) { super(userApp, config); userApp.describe(this); } + @Override + public StreamApplicationDescriptor withDefaultSystem(SystemDescriptor defaultSystemDescriptor) { + Preconditions.checkNotNull(defaultSystemDescriptor, "Provided defaultSystemDescriptor must not be null."); + Preconditions.checkState(inputOperators.isEmpty() && outputStreams.isEmpty(), + "Default system must be set before creating any input or output streams."); + addSystemDescriptor(defaultSystemDescriptor); + + defaultSystemDescriptorOptional = Optional.of(defaultSystemDescriptor); + return this; + } + @Override public MessageStream getInputStream(InputDescriptor inputDescriptor) { SystemDescriptor systemDescriptor = inputDescriptor.getSystemDescriptor(); @@ -94,7 +112,12 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor return expander.get().apply(this, inputDescriptor); } - addInputDescriptor(inputDescriptor); + // TODO: SAMZA-1841: need to add to the broadcast streams if isd is a broadcast stream + Preconditions.checkState(!inputDescriptors.containsKey(inputDescriptor.getStreamId()), + String.format("add input descriptors multiple times with the same streamId: %s", inputDescriptor.getStreamId())); + inputDescriptors.put(inputDescriptor.getStreamId(), inputDescriptor); + addSystemDescriptor(inputDescriptor.getSystemDescriptor()); + String streamId = inputDescriptor.getStreamId(); Preconditions.checkState(!inputOperators.containsKey(streamId), "getInputStream must not be called multiple times with the same streamId: " + streamId); @@ -121,7 +144,12 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor @Override public OutputStream getOutputStream(OutputDescriptor outputDescriptor) { - addOutputDescriptor(outputDescriptor); + // TODO: SAMZA-1841: need to add to the broadcast streams if osd is a broadcast stream + Preconditions.checkState(!outputDescriptors.containsKey(outputDescriptor.getStreamId()), + String.format("add output descriptors multiple times with the same streamId: %s", outputDescriptor.getStreamId())); + outputDescriptors.put(outputDescriptor.getStreamId(), outputDescriptor); + addSystemDescriptor(outputDescriptor.getSystemDescriptor()); + String streamId = outputDescriptor.getStreamId(); Preconditions.checkState(!outputStreams.containsKey(streamId), "getOutputStream must not be called multiple times with the same streamId: " + streamId); @@ -147,7 +175,10 @@ public Table> getTable(TableDescriptor tableDescriptor) String tableId = tableDescriptor.getTableId(); Preconditions.checkState(StringUtils.isNotBlank(tableId) && ID_PATTERN.matcher(tableId).matches(), String.format("tableId: %s must confirm to pattern: %s", tableId, ID_PATTERN.toString())); - addTableDescriptor(tableDescriptor); + Preconditions.checkState(!tableDescriptors.containsKey(tableDescriptor.getTableId()), + String.format("add table descriptors multiple times with the same tableId: %s", tableDescriptor.getTableId())); + tableDescriptors.put(tableDescriptor.getTableId(), tableDescriptor); + TableSpec tableSpec = ((BaseTableDescriptor) tableDescriptor).getTableSpec(); if (tables.containsKey(tableSpec)) { throw new IllegalStateException( @@ -157,6 +188,68 @@ public Table> getTable(TableDescriptor tableDescriptor) return tables.get(tableSpec); } + /** + * Get all the {@link InputDescriptor}s to this application + * + * @return an immutable map of streamId to {@link InputDescriptor} + */ + @Override + public Map getInputDescriptors() { + return Collections.unmodifiableMap(inputDescriptors); + } + + /** + * Get all the {@link OutputDescriptor}s from this application + * + * @return an immutable map of streamId to {@link OutputDescriptor} + */ + @Override + public Map getOutputDescriptors() { + return Collections.unmodifiableMap(outputDescriptors); + } + + /** + * Get all the broadcast streamIds from this application + * + * @return an immutable set of streamIds + */ + @Override + public Set getBroadcastStreams() { + return Collections.unmodifiableSet(broadcastStreams); + } + + /** + * Get all the {@link TableDescriptor}s in this application + * + * @return an immutable set of {@link TableDescriptor}s + */ + @Override + public Set getTableDescriptors() { + return Collections.unmodifiableSet(new HashSet<>(tableDescriptors.values())); + } + + /** + * Get all the unique {@link SystemDescriptor}s in this application + * + * @return an immutable set of {@link SystemDescriptor}s + */ + @Override + public Set getSystemDescriptors() { + // We enforce that users must not use different system descriptor instances for the same system name + // when getting an input/output stream or setting the default system descriptor + return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); + } + + /** + * Get the default {@link SystemDescriptor} in this application + * + * @return the default {@link SystemDescriptor} + */ + @Override + public Optional getDefaultSystemDescriptor() { + return defaultSystemDescriptorOptional; + } + public OperatorSpecGraph getOperatorSpecGraph() { return new OperatorSpecGraph(this); } @@ -198,6 +291,18 @@ public String getNextOpId(OpCode opCode) { return getNextOpId(opCode, null); } + public Map getInputOperators() { + return Collections.unmodifiableMap(inputOperators); + } + + public Map getOutputStreams() { + return Collections.unmodifiableMap(outputStreams); + } + + public Map getTables() { + return Collections.unmodifiableMap(tables); + } + /** * Internal helper for {@link MessageStreamImpl} to add an intermediate {@link MessageStream} to the graph. * An intermediate {@link MessageStream} is both an output and an input stream. @@ -219,8 +324,9 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI ". Key and message serdes configured for the job.default.system will be used."); } - if (isBroadcast) - addBroadcastStream(streamId); + if (isBroadcast) { + broadcastStreams.add(streamId); + } boolean isKeyed; KV kvSerdes; @@ -243,18 +349,6 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamId), outputStreams.get(streamId)); } - public Map getInputOperators() { - return Collections.unmodifiableMap(inputOperators); - } - - public Map getOutputStreams() { - return Collections.unmodifiableMap(outputStreams); - } - - public Map getTables() { - return Collections.unmodifiableMap(tables); - } - private KV getKVSerdes(String streamId, Serde serde) { Serde keySerde, valueSerde; @@ -278,8 +372,11 @@ private KV getKVSerdes(String streamId, Serde serde) { return KV.of(keySerde, valueSerde); } - @Override - protected boolean noInputOutputStreams() { - return inputOperators.isEmpty() && outputStreams.isEmpty(); + // check uniqueness of the {@code systemDescriptor} and add if it is unique + private void addSystemDescriptor(SystemDescriptor systemDescriptor) { + Preconditions.checkState(!systemDescriptors.containsKey(systemDescriptor.getSystemName()) + || systemDescriptors.get(systemDescriptor.getSystemName()) == systemDescriptor, + "Must not use different system descriptor instances for the same system name: " + systemDescriptor.getSystemName()); + systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); } } diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java index e7572b910b..b301fd0a5a 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java @@ -18,10 +18,17 @@ */ package org.apache.samza.application; +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; import org.apache.samza.config.Config; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.task.TaskFactory; @@ -35,7 +42,13 @@ public class TaskApplicationDescriptorImpl extends ApplicationDescriptorImpl implements TaskApplicationDescriptor { - TaskFactory taskFactory; + private final Map inputDescriptors = new LinkedHashMap<>(); + private final Map outputDescriptors = new LinkedHashMap<>(); + private final Set broadcastStreams = new HashSet<>(); + private final Map tableDescriptors = new LinkedHashMap<>(); + private final Map systemDescriptors = new LinkedHashMap<>(); + + private TaskFactory taskFactory = null; public TaskApplicationDescriptorImpl(TaskApplication userApp, Config config) { super(userApp, config); @@ -48,18 +61,55 @@ public void setTaskFactory(TaskFactory factory) { } @Override - public void addInputStream(InputDescriptor isd) { - addInputDescriptor(isd); + public void addInputStream(InputDescriptor inputDescriptor) { + // TODO: SAMZA-1841: need to add to the broadcast streams if isd is a broadcast stream + Preconditions.checkState(!inputDescriptors.containsKey(inputDescriptor.getStreamId()), + String.format("add input descriptors multiple times with the same streamId: %s", inputDescriptor.getStreamId())); + inputDescriptors.put(inputDescriptor.getStreamId(), inputDescriptor); + addSystemDescriptor(inputDescriptor.getSystemDescriptor()); } @Override - public void addOutputStream(OutputDescriptor osd) { - addOutputDescriptor(osd); + public void addOutputStream(OutputDescriptor outputDescriptor) { + // TODO: SAMZA-1841: need to add to the broadcast streams if osd is a broadcast stream + Preconditions.checkState(!outputDescriptors.containsKey(outputDescriptor.getStreamId()), + String.format("add output descriptors multiple times with the same streamId: %s", outputDescriptor.getStreamId())); + outputDescriptors.put(outputDescriptor.getStreamId(), outputDescriptor); + addSystemDescriptor(outputDescriptor.getSystemDescriptor()); } @Override public void addTable(TableDescriptor tableDescriptor) { - addTableDescriptor(tableDescriptor); + Preconditions.checkState(!tableDescriptors.containsKey(tableDescriptor.getTableId()), + String.format("add table descriptors multiple times with the same tableId: %s", tableDescriptor.getTableId())); + tableDescriptors.put(tableDescriptor.getTableId(), tableDescriptor); + } + + @Override + public Map getInputDescriptors() { + return Collections.unmodifiableMap(inputDescriptors); + } + + @Override + public Map getOutputDescriptors() { + return Collections.unmodifiableMap(outputDescriptors); + } + + @Override + public Set getBroadcastStreams() { + return Collections.unmodifiableSet(broadcastStreams); + } + + @Override + public Set getTableDescriptors() { + return Collections.unmodifiableSet(new HashSet<>(tableDescriptors.values())); + } + + @Override + public Set getSystemDescriptors() { + // We enforce that users must not use different system descriptor instances for the same system name + // when getting an input/output stream or setting the default system descriptor + return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); } /** @@ -70,8 +120,11 @@ public TaskFactory getTaskFactory() { return taskFactory; } - @Override - protected boolean noInputOutputStreams() { - return getInputDescriptors().isEmpty() && getOutputDescriptors().isEmpty(); + private void addSystemDescriptor(SystemDescriptor systemDescriptor) { + Preconditions.checkState(!systemDescriptors.containsKey(systemDescriptor.getSystemName()) + || systemDescriptors.get(systemDescriptor.getSystemName()) == systemDescriptor, + "Must not use different system descriptor instances for the same system name: " + systemDescriptor.getSystemName()); + systemDescriptors.put(systemDescriptor.getSystemName(), systemDescriptor); } + } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java index a6437fa92b..5b3636b0fb 100644 --- a/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java +++ b/samza-core/src/main/java/org/apache/samza/container/SamzaContainerListener.java @@ -25,7 +25,7 @@ public interface SamzaContainerListener { /** * Method invoked when the {@link SamzaContainer} state is {@link org.apache.samza.SamzaContainerStatus#NOT_STARTED} - * and is about to transit to {@link org.apache.samza.SamzaContainerStatus#STARTING} to start the initialization sequence. + * and is about to transition to {@link org.apache.samza.SamzaContainerStatus#STARTING} to start the initialization sequence. */ void beforeStart(); @@ -51,8 +51,8 @@ public interface SamzaContainerListener { * {@link org.apache.samza.SamzaContainerStatus#FAILED} state. Details on state transitions can be found in * {@link org.apache.samza.SamzaContainerStatus} *
    - * Note: {@link #afterFailed(Throwable)} is mutually exclusive to {@link #afterStop()}. + * Note: {@link #afterFailure(Throwable)} is mutually exclusive to {@link #afterStop()}. * @param t Throwable that caused the container failure. */ - void afterFailed(Throwable t); + void afterFailure(Throwable t); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 7d25e7986f..7d0767484a 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -28,12 +28,15 @@ import java.util.Map; import java.util.Queue; import java.util.Set; +import java.util.stream.Collectors; import org.apache.samza.SamzaException; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.StreamConfig; +import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; @@ -64,7 +67,7 @@ public ExecutionPlanner(Config config, StreamManager streamManager) { this.streamManager = streamManager; } - public ExecutionPlan plan(OperatorSpecGraph specGraph) throws Exception { + public ExecutionPlan plan(OperatorSpecGraph specGraph) { validateConfig(); // create physical job graph based on stream graph @@ -91,6 +94,44 @@ private void validateConfig() { } } + /** + * Create the single node physical graph from {@link org.apache.samza.application.TaskApplicationDescriptorImpl} + */ + // TODO: SAMZA-1811: fix this and consolidate w/ high-level application's createJobGrapg(OperatorSpecGraph) method + /* package private */ static JobGraph createJobGraph(Config config, TaskApplicationDescriptorImpl taskAppDesc) { + JobGraph jobGraph = new JobGraph(config, taskAppDesc); + StreamConfig streamConfig = new StreamConfig(config); + Set sourceStreams = getStreamSpecs(taskAppDesc.getInputDescriptors().keySet(), streamConfig); + Set sinkStreams = getStreamSpecs(taskAppDesc.getOutputDescriptors().keySet(), streamConfig); + Set intStreams = new HashSet<>(sourceStreams); + Set tables = new HashSet<>(taskAppDesc.getTableDescriptors().stream() + .map(tableDesc -> ((BaseTableDescriptor) tableDesc).getTableSpec()).collect(Collectors.toSet())); + intStreams.retainAll(sinkStreams); + sourceStreams.removeAll(intStreams); + sinkStreams.removeAll(intStreams); + + // For this phase, we have a single job node for the whole dag + String jobName = config.get(JobConfig.JOB_NAME()); + String jobId = config.get(JobConfig.JOB_ID(), "1"); + JobNode node = jobGraph.getOrCreateJobNode(jobName, jobId); + + // add sources + sourceStreams.forEach(spec -> jobGraph.addSource(spec, node)); + + // add sinks + sinkStreams.forEach(spec -> jobGraph.addSink(spec, node)); + + // add intermediate streams + intStreams.forEach(spec -> jobGraph.addIntermediateStream(spec, node, node)); + + // add tables + tables.forEach(spec -> jobGraph.addTable(spec, node)); + + jobGraph.validate(); + + return jobGraph; + } + /** * Create the physical graph from {@link OperatorSpecGraph} */ diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index f49e6db6fc..146a2bf9bd 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -31,6 +31,8 @@ import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -61,6 +63,8 @@ private final Config config; private final JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); private final OperatorSpecGraph specGraph; + private final Set broadcastStreams; + private final boolean isTaskApplication; /** * The JobGraph is only constructed by the {@link ExecutionPlanner}. @@ -69,6 +73,17 @@ JobGraph(Config config, OperatorSpecGraph specGraph) { this.config = config; this.specGraph = specGraph; + this.broadcastStreams = specGraph.getBroadcastStreams(); + this.isTaskApplication = false; + } + + JobGraph(Config config, TaskApplicationDescriptorImpl taskAppDesc) { + this.config = config; + // TODO: HACK!!! Need to be fixed after SAMZA-1811 + // create a dummy specGraph + this.specGraph = new StreamApplicationDescriptorImpl(appDesc -> { }, config).getOperatorSpecGraph(); + this.broadcastStreams = taskAppDesc.getBroadcastStreams(); + this.isTaskApplication = true; } @Override @@ -91,6 +106,11 @@ public List getIntermediateStreams() { .collect(Collectors.toList()); } + // TODO: SAMZA-1811: consolidate this with high-level application JobGraph + JobConfig getSingleNodeJobConfig(TaskApplicationDescriptorImpl taskAppDesc) { + return getJobNodes().get(0).generateTaskApplicationConfig(taskAppDesc); + } + void addTable(TableSpec tableSpec, JobNode node) { tables.add(tableSpec); node.addTable(tableSpec); @@ -187,7 +207,7 @@ StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec, boolean isIntermediate) String streamId = streamSpec.getId(); StreamEdge edge = edges.get(streamId); if (edge == null) { - boolean isBroadcast = specGraph.getBroadcastStreams().contains(streamId); + boolean isBroadcast = broadcastStreams.contains(streamId); edge = new StreamEdge(streamSpec, isIntermediate, isBroadcast, config); edges.put(streamId, edge); } @@ -299,6 +319,9 @@ private void validateInternalStreams() { * Validate all nodes are reachable by sources. */ private void validateReachability() { + if (isTaskApplication) { + return; + } // validate all nodes are reachable from the sources final Set reachable = findReachable(); if (reachable.size() != nodes.size()) { diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index a9f744cc74..923043da79 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -30,6 +30,7 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -39,12 +40,15 @@ import org.apache.samza.config.TaskConfig; import org.apache.samza.config.TaskConfigJava; import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.spec.StatefulOperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; +import org.apache.samza.serializers.KVSerde; import org.apache.samza.table.TableConfigGenerator; import org.apache.samza.util.MathUtil; import org.apache.samza.serializers.Serde; @@ -219,6 +223,75 @@ public JobConfig generateConfig(String executionPlanJson) { extractScopedConfig(new MapConfig(allowedConfigs), new MapConfig(configs), configPrefix))); } + //TODO: SAMZA-1811: duplicated most of the logic from config generation for high-level graph. Need to consolidate. + public JobConfig generateTaskApplicationConfig(TaskApplicationDescriptorImpl taskAppDesc) { + Map configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), jobName); + configs.put(JobConfig.JOB_ID(), jobId); + + final List inputs = new ArrayList<>(); + final List broadcasts = new ArrayList<>(); + for (StreamEdge inEdge : inEdges) { + String formattedSystemStream = inEdge.getName(); + if (inEdge.isBroadcast()) { + broadcasts.add(formattedSystemStream + "#0"); + } else { + inputs.add(formattedSystemStream); + } + } + + if (!broadcasts.isEmpty()) { + // TODO: remove this once we support defining broadcast input stream in high-level + // task.broadcast.input should be generated by the planner in the future. + final String taskBroadcasts = config.get(TaskConfigJava.BROADCAST_INPUT_STREAMS); + if (StringUtils.isNoneEmpty(taskBroadcasts)) { + broadcasts.add(taskBroadcasts); + } + configs.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcasts)); + } + + // write input/output streams to configs + inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); + + // write serialized serde instances and stream serde configs to configs + addSerdeConfigs(configs, taskAppDesc); + + configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); + + // Add side inputs to the inputs and mark the stream as bootstrap + tables.forEach(tableSpec -> { + List sideInputs = tableSpec.getSideInputs(); + if (sideInputs != null && !sideInputs.isEmpty()) { + sideInputs.stream() + .map(sideInput -> StreamUtil.getSystemStreamFromNameOrId(config, sideInput)) + .forEach(systemStream -> { + inputs.add(StreamUtil.getNameFromSystemStream(systemStream)); + configs.put(String.format(StreamConfig.STREAM_PREFIX() + StreamConfig.BOOTSTRAP(), + systemStream.getSystem(), systemStream.getStream()), "true"); + }); + } + }); + + configs.put(TaskConfig.INPUT_STREAMS(), Joiner.on(',').join(inputs)); + + log.info("Job {} has generated configs {}", jobName, configs); + + String configPrefix = String.format(CONFIG_JOB_PREFIX, jobName); + + // Disallow user specified job inputs/outputs. This info comes strictly from the user application. + Map allowedConfigs = new HashMap<>(config); + if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { + log.warn("Specifying task inputs in configuration is not allowed with Fluent API. " + + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); + allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); + } + + log.debug("Job {} has allowed configs {}", jobName, allowedConfigs); + return new JobConfig( + Util.rewriteConfig( + extractScopedConfig(new MapConfig(allowedConfigs), new MapConfig(configs), configPrefix))); + } + /** * Serializes the {@link Serde} instances for operators, adds them to the provided config, and * sets the serde configuration for the input/output/intermediate streams appropriately. @@ -315,6 +388,75 @@ void addSerdeConfigs(Map configs) { }); } + /** + * Serializes the {@link Serde} instances for operators, adds them to the provided config, and + * sets the serde configuration for the input/output/intermediate streams appropriately. + * + * We try to preserve the number of Serde instances before and after serialization. However we don't + * guarantee that references shared between these serdes instances (e.g. an Jackson ObjectMapper shared + * between two json serdes) are shared after deserialization too. + * + * Ideally all the user defined objects in the application should be serialized and de-serialized in one pass + * from the same output/input stream so that we can maintain reference sharing relationships. + * + * @param configs the configs to add serialized serde instances and stream serde configs to + */ + //TODO: SAMZA-1811: another duplicated method for low-level application that needs to be consolidated later. + void addSerdeConfigs(Map configs, TaskApplicationDescriptorImpl taskAppDesc) { + // collect all key and msg serde instances for streams + Map streamKeySerdes = new HashMap<>(); + Map streamMsgSerdes = new HashMap<>(); + Map inputDescriptors = taskAppDesc.getInputDescriptors(); + inEdges.forEach(edge -> { + String streamId = edge.getStreamSpec().getId(); + InputDescriptor inputDescriptor = inputDescriptors.get(streamId); + addSerde(streamId, inputDescriptor.getSerde(), streamKeySerdes, streamMsgSerdes); + }); + Map outputDescriptors = taskAppDesc.getOutputDescriptors(); + outEdges.forEach(edge -> { + String streamId = edge.getStreamSpec().getId(); + OutputDescriptor outputDescriptor = outputDescriptors.get(streamId); + addSerde(streamId, outputDescriptor.getSerde(), streamKeySerdes, streamMsgSerdes); + }); + + // for each unique stream or store serde instance, generate a unique name and serialize to config + HashSet serdes = new HashSet<>(streamKeySerdes.values()); + serdes.addAll(streamMsgSerdes.values()); + SerializableSerde serializableSerde = new SerializableSerde<>(); + Base64.Encoder base64Encoder = Base64.getEncoder(); + Map serdeUUIDs = new HashMap<>(); + serdes.forEach(serde -> { + String serdeName = serdeUUIDs.computeIfAbsent(serde, + s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); + configs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), + base64Encoder.encodeToString(serializableSerde.toBytes(serde))); + }); + + // set key and msg serdes for streams to the serde names generated above + streamKeySerdes.forEach((streamId, serde) -> { + String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); + String keySerdeConfigKey = streamIdPrefix + StreamConfig.KEY_SERDE(); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); + + streamMsgSerdes.forEach((streamId, serde) -> { + String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); + String valueSerdeConfigKey = streamIdPrefix + StreamConfig.MSG_SERDE(); + configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); + }); + } + + private void addSerde(String streamId, Serde serde, Map keySerdeMap, Map msgSerdeMap) { + if (serde != null) { + if (serde instanceof KVSerde) { + keySerdeMap.put(streamId, ((KVSerde) serde).getKeySerde()); + msgSerdeMap.put(streamId, ((KVSerde) serde).getValueSerde()); + } else { + msgSerdeMap.put(streamId, serde); + } + } + } + /** * Computes the triggering interval to use during the execution of this {@link JobNode} */ diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index 41b28a7b27..6f8a513f42 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.PrintWriter; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -31,7 +30,6 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.ApplicationDescriptors; import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; @@ -40,10 +38,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.config.ShellCommandConfig; import org.apache.samza.config.StreamConfig; -import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.table.TableConfigGenerator; -import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,22 +62,17 @@ public abstract class JobPlanner { public List prepareJobs() { String appId = new ApplicationConfig(appDesc.getConfig()).getGlobalAppId(); - return ApplicationDescriptors.forType( - taskAppDesc -> { - try { - return Collections.singletonList(prepareTaskJob(taskAppDesc)); - } catch (Exception e) { - throw new SamzaException("Failed to generate JobConfig for TaskApplication " + appId, e); - } - }, - streamAppDesc -> { - try { - return prepareStreamJobs(streamAppDesc); - } catch (Exception e) { - throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); - } - }, - appDesc); + if (appDesc instanceof TaskApplicationDescriptorImpl) { + return Collections.singletonList(prepareTaskJob((TaskApplicationDescriptorImpl) appDesc)); + } else if (appDesc instanceof StreamApplicationDescriptorImpl) { + try { + return prepareStreamJobs((StreamApplicationDescriptorImpl) appDesc); + } catch (Exception e) { + throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); + } + } + throw new IllegalArgumentException(String.format("AppDescriptorImpl has to be either TaskAppDescriptorImpl or " + + "StreamAppDescriptorImpl. class %s is not supported", appDesc.getClass().getName())); } abstract List prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception; @@ -93,12 +83,12 @@ StreamManager buildAndStartStreamManager(Config config) { return streamManager; } - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) throws Exception { + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) { return getExecutionPlan(specGraph, null); } /* package private */ - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) throws Exception { + ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) { // update application configs Map cfg = new HashMap<>(); @@ -117,7 +107,6 @@ ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) throws // descriptor generated configuration has higher priority Map systemStreamConfigs = expandSystemStreamConfigs(appDesc); cfg.putAll(systemStreamConfigs); - // TODO: should generate table configuration from table descriptors as well (SAMZA-1815) // adding app.class in the configuration cfg.put(ApplicationConfig.APP_CLASS, appDesc.getAppClass().getName()); @@ -156,20 +145,19 @@ final void writePlanJsonFile(String planJson) { } } + // TODO: SAMZA-1811: The following private helper to generate single node JobGraph for TaskApplication should be consolidated + // to ExecutionPlanner // helper method to generate a single node job configuration for low level task applications private JobConfig prepareTaskJob(TaskApplicationDescriptorImpl taskAppDesc) { // copy original configure - Map cfg = new HashMap<>(); + Map cfg = new HashMap<>(config); // expand system and streams configure Map systemStreamConfigs = expandSystemStreamConfigs(taskAppDesc); cfg.putAll(systemStreamConfigs); - // expand table configure - cfg.putAll(expandTableConfigs(cfg, taskAppDesc)); - // adding app.class in the configuration - cfg.put(ApplicationConfig.APP_CLASS, appDesc.getAppClass().getName()); - // create the physical execution plan and merge with overrides. This works for a single-stage job now - // TODO: This should all be consolidated with ExecutionPlanner after fixing SAMZA-1811 - return new JobConfig(JobNode.mergeJobConfig(config, new MapConfig(cfg))); + + Config generatedCfgFromStreamDescriptors = new MapConfig(cfg); + JobGraph singleNodeJobGraph = ExecutionPlanner.createJobGraph(generatedCfgFromStreamDescriptors, taskAppDesc); + return singleNodeJobGraph.getSingleNodeJobConfig(taskAppDesc); } private Map expandSystemStreamConfigs(ApplicationDescriptorImpl appDesc) { @@ -181,12 +169,4 @@ private Map expandSystemStreamConfigs(ApplicationDescriptorImpl< systemStreamConfigs.put(JobConfig.JOB_DEFAULT_SYSTEM(), dsd.getSystemName())); return systemStreamConfigs; } - - private Map expandTableConfigs(Map originConfig, - ApplicationDescriptorImpl appDesc) { - List tableSpecs = new ArrayList<>(); - appDesc.getTableDescriptors().stream().map(td -> ((BaseTableDescriptor) td).getTableSpec()) - .forEach(spec -> tableSpecs.add(spec)); - return TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(originConfig), tableSpecs); - } } 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 9a5eadb53c..4ef9f9c03b 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 @@ -105,7 +105,6 @@ public class StreamProcessor { private final Object lock = new Object(); private Throwable containerException = null; - private boolean processorOnStartCalled = false; volatile CountDownLatch containerShutdownLatch = new CountDownLatch(1); @@ -158,8 +157,8 @@ public State getState() { * @param taskFactory the {@link TaskFactory} to be used for creating task instances. * @param processorListener listener to the StreamProcessor life cycle. */ - public StreamProcessor(Config config, Map customMetricsReporters, - TaskFactory taskFactory, ProcessorLifecycleListener processorListener) { + public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, + ProcessorLifecycleListener processorListener) { this(config, customMetricsReporters, taskFactory, processorListener, null); } @@ -173,25 +172,13 @@ public StreamProcessor(Config config, Map customMetrics * @param jobCoordinator the instance of {@link JobCoordinator} */ public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, - ProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator) { - Preconditions.checkNotNull(processorListener, "ProcessorListener cannot be null."); - this.taskFactory = taskFactory; - this.config = config; - this.taskShutdownMs = new TaskConfigJava(config).getShutdownMs(); - this.processorListener = processorListener; - this.customMetricsReporter = customMetricsReporters; - this.jobCoordinator = (jobCoordinator != null) ? jobCoordinator : createJobCoordinator(); - this.jobCoordinatorListener = createJobCoordinatorListener(); - this.jobCoordinator.setListener(jobCoordinatorListener); - ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(CONTAINER_THREAD_NAME_FORMAT).setDaemon(true).build(); - this.executorService = Executors.newSingleThreadExecutor(threadFactory); - // TODO: remove the dependency on jobCoordinator for processorId after fixing SAMZA-1835 - this.processorId = this.jobCoordinator.getProcessorId(); + ProcessorLifecycleListener processorListener, JobCoordinator jobCoordinator) { + this(config, customMetricsReporters, taskFactory, sp -> processorListener, jobCoordinator); } /** * Same as {@link #StreamProcessor(Config, Map, TaskFactory, ProcessorLifecycleListener, JobCoordinator)}, except - * there is a {@link StreamProcessorListenerFactory} as input instead of {@link ProcessorLifecycleListener}. + * there is a {@link StreamProcessorLifecycleListenerFactory} as input instead of {@link ProcessorLifecycleListener}. * This is useful to create a {@link ProcessorLifecycleListener} with a reference to this {@link StreamProcessor} * * @param config configuration required to launch {@link JobCoordinator} and {@link SamzaContainer} @@ -201,7 +188,7 @@ public StreamProcessor(Config config, Map customMetrics * @param jobCoordinator the instance of {@link JobCoordinator} */ public StreamProcessor(Config config, Map customMetricsReporters, TaskFactory taskFactory, - StreamProcessorListenerFactory listenerFactory, JobCoordinator jobCoordinator) { + StreamProcessorLifecycleListenerFactory listenerFactory, JobCoordinator jobCoordinator) { Preconditions.checkNotNull(listenerFactory, "StreamProcessorListenerFactory cannot be null."); this.taskFactory = taskFactory; this.config = config; @@ -226,9 +213,9 @@ public StreamProcessor(Config config, Map customMetrics *

    */ public void start() { - processorListener.beforeStart(); synchronized (lock) { if (state == State.NEW) { + processorListener.beforeStart(); state = State.STARTED; jobCoordinator.start(); } else { @@ -256,7 +243,7 @@ public void start() { *
      *
    1. container is shutdown cleanly and {@link SamzaContainerListener#afterStop()} will trigger * {@link JobCoordinator#stop()}
    2. - *
    3. container fails to shutdown cleanly and {@link SamzaContainerListener#afterFailed(Throwable)} will + *
    4. container fails to shutdown cleanly and {@link SamzaContainerListener#afterFailure(Throwable)} will * trigger {@link JobCoordinator#stop()}
    5. *
    * If container is not running, then this method will simply shutdown the {@link JobCoordinator}. @@ -289,12 +276,12 @@ JobCoordinator getCurrentJobCoordinator() { return jobCoordinator; } - /* package private for testing */ @VisibleForTesting SamzaContainer getContainer() { return container; } + @VisibleForTesting SamzaContainer createSamzaContainer(String processorId, JobModel jobModel) { return SamzaContainer.apply(processorId, jobModel, config, ScalaJavaUtil.toScalaMap(customMetricsReporter), taskFactory); } @@ -400,12 +387,14 @@ public void onCoordinatorFailure(Throwable throwable) { * Interface to create a {@link ProcessorLifecycleListener} */ @FunctionalInterface - public interface StreamProcessorListenerFactory { + public interface StreamProcessorLifecycleListenerFactory { ProcessorLifecycleListener createInstance(StreamProcessor processor); } class ContainerListener implements SamzaContainerListener { + private boolean processorOnStartCalled = false; + @Override public void beforeStart() { // processorListener.beforeStart() is invoked in StreamProcessor.start() @@ -436,7 +425,7 @@ public void afterStop() { } @Override - public void afterFailed(Throwable t) { + public void afterFailure(Throwable t) { containerShutdownLatch.countDown(); synchronized (lock) { LOGGER.error(String.format("Container: %s failed with an exception. Stopping the stream processor: %s. Original exception:", container, processorId), containerException); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java index 7e15c16cef..17a9dc15a1 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ApplicationRunnerMain.java @@ -21,6 +21,7 @@ import joptsimple.OptionSet; import joptsimple.OptionSpec; +import org.apache.samza.application.ApplicationUtil; import org.apache.samza.config.Config; import org.apache.samza.util.CommandLine; import org.apache.samza.util.Util; @@ -54,7 +55,7 @@ public static void main(String[] args) throws Exception { ApplicationRunnerOperation op = cmdLine.getOperation(options); ApplicationRunner appRunner = - ApplicationRunners.getApplicationRunner(ApplicationClassUtils.fromConfig(config), config); + ApplicationRunners.getApplicationRunner(ApplicationUtil.fromConfig(config), config); switch (op) { case RUN: diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 88354d36f0..7100482f86 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -35,7 +35,7 @@ import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.ApplicationDescriptorUtil; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -65,19 +65,18 @@ public class LocalApplicationRunner implements ApplicationRunner { private ApplicationStatus appStatus = ApplicationStatus.New; /** - * Default public constructor that is required by any implementation of {@link ApplicationRunner} + * Constructors a {@link LocalApplicationRunner} to run the {@code app} with the {@code config}. * - * @param userApp user application - * @param config user configuration + * @param app application to run + * @param config configuration for the application */ - public LocalApplicationRunner(SamzaApplication userApp, Config config) { - this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); + public LocalApplicationRunner(SamzaApplication app, Config config) { + this.appDesc = ApplicationDescriptorUtil.getAppDescriptor(app, config); this.planner = new LocalJobPlanner(appDesc); } /** * Constructor only used in unit test to allow injection of {@link LocalJobPlanner} - * */ @VisibleForTesting LocalApplicationRunner(ApplicationDescriptorImpl appDesc, LocalJobPlanner planner) { @@ -160,9 +159,9 @@ CountDownLatch getShutdownLatch() { return shutdownLatch; } - /* package private */ + @VisibleForTesting StreamProcessor createStreamProcessor(Config config, ApplicationDescriptorImpl appDesc, - StreamProcessor.StreamProcessorListenerFactory listenerFactory) { + StreamProcessor.StreamProcessorLifecycleListenerFactory listenerFactory) { TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); Map reporters = new HashMap<>(); // TODO: the null processorId has to be fixed after SAMZA-1835 @@ -174,10 +173,16 @@ StreamProcessor createStreamProcessor(Config config, ApplicationDescriptorImpl appDesc = - ApplicationDescriptors.getAppDescriptor(ApplicationClassUtils.fromConfig(config), config); + ApplicationDescriptorUtil.getAppDescriptor(ApplicationUtil.fromConfig(config), config); run(appDesc, containerId, jobModel, config); System.exit(0); @@ -120,7 +121,7 @@ public void afterStop() { } @Override - public void afterFailed(Throwable t) { + public void afterFailure(Throwable t) { log.info("Container Failed"); containerRunnerException = t; listener.afterFailure(t); diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index e2cbfdcd34..69eb5fee53 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -25,7 +25,7 @@ import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.ApplicationDescriptorUtil; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.execution.RemoteJobPlanner; @@ -49,13 +49,13 @@ public class RemoteApplicationRunner implements ApplicationRunner { private final RemoteJobPlanner planner; /** - * Default public constructor that is required by any implementation of {@link ApplicationRunner} + * Constructors a {@link RemoteApplicationRunner} to run the {@code app} with the {@code config}. * - * @param userApp user application - * @param config user configuration + * @param app application to run + * @param config configuration for the application */ - public RemoteApplicationRunner(SamzaApplication userApp, Config config) { - this.appDesc = ApplicationDescriptors.getAppDescriptor(userApp, config); + public RemoteApplicationRunner(SamzaApplication app, Config config) { + this.appDesc = ApplicationDescriptorUtil.getAppDescriptor(app, config); this.planner = new RemoteJobPlanner(appDesc); } diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java index c350907fc5..5833df9f5c 100644 --- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java +++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java @@ -18,21 +18,19 @@ */ package org.apache.samza.task; +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.application.TaskApplicationDescriptorImpl; -import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; -import org.apache.samza.config.TaskConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.ExecutorService; -import static org.apache.samza.util.ScalaJavaUtil.toScalaFunction; - /** * This class provides utility functions to load task factory classes based on config, and to wrap {@link StreamTaskFactory} * in {@link AsyncStreamTaskFactory} when running {@link StreamTask}s in multi-thread mode @@ -47,10 +45,14 @@ public class TaskFactoryUtil { * @return {@link TaskFactory} object defined by {@code appDesc} */ public static TaskFactory getTaskFactory(ApplicationDescriptorImpl appDesc) { - return ApplicationDescriptors.forType(TaskApplicationDescriptorImpl::getTaskFactory, - streamAppDesc -> (StreamTaskFactory) () -> - new StreamOperatorTask(streamAppDesc.getOperatorSpecGraph(), streamAppDesc.getContextManager()), - appDesc); + if (appDesc instanceof TaskApplicationDescriptorImpl) { + return ((TaskApplicationDescriptorImpl) appDesc).getTaskFactory(); + } else if (appDesc instanceof StreamApplicationDescriptorImpl) { + return (StreamTaskFactory) () -> new StreamOperatorTask(((StreamApplicationDescriptorImpl) appDesc).getOperatorSpecGraph(), + ((StreamApplicationDescriptorImpl) appDesc).getContextManager()); + } + throw new IllegalArgumentException(String.format("AppDescriptorImpl has to be either TaskAppDescriptorImpl or " + + "StreamAppDescriptorImpl. class %s is not supported", appDesc.getClass().getName())); } /** @@ -58,16 +60,11 @@ public static TaskFactory getTaskFactory(ApplicationDescriptorImpl * This should only be used to create {@link TaskFactory} defined in task.class * - * @param config the {@link Config} for this job + * @param taskClassName the task class name for this job * @return a {@link TaskFactory} object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory} */ - public static TaskFactory getTaskFactoryFromConfig(Config config) { - // if there is configuration to set the job w/ a specific type of task, instantiate the corresponding task factory - String taskClassName = new TaskConfig(config).getTaskClass().getOrElse(toScalaFunction( - () -> { - throw new ConfigException("No task class defined in the configuration."); - })); - + public static TaskFactory getTaskFactory(String taskClassName) { + Preconditions.checkArgument(StringUtils.isNotBlank(taskClassName), "task.class cannot be empty"); log.info("Got task class name: {}", taskClassName); boolean isAsyncTaskClass; diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 16b9813582..68de4a630c 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -868,7 +868,7 @@ class SamzaContainer( } case SamzaContainerStatus.FAILED => if (containerListener != null) { - containerListener.afterFailed(exceptionSeen) + containerListener.afterFailure(exceptionSeen) } } } @@ -881,7 +881,7 @@ class SamzaContainer( * Implementation: Stops the [[RunLoop]], which will eventually transition the container from * [[SamzaContainerStatus.STARTED]] to either [[SamzaContainerStatus.STOPPED]] or [[SamzaContainerStatus.FAILED]]]. * Based on the final `status`, [[SamzaContainerListener#afterStop()]] or - * [[SamzaContainerListener#afterFailed(Throwable]] will be invoked respectively. + * [[SamzaContainerListener#afterFailure(Throwable]] will be invoked respectively. * * @throws SamzaException, Thrown when the container has already been stopped or failed */ diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala index d97684971d..abd7f65689 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala @@ -19,7 +19,7 @@ package org.apache.samza.job.local -import org.apache.samza.application.ApplicationDescriptors +import org.apache.samza.application.{ApplicationDescriptorUtil, ApplicationUtil} import org.apache.samza.config.{Config, TaskConfigJava} import org.apache.samza.config.JobConfig._ import org.apache.samza.config.ShellCommandConfig._ @@ -28,7 +28,7 @@ import org.apache.samza.coordinator.JobModelManager import org.apache.samza.coordinator.stream.CoordinatorStreamManager import org.apache.samza.job.{StreamJob, StreamJobFactory} import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter} -import org.apache.samza.runtime.{ApplicationClassUtils, ProcessorContext} +import org.apache.samza.runtime.ProcessorContext import org.apache.samza.storage.ChangelogStreamManager import org.apache.samza.task.TaskFactory import org.apache.samza.task.TaskFactoryUtil @@ -73,7 +73,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging { val containerId = "0" val jmxServer = new JmxServer - val appDesc = ApplicationDescriptors.getAppDescriptor(ApplicationClassUtils.fromConfig(config), config) + val appDesc = ApplicationDescriptorUtil.getAppDescriptor(ApplicationUtil.fromConfig(config), config) val taskFactory : TaskFactory[_] = TaskFactoryUtil.getTaskFactory(appDesc) // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job. @@ -83,24 +83,24 @@ class ThreadJobFactory extends StreamJobFactory with Logging { case _ => None } - val listener = { - val userListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(new ProcessorContext() { }, config) + val containerListener = { + val processorLifecycleListener = appDesc.getProcessorLifecycleListenerFactory().createInstance(new ProcessorContext() { }, config) new SamzaContainerListener { - override def afterFailed(t: Throwable): Unit = { - userListener.afterFailure(t) - throw t; + override def afterFailure(t: Throwable): Unit = { + processorLifecycleListener.afterFailure(t) + throw t } override def afterStart(): Unit = { - userListener.afterStart() + processorLifecycleListener.afterStart() } override def afterStop(): Unit = { - userListener.afterStop() + processorLifecycleListener.afterStop() } override def beforeStart(): Unit = { - userListener.beforeStart() + processorLifecycleListener.beforeStart() } } @@ -114,7 +114,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging { config, Map[String, MetricsReporter](), taskFactory) - container.setContainerListener(listener) + container.setContainerListener(containerListener) val threadJob = new ThreadJob(container) threadJob diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java b/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java similarity index 76% rename from samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java rename to samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java index faba6d844b..9b590c4941 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationClassUtils.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestApplicationUtil.java @@ -16,15 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.runtime; +package org.apache.samza.application; import java.util.HashMap; import java.util.Map; -import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.TaskApplicationDescriptor; -import org.apache.samza.application.TaskApplicationDescriptorImpl; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.MockStreamApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; @@ -37,19 +32,19 @@ /** - * Unit tests for {@link ApplicationClassUtils} + * Unit tests for {@link ApplicationUtil} */ -public class TestApplicationClassUtils { +public class TestApplicationUtil { @Test public void testStreamAppClass() { Map configMap = new HashMap<>(); configMap.put(ApplicationConfig.APP_CLASS, MockStreamApplication.class.getName()); - SamzaApplication app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + SamzaApplication app = ApplicationUtil.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockStreamApplication); configMap.put(TaskConfig.TASK_CLASS(), MockStreamTask.class.getName()); - app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + app = ApplicationUtil.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockStreamApplication); } @@ -57,11 +52,11 @@ public void testStreamAppClass() { public void testTaskAppClass() { Map configMap = new HashMap<>(); configMap.put(ApplicationConfig.APP_CLASS, MockTaskApplication.class.getName()); - SamzaApplication app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + SamzaApplication app = ApplicationUtil.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockTaskApplication); configMap.put(TaskConfig.TASK_CLASS(), MockStreamTask.class.getName()); - app = ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + app = ApplicationUtil.fromConfig(new MapConfig(configMap)); assertTrue(app instanceof MockTaskApplication); } @@ -70,16 +65,23 @@ public void testTaskClassOnly() { Map configMap = new HashMap<>(); configMap.put(TaskConfig.TASK_CLASS(), MockStreamTask.class.getName()); Config config = new MapConfig(configMap); - SamzaApplication app = ApplicationClassUtils.fromConfig(config); + SamzaApplication app = ApplicationUtil.fromConfig(config); assertTrue(app instanceof TaskApplication); TaskApplicationDescriptorImpl appSpec = new TaskApplicationDescriptorImpl((TaskApplication) app, config); assertTrue(appSpec.getTaskFactory().createInstance() instanceof MockStreamTask); } + @Test(expected = ConfigException.class) + public void testEmptyTaskClassOnly() { + Map configMap = new HashMap<>(); + configMap.put(TaskConfig.TASK_CLASS(), ""); + ApplicationUtil.fromConfig(new MapConfig(configMap)); + } + @Test(expected = ConfigException.class) public void testNoAppClassNoTaskClass() { Map configMap = new HashMap<>(); - ApplicationClassUtils.fromConfig(new MapConfig(configMap)); + ApplicationUtil.fromConfig(new MapConfig(configMap)); } /** 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 61cf6c52f1..ea74540590 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 @@ -24,11 +24,14 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.samza.Partition; import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -36,9 +39,13 @@ import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.GenericInputDescriptor; import org.apache.samza.operators.descriptors.GenericOutputDescriptor; import org.apache.samza.operators.descriptors.GenericSystemDescriptor; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.windows.Windows; import org.apache.samza.serializers.KVSerde; @@ -62,6 +69,11 @@ public class TestExecutionPlanner { private static final String DEFAULT_SYSTEM = "test-system"; private static final int DEFAULT_PARTITIONS = 10; + private final Set systemDescriptors = new HashSet<>(); + private final Map inputDescriptors = new HashMap<>(); + private final Map outputDescriptors = new HashMap<>(); + private final Set tableDescriptors = new HashSet<>(); + private SystemAdmins systemAdmins; private StreamManager streamManager; private Config config; @@ -78,6 +90,8 @@ public class TestExecutionPlanner { private GenericOutputDescriptor> output1Descriptor; private StreamSpec output2Spec; private GenericOutputDescriptor> output2Descriptor; + private GenericSystemDescriptor system1Descriptor; + private GenericSystemDescriptor system2Descriptor; static SystemAdmin createSystemAdmin(Map streamToPartitions) { @@ -232,14 +246,29 @@ public void setup() { KVSerde kvSerde = new KVSerde<>(new NoOpSerde(), new NoOpSerde()); String mockSystemFactoryClass = "factory.class.name"; - GenericSystemDescriptor system1 = new GenericSystemDescriptor("system1", mockSystemFactoryClass); - GenericSystemDescriptor system2 = new GenericSystemDescriptor("system2", mockSystemFactoryClass); - input1Descriptor = system1.getInputDescriptor("input1", kvSerde); - input2Descriptor = system2.getInputDescriptor("input2", kvSerde); - input3Descriptor = system2.getInputDescriptor("input3", kvSerde); - input4Descriptor = system1.getInputDescriptor("input4", kvSerde); - output1Descriptor = system1.getOutputDescriptor("output1", kvSerde); - output2Descriptor = system2.getOutputDescriptor("output2", kvSerde); + system1Descriptor = new GenericSystemDescriptor("system1", mockSystemFactoryClass); + system2Descriptor = new GenericSystemDescriptor("system2", mockSystemFactoryClass); + input1Descriptor = system1Descriptor.getInputDescriptor("input1", kvSerde); + input2Descriptor = system2Descriptor.getInputDescriptor("input2", kvSerde); + input3Descriptor = system2Descriptor.getInputDescriptor("input3", kvSerde); + input4Descriptor = system1Descriptor.getInputDescriptor("input4", kvSerde); + output1Descriptor = system1Descriptor.getOutputDescriptor("output1", kvSerde); + output2Descriptor = system2Descriptor.getOutputDescriptor("output2", kvSerde); + + // clean and set up sets and maps of descriptors + systemDescriptors.clear(); + inputDescriptors.clear(); + outputDescriptors.clear(); + tableDescriptors.clear(); + systemDescriptors.add(system1Descriptor); + systemDescriptors.add(system2Descriptor); + inputDescriptors.put(input1Descriptor.getStreamId(), input1Descriptor); + inputDescriptors.put(input2Descriptor.getStreamId(), input2Descriptor); + inputDescriptors.put(input3Descriptor.getStreamId(), input3Descriptor); + inputDescriptors.put(input4Descriptor.getStreamId(), input4Descriptor); + outputDescriptors.put(output1Descriptor.getStreamId(), output1Descriptor); + outputDescriptors.put(output2Descriptor.getStreamId(), output2Descriptor); + // set up external partition count Map system1Map = new HashMap<>(); @@ -447,4 +476,52 @@ public void testMaxPartitionLimit() throws Exception { assertEquals(partitionLimit, edge.getPartitionCount()); // max of input1 and output1 }); } + + @Test + public void testCreateJobGraphForTaskApplication() { + TaskApplicationDescriptorImpl taskAppDesc = mock(TaskApplicationDescriptorImpl.class); + // add interemediate streams + String intermediateStream1 = "intermediate-stream1"; + String intermediateBroadcast = "intermediate-broadcast1"; + // intermediate stream1, not broadcast + GenericInputDescriptor> intermediateInput1 = system1Descriptor.getInputDescriptor( + intermediateStream1, new KVSerde<>(new NoOpSerde(), new NoOpSerde())); + GenericOutputDescriptor> intermediateOutput1 = system1Descriptor.getOutputDescriptor( + intermediateStream1, new KVSerde<>(new NoOpSerde(), new NoOpSerde())); + // intermediate stream2, broadcast + GenericInputDescriptor> intermediateBroacastInput1 = system1Descriptor.getInputDescriptor( + intermediateBroadcast, new KVSerde<>(new NoOpSerde<>(), new NoOpSerde<>())); + GenericOutputDescriptor> intermediateBroacastOutput1 = system1Descriptor.getOutputDescriptor( + intermediateBroadcast, new KVSerde<>(new NoOpSerde<>(), new NoOpSerde<>())); + inputDescriptors.put(intermediateStream1, intermediateInput1); + outputDescriptors.put(intermediateStream1, intermediateOutput1); + inputDescriptors.put(intermediateBroadcast, intermediateBroacastInput1); + outputDescriptors.put(intermediateBroadcast, intermediateBroacastOutput1); + Set broadcastStreams = new HashSet<>(); + broadcastStreams.add(intermediateBroadcast); + + when(taskAppDesc.getInputDescriptors()).thenReturn(inputDescriptors); + when(taskAppDesc.getOutputDescriptors()).thenReturn(outputDescriptors); + when(taskAppDesc.getTableDescriptors()).thenReturn(tableDescriptors); + when(taskAppDesc.getSystemDescriptors()).thenReturn(systemDescriptors); + when(taskAppDesc.getBroadcastStreams()).thenReturn(broadcastStreams); + + Map systemStreamConfigs = new HashMap<>(); + inputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); + outputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); + systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); + + JobGraph jobGraph = ExecutionPlanner.createJobGraph(config, taskAppDesc); + assertEquals(1, jobGraph.getJobNodes().size()); + assertTrue(jobGraph.getSources().stream().map(edge -> edge.getName()) + .filter(streamId -> inputDescriptors.containsKey(streamId)).collect(Collectors.toList()).isEmpty()); + Set intermediateStreams = new HashSet<>(inputDescriptors.keySet()); + jobGraph.getSources().forEach(edge -> { + if (intermediateStreams.contains(edge.getName())) { + intermediateStreams.remove(edge.getName()); + } + }); + assertEquals(new HashSet() { { this.add(intermediateStream1); this.add(intermediateBroadcast); } }.toArray(), + intermediateStreams.toArray()); + } } diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 6a74b1945a..19ee74f9a7 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -26,7 +26,7 @@ import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.SamzaApplication; -import org.apache.samza.application.ApplicationDescriptors; +import org.apache.samza.application.ApplicationDescriptorUtil; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.TaskApplication; import org.apache.samza.config.ApplicationConfig; @@ -79,8 +79,8 @@ public void testRunStreamTask() StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerFactory.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorLifecycleListenerFactory.class); doAnswer(i -> { @@ -114,8 +114,8 @@ public void testRunComplete() doReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))).when(localPlanner).prepareJobs(); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerFactory.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorLifecycleListenerFactory.class); doAnswer(i -> { @@ -149,8 +149,8 @@ public void testRunFailure() doReturn(Collections.singletonList(new JobConfig(new MapConfig(config)))).when(localPlanner).prepareJobs(); StreamProcessor sp = mock(StreamProcessor.class); - ArgumentCaptor captor = - ArgumentCaptor.forClass(StreamProcessor.StreamProcessorListenerFactory.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(StreamProcessor.StreamProcessorLifecycleListenerFactory.class); doAnswer(i -> { @@ -186,7 +186,7 @@ public void testWaitForFinishTimesout() { } private void prepareTest() { - ApplicationDescriptorImpl appDesc = ApplicationDescriptors.getAppDescriptor(mockApp, config); + ApplicationDescriptorImpl appDesc = ApplicationDescriptorUtil.getAppDescriptor(mockApp, config); localPlanner = spy(new LocalJobPlanner(appDesc)); runner = spy(new LocalApplicationRunner(appDesc, localPlanner)); } diff --git a/samza-core/src/test/java/org/apache/samza/task/TestAsyncStreamTask.java b/samza-core/src/test/java/org/apache/samza/task/MockAsyncStreamTask.java similarity index 94% rename from samza-core/src/test/java/org/apache/samza/task/TestAsyncStreamTask.java rename to samza-core/src/test/java/org/apache/samza/task/MockAsyncStreamTask.java index 26e8c8579e..e449ecc786 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestAsyncStreamTask.java +++ b/samza-core/src/test/java/org/apache/samza/task/MockAsyncStreamTask.java @@ -23,7 +23,7 @@ /** * Test implementation class for {@link AsyncStreamTask} */ -public class TestAsyncStreamTask implements AsyncStreamTask { +public class MockAsyncStreamTask implements AsyncStreamTask { @Override public void processAsync(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator, TaskCallback callback) { diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java index 297016a948..f96ab197c7 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java @@ -19,15 +19,12 @@ package org.apache.samza.task; import java.lang.reflect.Field; -import java.util.HashMap; import java.util.concurrent.ExecutorService; import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.application.TaskApplicationDescriptorImpl; -import org.apache.samza.config.Config; import org.apache.samza.config.ConfigException; -import org.apache.samza.config.MapConfig; import org.apache.samza.operators.OperatorSpecGraph; import org.junit.Test; @@ -46,22 +43,12 @@ public class TestTaskFactoryUtil { @Test public void testStreamTaskClass() { - Config config = new MapConfig(new HashMap() { - { - this.put("task.class", MockStreamTask.class.getName()); - } - }); - Object retFactory = TaskFactoryUtil.getTaskFactoryFromConfig(config); + TaskFactory retFactory = TaskFactoryUtil.getTaskFactory(MockStreamTask.class.getName()); assertTrue(retFactory instanceof StreamTaskFactory); assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof MockStreamTask); - config = new MapConfig(new HashMap() { - { - this.put("task.class", "no.such.class"); - } - }); try { - TaskFactoryUtil.getTaskFactoryFromConfig(config); + TaskFactoryUtil.getTaskFactory("no.such.class"); fail("Should have failed w/ no.such.class"); } catch (ConfigException cfe) { // expected @@ -70,22 +57,12 @@ public void testStreamTaskClass() { @Test public void testAsyncStreamTask() { - Config config = new MapConfig(new HashMap() { - { - this.put("task.class", TestAsyncStreamTask.class.getName()); - } - }); - Object retFactory = TaskFactoryUtil.getTaskFactoryFromConfig(config); + TaskFactory retFactory = TaskFactoryUtil.getTaskFactory(MockAsyncStreamTask.class.getName()); assertTrue(retFactory instanceof AsyncStreamTaskFactory); - assertTrue(((AsyncStreamTaskFactory) retFactory).createInstance() instanceof TestAsyncStreamTask); + assertTrue(((AsyncStreamTaskFactory) retFactory).createInstance() instanceof MockAsyncStreamTask); - config = new MapConfig(new HashMap() { - { - this.put("task.class", "no.such.class"); - } - }); try { - TaskFactoryUtil.getTaskFactoryFromConfig(config); + TaskFactoryUtil.getTaskFactory("no.such.class"); fail("Should have failed w/ no.such.class"); } catch (ConfigException cfe) { // expected diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala index c586766122..ff57047069 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala @@ -199,7 +199,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def afterFailed(t: Throwable): Unit = { + override def afterFailure(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } @@ -287,7 +287,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { producerMultiplexer = producerMultiplexer, metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def afterFailed(t: Throwable): Unit = { + override def afterFailure(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } @@ -379,7 +379,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { producerMultiplexer = producerMultiplexer, metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def afterFailed(t: Throwable): Unit = { + override def afterFailure(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } @@ -471,7 +471,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { producerMultiplexer = producerMultiplexer, metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def afterFailed(t: Throwable): Unit = { + override def afterFailure(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } @@ -559,7 +559,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { metrics = new SamzaContainerMetrics) val containerListener = new SamzaContainerListener { - override def afterFailed(t: Throwable): Unit = { + override def afterFailure(t: Throwable): Unit = { onContainerFailedCalled = true onContainerFailedThrowable = t } diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java index 43bd62ed20..34b264f5aa 100644 --- a/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java +++ b/samza-test/src/main/java/org/apache/samza/test/integration/LocalApplicationRunnerMain.java @@ -21,7 +21,7 @@ import joptsimple.OptionSet; import org.apache.samza.application.SamzaApplication; -import org.apache.samza.runtime.ApplicationClassUtils; +import org.apache.samza.application.ApplicationUtil; import org.apache.samza.config.Config; import org.apache.samza.runtime.ApplicationRunnerMain; import org.apache.samza.runtime.ApplicationRunner; @@ -46,7 +46,7 @@ public static void main(String[] args) throws Exception { Config orgConfig = cmdLine.loadConfig(options); Config config = Util.rewriteConfig(orgConfig); - SamzaApplication app = ApplicationClassUtils.fromConfig(config); + SamzaApplication app = ApplicationUtil.fromConfig(config); ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config); try { From 05637e6e6cb53e053d495123112b5efe88c746d2 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Tue, 11 Sep 2018 11:09:25 -0700 Subject: [PATCH 28/38] SAMZA-1814: WIP consolidate all JobGraph and JobNode Json and Config generation code to support both high- and low-level applications --- .../samza/execution/ExecutionPlanner.java | 210 +------- .../IntermediateStreamPartitionPlanner.java | 211 ++++++++ .../org/apache/samza/execution/JobGraph.java | 52 +- .../execution/JobGraphConfigureGenerator.java | 453 +++++++++++++++++ .../execution/JobGraphJsonGenerator.java | 125 +++-- .../org/apache/samza/execution/JobNode.java | 454 +----------------- .../apache/samza/execution/JobPlanner.java | 47 +- .../samza/execution/LocalJobPlanner.java | 6 +- .../samza/execution/RemoteJobPlanner.java | 6 +- .../samza/execution/TestExecutionPlanner.java | 36 +- .../apache/samza/execution/TestJobGraph.java | 36 +- .../TestJobGraphConfigureGenerator.java | 339 +++++++++++++ .../execution/TestJobGraphJsonGenerator.java | 7 +- .../apache/samza/execution/TestJobNode.java | 172 ------- .../samza/execution/TestRemoteJobPlanner.java | 2 +- 15 files changed, 1181 insertions(+), 975 deletions(-) create mode 100644 samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java create mode 100644 samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java create mode 100644 samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 7d0767484a..990f7ebe37 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -24,22 +24,17 @@ import java.util.Collection; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.Map; -import java.util.Queue; import java.util.Set; -import java.util.stream.Collectors; import org.apache.samza.SamzaException; -import org.apache.samza.application.TaskApplicationDescriptorImpl; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.StreamConfig; -import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.spec.JoinOperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; import org.apache.samza.table.TableSpec; @@ -57,8 +52,6 @@ public class ExecutionPlanner { private static final Logger log = LoggerFactory.getLogger(ExecutionPlanner.class); - static final int MAX_INFERRED_PARTITIONS = 256; - private final Config config; private final StreamManager streamManager; @@ -67,18 +60,18 @@ public ExecutionPlanner(Config config, StreamManager streamManager) { this.streamManager = streamManager; } - public ExecutionPlan plan(OperatorSpecGraph specGraph) { + public ExecutionPlan plan(ApplicationDescriptorImpl appDesc) { validateConfig(); // create physical job graph based on stream graph - JobGraph jobGraph = createJobGraph(specGraph); + JobGraph jobGraph = createJobGraph(config, new JobGraphJsonGenerator(appDesc), new JobGraphConfigureGenerator(appDesc)); // fetch the external streams partition info updateExistingPartitions(jobGraph, streamManager); if (!jobGraph.getIntermediateStreamEdges().isEmpty()) { // figure out the partitions for internal streams - calculatePartitions(jobGraph); + new IntermediateStreamPartitionPlanner(config, appDesc).calculatePartitions(jobGraph); } return jobGraph; @@ -94,54 +87,17 @@ private void validateConfig() { } } - /** - * Create the single node physical graph from {@link org.apache.samza.application.TaskApplicationDescriptorImpl} - */ - // TODO: SAMZA-1811: fix this and consolidate w/ high-level application's createJobGrapg(OperatorSpecGraph) method - /* package private */ static JobGraph createJobGraph(Config config, TaskApplicationDescriptorImpl taskAppDesc) { - JobGraph jobGraph = new JobGraph(config, taskAppDesc); - StreamConfig streamConfig = new StreamConfig(config); - Set sourceStreams = getStreamSpecs(taskAppDesc.getInputDescriptors().keySet(), streamConfig); - Set sinkStreams = getStreamSpecs(taskAppDesc.getOutputDescriptors().keySet(), streamConfig); - Set intStreams = new HashSet<>(sourceStreams); - Set tables = new HashSet<>(taskAppDesc.getTableDescriptors().stream() - .map(tableDesc -> ((BaseTableDescriptor) tableDesc).getTableSpec()).collect(Collectors.toSet())); - intStreams.retainAll(sinkStreams); - sourceStreams.removeAll(intStreams); - sinkStreams.removeAll(intStreams); - - // For this phase, we have a single job node for the whole dag - String jobName = config.get(JobConfig.JOB_NAME()); - String jobId = config.get(JobConfig.JOB_ID(), "1"); - JobNode node = jobGraph.getOrCreateJobNode(jobName, jobId); - - // add sources - sourceStreams.forEach(spec -> jobGraph.addSource(spec, node)); - - // add sinks - sinkStreams.forEach(spec -> jobGraph.addSink(spec, node)); - - // add intermediate streams - intStreams.forEach(spec -> jobGraph.addIntermediateStream(spec, node, node)); - - // add tables - tables.forEach(spec -> jobGraph.addTable(spec, node)); - - jobGraph.validate(); - - return jobGraph; - } - /** * Create the physical graph from {@link OperatorSpecGraph} */ - /* package private */ JobGraph createJobGraph(OperatorSpecGraph specGraph) { - JobGraph jobGraph = new JobGraph(config, specGraph); + /* package private */ static JobGraph createJobGraph(Config config, JobGraphJsonGenerator jobJsonGenerator, + JobGraphConfigureGenerator jobConfigureGenerator) { + JobGraph jobGraph = new JobGraph(config, jobJsonGenerator, jobConfigureGenerator); StreamConfig streamConfig = new StreamConfig(config); - Set sourceStreams = getStreamSpecs(specGraph.getInputOperators().keySet(), streamConfig); - Set sinkStreams = getStreamSpecs(specGraph.getOutputStreams().keySet(), streamConfig); + Set sourceStreams = getStreamSpecs(jobConfigureGenerator.getInputStreamIds(), streamConfig); + Set sinkStreams = getStreamSpecs(jobConfigureGenerator.getOutputStreamIds(), streamConfig); Set intStreams = new HashSet<>(sourceStreams); - Set tables = new HashSet<>(specGraph.getTables().keySet()); + Set tables = new HashSet<>(jobConfigureGenerator.getTableSpecs()); intStreams.retainAll(sinkStreams); sourceStreams.removeAll(intStreams); sinkStreams.removeAll(intStreams); @@ -168,20 +124,6 @@ private void validateConfig() { return jobGraph; } - /** - * Figure out the number of partitions of all streams - */ - /* package private */ void calculatePartitions(JobGraph jobGraph) { - // calculate the partitions for the input streams of join operators - calculateJoinInputPartitions(jobGraph, config); - - // calculate the partitions for the rest of intermediate streams - calculateIntStreamPartitions(jobGraph, config); - - // validate all the partitions are assigned - validatePartitions(jobGraph); - } - /** * Fetch the partitions of source/sink streams and update the StreamEdges. * @param jobGraph {@link JobGraph} @@ -213,134 +155,4 @@ private void validateConfig() { }); } } - - /** - * Calculate the partitions for the input streams of join operators - */ - /* package private */ static void calculateJoinInputPartitions(JobGraph jobGraph, Config config) { - // mapping from a source stream to all join specs reachable from it - Multimap joinSpecToStreamEdges = HashMultimap.create(); - // reverse mapping of the above - Multimap streamEdgeToJoinSpecs = HashMultimap.create(); - // A queue of joins with known input partitions - Queue joinQ = new LinkedList<>(); - // The visited set keeps track of the join specs that have been already inserted in the queue before - Set visited = new HashSet<>(); - - StreamConfig streamConfig = new StreamConfig(config); - - jobGraph.getSpecGraph().getInputOperators().forEach((key, value) -> { - StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(getStreamSpec(key, streamConfig)); - // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge - findReachableJoins(value, streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, visited); - }); - - // At this point, joinQ contains joinSpecs where at least one of the input stream edge partitions is known. - while (!joinQ.isEmpty()) { - OperatorSpec join = joinQ.poll(); - int partitions = StreamEdge.PARTITIONS_UNKNOWN; - // loop through the input streams to the join and find the partition count - for (StreamEdge edge : joinSpecToStreamEdges.get(join)) { - int edgePartitions = edge.getPartitionCount(); - if (edgePartitions != StreamEdge.PARTITIONS_UNKNOWN) { - if (partitions == StreamEdge.PARTITIONS_UNKNOWN) { - //if the partition is not assigned - partitions = edgePartitions; - log.info("Inferred the partition count {} for the join operator {} from {}." - , new Object[] {partitions, join.getOpId(), edge.getName()}); - } else if (partitions != edgePartitions) { - throw new SamzaException(String.format( - "Unable to resolve input partitions of stream %s for the join %s. Expected: %d, Actual: %d", - edge.getName(), join.getOpId(), partitions, edgePartitions)); - } - } - } - - // assign the partition count for intermediate streams - for (StreamEdge edge : joinSpecToStreamEdges.get(join)) { - if (edge.getPartitionCount() <= 0) { - log.info("Set the partition count to {} for input stream {} to the join {}.", - new Object[] {partitions, edge.getName(), join.getOpId()}); - edge.setPartitionCount(partitions); - - // find other joins can be inferred by setting this edge - for (OperatorSpec op : streamEdgeToJoinSpecs.get(edge)) { - if (!visited.contains(op)) { - joinQ.add(op); - visited.add(op); - } - } - } - } - } - } - - /** - * This function traverses the {@link OperatorSpec} graph to find and update mappings for all Joins reachable - * from this input {@link StreamEdge}. - * @param operatorSpec the {@link OperatorSpec} to traverse - * @param sourceStreamEdge source {@link StreamEdge} - * @param joinSpecToStreamEdges mapping from join spec to its source {@link StreamEdge}s - * @param streamEdgeToJoinSpecs mapping from source {@link StreamEdge} to the join specs that consumes it - * @param joinQ queue that contains joinSpecs where at least one of the input stream edge partitions is known. - */ - private static void findReachableJoins(OperatorSpec operatorSpec, StreamEdge sourceStreamEdge, - Multimap joinSpecToStreamEdges, - Multimap streamEdgeToJoinSpecs, - Queue joinQ, Set visited) { - if (operatorSpec instanceof JoinOperatorSpec) { - joinSpecToStreamEdges.put(operatorSpec, sourceStreamEdge); - streamEdgeToJoinSpecs.put(sourceStreamEdge, operatorSpec); - - if (!visited.contains(operatorSpec) && sourceStreamEdge.getPartitionCount() > 0) { - // put the joins with known input partitions into the queue and mark as visited - joinQ.add(operatorSpec); - visited.add(operatorSpec); - } - } - - Collection registeredOperatorSpecs = operatorSpec.getRegisteredOperatorSpecs(); - for (OperatorSpec registeredOpSpec : registeredOperatorSpecs) { - findReachableJoins(registeredOpSpec, sourceStreamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, - visited); - } - } - - private static void calculateIntStreamPartitions(JobGraph jobGraph, Config config) { - int partitions = config.getInt(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), StreamEdge.PARTITIONS_UNKNOWN); - if (partitions < 0) { - // use the following simple algo to figure out the partitions - // partition = MAX(MAX(Input topic partitions), MAX(Output topic partitions)) - // partition will be further bounded by MAX_INFERRED_PARTITIONS. - // This is important when running in hadoop where an HDFS input can have lots of files (partitions). - int maxInPartitions = maxPartition(jobGraph.getSources()); - int maxOutPartitions = maxPartition(jobGraph.getSinks()); - partitions = Math.max(maxInPartitions, maxOutPartitions); - - if (partitions > MAX_INFERRED_PARTITIONS) { - partitions = MAX_INFERRED_PARTITIONS; - log.warn(String.format("Inferred intermediate stream partition count %d is greater than the max %d. Using the max.", - partitions, MAX_INFERRED_PARTITIONS)); - } - } - for (StreamEdge edge : jobGraph.getIntermediateStreamEdges()) { - if (edge.getPartitionCount() <= 0) { - log.info("Set the partition count for intermediate stream {} to {}.", edge.getName(), partitions); - edge.setPartitionCount(partitions); - } - } - } - - private static void validatePartitions(JobGraph jobGraph) { - for (StreamEdge edge : jobGraph.getIntermediateStreamEdges()) { - if (edge.getPartitionCount() <= 0) { - throw new SamzaException(String.format("Failure to assign the partitions to Stream %s", edge.getName())); - } - } - } - - /* package private */ static int maxPartition(Collection edges) { - return edges.stream().map(StreamEdge::getPartitionCount).reduce(Integer::max).orElse(StreamEdge.PARTITIONS_UNKNOWN); - } - } diff --git a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java new file mode 100644 index 0000000000..126d15b735 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java @@ -0,0 +1,211 @@ +/* + * 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.execution; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.StreamConfig; +import org.apache.samza.operators.spec.InputOperatorSpec; +import org.apache.samza.operators.spec.JoinOperatorSpec; +import org.apache.samza.operators.spec.OperatorSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.samza.util.StreamUtil.getStreamSpec; + + +/** + * {@link IntermediateStreamPartitionPlanner} calculates intermediate stream partitions based on the high-level application graph. + */ +class IntermediateStreamPartitionPlanner { + + private static final Logger log = LoggerFactory.getLogger(IntermediateStreamPartitionPlanner.class); + + private final Config config; + private final Map inputOperators; + + @VisibleForTesting + static final int MAX_INFERRED_PARTITIONS = 256; + + IntermediateStreamPartitionPlanner(Config config, ApplicationDescriptorImpl appDesc) { + this.config = config; + this.inputOperators = appDesc instanceof StreamApplicationDescriptorImpl ? + ((StreamApplicationDescriptorImpl) appDesc).getInputOperators() : new HashMap<>(); + } + + /** + * Figure out the number of partitions of all streams + */ + /* package private */ void calculatePartitions(JobGraph jobGraph) { + // only compute partitions for high-level API applications + + // calculate the partitions for the input streams of join operators + calculateJoinInputPartitions(jobGraph); + + // calculate the partitions for the rest of intermediate streams + calculateIntStreamPartitions(jobGraph); + + // validate all the partitions are assigned + validatePartitions(jobGraph); + } + + /** + * Calculate the partitions for the input streams of join operators + */ + /* package private */ void calculateJoinInputPartitions(JobGraph jobGraph) { + // mapping from a source stream to all join specs reachable from it + Multimap joinSpecToStreamEdges = HashMultimap.create(); + // reverse mapping of the above + Multimap streamEdgeToJoinSpecs = HashMultimap.create(); + // A queue of joins with known input partitions + Queue joinQ = new LinkedList<>(); + // The visited set keeps track of the join specs that have been already inserted in the queue before + Set visited = new HashSet<>(); + + StreamConfig streamConfig = new StreamConfig(config); + + inputOperators.forEach((key, value) -> { + StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(getStreamSpec(key, streamConfig)); + // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge + findReachableJoins(value, streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, visited); + }); + + // At this point, joinQ contains joinSpecs where at least one of the input stream edge partitions is known. + while (!joinQ.isEmpty()) { + OperatorSpec join = joinQ.poll(); + int partitions = StreamEdge.PARTITIONS_UNKNOWN; + // loop through the input streams to the join and find the partition count + for (StreamEdge edge : joinSpecToStreamEdges.get(join)) { + int edgePartitions = edge.getPartitionCount(); + if (edgePartitions != StreamEdge.PARTITIONS_UNKNOWN) { + if (partitions == StreamEdge.PARTITIONS_UNKNOWN) { + //if the partition is not assigned + partitions = edgePartitions; + log.info("Inferred the partition count {} for the join operator {} from {}." + , new Object[] {partitions, join.getOpId(), edge.getName()}); + } else if (partitions != edgePartitions) { + throw new SamzaException(String.format( + "Unable to resolve input partitions of stream %s for the join %s. Expected: %d, Actual: %d", + edge.getName(), join.getOpId(), partitions, edgePartitions)); + } + } + } + + // assign the partition count for intermediate streams + for (StreamEdge edge : joinSpecToStreamEdges.get(join)) { + if (edge.getPartitionCount() <= 0) { + log.info("Set the partition count to {} for input stream {} to the join {}.", + new Object[] {partitions, edge.getName(), join.getOpId()}); + edge.setPartitionCount(partitions); + + // find other joins can be inferred by setting this edge + for (OperatorSpec op : streamEdgeToJoinSpecs.get(edge)) { + if (!visited.contains(op)) { + joinQ.add(op); + visited.add(op); + } + } + } + } + } + } + + /** + * This function traverses the {@link OperatorSpec} graph to find and update mappings for all Joins reachable + * from this input {@link StreamEdge}. + * @param operatorSpec the {@link OperatorSpec} to traverse + * @param sourceStreamEdge source {@link StreamEdge} + * @param joinSpecToStreamEdges mapping from join spec to its source {@link StreamEdge}s + * @param streamEdgeToJoinSpecs mapping from source {@link StreamEdge} to the join specs that consumes it + * @param joinQ queue that contains joinSpecs where at least one of the input stream edge partitions is known. + */ + private static void findReachableJoins(OperatorSpec operatorSpec, StreamEdge sourceStreamEdge, + Multimap joinSpecToStreamEdges, + Multimap streamEdgeToJoinSpecs, + Queue joinQ, Set visited) { + if (operatorSpec instanceof JoinOperatorSpec) { + joinSpecToStreamEdges.put(operatorSpec, sourceStreamEdge); + streamEdgeToJoinSpecs.put(sourceStreamEdge, operatorSpec); + + if (!visited.contains(operatorSpec) && sourceStreamEdge.getPartitionCount() > 0) { + // put the joins with known input partitions into the queue and mark as visited + joinQ.add(operatorSpec); + visited.add(operatorSpec); + } + } + + Collection registeredOperatorSpecs = operatorSpec.getRegisteredOperatorSpecs(); + for (OperatorSpec registeredOpSpec : registeredOperatorSpecs) { + findReachableJoins(registeredOpSpec, sourceStreamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, + visited); + } + } + + private void calculateIntStreamPartitions(JobGraph jobGraph) { + int partitions = config.getInt(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), StreamEdge.PARTITIONS_UNKNOWN); + if (partitions < 0) { + // use the following simple algo to figure out the partitions + // partition = MAX(MAX(Input topic partitions), MAX(Output topic partitions)) + // partition will be further bounded by MAX_INFERRED_PARTITIONS. + // This is important when running in hadoop where an HDFS input can have lots of files (partitions). + int maxInPartitions = maxPartition(jobGraph.getSources()); + int maxOutPartitions = maxPartition(jobGraph.getSinks()); + partitions = Math.max(maxInPartitions, maxOutPartitions); + + if (partitions > MAX_INFERRED_PARTITIONS) { + partitions = MAX_INFERRED_PARTITIONS; + log.warn(String.format("Inferred intermediate stream partition count %d is greater than the max %d. Using the max.", + partitions, MAX_INFERRED_PARTITIONS)); + } + } + for (StreamEdge edge : jobGraph.getIntermediateStreamEdges()) { + if (edge.getPartitionCount() <= 0) { + log.info("Set the partition count for intermediate stream {} to {}.", edge.getName(), partitions); + edge.setPartitionCount(partitions); + } + } + } + + private static void validatePartitions(JobGraph jobGraph) { + for (StreamEdge edge : jobGraph.getIntermediateStreamEdges()) { + if (edge.getPartitionCount() <= 0) { + throw new SamzaException(String.format("Failure to assign the partitions to Stream %s", edge.getName())); + } + } + } + + /* package private */ static int maxPartition(Collection edges) { + return edges.stream().map(StreamEdge::getPartitionCount).reduce(Integer::max).orElse(StreamEdge.PARTITIONS_UNKNOWN); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 146a2bf9bd..2d0c46f140 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -31,12 +31,9 @@ import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.StreamApplicationDescriptorImpl; -import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.operators.OperatorSpecGraph; import org.apache.samza.system.StreamSpec; import org.apache.samza.table.TableSpec; import org.slf4j.Logger; @@ -61,29 +58,17 @@ private final Set intermediateStreams = new HashSet<>(); private final Set tables = new HashSet<>(); private final Config config; - private final JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); - private final OperatorSpecGraph specGraph; - private final Set broadcastStreams; - private final boolean isTaskApplication; + private final JobGraphJsonGenerator jsonGenerator; + private final JobGraphConfigureGenerator configGenerator; /** * The JobGraph is only constructed by the {@link ExecutionPlanner}. * @param config Config */ - JobGraph(Config config, OperatorSpecGraph specGraph) { + JobGraph(Config config, JobGraphJsonGenerator jsonGenerator, JobGraphConfigureGenerator configureGenerator) { + this.jsonGenerator = jsonGenerator; + this.configGenerator = configureGenerator; this.config = config; - this.specGraph = specGraph; - this.broadcastStreams = specGraph.getBroadcastStreams(); - this.isTaskApplication = false; - } - - JobGraph(Config config, TaskApplicationDescriptorImpl taskAppDesc) { - this.config = config; - // TODO: HACK!!! Need to be fixed after SAMZA-1811 - // create a dummy specGraph - this.specGraph = new StreamApplicationDescriptorImpl(appDesc -> { }, config).getOperatorSpecGraph(); - this.broadcastStreams = taskAppDesc.getBroadcastStreams(); - this.isTaskApplication = true; } @Override @@ -106,16 +91,6 @@ public List getIntermediateStreams() { .collect(Collectors.toList()); } - // TODO: SAMZA-1811: consolidate this with high-level application JobGraph - JobConfig getSingleNodeJobConfig(TaskApplicationDescriptorImpl taskAppDesc) { - return getJobNodes().get(0).generateTaskApplicationConfig(taskAppDesc); - } - - void addTable(TableSpec tableSpec, JobNode node) { - tables.add(tableSpec); - node.addTable(tableSpec); - } - @Override public String getPlanAsJson() throws Exception { return jsonGenerator.toJson(this); @@ -125,14 +100,11 @@ public String getPlanAsJson() throws Exception { * Returns the config for this application * @return {@link ApplicationConfig} */ + @Override public ApplicationConfig getApplicationConfig() { return new ApplicationConfig(config); } - public OperatorSpecGraph getSpecGraph() { - return specGraph; - } - /** * Add a source stream to a {@link JobNode} * @param input source stream @@ -172,6 +144,11 @@ void addIntermediateStream(StreamSpec streamSpec, JobNode from, JobNode to) { intermediateStreams.add(edge); } + void addTable(TableSpec tableSpec, JobNode node) { + tables.add(tableSpec); + node.addTable(tableSpec); + } + /** * Get the {@link JobNode}. Create one if it does not exist. * @param jobName name of the job @@ -182,7 +159,7 @@ JobNode getOrCreateJobNode(String jobName, String jobId) { String nodeId = JobNode.createId(jobName, jobId); JobNode node = nodes.get(nodeId); if (node == null) { - node = new JobNode(jobName, jobId, specGraph, config); + node = new JobNode(jobName, jobId, config, configGenerator); nodes.put(nodeId, node); } return node; @@ -207,7 +184,7 @@ StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec, boolean isIntermediate) String streamId = streamSpec.getId(); StreamEdge edge = edges.get(streamId); if (edge == null) { - boolean isBroadcast = broadcastStreams.contains(streamId); + boolean isBroadcast = configGenerator.isBroadcastStream(streamId); edge = new StreamEdge(streamSpec, isIntermediate, isBroadcast, config); edges.put(streamId, edge); } @@ -319,9 +296,6 @@ private void validateInternalStreams() { * Validate all nodes are reachable by sources. */ private void validateReachability() { - if (isTaskApplication) { - return; - } // validate all nodes are reachable from the sources final Set reachable = findReachable(); if (reachable.size() != nodes.size()) { diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java new file mode 100644 index 0000000000..9e44635b90 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java @@ -0,0 +1,453 @@ +/* + * 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.execution; + +import com.google.common.base.Joiner; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptor; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.SerializerConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.config.StreamConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.config.TaskConfigJava; +import org.apache.samza.operators.BaseTableDescriptor; +import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; +import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; +import org.apache.samza.operators.spec.InputOperatorSpec; +import org.apache.samza.operators.spec.JoinOperatorSpec; +import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.operators.spec.OutputStreamImpl; +import org.apache.samza.operators.spec.StatefulOperatorSpec; +import org.apache.samza.operators.spec.StoreDescriptor; +import org.apache.samza.operators.spec.WindowOperatorSpec; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerializableSerde; +import org.apache.samza.table.TableConfigGenerator; +import org.apache.samza.table.TableSpec; +import org.apache.samza.util.MathUtil; +import org.apache.samza.util.StreamUtil; +import org.apache.samza.util.Util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This class provides methods to generate configuration for a {@link JobNode} + */ +public class JobGraphConfigureGenerator { + + private static final Logger LOG = LoggerFactory.getLogger(JobGraphConfigureGenerator.class); + private static final String CONFIG_JOB_PREFIX = "jobs.%s."; + private static final String CONFIG_INTERNAL_EXECUTION_PLAN = "samza.internal.execution.plan"; + + private final StreamSerdeManager streamSerdeManager; + private final Set broadcastStreamIds; + private final Set inputStreamIds; + private final Set outputStreamIds; + private final Set tableSpecs; + + JobGraphConfigureGenerator(ApplicationDescriptorImpl appDesc) { + this.streamSerdeManager = new StreamSerdeManager(appDesc); + this.broadcastStreamIds = appDesc.getBroadcastStreams(); + this.inputStreamIds = appDesc instanceof StreamApplicationDescriptor ? + ((StreamApplicationDescriptorImpl) appDesc).getInputOperators().keySet() : appDesc.getInputDescriptors().keySet(); + this.outputStreamIds = appDesc instanceof StreamApplicationDescriptor ? + ((StreamApplicationDescriptorImpl) appDesc).getOutputStreams().keySet() : appDesc.getOutputDescriptors().keySet(); + this.tableSpecs = appDesc.getTableDescriptors().stream().map(tableDesc -> ((BaseTableDescriptor) tableDesc).getTableSpec()) + .collect(Collectors.toSet()); + } + + static Config mergeJobConfig(Config fullConfig, Config generatedConfig) { + return new JobConfig(Util.rewriteConfig(extractScopedConfig( + fullConfig, generatedConfig, String.format(CONFIG_JOB_PREFIX, new JobConfig(fullConfig).getName().get())))); + } + + boolean isBroadcastStream(String streamId) { + return this.broadcastStreamIds.contains(streamId); + } + + Set getInputStreamIds() { + return inputStreamIds; + } + + Set getOutputStreamIds() { + return outputStreamIds; + } + + Set getTableSpecs() { + return tableSpecs; + } + + JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { + Map configs = new HashMap<>(); + // set up job name and job ID + configs.put(JobConfig.JOB_NAME(), jobNode.getJobName()); + configs.put(JobConfig.JOB_ID(), jobNode.getJobId()); + + List inEdges = jobNode.getInEdges(); + List outEdges = jobNode.getOutEdges(); + Collection reachableOperators = getReachableOperators(jobNode); + boolean hasWindowOrJoin = reachableOperators.stream().anyMatch(op -> op.getOpCode() == OperatorSpec.OpCode.WINDOW + || op.getOpCode() == OperatorSpec.OpCode.JOIN); + List stores = getStoreDescriptors(reachableOperators); + Config config = jobNode.getConfig(); + + // check all inputs to the node for broadcast and input streams + final List inputs = new ArrayList<>(); + final List broadcasts = new ArrayList<>(); + for (StreamEdge inEdge : inEdges) { + String formattedSystemStream = inEdge.getName(); + if (inEdge.isBroadcast()) { + broadcasts.add(formattedSystemStream + "#0"); + } else { + inputs.add(formattedSystemStream); + } + } + + if (!broadcasts.isEmpty()) { + configureBroadcastInputs(configs, config, broadcasts); + } + + // compute window and join operator intervals in this node + if (hasWindowOrJoin) { + configureWindowAndJoinInterval(configs, config, reachableOperators); + } + + // set store configuration for stateful operators. + stores.forEach(sd -> configs.putAll(sd.getStorageConfigs())); + + // set the execution plan in json + configs.put(CONFIG_INTERNAL_EXECUTION_PLAN, executionPlanJson); + + // write intermediate input/output streams to configs + inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); + + // write serialized serde instances and stream /store serdes to configs + configureSerdes(configs, inEdges, outEdges, stores); + + // generate table configuration and potential side input configuration + configureTables(configs, config, jobNode.getTables(), inputs); + + // finalize the task.inputs configuration + configs.put(TaskConfig.INPUT_STREAMS(), Joiner.on(',').join(inputs)); + + LOG.info("Job {} has generated configs {}", jobNode.getId(), configs); + + // apply configure rewriters and user configure overrides + return applyConfigureRewritersAndOverrides(configs, config, jobNode); + } + + private void configureBroadcastInputs(Map configs, Config config, List broadcasts) { + // TODO: remove this once we support defining broadcast input stream in high-level + // task.broadcast.input should be generated by the planner in the future. + final String taskBroadcasts = config.get(TaskConfigJava.BROADCAST_INPUT_STREAMS); + if (StringUtils.isNoneEmpty(taskBroadcasts)) { + broadcasts.add(taskBroadcasts); + } + configs.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcasts)); + } + + private void configureWindowAndJoinInterval(Map configs, Config config, + Collection reachableOperators) { + // set triggering interval if a window or join is defined. Only applies to high-level applications + if ("-1".equals(config.get(TaskConfig.WINDOW_MS(), "-1"))) { + long triggerInterval = computeTriggerInterval(reachableOperators); + LOG.info("Using triggering interval: {}", triggerInterval); + + configs.put(TaskConfig.WINDOW_MS(), String.valueOf(triggerInterval)); + } + } + + private JobConfig applyConfigureRewritersAndOverrides(Map configs, Config config, JobNode jobNode) { + String configPrefix = String.format(CONFIG_JOB_PREFIX, jobNode.getJobName()); + + // Disallow user specified job inputs/outputs. This info comes strictly from the user application. + Map allowedConfigs = new HashMap<>(config); + if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { + LOG.warn("Specifying task inputs in configuration is not allowed with Fluent API. " + + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); + allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); + } + + LOG.debug("Job {} has allowed configs {}", jobNode.getId(), allowedConfigs); + return new JobConfig(Util.rewriteConfig( + extractScopedConfig(new MapConfig(allowedConfigs), new MapConfig(configs), configPrefix))); + } + + /** + * This function extract the subset of configs from the full config, and use it to override the generated configs + * from the job. + * @param fullConfig full config + * @param generatedConfig config generated for the job + * @param configPrefix prefix to extract the subset of the config overrides + * @return config that merges the generated configs and overrides + */ + private static Config extractScopedConfig(Config fullConfig, Config generatedConfig, String configPrefix) { + Config scopedConfig = fullConfig.subset(configPrefix); + + Config[] configPrecedence = new Config[] {fullConfig, generatedConfig, scopedConfig}; + // Strip empty configs so they don't override the configs before them. + Map mergedConfig = new HashMap<>(); + for (Map config : configPrecedence) { + for (Map.Entry property : config.entrySet()) { + String value = property.getValue(); + if (!(value == null || value.isEmpty())) { + mergedConfig.put(property.getKey(), property.getValue()); + } + } + } + scopedConfig = new MapConfig(mergedConfig); + LOG.debug("Prefix '{}' has merged config {}", configPrefix, scopedConfig); + + return scopedConfig; + } + + private List getStoreDescriptors(Collection reachableOperators) { + return reachableOperators.stream().filter(operatorSpec -> operatorSpec instanceof StatefulOperatorSpec) + .map(operatorSpec -> ((StatefulOperatorSpec) operatorSpec).getStoreDescriptors()).flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + private void configureTables(Map configs, Config config, List tables, List inputs) { + configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); + + // Add side inputs to the inputs and mark the stream as bootstrap + tables.forEach(tableSpec -> { + List sideInputs = tableSpec.getSideInputs(); + if (sideInputs != null && !sideInputs.isEmpty()) { + sideInputs.stream() + .map(sideInput -> StreamUtil.getSystemStreamFromNameOrId(config, sideInput)) + .forEach(systemStream -> { + inputs.add(StreamUtil.getNameFromSystemStream(systemStream)); + configs.put(String.format(StreamConfig.STREAM_PREFIX() + StreamConfig.BOOTSTRAP(), + systemStream.getSystem(), systemStream.getStream()), "true"); + }); + } + }); + } + + /** + * Serializes the {@link Serde} instances for operators, adds them to the provided config, and + * sets the serde configuration for the input/output/intermediate streams appropriately. + * + * We try to preserve the number of Serde instances before and after serialization. However we don't + * guarantee that references shared between these serdes instances (e.g. an Jackson ObjectMapper shared + * between two json serdes) are shared after deserialization too. + * + * Ideally all the user defined objects in the application should be serialized and de-serialized in one pass + * from the same output/input stream so that we can maintain reference sharing relationships. + * + * @param configs the configs to add serialized serde instances and stream serde configs to + */ + private void configureSerdes(Map configs, List inEdges, List outEdges, + List stores) { + // collect all key and msg serde instances for streams + Map streamKeySerdes = new HashMap<>(); + Map streamMsgSerdes = new HashMap<>(); + inEdges.forEach(edge -> { + String streamId = edge.getStreamSpec().getId(); + streamSerdeManager.addSerde(sid -> streamSerdeManager.getInputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); + }); + outEdges.forEach(edge -> { + String streamId = edge.getStreamSpec().getId(); + streamSerdeManager.addSerde(sid -> streamSerdeManager.getOutputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); + }); + + Map storeKeySerdes = new HashMap<>(); + Map storeMsgSerdes = new HashMap<>(); + stores.forEach(storeDescriptor -> { + storeKeySerdes.put(storeDescriptor.getStoreName(), storeDescriptor.getKeySerde()); + storeMsgSerdes.put(storeDescriptor.getStoreName(), storeDescriptor.getMsgSerde()); + }); + + // for each unique stream or store serde instance, generate a unique name and serialize to config + HashSet serdes = new HashSet<>(streamKeySerdes.values()); + serdes.addAll(streamMsgSerdes.values()); + serdes.addAll(storeKeySerdes.values()); + serdes.addAll(storeMsgSerdes.values()); + SerializableSerde serializableSerde = new SerializableSerde<>(); + Base64.Encoder base64Encoder = Base64.getEncoder(); + Map serdeUUIDs = new HashMap<>(); + serdes.forEach(serde -> { + String serdeName = serdeUUIDs.computeIfAbsent(serde, + s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); + configs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), + base64Encoder.encodeToString(serializableSerde.toBytes(serde))); + }); + + // set key and msg serdes for streams to the serde names generated above + streamKeySerdes.forEach((streamId, serde) -> { + String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); + String keySerdeConfigKey = streamIdPrefix + StreamConfig.KEY_SERDE(); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); + + streamMsgSerdes.forEach((streamId, serde) -> { + String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); + String valueSerdeConfigKey = streamIdPrefix + StreamConfig.MSG_SERDE(); + configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); + }); + + // set key and msg serdes for stores to the serde names generated above + storeKeySerdes.forEach((storeName, serde) -> { + String keySerdeConfigKey = String.format(StorageConfig.KEY_SERDE(), storeName); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); + + storeMsgSerdes.forEach((storeName, serde) -> { + String msgSerdeConfigKey = String.format(StorageConfig.MSG_SERDE(), storeName); + configs.put(msgSerdeConfigKey, serdeUUIDs.get(serde)); + }); + } + + /** + * Computes the triggering interval to use during the execution of this {@link JobNode} + */ + private long computeTriggerInterval(Collection reachableOperators) { + List windowTimerIntervals = reachableOperators.stream() + .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.WINDOW) + .map(spec -> ((WindowOperatorSpec) spec).getDefaultTriggerMs()) + .collect(Collectors.toList()); + + // Filter out the join operators, and obtain a list of their ttl values + List joinTtlIntervals = reachableOperators.stream() + .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.JOIN) + .map(spec -> ((JoinOperatorSpec) spec).getTtlMs()) + .collect(Collectors.toList()); + + // Combine both the above lists + List candidateTimerIntervals = new ArrayList<>(joinTtlIntervals); + candidateTimerIntervals.addAll(windowTimerIntervals); + + if (candidateTimerIntervals.isEmpty()) { + return -1; + } + + // Compute the gcd of the resultant list + return MathUtil.gcd(candidateTimerIntervals); + } + + Collection getReachableOperators(JobNode jobNode) { + // Filter out window operators, and obtain a list of their triggering interval values + Set inputOperatorsInJobNode = jobNode.getInEdges().stream() + .filter(streamEdge -> streamSerdeManager.inputOperators.containsKey(streamEdge.getStreamSpec().getId())) + .map(streamEdge -> streamSerdeManager.inputOperators.get(streamEdge.getStreamSpec().getId())) + .collect(Collectors.toSet()); + Set reachableOperators = new HashSet<>(); + findReachableOperators(inputOperatorsInJobNode, reachableOperators); + return reachableOperators; + } + + private void findReachableOperators(Collection inputOperatorsInJobNode, Set reachableOperators) { + inputOperatorsInJobNode.forEach(op -> { + if (reachableOperators.contains(op)) { + return; + } + reachableOperators.add(op); + findReachableOperators(op.getRegisteredOperatorSpecs(), reachableOperators); + }); + } + + private class StreamSerdeManager { + private final Map inputDescriptors; + private final Map inputOperators; + private final Map outputDescriptors; + private final Map outputStreams; + + private StreamSerdeManager(ApplicationDescriptorImpl appDesc) { + this.inputDescriptors = appDesc.getInputDescriptors(); + this.outputDescriptors = appDesc.getOutputDescriptors(); + if (appDesc instanceof StreamApplicationDescriptorImpl) { + StreamApplicationDescriptorImpl streamAppDesc = (StreamApplicationDescriptorImpl) appDesc; + this.inputOperators = streamAppDesc.getInputOperators(); + this.outputStreams = streamAppDesc.getOutputStreams(); + } else { + this.inputOperators = new HashMap<>(); + this.outputStreams = new HashMap<>(); + } + } + + private Serde getInputSerde(String streamId) { + InputDescriptor inputDescriptor = inputDescriptors.get(streamId); + if (inputDescriptor != null) { + return inputDescriptor.getSerde(); + } + + // for high-level applications, the intermediate streams don't have the input descriptor yet + InputOperatorSpec inputOp = inputOperators.get(streamId); + if (inputOp == null) { + LOG.warn("Input stream {} don't have any corresponding InputDescriptor or InputOperatorSpec.", streamId); + return null; + } + return KVSerde.of(inputOp.getKeySerde(), inputOp.getValueSerde()); + } + + private Serde getOutputSerde(String streamId) { + OutputDescriptor outputDescriptor = outputDescriptors.get(streamId); + if (outputDescriptor != null) { + return outputDescriptor.getSerde(); + } + + // for high-level applications, the intermediate streams don't have the input descriptor yet + OutputStreamImpl outputStream = outputStreams.get(streamId); + if (outputStream == null) { + LOG.warn("Output stream {} don't have any corresponding OutputDescriptor or OutputStreamImpl.", streamId); + return null; + } + return KVSerde.of(outputStream.getKeySerde(), outputStream.getValueSerde()); + } + + private void addSerde(Function serdeFinder, String streamId, Map keySerdeMap, + Map msgSerdeMap) { + Serde serde = serdeFinder.apply(streamId); + if (serde != null) { + if (serde instanceof KVSerde) { + KVSerde kvSerde = (KVSerde) serde; + if (kvSerde.getKeySerde() != null) { + keySerdeMap.put(streamId, ((KVSerde) serde).getKeySerde()); + } + if (kvSerde.getValueSerde() != null) { + msgSerdeMap.put(streamId, ((KVSerde) serde).getValueSerde()); + } + } else { + msgSerdeMap.put(streamId, serde); + } + } + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 3a8d5c9427..30ef6c9caf 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -29,10 +29,13 @@ import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.operators.spec.OutputOperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.spec.PartitionByOperatorSpec; @@ -117,6 +120,14 @@ static final class JobGraphJson { String applicationId; } + // input operators for the application. For low-level task applications, this is empty. + private final Map inputOperators; + + JobGraphJsonGenerator(ApplicationDescriptorImpl appDesc) { + this.inputOperators = appDesc instanceof StreamApplicationDescriptorImpl ? + ((StreamApplicationDescriptorImpl) appDesc).getInputOperators() : new HashMap<>(); + } + /** * Returns the JSON representation of a {@link JobGraph} * @param jobGraph {@link JobGraph} @@ -149,54 +160,8 @@ static final class JobGraphJson { return new String(out.toByteArray()); } - /** - * Create JSON POJO for a {@link JobNode}, including the {@link org.apache.samza.operators.StreamGraph} for this job - * @param jobNode job node in the {@link JobGraph} - * @return {@link org.apache.samza.execution.JobGraphJsonGenerator.JobNodeJson} - */ - private JobNodeJson buildJobNodeJson(JobNode jobNode) { - JobNodeJson job = new JobNodeJson(); - job.jobName = jobNode.getJobName(); - job.jobId = jobNode.getJobId(); - job.operatorGraph = buildOperatorGraphJson(jobNode); - return job; - } - - /** - * Traverse the {@link OperatorSpec} graph and build the operator graph JSON POJO. - * @param jobNode job node in the {@link JobGraph} - * @return {@link org.apache.samza.execution.JobGraphJsonGenerator.OperatorGraphJson} - */ - private OperatorGraphJson buildOperatorGraphJson(JobNode jobNode) { - OperatorGraphJson opGraph = new OperatorGraphJson(); - opGraph.inputStreams = new ArrayList<>(); - jobNode.getSpecGraph().getInputOperators().forEach((streamId, operatorSpec) -> { - StreamJson inputJson = new StreamJson(); - opGraph.inputStreams.add(inputJson); - inputJson.streamId = streamId; - inputJson.nextOperatorIds = operatorSpec.getRegisteredOperatorSpecs().stream() - .map(OperatorSpec::getOpId).collect(Collectors.toSet()); - - updateOperatorGraphJson(operatorSpec, opGraph); - }); - - opGraph.outputStreams = new ArrayList<>(); - jobNode.getSpecGraph().getOutputStreams().keySet().forEach(streamId -> { - StreamJson outputJson = new StreamJson(); - outputJson.streamId = streamId; - opGraph.outputStreams.add(outputJson); - }); - return opGraph; - } - - /** - * Traverse the {@link OperatorSpec} graph recursively and update the operator graph JSON POJO. - * @param operatorSpec input - * @param opGraph operator graph to build - */ private void updateOperatorGraphJson(OperatorSpec operatorSpec, OperatorGraphJson opGraph) { - // TODO xiliu: render input operators instead of input streams - if (operatorSpec.getOpCode() != OpCode.INPUT) { + if (operatorSpec.getOpCode() != OperatorSpec.OpCode.INPUT) { opGraph.operators.put(operatorSpec.getOpId(), operatorToMap(operatorSpec)); } Collection specs = operatorSpec.getRegisteredOperatorSpecs(); @@ -242,6 +207,70 @@ private Map operatorToMap(OperatorSpec spec) { return map; } + // get all next operators consuming from the input {@code streamId} + private Set getNextOperatorIds(String streamId) { + if (!this.inputOperators.containsKey(streamId)) { + return new HashSet<>(); + } + return this.inputOperators.get(streamId).getRegisteredOperatorSpecs().stream() + .map(op -> op.getOpId()).collect(Collectors.toSet()); + } + + /** + * Traverse the {@link OperatorSpec} graph recursively and update the operator graph JSON POJO. + * @param inputStreamId input streamId + * @param opGraph operator graph to build + */ + private void updateOperatorGraphJson(String inputStreamId, OperatorGraphJson opGraph) { + // TODO xiliu: render input operators instead of input streams + InputOperatorSpec operatorSpec = this.inputOperators.get(inputStreamId); + if (operatorSpec == null) { + // no corresponding input operator for input stream + return; + } + updateOperatorGraphJson(operatorSpec, opGraph); + } + + /** + * Create JSON POJO for a {@link JobNode}, including the {@link org.apache.samza.application.ApplicationDescriptorImpl} + * for this job + * + * @param jobNode job node in the {@link JobGraph} + * @return {@link org.apache.samza.execution.JobGraphJsonGenerator.JobNodeJson} + */ + private JobNodeJson buildJobNodeJson(JobNode jobNode) { + JobNodeJson job = new JobNodeJson(); + job.jobName = jobNode.getJobName(); + job.jobId = jobNode.getJobId(); + job.operatorGraph = buildOperatorGraphJson(jobNode); + return job; + } + + /** + * Traverse the {@link OperatorSpec} graph and build the operator graph JSON POJO. + * @param jobNode job node in the {@link JobGraph} + * @return {@link org.apache.samza.execution.JobGraphJsonGenerator.OperatorGraphJson} + */ + private OperatorGraphJson buildOperatorGraphJson(JobNode jobNode) { + OperatorGraphJson opGraph = new OperatorGraphJson(); + opGraph.inputStreams = new ArrayList<>(); + jobNode.getInEdges().forEach(inStream -> { + StreamJson inputJson = new StreamJson(); + opGraph.inputStreams.add(inputJson); + inputJson.streamId = inStream.getStreamSpec().getId(); + inputJson.nextOperatorIds = getNextOperatorIds(inputJson.streamId); + updateOperatorGraphJson(inputJson.streamId, opGraph); + }); + + opGraph.outputStreams = new ArrayList<>(); + jobNode.getOutEdges().forEach(outStream -> { + StreamJson outputJson = new StreamJson(); + outputJson.streamId = outStream.getStreamSpec().getId(); + opGraph.outputStreams.add(outputJson); + }); + return opGraph; + } + /** * Get or create the JSON POJO for a {@link StreamEdge} * @param edge {@link StreamEdge} diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index 923043da79..5e64947d5c 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -20,48 +20,13 @@ package org.apache.samza.execution; import java.util.ArrayList; -import java.util.Base64; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Collectors; - -import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; -import org.apache.samza.config.SerializerConfig; -import org.apache.samza.config.StorageConfig; -import org.apache.samza.config.StreamConfig; -import org.apache.samza.config.TaskConfig; -import org.apache.samza.config.TaskConfigJava; -import org.apache.samza.operators.OperatorSpecGraph; -import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; -import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; -import org.apache.samza.operators.spec.InputOperatorSpec; -import org.apache.samza.operators.spec.JoinOperatorSpec; -import org.apache.samza.operators.spec.OperatorSpec; -import org.apache.samza.operators.spec.OutputStreamImpl; -import org.apache.samza.operators.spec.StatefulOperatorSpec; -import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.serializers.KVSerde; -import org.apache.samza.table.TableConfigGenerator; -import org.apache.samza.util.MathUtil; -import org.apache.samza.serializers.Serde; -import org.apache.samza.serializers.SerializableSerde; import org.apache.samza.table.TableSpec; -import org.apache.samza.util.StreamUtil; -import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Joiner; - - /** * A JobNode is a physical execution unit. In RemoteExecutionEnvironment, it's a job that will be submitted * to remote cluster. In LocalExecutionEnvironment, it's a set of StreamProcessors for local execution. @@ -69,47 +34,40 @@ */ public class JobNode { private static final Logger log = LoggerFactory.getLogger(JobNode.class); - private static final String CONFIG_JOB_PREFIX = "jobs.%s."; - private static final String CONFIG_INTERNAL_EXECUTION_PLAN = "samza.internal.execution.plan"; private final String jobName; private final String jobId; private final String id; - private final OperatorSpecGraph specGraph; + private final Config config; + private final JobGraphConfigureGenerator configGenerator; private final List inEdges = new ArrayList<>(); private final List outEdges = new ArrayList<>(); private final List tables = new ArrayList<>(); - private final Config config; - JobNode(String jobName, String jobId, OperatorSpecGraph specGraph, Config config) { + JobNode(String jobName, String jobId, Config config, JobGraphConfigureGenerator configureGenerator) { this.jobName = jobName; this.jobId = jobId; this.id = createId(jobName, jobId); - this.specGraph = specGraph; this.config = config; + this.configGenerator = configureGenerator; } - public static Config mergeJobConfig(Config fullConfig, Config generatedConfig) { - return new JobConfig(Util.rewriteConfig(extractScopedConfig( - fullConfig, generatedConfig, String.format(CONFIG_JOB_PREFIX, new JobConfig(fullConfig).getName().get())))); - } - - public OperatorSpecGraph getSpecGraph() { - return this.specGraph; - } - - public String getId() { + String getId() { return id; } - public String getJobName() { + String getJobName() { return jobName; } - public String getJobId() { + String getJobId() { return jobId; } + Config getConfig() { + return config; + } + void addInEdge(StreamEdge in) { inEdges.add(in); } @@ -118,6 +76,10 @@ void addOutEdge(StreamEdge out) { outEdges.add(out); } + void addTable(TableSpec tableSpec) { + tables.add(tableSpec); + } + List getInEdges() { return inEdges; } @@ -126,8 +88,8 @@ List getOutEdges() { return outEdges; } - void addTable(TableSpec tableSpec) { - tables.add(tableSpec); + List getTables() { + return tables; } /** @@ -135,388 +97,12 @@ void addTable(TableSpec tableSpec) { * @param executionPlanJson JSON representation of the execution plan * @return config of the job */ - public JobConfig generateConfig(String executionPlanJson) { - Map configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), jobName); - configs.put(JobConfig.JOB_ID(), jobId); - - final List inputs = new ArrayList<>(); - final List broadcasts = new ArrayList<>(); - for (StreamEdge inEdge : inEdges) { - String formattedSystemStream = inEdge.getName(); - if (inEdge.isBroadcast()) { - broadcasts.add(formattedSystemStream + "#0"); - } else { - inputs.add(formattedSystemStream); - } - } - - if (!broadcasts.isEmpty()) { - // TODO: remove this once we support defining broadcast input stream in high-level - // task.broadcast.input should be generated by the planner in the future. - final String taskBroadcasts = config.get(TaskConfigJava.BROADCAST_INPUT_STREAMS); - if (StringUtils.isNoneEmpty(taskBroadcasts)) { - broadcasts.add(taskBroadcasts); - } - configs.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcasts)); - } - - // set triggering interval if a window or join is defined - if (specGraph.hasWindowOrJoins()) { - if ("-1".equals(config.get(TaskConfig.WINDOW_MS(), "-1"))) { - long triggerInterval = computeTriggerInterval(); - log.info("Using triggering interval: {} for jobName: {}", triggerInterval, jobName); - - configs.put(TaskConfig.WINDOW_MS(), String.valueOf(triggerInterval)); - } - } - - specGraph.getAllOperatorSpecs().forEach(opSpec -> { - if (opSpec instanceof StatefulOperatorSpec) { - ((StatefulOperatorSpec) opSpec).getStoreDescriptors() - .forEach(sd -> configs.putAll(sd.getStorageConfigs())); - // store key and message serdes are configured separately in #addSerdeConfigs - } - }); - - configs.put(CONFIG_INTERNAL_EXECUTION_PLAN, executionPlanJson); - - // write input/output streams to configs - inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); - - // write serialized serde instances and stream serde configs to configs - addSerdeConfigs(configs); - - configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); - - // Add side inputs to the inputs and mark the stream as bootstrap - tables.forEach(tableSpec -> { - List sideInputs = tableSpec.getSideInputs(); - if (sideInputs != null && !sideInputs.isEmpty()) { - sideInputs.stream() - .map(sideInput -> StreamUtil.getSystemStreamFromNameOrId(config, sideInput)) - .forEach(systemStream -> { - inputs.add(StreamUtil.getNameFromSystemStream(systemStream)); - configs.put(String.format(StreamConfig.STREAM_PREFIX() + StreamConfig.BOOTSTRAP(), - systemStream.getSystem(), systemStream.getStream()), "true"); - }); - } - }); - - configs.put(TaskConfig.INPUT_STREAMS(), Joiner.on(',').join(inputs)); - - log.info("Job {} has generated configs {}", jobName, configs); - - String configPrefix = String.format(CONFIG_JOB_PREFIX, jobName); - - // Disallow user specified job inputs/outputs. This info comes strictly from the user application. - Map allowedConfigs = new HashMap<>(config); - if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { - log.warn("Specifying task inputs in configuration is not allowed with Fluent API. " - + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); - allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); - } - - log.debug("Job {} has allowed configs {}", jobName, allowedConfigs); - return new JobConfig( - Util.rewriteConfig( - extractScopedConfig(new MapConfig(allowedConfigs), new MapConfig(configs), configPrefix))); - } - - //TODO: SAMZA-1811: duplicated most of the logic from config generation for high-level graph. Need to consolidate. - public JobConfig generateTaskApplicationConfig(TaskApplicationDescriptorImpl taskAppDesc) { - Map configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), jobName); - configs.put(JobConfig.JOB_ID(), jobId); - - final List inputs = new ArrayList<>(); - final List broadcasts = new ArrayList<>(); - for (StreamEdge inEdge : inEdges) { - String formattedSystemStream = inEdge.getName(); - if (inEdge.isBroadcast()) { - broadcasts.add(formattedSystemStream + "#0"); - } else { - inputs.add(formattedSystemStream); - } - } - - if (!broadcasts.isEmpty()) { - // TODO: remove this once we support defining broadcast input stream in high-level - // task.broadcast.input should be generated by the planner in the future. - final String taskBroadcasts = config.get(TaskConfigJava.BROADCAST_INPUT_STREAMS); - if (StringUtils.isNoneEmpty(taskBroadcasts)) { - broadcasts.add(taskBroadcasts); - } - configs.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcasts)); - } - - // write input/output streams to configs - inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); - - // write serialized serde instances and stream serde configs to configs - addSerdeConfigs(configs, taskAppDesc); - - configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); - - // Add side inputs to the inputs and mark the stream as bootstrap - tables.forEach(tableSpec -> { - List sideInputs = tableSpec.getSideInputs(); - if (sideInputs != null && !sideInputs.isEmpty()) { - sideInputs.stream() - .map(sideInput -> StreamUtil.getSystemStreamFromNameOrId(config, sideInput)) - .forEach(systemStream -> { - inputs.add(StreamUtil.getNameFromSystemStream(systemStream)); - configs.put(String.format(StreamConfig.STREAM_PREFIX() + StreamConfig.BOOTSTRAP(), - systemStream.getSystem(), systemStream.getStream()), "true"); - }); - } - }); - - configs.put(TaskConfig.INPUT_STREAMS(), Joiner.on(',').join(inputs)); - - log.info("Job {} has generated configs {}", jobName, configs); - - String configPrefix = String.format(CONFIG_JOB_PREFIX, jobName); - - // Disallow user specified job inputs/outputs. This info comes strictly from the user application. - Map allowedConfigs = new HashMap<>(config); - if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { - log.warn("Specifying task inputs in configuration is not allowed with Fluent API. " - + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); - allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); - } - - log.debug("Job {} has allowed configs {}", jobName, allowedConfigs); - return new JobConfig( - Util.rewriteConfig( - extractScopedConfig(new MapConfig(allowedConfigs), new MapConfig(configs), configPrefix))); - } - - /** - * Serializes the {@link Serde} instances for operators, adds them to the provided config, and - * sets the serde configuration for the input/output/intermediate streams appropriately. - * - * We try to preserve the number of Serde instances before and after serialization. However we don't - * guarantee that references shared between these serdes instances (e.g. an Jackson ObjectMapper shared - * between two json serdes) are shared after deserialization too. - * - * Ideally all the user defined objects in the application should be serialized and de-serialized in one pass - * from the same output/input stream so that we can maintain reference sharing relationships. - * - * @param configs the configs to add serialized serde instances and stream serde configs to - */ - void addSerdeConfigs(Map configs) { - // collect all key and msg serde instances for streams - Map streamKeySerdes = new HashMap<>(); - Map streamMsgSerdes = new HashMap<>(); - Map inputOperators = specGraph.getInputOperators(); - inEdges.forEach(edge -> { - String streamId = edge.getStreamSpec().getId(); - InputOperatorSpec inputOperatorSpec = inputOperators.get(streamId); - Serde keySerde = inputOperatorSpec.getKeySerde(); - if (keySerde != null) { - streamKeySerdes.put(streamId, keySerde); - } - Serde valueSerde = inputOperatorSpec.getValueSerde(); - if (valueSerde != null) { - streamMsgSerdes.put(streamId, valueSerde); - } - }); - Map outputStreams = specGraph.getOutputStreams(); - outEdges.forEach(edge -> { - String streamId = edge.getStreamSpec().getId(); - OutputStreamImpl outputStream = outputStreams.get(streamId); - Serde keySerde = outputStream.getKeySerde(); - if (keySerde != null) { - streamKeySerdes.put(streamId, keySerde); - } - Serde valueSerde = outputStream.getValueSerde(); - if (valueSerde != null) { - streamMsgSerdes.put(streamId, valueSerde); - } - }); - - // collect all key and msg serde instances for stores - Map storeKeySerdes = new HashMap<>(); - Map storeMsgSerdes = new HashMap<>(); - specGraph.getAllOperatorSpecs().forEach(opSpec -> { - if (opSpec instanceof StatefulOperatorSpec) { - ((StatefulOperatorSpec) opSpec).getStoreDescriptors().forEach(storeDescriptor -> { - storeKeySerdes.put(storeDescriptor.getStoreName(), storeDescriptor.getKeySerde()); - storeMsgSerdes.put(storeDescriptor.getStoreName(), storeDescriptor.getMsgSerde()); - }); - } - }); - - // for each unique stream or store serde instance, generate a unique name and serialize to config - HashSet serdes = new HashSet<>(streamKeySerdes.values()); - serdes.addAll(streamMsgSerdes.values()); - serdes.addAll(storeKeySerdes.values()); - serdes.addAll(storeMsgSerdes.values()); - SerializableSerde serializableSerde = new SerializableSerde<>(); - Base64.Encoder base64Encoder = Base64.getEncoder(); - Map serdeUUIDs = new HashMap<>(); - serdes.forEach(serde -> { - String serdeName = serdeUUIDs.computeIfAbsent(serde, - s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); - configs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), - base64Encoder.encodeToString(serializableSerde.toBytes(serde))); - }); - - // set key and msg serdes for streams to the serde names generated above - streamKeySerdes.forEach((streamId, serde) -> { - String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); - String keySerdeConfigKey = streamIdPrefix + StreamConfig.KEY_SERDE(); - configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); - }); - - streamMsgSerdes.forEach((streamId, serde) -> { - String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); - String valueSerdeConfigKey = streamIdPrefix + StreamConfig.MSG_SERDE(); - configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); - }); - - // set key and msg serdes for stores to the serde names generated above - storeKeySerdes.forEach((storeName, serde) -> { - String keySerdeConfigKey = String.format(StorageConfig.KEY_SERDE(), storeName); - configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); - }); - - storeMsgSerdes.forEach((storeName, serde) -> { - String msgSerdeConfigKey = String.format(StorageConfig.MSG_SERDE(), storeName); - configs.put(msgSerdeConfigKey, serdeUUIDs.get(serde)); - }); - } - - /** - * Serializes the {@link Serde} instances for operators, adds them to the provided config, and - * sets the serde configuration for the input/output/intermediate streams appropriately. - * - * We try to preserve the number of Serde instances before and after serialization. However we don't - * guarantee that references shared between these serdes instances (e.g. an Jackson ObjectMapper shared - * between two json serdes) are shared after deserialization too. - * - * Ideally all the user defined objects in the application should be serialized and de-serialized in one pass - * from the same output/input stream so that we can maintain reference sharing relationships. - * - * @param configs the configs to add serialized serde instances and stream serde configs to - */ - //TODO: SAMZA-1811: another duplicated method for low-level application that needs to be consolidated later. - void addSerdeConfigs(Map configs, TaskApplicationDescriptorImpl taskAppDesc) { - // collect all key and msg serde instances for streams - Map streamKeySerdes = new HashMap<>(); - Map streamMsgSerdes = new HashMap<>(); - Map inputDescriptors = taskAppDesc.getInputDescriptors(); - inEdges.forEach(edge -> { - String streamId = edge.getStreamSpec().getId(); - InputDescriptor inputDescriptor = inputDescriptors.get(streamId); - addSerde(streamId, inputDescriptor.getSerde(), streamKeySerdes, streamMsgSerdes); - }); - Map outputDescriptors = taskAppDesc.getOutputDescriptors(); - outEdges.forEach(edge -> { - String streamId = edge.getStreamSpec().getId(); - OutputDescriptor outputDescriptor = outputDescriptors.get(streamId); - addSerde(streamId, outputDescriptor.getSerde(), streamKeySerdes, streamMsgSerdes); - }); - - // for each unique stream or store serde instance, generate a unique name and serialize to config - HashSet serdes = new HashSet<>(streamKeySerdes.values()); - serdes.addAll(streamMsgSerdes.values()); - SerializableSerde serializableSerde = new SerializableSerde<>(); - Base64.Encoder base64Encoder = Base64.getEncoder(); - Map serdeUUIDs = new HashMap<>(); - serdes.forEach(serde -> { - String serdeName = serdeUUIDs.computeIfAbsent(serde, - s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); - configs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), - base64Encoder.encodeToString(serializableSerde.toBytes(serde))); - }); - - // set key and msg serdes for streams to the serde names generated above - streamKeySerdes.forEach((streamId, serde) -> { - String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); - String keySerdeConfigKey = streamIdPrefix + StreamConfig.KEY_SERDE(); - configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); - }); - - streamMsgSerdes.forEach((streamId, serde) -> { - String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); - String valueSerdeConfigKey = streamIdPrefix + StreamConfig.MSG_SERDE(); - configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); - }); - } - - private void addSerde(String streamId, Serde serde, Map keySerdeMap, Map msgSerdeMap) { - if (serde != null) { - if (serde instanceof KVSerde) { - keySerdeMap.put(streamId, ((KVSerde) serde).getKeySerde()); - msgSerdeMap.put(streamId, ((KVSerde) serde).getValueSerde()); - } else { - msgSerdeMap.put(streamId, serde); - } - } - } - - /** - * Computes the triggering interval to use during the execution of this {@link JobNode} - */ - private long computeTriggerInterval() { - // Obtain the operator specs from the specGraph - Collection operatorSpecs = specGraph.getAllOperatorSpecs(); - - // Filter out window operators, and obtain a list of their triggering interval values - List windowTimerIntervals = operatorSpecs.stream() - .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.WINDOW) - .map(spec -> ((WindowOperatorSpec) spec).getDefaultTriggerMs()) - .collect(Collectors.toList()); - - // Filter out the join operators, and obtain a list of their ttl values - List joinTtlIntervals = operatorSpecs.stream() - .filter(spec -> spec instanceof JoinOperatorSpec) - .map(spec -> ((JoinOperatorSpec) spec).getTtlMs()) - .collect(Collectors.toList()); - - // Combine both the above lists - List candidateTimerIntervals = new ArrayList<>(joinTtlIntervals); - candidateTimerIntervals.addAll(windowTimerIntervals); - - if (candidateTimerIntervals.isEmpty()) { - return -1; - } - - // Compute the gcd of the resultant list - return MathUtil.gcd(candidateTimerIntervals); - } - - /** - * This function extract the subset of configs from the full config, and use it to override the generated configs - * from the job. - * @param fullConfig full config - * @param generatedConfig config generated for the job - * @param configPrefix prefix to extract the subset of the config overrides - * @return config that merges the generated configs and overrides - */ - private static Config extractScopedConfig(Config fullConfig, Config generatedConfig, String configPrefix) { - Config scopedConfig = fullConfig.subset(configPrefix); - - Config[] configPrecedence = new Config[] {fullConfig, generatedConfig, scopedConfig}; - // Strip empty configs so they don't override the configs before them. - Map mergedConfig = new HashMap<>(); - for (Map config : configPrecedence) { - for (Map.Entry property : config.entrySet()) { - String value = property.getValue(); - if (!(value == null || value.isEmpty())) { - mergedConfig.put(property.getKey(), property.getValue()); - } - } - } - scopedConfig = new MapConfig(mergedConfig); - log.debug("Prefix '{}' has merged config {}", configPrefix, scopedConfig); - - return scopedConfig; + JobConfig generateConfig(String executionPlanJson) { + return configGenerator.generateJobConfig(this, executionPlanJson); } static String createId(String jobName, String jobId) { return String.format("%s-%s", jobName, jobId); } + } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index 0b09c05afd..9d47428b57 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -60,22 +60,7 @@ public abstract class JobPlanner { this.config = descriptor.getConfig(); } - public List prepareJobs() { - String appId = new ApplicationConfig(appDesc.getConfig()).getGlobalAppId(); - if (appDesc instanceof TaskApplicationDescriptorImpl) { - return Collections.singletonList(prepareTaskJob((TaskApplicationDescriptorImpl) appDesc)); - } else if (appDesc instanceof StreamApplicationDescriptorImpl) { - try { - return prepareStreamJobs((StreamApplicationDescriptorImpl) appDesc); - } catch (Exception e) { - throw new SamzaException("Failed to generate JobConfig for StreamApplication " + appId, e); - } - } - throw new IllegalArgumentException(String.format("ApplicationDescriptorImpl has to be either TaskApplicationDescriptorImpl or " - + "StreamApplicationDescriptorImpl. class %s is not supported", appDesc.getClass().getName())); - } - - abstract List prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception; + public abstract List prepareJobs() throws Exception; StreamManager buildAndStartStreamManager(Config config) { StreamManager streamManager = new StreamManager(config); @@ -83,12 +68,12 @@ StreamManager buildAndStartStreamManager(Config config) { return streamManager; } - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph) { - return getExecutionPlan(specGraph, null); + ExecutionPlan getExecutionPlan() { + return getExecutionPlan(null); } /* package private */ - ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) { + ExecutionPlan getExecutionPlan(String runId) { // update application configs Map cfg = new HashMap<>(); @@ -97,8 +82,9 @@ ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) { } StreamConfig streamConfig = new StreamConfig(config); - Set inputStreams = new HashSet<>(specGraph.getInputOperators().keySet()); - inputStreams.removeAll(specGraph.getOutputStreams().keySet()); + JobGraphConfigureGenerator jobGraphConfigureGenerator = new JobGraphConfigureGenerator(appDesc); + Set inputStreams = jobGraphConfigureGenerator.getInputStreamIds(); + inputStreams.removeAll(jobGraphConfigureGenerator.getOutputStreamIds()); ApplicationConfig.ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) ? ApplicationConfig.ApplicationMode.BATCH : ApplicationConfig.ApplicationMode.STREAM; cfg.put(ApplicationConfig.APP_MODE, mode.name()); @@ -113,12 +99,12 @@ ExecutionPlan getExecutionPlan(OperatorSpecGraph specGraph, String runId) { // create the physical execution plan and merge with overrides. This works for a single-stage job now // TODO: This should all be consolidated with ExecutionPlanner after fixing SAMZA-1811 - Config mergedConfig = JobNode.mergeJobConfig(config, new MapConfig(cfg)); + Config mergedConfig = JobGraphConfigureGenerator.mergeJobConfig(config, new MapConfig(cfg)); // creating the StreamManager to get all input/output streams' metadata for planning StreamManager streamManager = buildAndStartStreamManager(mergedConfig); try { ExecutionPlanner planner = new ExecutionPlanner(mergedConfig, streamManager); - return planner.plan(specGraph); + return planner.plan(appDesc); } finally { streamManager.stop(); } @@ -145,21 +131,6 @@ final void writePlanJsonFile(String planJson) { } } - // TODO: SAMZA-1811: The following private helper to generate single node JobGraph for TaskApplication should be consolidated - // to ExecutionPlanner - // helper method to generate a single node job configuration for low level task applications - private JobConfig prepareTaskJob(TaskApplicationDescriptorImpl taskAppDesc) { - // copy original configure - Map cfg = new HashMap<>(config); - // expand system and streams configure - Map systemStreamConfigs = expandSystemStreamConfigs(taskAppDesc); - cfg.putAll(systemStreamConfigs); - - Config generatedCfgFromStreamDescriptors = new MapConfig(cfg); - JobGraph singleNodeJobGraph = ExecutionPlanner.createJobGraph(generatedCfgFromStreamDescriptors, taskAppDesc); - return singleNodeJobGraph.getSingleNodeJobConfig(taskAppDesc); - } - private Map expandSystemStreamConfigs(ApplicationDescriptorImpl appDesc) { Map systemStreamConfigs = new HashMap<>(); appDesc.getInputDescriptors().forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java index 7996d6bb27..b360ea1633 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -37,7 +37,7 @@ /** - * Temporarily helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamApplicationDescriptorImpl)} + * Temporarily helper class with specific implementation of {@link JobPlanner#prepareJobs()} * for standalone Samza processors. * * TODO: we need to consolidate this with {@link ExecutionPlanner} after SAMZA-1811. @@ -53,10 +53,10 @@ public LocalJobPlanner(ApplicationDescriptorImpl prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception { + public List prepareJobs() throws Exception { // for high-level DAG, generating the plan and job configs // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph()); + ExecutionPlan plan = getExecutionPlan(); String executionPlanJson = plan.getPlanAsJson(); writePlanJsonFile(executionPlanJson); diff --git a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java index 254ff97c51..1c65ac6082 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java @@ -34,7 +34,7 @@ /** - * Temporary helper class with specific implementation of {@link JobPlanner#prepareStreamJobs(StreamApplicationDescriptorImpl)} + * Temporary helper class with specific implementation of {@link JobPlanner#prepareJobs()} * for remote-launched Samza processors (e.g. in YARN). * * TODO: we need to consolidate this class with {@link ExecutionPlanner} after SAMZA-1811. @@ -47,7 +47,7 @@ public RemoteJobPlanner(ApplicationDescriptorImpl prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) throws Exception { + public List prepareJobs() throws Exception { // for high-level DAG, generate the plan and job configs // TODO: run.id needs to be set for standalone: SAMZA-1531 // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision @@ -55,7 +55,7 @@ List prepareStreamJobs(StreamApplicationDescriptorImpl streamAppDesc) LOG.info("The run id for this run is {}", runId); // 1. initialize and plan - ExecutionPlan plan = getExecutionPlan(streamAppDesc.getOperatorSpecGraph(), runId); + ExecutionPlan plan = getExecutionPlan(runId); writePlanJsonFile(plan.getPlanAsJson()); if (plan.getJobConfigs().isEmpty()) { 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 ea74540590..8d52b55e08 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 @@ -274,7 +274,7 @@ public void setup() { Map system1Map = new HashMap<>(); system1Map.put("input1", 64); system1Map.put("output1", 8); - system1Map.put("input4", ExecutionPlanner.MAX_INFERRED_PARTITIONS * 2); + system1Map.put("input4", IntermediateStreamPartitionPlanner.MAX_INFERRED_PARTITIONS * 2); Map system2Map = new HashMap<>(); system2Map.put("input2", 16); system2Map.put("input3", 32); @@ -293,7 +293,7 @@ public void testCreateProcessorGraph() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); assertTrue(jobGraph.getSources().size() == 3); assertTrue(jobGraph.getSinks().size() == 2); assertTrue(jobGraph.getIntermediateStreams().size() == 2); // two streams generated by partitionBy @@ -303,7 +303,7 @@ public void testCreateProcessorGraph() { public void testFetchExistingStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); assertTrue(jobGraph.getOrCreateStreamEdge(input1Spec).getPartitionCount() == 64); @@ -321,10 +321,10 @@ public void testFetchExistingStreamPartitions() { public void testCalculateJoinInputPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); - ExecutionPlanner.calculateJoinInputPartitions(jobGraph, config); + new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -340,8 +340,8 @@ public void testDefaultPartitions() { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph()); - planner.calculatePartitions(jobGraph); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); + new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -357,7 +357,7 @@ public void testTriggerIntervalForJoins() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); List jobConfigs = plan.getJobConfigs(); for (JobConfig config : jobConfigs) { System.out.println(config); @@ -372,7 +372,7 @@ public void testTriggerIntervalForWindowsAndJoins() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -389,7 +389,7 @@ public void testTriggerIntervalWithInvalidWindowMs() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoinAndWindow(); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); @@ -406,7 +406,7 @@ public void testTriggerIntervalForStatelessOperators() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); assertFalse(jobConfigs.get(0).containsKey(TaskConfig.WINDOW_MS())); @@ -421,7 +421,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); List jobConfigs = plan.getJobConfigs(); assertEquals(1, jobConfigs.size()); assertEquals("2000", jobConfigs.get(0).get(TaskConfig.WINDOW_MS())); @@ -431,7 +431,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { public void testCalculateIntStreamPartitions() throws Exception { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); - JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph()); + JobGraph jobGraph = (JobGraph) planner.plan(graphSpec); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -452,15 +452,15 @@ public void testMaxPartition() { edge.setPartitionCount(16); edges.add(edge); - assertEquals(32, ExecutionPlanner.maxPartition(edges)); + assertEquals(32, IntermediateStreamPartitionPlanner.maxPartition(edges)); edges = Collections.emptyList(); - assertEquals(StreamEdge.PARTITIONS_UNKNOWN, ExecutionPlanner.maxPartition(edges)); + assertEquals(StreamEdge.PARTITIONS_UNKNOWN, IntermediateStreamPartitionPlanner.maxPartition(edges)); } @Test public void testMaxPartitionLimit() throws Exception { - int partitionLimit = ExecutionPlanner.MAX_INFERRED_PARTITIONS; + int partitionLimit = IntermediateStreamPartitionPlanner.MAX_INFERRED_PARTITIONS; ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { @@ -469,7 +469,7 @@ public void testMaxPartitionLimit() throws Exception { input1.partitionBy(m -> m.key, m -> m.value, "p1").map(kv -> kv).sendTo(output1); }, config); - JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph()); + JobGraph jobGraph = (JobGraph) planner.plan(graphSpec); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -511,7 +511,7 @@ public void testCreateJobGraphForTaskApplication() { outputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); - JobGraph jobGraph = ExecutionPlanner.createJobGraph(config, taskAppDesc); + JobGraph jobGraph = ExecutionPlanner.createJobGraph(config, new JobGraphJsonGenerator(taskAppDesc), new JobGraphConfigureGenerator(taskAppDesc)); assertEquals(1, jobGraph.getJobNodes().size()); assertTrue(jobGraph.getSources().stream().map(edge -> edge.getName()) .filter(streamId -> inputDescriptors.containsKey(streamId)).collect(Collectors.toList()).isEmpty()); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java index 73452d8b77..e6067adcc7 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java @@ -61,9 +61,9 @@ private StreamSpec genStream() { * 2 9 10 */ private void createGraph1() { - OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); - when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph1 = new JobGraph(null, specGraph); +// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); +// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); + graph1 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); JobNode n2 = graph1.getOrCreateJobNode("2", "1"); JobNode n3 = graph1.getOrCreateJobNode("3", "1"); @@ -96,9 +96,9 @@ private void createGraph1() { * |<---6 <--| <> */ private void createGraph2() { - OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); - when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph2 = new JobGraph(null, specGraph); +// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); +// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); + graph2 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); JobNode n1 = graph2.getOrCreateJobNode("1", "1"); JobNode n2 = graph2.getOrCreateJobNode("2", "1"); @@ -125,9 +125,9 @@ private void createGraph2() { * 1<->1 -> 2<->2 */ private void createGraph3() { - OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); - when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph3 = new JobGraph(null, specGraph); +// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); +// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); + graph3 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); JobNode n1 = graph3.getOrCreateJobNode("1", "1"); JobNode n2 = graph3.getOrCreateJobNode("2", "1"); @@ -143,9 +143,9 @@ private void createGraph3() { * 1<->1 */ private void createGraph4() { - OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); - when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph4 = new JobGraph(null, specGraph); +// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); +// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); + graph4 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); JobNode n1 = graph4.getOrCreateJobNode("1", "1"); @@ -163,9 +163,9 @@ public void setup() { @Test public void testAddSource() { - OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); - when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - JobGraph graph = new JobGraph(null, specGraph); +// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); +// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); + JobGraph graph = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); /** * s1 -> 1 @@ -206,9 +206,9 @@ public void testAddSink() { * 2 -> s2 * 2 -> s3 */ - OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); - when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - JobGraph graph = new JobGraph(null, specGraph); +// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); +// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); + JobGraph graph = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); JobNode n1 = graph.getOrCreateJobNode("1", "1"); JobNode n2 = graph.getOrCreateJobNode("2", "1"); StreamSpec s1 = genStream(); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java new file mode 100644 index 0000000000..7580cb1e2a --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java @@ -0,0 +1,339 @@ +/* + * 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.execution; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.samza.application.LegacyTaskApplication; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.TaskApplicationDescriptorImpl; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.SerializerConfig; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.descriptors.GenericInputDescriptor; +import org.apache.samza.operators.descriptors.GenericOutputDescriptor; +import org.apache.samza.operators.descriptors.GenericSystemDescriptor; +import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.operators.impl.store.TimestampedValueSerde; +import org.apache.samza.serializers.JsonSerdeV2; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerializableSerde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.system.StreamSpec; +import org.apache.samza.task.IdentityStreamTask; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +/** + * Unit test for {@link JobGraphConfigureGenerator} + */ +public class TestJobGraphConfigureGenerator { + + private StreamApplicationDescriptorImpl mockStreamAppDesc; + private Config mockConfig; + private JobNode mockJobNode; + private StreamSpec input1Spec; + private StreamSpec input2Spec; + private StreamSpec outputSpec; + private StreamSpec repartitionSpec; + private KVSerde defaultSerde; + private GenericSystemDescriptor inputSystemDescriptor; + private GenericSystemDescriptor outputSystemDescriptor; + private GenericSystemDescriptor intermediateSystemDescriptor; + private GenericInputDescriptor> input1Descriptor; + private GenericInputDescriptor> input2Descriptor; + private GenericInputDescriptor> intermediateInputDescriptor; + private GenericOutputDescriptor> outputDescriptor; + private GenericOutputDescriptor> intermediateOutputDescriptor; + + @Before + public void setUp() { + input1Spec = new StreamSpec("input1", "input1", "input-system"); + input2Spec = new StreamSpec("input2", "input2", "input-system"); + outputSpec = new StreamSpec("output", "output", "output-system"); + repartitionSpec = + new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); + + defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); + inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); + outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); + intermediateSystemDescriptor = new GenericSystemDescriptor("intermediate-system", "mockSystemFactoryClassName"); + input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); + input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); + outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); + intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("partition_by-p1"); + intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("partition_by-p1"); + + mockConfig = mock(Config.class); + when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); + when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); + + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + + mockJobNode = mock(JobNode.class); + StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, mockConfig); + StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); + StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); + StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); + List inputEdges = new ArrayList<>(); + inputEdges.add(input1Edge); + inputEdges.add(input2Edge); + inputEdges.add(repartitionEdge); + List outputEdges = new ArrayList<>(); + outputEdges.add(outputEdge); + outputEdges.add(repartitionEdge); + when(mockJobNode.getInEdges()).thenReturn(inputEdges); + when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + when(mockJobNode.getJobName()).thenReturn("jobName"); + when(mockJobNode.getJobId()).thenReturn("jobId"); + } + + @Test + public void testConfigureSerdesWithRepartitionJoinApplication() { + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); +// Collection reachableOperators = configureGenerator.getReachableOperators(mockJobNode); +// assertEquals(reachableOperators.toArray(), mockStreamAppDesc.getOperatorSpecGraph().getAllOperatorSpecs().toArray()); +// List stores = configureGenerator.getStoreDescriptors(reachableOperators); +// Optional joinOpOptional = reachableOperators.stream().filter(operatorSpec -> +// operatorSpec.getOpCode() == OperatorSpec.OpCode.JOIN).findFirst(); +// assertEquals(2, stores.size()); +// List nonJoinStores = stores.stream().filter(store -> +// !store.getStoreName().contains(joinOpOptional.get().getOpId())).collect(Collectors.toList()); +// assertEquals(0, nonJoinStores.size()); + + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + + Config serializers = jobConfig.subset("serializers.registry.", true); + + // make sure that the serializers deserialize correctly + SerializableSerde serializableSerde = new SerializableSerde<>(); + Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( + e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), + e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) + )); + assertEquals(5, serializers.size()); // 2 default + 3 specific for join + validateStreamSerdeConfigures(jobConfig, deserializedSerdes); + validateJoinStoreSerdeConfigures(jobConfig, deserializedSerdes); + } + + @Test + public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionOnlyStreamApplication(), mockConfig); + + StreamEdge reparStreamEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); + List inputEdges = new ArrayList<>(); + inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); + inputEdges.add(reparStreamEdge); + List outputEdges = new ArrayList<>(); + outputEdges.add(reparStreamEdge); + when(mockJobNode.getInEdges()).thenReturn(inputEdges); + when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + +// Collection reachableOperators = configureGenerator.getReachableOperators(mockJobNode); +// assertEquals(reachableOperators.toArray(), mockStreamAppDesc.getOperatorSpecGraph().getAllOperatorSpecs().toArray()); +// List stores = configureGenerator.getStoreDescriptors(reachableOperators); +// assertEquals(0, stores.size()); + + MapConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config serializers = jobConfig.subset("serializers.registry.", true); + + // make sure that the serializers deserialize correctly + SerializableSerde serializableSerde = new SerializableSerde<>(); + Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( + e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), + e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) + )); + assertEquals(2, serializers.size()); // 2 input stream + + String partitionByKeySerde = jobConfig.get("streams.jobName-jobId-partition_by-p1.samza.key.serde"); + String partitionByMsgSerde = jobConfig.get("streams.jobName-jobId-partition_by-p1.samza.msg.serde"); + assertTrue("Serialized serdes should not contain intermediate stream key serde", + !deserializedSerdes.containsKey(partitionByKeySerde)); + assertTrue("Serialized serdes should not contain intermediate stream msg serde", + !deserializedSerdes.containsKey(partitionByMsgSerde)); + } + + @Test + public void testGenerateJobConfigWithTaskApplication() { + TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); + + Config serializers = jobConfig.subset("serializers.registry.", true); + SerializableSerde serializableSerde = new SerializableSerde<>(); + Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( + e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), + e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) + )); + assertEquals(2, serializers.size()); // 2 default + validateStreamSerdeConfigures(jobConfig, deserializedSerdes); + } + + @Test + public void testGenerateJobConfigWithLegacyTaskApplication() { + Map originConfig = new HashMap<>(); + originConfig.put(JobConfig.JOB_NAME(), "jobName1"); + originConfig.put(JobConfig.JOB_ID(), "jobId1"); + TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); + when(mockJobNode.getInEdges()).thenReturn(new ArrayList<>()); + when(mockJobNode.getOutEdges()).thenReturn(new ArrayList<>()); + when(mockJobNode.getJobName()).thenReturn("jobName1"); + when(mockJobNode.getJobId()).thenReturn("jobId1"); + when(mockJobNode.getConfig()).thenReturn(new MapConfig(originConfig)); + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); + // jobConfig should be exactly the same as original config + Map generatedConfig = new HashMap<>(jobConfig); + assertEquals(originConfig, generatedConfig); + } + + private void validateJoinStoreSerdeConfigures(MapConfig mapConfig, Map deserializedSerdes) { + String leftJoinStoreKeySerde = mapConfig.get("stores.jobName-jobId-join-j1-L.key.serde"); + String leftJoinStoreMsgSerde = mapConfig.get("stores.jobName-jobId-join-j1-L.msg.serde"); + assertTrue("Serialized serdes should contain left join store key serde", + deserializedSerdes.containsKey(leftJoinStoreKeySerde)); + assertTrue("Serialized left join store key serde should be a StringSerde", + leftJoinStoreKeySerde.startsWith(StringSerde.class.getSimpleName())); + assertTrue("Serialized serdes should contain left join store msg serde", + deserializedSerdes.containsKey(leftJoinStoreMsgSerde)); + assertTrue("Serialized left join store msg serde should be a TimestampedValueSerde", + leftJoinStoreMsgSerde.startsWith(TimestampedValueSerde.class.getSimpleName())); + + String rightJoinStoreKeySerde = mapConfig.get("stores.jobName-jobId-join-j1-R.key.serde"); + String rightJoinStoreMsgSerde = mapConfig.get("stores.jobName-jobId-join-j1-R.msg.serde"); + assertTrue("Serialized serdes should contain right join store key serde", + deserializedSerdes.containsKey(rightJoinStoreKeySerde)); + assertTrue("Serialized right join store key serde should be a StringSerde", + rightJoinStoreKeySerde.startsWith(StringSerde.class.getSimpleName())); + assertTrue("Serialized serdes should contain right join store msg serde", + deserializedSerdes.containsKey(rightJoinStoreMsgSerde)); + assertTrue("Serialized right join store msg serde should be a TimestampedValueSerde", + rightJoinStoreMsgSerde.startsWith(TimestampedValueSerde.class.getSimpleName())); + } + + private void validateStreamSerdeConfigures(Config config, Map deserializedSerdes) { + String input1KeySerde = config.get("streams.input1.samza.key.serde"); + String input1MsgSerde = config.get("streams.input1.samza.msg.serde"); + assertTrue("Serialized serdes should contain input1 key serde", + deserializedSerdes.containsKey(input1KeySerde)); + assertTrue("Serialized input1 key serde should be a StringSerde", + input1KeySerde.startsWith(StringSerde.class.getSimpleName())); + assertTrue("Serialized serdes should contain input1 msg serde", + deserializedSerdes.containsKey(input1MsgSerde)); + assertTrue("Serialized input1 msg serde should be a JsonSerdeV2", + input1MsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); + + String input2KeySerde = config.get("streams.input2.samza.key.serde"); + String input2MsgSerde = config.get("streams.input2.samza.msg.serde"); + assertTrue("Serialized serdes should contain input2 key serde", + deserializedSerdes.containsKey(input2KeySerde)); + assertTrue("Serialized input2 key serde should be a StringSerde", + input2KeySerde.startsWith(StringSerde.class.getSimpleName())); + assertTrue("Serialized serdes should contain input2 msg serde", + deserializedSerdes.containsKey(input2MsgSerde)); + assertTrue("Serialized input2 msg serde should be a JsonSerdeV2", + input2MsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); + + String outputKeySerde = config.get("streams.output.samza.key.serde"); + String outputMsgSerde = config.get("streams.output.samza.msg.serde"); + assertTrue("Serialized serdes should contain output key serde", + deserializedSerdes.containsKey(outputKeySerde)); + assertTrue("Serialized output key serde should be a StringSerde", + outputKeySerde.startsWith(StringSerde.class.getSimpleName())); + assertTrue("Serialized serdes should contain output msg serde", + deserializedSerdes.containsKey(outputMsgSerde)); + assertTrue("Serialized output msg serde should be a JsonSerdeV2", + outputMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); + + String partitionByKeySerde = config.get("streams.jobName-jobId-partition_by-p1.samza.key.serde"); + String partitionByMsgSerde = config.get("streams.jobName-jobId-partition_by-p1.samza.msg.serde"); + assertTrue("Serialized serdes should contain intermediate stream key serde", + deserializedSerdes.containsKey(partitionByKeySerde)); + assertTrue("Serialized intermediate stream key serde should be a StringSerde", + partitionByKeySerde.startsWith(StringSerde.class.getSimpleName())); + assertTrue("Serialized serdes should contain intermediate stream msg serde", + deserializedSerdes.containsKey(partitionByMsgSerde)); + assertTrue( + "Serialized intermediate stream msg serde should be a JsonSerdeV2", + partitionByMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); + } + + private TaskApplication getTaskApplication() { + return appDesc -> { + appDesc.addInputStream(input1Descriptor); + appDesc.addInputStream(input2Descriptor); + appDesc.addInputStream(intermediateInputDescriptor); + appDesc.addOutputStream(intermediateOutputDescriptor); + appDesc.addOutputStream(outputDescriptor); + appDesc.setTaskFactory(() -> new IdentityStreamTask()); + }; + } + + private TaskApplication getLegacyTaskApplication() { + return new LegacyTaskApplication(IdentityStreamTask.class.getName()); + } + + private StreamApplication getRepartitionJoinStreamApplication() { + return appDesc -> { + MessageStream> input1 = appDesc.getInputStream(input1Descriptor); + MessageStream> input2 = appDesc.getInputStream(input2Descriptor); + OutputStream> output = appDesc.getOutputStream(outputDescriptor); + JoinFunction> mockJoinFn = mock(JoinFunction.class); + input1 + .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") + .map(kv -> kv.value) + .join(input2.map(kv -> kv.value), mockJoinFn, + new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), + Duration.ofHours(1), "j1") + .sendTo(output); + }; + } + + private StreamApplication getRepartitionOnlyStreamApplication() { + return appDesc -> { + MessageStream> input = appDesc.getInputStream(input1Descriptor); + input.partitionBy(KV::getKey, KV::getValue, "p1"); + }; + } +} diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index ae6e25e5ee..9f666452e2 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -51,6 +51,9 @@ import static org.mockito.Mockito.*; +/** + * Unit test for {@link JobGraphJsonGenerator} + */ public class TestJobGraphJsonGenerator { public class PageViewEvent { @@ -142,7 +145,7 @@ public void test() throws Exception { }, config); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); String json = plan.getPlanAsJson(); System.out.println(json); @@ -202,7 +205,7 @@ public void test2() throws Exception { }, config); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph()); + ExecutionPlan plan = planner.plan(graphSpec); String json = plan.getPlanAsJson(); System.out.println(json); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index 163b094960..dbaa72ab66 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -53,176 +53,4 @@ public class TestJobNode { - @Test - public void testAddSerdeConfigs() { - StreamSpec input1Spec = new StreamSpec("input1", "input1", "input-system"); - StreamSpec input2Spec = new StreamSpec("input2", "input2", "input-system"); - StreamSpec outputSpec = new StreamSpec("output", "output", "output-system"); - StreamSpec partitionBySpec = - new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); - - Config mockConfig = mock(Config.class); - when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); - when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - - StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { - KVSerde serde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); - GenericSystemDescriptor sd = new GenericSystemDescriptor("system1", "mockSystemFactoryClass"); - GenericInputDescriptor> inputDescriptor1 = sd.getInputDescriptor("input1", serde); - GenericInputDescriptor> inputDescriptor2 = sd.getInputDescriptor("input2", serde); - GenericOutputDescriptor> outputDescriptor = sd.getOutputDescriptor("output", serde); - MessageStream> input1 = appDesc.getInputStream(inputDescriptor1); - MessageStream> input2 = appDesc.getInputStream(inputDescriptor2); - OutputStream> output = appDesc.getOutputStream(outputDescriptor); - JoinFunction> mockJoinFn = mock(JoinFunction.class); - input1 - .partitionBy(KV::getKey, KV::getValue, serde, "p1") - .map(kv -> kv.value) - .join(input2.map(kv -> kv.value), mockJoinFn, - new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), - Duration.ofHours(1), "j1") - .sendTo(output); - }, mockConfig); - - JobNode jobNode = new JobNode("jobName", "jobId", graphSpec.getOperatorSpecGraph(), mockConfig); - Config config = new MapConfig(); - StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, config); - StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, config); - StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, config); - StreamEdge repartitionEdge = new StreamEdge(partitionBySpec, true, false, config); - jobNode.addInEdge(input1Edge); - jobNode.addInEdge(input2Edge); - jobNode.addOutEdge(outputEdge); - jobNode.addInEdge(repartitionEdge); - jobNode.addOutEdge(repartitionEdge); - - Map configs = new HashMap<>(); - jobNode.addSerdeConfigs(configs); - - MapConfig mapConfig = new MapConfig(configs); - Config serializers = mapConfig.subset("serializers.registry.", true); - - // make sure that the serializers deserialize correctly - SerializableSerde serializableSerde = new SerializableSerde<>(); - Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( - e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), - e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) - )); - assertEquals(5, serializers.size()); // 2 default + 3 specific for join - - String input1KeySerde = mapConfig.get("streams.input1.samza.key.serde"); - String input1MsgSerde = mapConfig.get("streams.input1.samza.msg.serde"); - assertTrue("Serialized serdes should contain input1 key serde", - deserializedSerdes.containsKey(input1KeySerde)); - assertTrue("Serialized input1 key serde should be a StringSerde", - input1KeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain input1 msg serde", - deserializedSerdes.containsKey(input1MsgSerde)); - assertTrue("Serialized input1 msg serde should be a JsonSerdeV2", - input1MsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String input2KeySerde = mapConfig.get("streams.input2.samza.key.serde"); - String input2MsgSerde = mapConfig.get("streams.input2.samza.msg.serde"); - assertTrue("Serialized serdes should contain input2 key serde", - deserializedSerdes.containsKey(input2KeySerde)); - assertTrue("Serialized input2 key serde should be a StringSerde", - input2KeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain input2 msg serde", - deserializedSerdes.containsKey(input2MsgSerde)); - assertTrue("Serialized input2 msg serde should be a JsonSerdeV2", - input2MsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String outputKeySerde = mapConfig.get("streams.output.samza.key.serde"); - String outputMsgSerde = mapConfig.get("streams.output.samza.msg.serde"); - assertTrue("Serialized serdes should contain output key serde", - deserializedSerdes.containsKey(outputKeySerde)); - assertTrue("Serialized output key serde should be a StringSerde", - outputKeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain output msg serde", - deserializedSerdes.containsKey(outputMsgSerde)); - assertTrue("Serialized output msg serde should be a JsonSerdeV2", - outputMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String partitionByKeySerde = mapConfig.get("streams.jobName-jobId-partition_by-p1.samza.key.serde"); - String partitionByMsgSerde = mapConfig.get("streams.jobName-jobId-partition_by-p1.samza.msg.serde"); - assertTrue("Serialized serdes should contain intermediate stream key serde", - deserializedSerdes.containsKey(partitionByKeySerde)); - assertTrue("Serialized intermediate stream key serde should be a StringSerde", - partitionByKeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain intermediate stream msg serde", - deserializedSerdes.containsKey(partitionByMsgSerde)); - assertTrue( - "Serialized intermediate stream msg serde should be a JsonSerdeV2", - partitionByMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String leftJoinStoreKeySerde = mapConfig.get("stores.jobName-jobId-join-j1-L.key.serde"); - String leftJoinStoreMsgSerde = mapConfig.get("stores.jobName-jobId-join-j1-L.msg.serde"); - assertTrue("Serialized serdes should contain left join store key serde", - deserializedSerdes.containsKey(leftJoinStoreKeySerde)); - assertTrue("Serialized left join store key serde should be a StringSerde", - leftJoinStoreKeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain left join store msg serde", - deserializedSerdes.containsKey(leftJoinStoreMsgSerde)); - assertTrue("Serialized left join store msg serde should be a TimestampedValueSerde", - leftJoinStoreMsgSerde.startsWith(TimestampedValueSerde.class.getSimpleName())); - - String rightJoinStoreKeySerde = mapConfig.get("stores.jobName-jobId-join-j1-R.key.serde"); - String rightJoinStoreMsgSerde = mapConfig.get("stores.jobName-jobId-join-j1-R.msg.serde"); - assertTrue("Serialized serdes should contain right join store key serde", - deserializedSerdes.containsKey(rightJoinStoreKeySerde)); - assertTrue("Serialized right join store key serde should be a StringSerde", - rightJoinStoreKeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain right join store msg serde", - deserializedSerdes.containsKey(rightJoinStoreMsgSerde)); - assertTrue("Serialized right join store msg serde should be a TimestampedValueSerde", - rightJoinStoreMsgSerde.startsWith(TimestampedValueSerde.class.getSimpleName())); - } - - @Test - public void testAddSerdeConfigsForRepartitionWithNoDefaultSystem() { - StreamSpec inputSpec = new StreamSpec("input", "input", "input-system"); - StreamSpec partitionBySpec = - new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); - - Config mockConfig = mock(Config.class); - when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); - when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); - - StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { - GenericSystemDescriptor sd = new GenericSystemDescriptor("system1", "mockSystemFactoryClassName"); - GenericInputDescriptor> inputDescriptor1 = - sd.getInputDescriptor("input", KVSerde.of(new StringSerde(), new JsonSerdeV2<>())); - MessageStream> input = appDesc.getInputStream(inputDescriptor1); - input.partitionBy(KV::getKey, KV::getValue, "p1"); - }, mockConfig); - - JobNode jobNode = new JobNode("jobName", "jobId", graphSpec.getOperatorSpecGraph(), mockConfig); - Config config = new MapConfig(); - StreamEdge input1Edge = new StreamEdge(inputSpec, false, false, config); - StreamEdge repartitionEdge = new StreamEdge(partitionBySpec, true, false, config); - jobNode.addInEdge(input1Edge); - jobNode.addInEdge(repartitionEdge); - jobNode.addOutEdge(repartitionEdge); - - Map configs = new HashMap<>(); - jobNode.addSerdeConfigs(configs); - - MapConfig mapConfig = new MapConfig(configs); - Config serializers = mapConfig.subset("serializers.registry.", true); - - // make sure that the serializers deserialize correctly - SerializableSerde serializableSerde = new SerializableSerde<>(); - Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( - e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), - e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) - )); - assertEquals(2, serializers.size()); // 2 input stream - - String partitionByKeySerde = mapConfig.get("streams.jobName-jobId-partition_by-p1.samza.key.serde"); - String partitionByMsgSerde = mapConfig.get("streams.jobName-jobId-partition_by-p1.samza.msg.serde"); - assertTrue("Serialized serdes should not contain intermediate stream key serde", - !deserializedSerdes.containsKey(partitionByKeySerde)); - assertTrue("Serialized serdes should not contain intermediate stream msg serde", - !deserializedSerdes.containsKey(partitionByMsgSerde)); - } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java index 988fb341f8..85921f4def 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestRemoteJobPlanner.java @@ -69,7 +69,7 @@ public void testStreamCreation() ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); when(mockAppConfig.getAppMode()).thenReturn(ApplicationConfig.ApplicationMode.STREAM); when(plan.getApplicationConfig()).thenReturn(mockAppConfig); - doReturn(plan).when(remotePlanner).getExecutionPlan(any(), any()); + doReturn(plan).when(remotePlanner).getExecutionPlan(any()); remotePlanner.prepareJobs(); From 97c00a2e0a29f55d92cc265ffd8938c67f617931 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 14 Sep 2018 00:00:15 -0700 Subject: [PATCH 29/38] SAMZA-1814: WIP unit tests fixed for configure generation. --- .../samza/execution/ExecutionPlanner.java | 13 +- .../IntermediateStreamPartitionPlanner.java | 1 - .../execution/JobGraphConfigureGenerator.java | 40 +- .../samza/execution/TestExecutionPlanner.java | 48 +- ...estIntermediateStreamPartitionPlanner.java | 159 ++++++ .../TestJobGraphConfigureGenerator.java | 510 ++++++++++++++---- .../execution/TestJobGraphJsonGenerator.java | 172 +++++- 7 files changed, 809 insertions(+), 134 deletions(-) create mode 100644 samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 990f7ebe37..7cb96e0b70 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -29,6 +29,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; @@ -64,7 +65,8 @@ public ExecutionPlan plan(ApplicationDescriptorImpl sourceStreams = getStreamSpecs(jobConfigureGenerator.getInputStreamIds(), streamConfig); @@ -119,7 +121,10 @@ private void validateConfig() { // add tables tables.forEach(spec -> jobGraph.addTable(spec, node)); - jobGraph.validate(); + if (!isLegacyTaskApplication) { + // skip the validation when input streamIds are empty. This is only possible for LegacyApplication + jobGraph.validate(); + } return jobGraph; } diff --git a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java index 126d15b735..0c5e818239 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java @@ -68,7 +68,6 @@ class IntermediateStreamPartitionPlanner { * Figure out the number of partitions of all streams */ /* package private */ void calculatePartitions(JobGraph jobGraph) { - // only compute partitions for high-level API applications // calculate the partitions for the input streams of join operators calculateJoinInputPartitions(jobGraph); diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java index 9e44635b90..016cb30d8c 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java @@ -26,7 +26,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.function.Function; @@ -72,17 +71,18 @@ public class JobGraphConfigureGenerator { private static final Logger LOG = LoggerFactory.getLogger(JobGraphConfigureGenerator.class); - private static final String CONFIG_JOB_PREFIX = "jobs.%s."; - private static final String CONFIG_INTERNAL_EXECUTION_PLAN = "samza.internal.execution.plan"; - private final StreamSerdeManager streamSerdeManager; + private final StreamSerdeConfigureGenerator streamSerdeConfigureGenerator; private final Set broadcastStreamIds; private final Set inputStreamIds; private final Set outputStreamIds; private final Set tableSpecs; + static final String CONFIG_JOB_PREFIX = "jobs.%s."; + static final String CONFIG_INTERNAL_EXECUTION_PLAN = "samza.internal.execution.plan"; + JobGraphConfigureGenerator(ApplicationDescriptorImpl appDesc) { - this.streamSerdeManager = new StreamSerdeManager(appDesc); + this.streamSerdeConfigureGenerator = new StreamSerdeConfigureGenerator(appDesc); this.broadcastStreamIds = appDesc.getBroadcastStreams(); this.inputStreamIds = appDesc instanceof StreamApplicationDescriptor ? ((StreamApplicationDescriptorImpl) appDesc).getInputOperators().keySet() : appDesc.getInputDescriptors().keySet(); @@ -92,9 +92,11 @@ public class JobGraphConfigureGenerator { .collect(Collectors.toSet()); } - static Config mergeJobConfig(Config fullConfig, Config generatedConfig) { - return new JobConfig(Util.rewriteConfig(extractScopedConfig( - fullConfig, generatedConfig, String.format(CONFIG_JOB_PREFIX, new JobConfig(fullConfig).getName().get())))); + static Config mergeJobConfig(Config originalConfig, Config generatedConfig) { + JobConfig jobConfig = new JobConfig(originalConfig); + String jobId = JobNode.createId(jobConfig.getName().get(), jobConfig.getJobId().get()); + return new JobConfig(Util.rewriteConfig(extractScopedConfig(originalConfig, generatedConfig, + String.format(CONFIG_JOB_PREFIX, jobId)))); } boolean isBroadcastStream(String streamId) { @@ -128,8 +130,8 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { Config config = jobNode.getConfig(); // check all inputs to the node for broadcast and input streams - final List inputs = new ArrayList<>(); - final List broadcasts = new ArrayList<>(); + final Set inputs = new HashSet<>(); + final Set broadcasts = new HashSet<>(); for (StreamEdge inEdge : inEdges) { String formattedSystemStream = inEdge.getName(); if (inEdge.isBroadcast()) { @@ -172,7 +174,7 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { return applyConfigureRewritersAndOverrides(configs, config, jobNode); } - private void configureBroadcastInputs(Map configs, Config config, List broadcasts) { + private void configureBroadcastInputs(Map configs, Config config, Set broadcasts) { // TODO: remove this once we support defining broadcast input stream in high-level // task.broadcast.input should be generated by the planner in the future. final String taskBroadcasts = config.get(TaskConfigJava.BROADCAST_INPUT_STREAMS); @@ -194,7 +196,7 @@ private void configureWindowAndJoinInterval(Map configs, Config } private JobConfig applyConfigureRewritersAndOverrides(Map configs, Config config, JobNode jobNode) { - String configPrefix = String.format(CONFIG_JOB_PREFIX, jobNode.getJobName()); + String configPrefix = String.format(CONFIG_JOB_PREFIX, jobNode.getId()); // Disallow user specified job inputs/outputs. This info comes strictly from the user application. Map allowedConfigs = new HashMap<>(config); @@ -243,7 +245,7 @@ private List getStoreDescriptors(Collection reach .collect(Collectors.toList()); } - private void configureTables(Map configs, Config config, List tables, List inputs) { + private void configureTables(Map configs, Config config, List tables, Set inputs) { configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); // Add side inputs to the inputs and mark the stream as bootstrap @@ -281,11 +283,11 @@ private void configureSerdes(Map configs, List inEdg Map streamMsgSerdes = new HashMap<>(); inEdges.forEach(edge -> { String streamId = edge.getStreamSpec().getId(); - streamSerdeManager.addSerde(sid -> streamSerdeManager.getInputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); + streamSerdeConfigureGenerator.addSerde(sid -> streamSerdeConfigureGenerator.getInputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); }); outEdges.forEach(edge -> { String streamId = edge.getStreamSpec().getId(); - streamSerdeManager.addSerde(sid -> streamSerdeManager.getOutputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); + streamSerdeConfigureGenerator.addSerde(sid -> streamSerdeConfigureGenerator.getOutputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); }); Map storeKeySerdes = new HashMap<>(); @@ -365,8 +367,8 @@ private long computeTriggerInterval(Collection reachableOperators) Collection getReachableOperators(JobNode jobNode) { // Filter out window operators, and obtain a list of their triggering interval values Set inputOperatorsInJobNode = jobNode.getInEdges().stream() - .filter(streamEdge -> streamSerdeManager.inputOperators.containsKey(streamEdge.getStreamSpec().getId())) - .map(streamEdge -> streamSerdeManager.inputOperators.get(streamEdge.getStreamSpec().getId())) + .filter(streamEdge -> streamSerdeConfigureGenerator.inputOperators.containsKey(streamEdge.getStreamSpec().getId())) + .map(streamEdge -> streamSerdeConfigureGenerator.inputOperators.get(streamEdge.getStreamSpec().getId())) .collect(Collectors.toSet()); Set reachableOperators = new HashSet<>(); findReachableOperators(inputOperatorsInJobNode, reachableOperators); @@ -383,13 +385,13 @@ private void findReachableOperators(Collection inputOperatorsInJob }); } - private class StreamSerdeManager { + private class StreamSerdeConfigureGenerator { private final Map inputDescriptors; private final Map inputOperators; private final Map outputDescriptors; private final Map outputStreams; - private StreamSerdeManager(ApplicationDescriptorImpl appDesc) { + private StreamSerdeConfigureGenerator(ApplicationDescriptorImpl appDesc) { this.inputDescriptors = appDesc.getInputDescriptors(); this.outputDescriptors = appDesc.getOutputDescriptors(); if (appDesc instanceof StreamApplicationDescriptorImpl) { 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 8d52b55e08..82ae0c8a57 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 @@ -293,7 +293,8 @@ public void testCreateProcessorGraph() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), + new JobGraphConfigureGenerator(graphSpec), false); assertTrue(jobGraph.getSources().size() == 3); assertTrue(jobGraph.getSinks().size() == 2); assertTrue(jobGraph.getIntermediateStreams().size() == 2); // two streams generated by partitionBy @@ -303,7 +304,8 @@ public void testCreateProcessorGraph() { public void testFetchExistingStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), + new JobGraphConfigureGenerator(graphSpec), false); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); assertTrue(jobGraph.getOrCreateStreamEdge(input1Spec).getPartitionCount() == 64); @@ -321,7 +323,8 @@ public void testFetchExistingStreamPartitions() { public void testCalculateJoinInputPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), + new JobGraphConfigureGenerator(graphSpec), false); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); @@ -340,8 +343,7 @@ public void testDefaultPartitions() { ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), new JobGraphConfigureGenerator(graphSpec)); - new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); + JobGraph jobGraph = (JobGraph) planner.plan(graphSpec); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -511,17 +513,47 @@ public void testCreateJobGraphForTaskApplication() { outputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); - JobGraph jobGraph = ExecutionPlanner.createJobGraph(config, new JobGraphJsonGenerator(taskAppDesc), new JobGraphConfigureGenerator(taskAppDesc)); + ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); + JobGraph jobGraph = planner.createJobGraph(config, new JobGraphJsonGenerator(taskAppDesc), + new JobGraphConfigureGenerator(taskAppDesc), false); assertEquals(1, jobGraph.getJobNodes().size()); assertTrue(jobGraph.getSources().stream().map(edge -> edge.getName()) .filter(streamId -> inputDescriptors.containsKey(streamId)).collect(Collectors.toList()).isEmpty()); Set intermediateStreams = new HashSet<>(inputDescriptors.keySet()); jobGraph.getSources().forEach(edge -> { - if (intermediateStreams.contains(edge.getName())) { - intermediateStreams.remove(edge.getName()); + if (intermediateStreams.contains(edge.getStreamSpec().getId())) { + intermediateStreams.remove(edge.getStreamSpec().getId()); } }); assertEquals(new HashSet() { { this.add(intermediateStream1); this.add(intermediateBroadcast); } }.toArray(), intermediateStreams.toArray()); } + + @Test + public void testCreateJobGraphForLegacyTaskApplication() { + TaskApplicationDescriptorImpl taskAppDesc = mock(TaskApplicationDescriptorImpl.class); + + when(taskAppDesc.getInputDescriptors()).thenReturn(new HashMap<>()); + when(taskAppDesc.getOutputDescriptors()).thenReturn(new HashMap<>()); + when(taskAppDesc.getTableDescriptors()).thenReturn(new HashSet<>()); + when(taskAppDesc.getSystemDescriptors()).thenReturn(new HashSet<>()); + when(taskAppDesc.getBroadcastStreams()).thenReturn(new HashSet<>()); + + Map systemStreamConfigs = new HashMap<>(); + inputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); + outputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); + systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); + + ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); + JobGraph jobGraph = planner.createJobGraph(config, new JobGraphJsonGenerator(taskAppDesc), + new JobGraphConfigureGenerator(taskAppDesc), true); + assertEquals(1, jobGraph.getJobNodes().size()); + JobNode jobNode = jobGraph.getJobNodes().get(0); + assertEquals("test-app", jobNode.getJobName()); + assertEquals("test-app-1", jobNode.getId()); + assertEquals(0, jobNode.getInEdges().size()); + assertEquals(0, jobNode.getOutEdges().size()); + assertEquals(0, jobNode.getTables().size()); + assertEquals(config, jobNode.getConfig()); + } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java new file mode 100644 index 0000000000..61fc09b159 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java @@ -0,0 +1,159 @@ +/* + * 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.execution; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.descriptors.GenericInputDescriptor; +import org.apache.samza.operators.descriptors.GenericOutputDescriptor; +import org.apache.samza.operators.descriptors.GenericSystemDescriptor; +import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.operators.spec.InputOperatorSpec; +import org.apache.samza.serializers.JsonSerdeV2; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.system.StreamSpec; +import org.junit.Before; +import org.junit.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + + +/** + * Unit tests for {@link IntermediateStreamPartitionPlanner} + */ +public class TestIntermediateStreamPartitionPlanner { + + private JobGraph mockGraph; + private StreamApplicationDescriptorImpl mockStreamAppDesc; + private Config mockConfig; + private JobNode mockJobNode; + private StreamSpec input1Spec; + private StreamSpec input2Spec; + private StreamSpec outputSpec; + private StreamSpec repartitionSpec; + private StreamSpec broadcastSpec; + private KVSerde defaultSerde; + private GenericSystemDescriptor inputSystemDescriptor; + private GenericSystemDescriptor outputSystemDescriptor; + private GenericSystemDescriptor intermediateSystemDescriptor; + private GenericInputDescriptor> input1Descriptor; + private GenericInputDescriptor> input2Descriptor; + private GenericInputDescriptor> intermediateInputDescriptor; + private GenericInputDescriptor> broadcastInputDesriptor; + private GenericOutputDescriptor> outputDescriptor; + private GenericOutputDescriptor> intermediateOutputDescriptor; + + @Before + public void setUp() { + input1Spec = new StreamSpec("input1", "input1", "input-system"); + input2Spec = new StreamSpec("input2", "input2", "input-system"); + outputSpec = new StreamSpec("output", "output", "output-system"); + repartitionSpec = + new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); + broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "broadcast-b1", "intermediate-system"); + + + defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); + inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); + outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); + intermediateSystemDescriptor = new GenericSystemDescriptor("intermediate-system", "mockSystemFactoryClassName"); + input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); + input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); + outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); + intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("partition_by-p1"); + intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("partition_by-p1"); + broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde); + + Map configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + mockConfig = spy(new MapConfig(configs)); + + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + + mockJobNode = mock(JobNode.class); + StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, mockConfig); + StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); + StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); + StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); + List inputEdges = new ArrayList<>(); + inputEdges.add(input1Edge); + inputEdges.add(input2Edge); + inputEdges.add(repartitionEdge); + List outputEdges = new ArrayList<>(); + outputEdges.add(outputEdge); + outputEdges.add(repartitionEdge); + when(mockJobNode.getInEdges()).thenReturn(inputEdges); + when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + when(mockJobNode.getJobName()).thenReturn("jobName"); + when(mockJobNode.getJobId()).thenReturn("jobId"); + when(mockJobNode.getId()).thenReturn(JobNode.createId("jobName", "jobId")); + + mockGraph = mock(JobGraph.class); + } + + @Test + public void testConstructor() { + StreamApplicationDescriptorImpl mockAppDesc = spy(new StreamApplicationDescriptorImpl(appDesc -> { }, + mock(Config.class))); + InputOperatorSpec inputOp1 = mock(InputOperatorSpec.class); + InputOperatorSpec inputOp2 = mock(InputOperatorSpec.class); + Map inputOpMaps = new HashMap<>(); + inputOpMaps.put("input-op1", inputOp1); + inputOpMaps.put("input-op2", inputOp2); + when(mockAppDesc.getInputOperators()).thenReturn(inputOpMaps); + IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mock(Config.class), + mockAppDesc); + JobGraph mockGraph = mock(JobGraph.class); + partitionPlanner.calculatePartitions(mockGraph); + } + + private StreamApplication getRepartitionJoinStreamApplication() { + return appDesc -> { + MessageStream> input1 = appDesc.getInputStream(input1Descriptor); + MessageStream> input2 = appDesc.getInputStream(input2Descriptor); + OutputStream> output = appDesc.getOutputStream(outputDescriptor); + JoinFunction> mockJoinFn = mock(JoinFunction.class); + input1 + .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") + .map(kv -> kv.value) + .join(input2.map(kv -> kv.value), mockJoinFn, + new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), + Duration.ofHours(1), "j1") + .sendTo(output); + }; + } +} diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java index 7580cb1e2a..8b7e26614a 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java @@ -18,12 +18,15 @@ */ package org.apache.samza.execution; +import com.google.common.base.Joiner; import java.time.Duration; import java.util.ArrayList; import java.util.Base64; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.StreamApplication; @@ -31,12 +34,18 @@ import org.apache.samza.application.TaskApplication; import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigRewriter; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.SerializerConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.config.TaskConfigJava; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.KV; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.GenericInputDescriptor; import org.apache.samza.operators.descriptors.GenericOutputDescriptor; import org.apache.samza.operators.descriptors.GenericSystemDescriptor; @@ -48,15 +57,19 @@ import org.apache.samza.serializers.SerializableSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.system.StreamSpec; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableProviderFactory; +import org.apache.samza.table.TableSpec; import org.apache.samza.task.IdentityStreamTask; +import org.apache.samza.task.TaskContext; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -72,6 +85,7 @@ public class TestJobGraphConfigureGenerator { private StreamSpec input2Spec; private StreamSpec outputSpec; private StreamSpec repartitionSpec; + private StreamSpec broadcastSpec; private KVSerde defaultSerde; private GenericSystemDescriptor inputSystemDescriptor; private GenericSystemDescriptor outputSystemDescriptor; @@ -79,6 +93,7 @@ public class TestJobGraphConfigureGenerator { private GenericInputDescriptor> input1Descriptor; private GenericInputDescriptor> input2Descriptor; private GenericInputDescriptor> intermediateInputDescriptor; + private GenericInputDescriptor> broadcastInputDesriptor; private GenericOutputDescriptor> outputDescriptor; private GenericOutputDescriptor> intermediateOutputDescriptor; @@ -89,6 +104,8 @@ public void setUp() { outputSpec = new StreamSpec("output", "output", "output-system"); repartitionSpec = new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); + broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "broadcast-b1", "intermediate-system"); + defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); @@ -101,10 +118,12 @@ public void setUp() { .withPhysicalName("partition_by-p1"); intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) .withPhysicalName("partition_by-p1"); + broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde); - mockConfig = mock(Config.class); - when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName"); - when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId"); + Map configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + mockConfig = spy(new MapConfig(configs)); mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); @@ -125,40 +144,30 @@ public void setUp() { when(mockJobNode.getConfig()).thenReturn(mockConfig); when(mockJobNode.getJobName()).thenReturn("jobName"); when(mockJobNode.getJobId()).thenReturn("jobId"); + when(mockJobNode.getId()).thenReturn(JobNode.createId("jobName", "jobId")); } @Test public void testConfigureSerdesWithRepartitionJoinApplication() { + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); -// Collection reachableOperators = configureGenerator.getReachableOperators(mockJobNode); -// assertEquals(reachableOperators.toArray(), mockStreamAppDesc.getOperatorSpecGraph().getAllOperatorSpecs().toArray()); -// List stores = configureGenerator.getStoreDescriptors(reachableOperators); -// Optional joinOpOptional = reachableOperators.stream().filter(operatorSpec -> -// operatorSpec.getOpCode() == OperatorSpec.OpCode.JOIN).findFirst(); -// assertEquals(2, stores.size()); -// List nonJoinStores = stores.stream().filter(store -> -// !store.getStoreName().contains(joinOpOptional.get().getOpId())).collect(Collectors.toList()); -// assertEquals(0, nonJoinStores.size()); - JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); - Config serializers = jobConfig.subset("serializers.registry.", true); - - // make sure that the serializers deserialize correctly - SerializableSerde serializableSerde = new SerializableSerde<>(); - Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( - e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), - e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) - )); - assertEquals(5, serializers.size()); // 2 default + 3 specific for join - validateStreamSerdeConfigures(jobConfig, deserializedSerdes); - validateJoinStoreSerdeConfigures(jobConfig, deserializedSerdes); + // Verify the results + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedJobConfig, jobConfig); + // additional, check the computed window.ms for join + assertEquals("3600000", jobConfig.get(TaskConfig.WINDOW_MS())); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 5); + validateStreamConfigures(jobConfig, deserializedSerdes); + validateJoinStoreConfigures(jobConfig, deserializedSerdes); } @Test public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { + // set the application to RepartitionOnlyStreamApplication mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionOnlyStreamApplication(), mockConfig); - + // add the stream edges to the node StreamEdge reparStreamEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); List inputEdges = new ArrayList<>(); inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); @@ -168,23 +177,16 @@ public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { when(mockJobNode.getInEdges()).thenReturn(inputEdges); when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); -// Collection reachableOperators = configureGenerator.getReachableOperators(mockJobNode); -// assertEquals(reachableOperators.toArray(), mockStreamAppDesc.getOperatorSpecGraph().getAllOperatorSpecs().toArray()); -// List stores = configureGenerator.getStoreDescriptors(reachableOperators); -// assertEquals(0, stores.size()); + // Verify the results + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedJobConfig, jobConfig); - MapConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); - Config serializers = jobConfig.subset("serializers.registry.", true); - - // make sure that the serializers deserialize correctly - SerializableSerde serializableSerde = new SerializableSerde<>(); - Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( - e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), - e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) - )); - assertEquals(2, serializers.size()); // 2 input stream + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); + validateStreamConfigures(jobConfig, null); String partitionByKeySerde = jobConfig.get("streams.jobName-jobId-partition_by-p1.samza.key.serde"); String partitionByMsgSerde = jobConfig.get("streams.jobName-jobId-partition_by-p1.samza.msg.serde"); @@ -196,31 +198,34 @@ public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { @Test public void testGenerateJobConfigWithTaskApplication() { + // set the application to TaskApplication, which still wire up all input/output/intermediate streams TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); - JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); - Config serializers = jobConfig.subset("serializers.registry.", true); - SerializableSerde serializableSerde = new SerializableSerde<>(); - Map deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap( - e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), - e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) - )); - assertEquals(2, serializers.size()); // 2 default - validateStreamSerdeConfigures(jobConfig, deserializedSerdes); + // Verify the results + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedJobConfig, jobConfig); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); + validateStreamConfigures(jobConfig, deserializedSerdes); } @Test public void testGenerateJobConfigWithLegacyTaskApplication() { + // set the application to LegacyTaskApplication, which only has configuration and no descriptors Map originConfig = new HashMap<>(); originConfig.put(JobConfig.JOB_NAME(), "jobName1"); originConfig.put(JobConfig.JOB_ID(), "jobId1"); TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); + //clear the JobNode's stream edges since legacy task application does not wire up any input/output streams when(mockJobNode.getInEdges()).thenReturn(new ArrayList<>()); when(mockJobNode.getOutEdges()).thenReturn(new ArrayList<>()); when(mockJobNode.getJobName()).thenReturn("jobName1"); when(mockJobNode.getJobId()).thenReturn("jobId1"); when(mockJobNode.getConfig()).thenReturn(new MapConfig(originConfig)); + + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); // jobConfig should be exactly the same as original config @@ -228,7 +233,295 @@ public void testGenerateJobConfigWithLegacyTaskApplication() { assertEquals(originConfig, generatedConfig); } - private void validateJoinStoreSerdeConfigures(MapConfig mapConfig, Map deserializedSerdes) { + @Test + public void testBroadcastStreamApplication() { + // set the application to BroadcastStreamApplication + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(defaultSerde), mockConfig); + // add the stream edges to the node + StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); + List inputEdges = new ArrayList<>(); + inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); + inputEdges.add(broadcastStreamEdge); + List outputEdges = new ArrayList<>(); + outputEdges.add(broadcastStreamEdge); + when(mockJobNode.getInEdges()).thenReturn(inputEdges); + when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config expectedJobConfig = getExpectedJobConfig(mockConfig, inputEdges); + validateJobConfig(expectedJobConfig, jobConfig); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); + validateStreamSerdeConfigure(broadcastInputDesriptor.getStreamId(), jobConfig, deserializedSerdes); + validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), + broadcastStreamEdge.getStreamSpec().getPhysicalName(), jobConfig); + } + + @Test + public void testBroadcastStreamApplicationWithoutSerde() { + // set the application to BroadcastStreamApplication withoutSerde + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(null), mockConfig); + // add the stream edges to the node + StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); + List inputEdges = new ArrayList<>(); + inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); + inputEdges.add(broadcastStreamEdge); + List outputEdges = new ArrayList<>(); + outputEdges.add(broadcastStreamEdge); + when(mockJobNode.getInEdges()).thenReturn(inputEdges); + when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config expectedJobConfig = getExpectedJobConfig(mockConfig, inputEdges); + validateJobConfig(expectedJobConfig, jobConfig); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); + validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), + broadcastStreamEdge.getStreamSpec().getPhysicalName(), jobConfig); + + String keySerde = jobConfig.get(String.format("streams.%s.samza.key.serde", broadcastInputDesriptor.getStreamId())); + String msgSerde = jobConfig.get(String.format("streams.%s.samza.msg.serde", broadcastInputDesriptor.getStreamId())); + assertTrue("Serialized serdes should not contain intermediate stream key serde", + !deserializedSerdes.containsKey(keySerde)); + assertTrue("Serialized serdes should not contain intermediate stream msg serde", + !deserializedSerdes.containsKey(msgSerde)); + } + + @Test + public void testStreamApplicationWithTableAndSideInput() { + // add table to the RepartitionJoinStreamApplication + GenericInputDescriptor> sideInput1 = inputSystemDescriptor.getInputDescriptor("sideInput1", defaultSerde); + BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); + TableSpec mockTableSpec = mock(TableSpec.class); + when(mockTableSpec.getId()).thenReturn("testTable"); + when(mockTableSpec.getSerde()).thenReturn((KVSerde) defaultSerde); + when(mockTableSpec.getTableProviderFactoryClassName()).thenReturn(MockTableProviderFactory.class.getName()); + List sideInputs = new ArrayList<>(); + sideInputs.add(sideInput1.getStreamId()); + when(mockTableSpec.getSideInputs()).thenReturn(sideInputs); + when(mockTableDescriptor.getTableId()).thenReturn("testTable"); + when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); + // add side input and terminate at table in the appplication + mockStreamAppDesc.getInputStream(sideInput1).sendTo(mockStreamAppDesc.getTable(mockTableDescriptor)); + // add table and input edge to the node + List tables = new ArrayList<>(); + tables.add(mockTableSpec); + List inEdges = new ArrayList<>(mockJobNode.getInEdges()); + StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", + inputSystemDescriptor.getSystemName()), false, false, mockConfig); + inEdges.add(sideInputEdge); + // need to put the sideInput related stream configuration to the original config + // TODO: this is confusing since part of the system and stream related configuration is generated outside the JobGraphConfigureGenerator + // It would be nice if all system and stream related configuration is generated in one place and only intermediate stream + // configuration is generated by JobGraphConfigureGenerator + Map configs = new HashMap<>(mockConfig); + configs.putAll(sideInputEdge.generateConfig()); + mockConfig = spy(new MapConfig(configs)); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + when(mockJobNode.getInEdges()).thenReturn(inEdges); + when(mockJobNode.getTables()).thenReturn(tables); + + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedJobConfig, jobConfig); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 7); + validateTableConfigure(jobConfig, deserializedSerdes, mockTableDescriptor); + } + + @Test + public void testTaskApplicationWithTableAndSideInput() { + // add table to the RepartitionJoinStreamApplication + GenericInputDescriptor> sideInput1 = inputSystemDescriptor.getInputDescriptor("sideInput1", defaultSerde); + BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); + TableSpec mockTableSpec = mock(TableSpec.class); + when(mockTableSpec.getId()).thenReturn("testTable"); + when(mockTableSpec.getSerde()).thenReturn((KVSerde) defaultSerde); + when(mockTableSpec.getTableProviderFactoryClassName()).thenReturn(MockTableProviderFactory.class.getName()); + List sideInputs = new ArrayList<>(); + sideInputs.add(sideInput1.getStreamId()); + when(mockTableSpec.getSideInputs()).thenReturn(sideInputs); + when(mockTableDescriptor.getTableId()).thenReturn("testTable"); + when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); + // add table and input edge to the node + List tables = new ArrayList<>(); + tables.add(mockTableSpec); + List inEdges = new ArrayList<>(mockJobNode.getInEdges()); + StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", + inputSystemDescriptor.getSystemName()), false, false, mockConfig); + inEdges.add(sideInputEdge); + // need to put the sideInput related stream configuration to the original config + // TODO: this is confusing since part of the system and stream related configuration is generated outside the JobGraphConfigureGenerator + // It would be nice if all system and stream related configuration is generated in one place and only intermediate stream + // configuration is generated by JobGraphConfigureGenerator + Map configs = new HashMap<>(mockConfig); + configs.putAll(sideInputEdge.generateConfig()); + mockConfig = spy(new MapConfig(configs)); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + when(mockJobNode.getInEdges()).thenReturn(inEdges); + when(mockJobNode.getTables()).thenReturn(tables); + + // set the application to TaskApplication, which still wire up all input/output/intermediate streams + TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); + // add table to the task application + taskAppDesc.addTable(mockTableDescriptor); + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + + // Verify the results + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedJobConfig, jobConfig); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 4); + validateStreamConfigures(jobConfig, deserializedSerdes); + validateTableConfigure(jobConfig, deserializedSerdes, mockTableDescriptor); + } + + @Test + public void testTaskInputsRemovedFromOriginalConfig() { + Map configs = new HashMap<>(mockConfig); + configs.put(TaskConfig.INPUT_STREAMS(), "not.allowed1,not.allowed2"); + mockConfig = spy(new MapConfig(configs)); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedConfig, jobConfig); + } + + @Test + public void testOverrideConfigs() { + Map configs = new HashMap<>(mockConfig); + String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); + String overrideCfgKey = String.format(JobGraphConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; + configs.put(overrideCfgKey, "customized-system"); + mockConfig = spy(new MapConfig(configs)); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedConfig, jobConfig); + assertEquals("customized-system", jobConfig.get(streamCfgToOverride)); + } + + @Test + public void testConfigureRewriter(){ + Map configs = new HashMap<>(mockConfig); + String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); + String overrideCfgKey = String.format(JobGraphConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; + configs.put(overrideCfgKey, "customized-system"); + configs.put(String.format(JobConfig.CONFIG_REWRITER_CLASS(), "mock"), MockConfigRewriter.class.getName()); + configs.put(JobConfig.CONFIG_REWRITERS(), "mock"); + configs.put(String.format("job.config.rewriter.mock.%s", streamCfgToOverride), "rewritten-system"); + mockConfig = spy(new MapConfig(configs)); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + + JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); + Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); + validateJobConfig(expectedConfig, jobConfig); + assertEquals("rewritten-system", jobConfig.get(streamCfgToOverride)); + } + + private void validateTableConfigure(JobConfig jobConfig, Map deserializedSerdes, + TableDescriptor tableDescriptor) { + Config tableConfig = jobConfig.subset(String.format("tables.%s.", tableDescriptor.getTableId())); + assertEquals(MockTableProviderFactory.class.getName(), tableConfig.get("provider.factory")); + MockTableProvider mockTableProvider = + (MockTableProvider) new MockTableProviderFactory().getTableProvider(((BaseTableDescriptor) tableDescriptor).getTableSpec()); + assertEquals(mockTableProvider.configMap.get("mock.table.provider.config"), jobConfig.get("mock.table.provider.config")); + validateTableSerdeConfigure(tableDescriptor.getTableId(), jobConfig, deserializedSerdes); + } + + private Config getExpectedJobConfig(Config originConfig, List inputEdges) { + Map configMap = new HashMap<>(originConfig); + Set inputs = new HashSet<>(); + Set broadcasts = new HashSet<>(); + for (StreamEdge inputEdge : inputEdges) { + if (inputEdge.isBroadcast()) { + broadcasts.add(inputEdge.getName() + "#0"); + } else { + inputs.add(inputEdge.getName()); + } + } + if (!inputs.isEmpty()) { + configMap.put(TaskConfig.INPUT_STREAMS(), Joiner.on(',').join(inputs)); + } + if (!broadcasts.isEmpty()) { + configMap.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcasts)); + } + return new MapConfig(configMap); + } + + private Map validateAndGetDeserializedSerdes(Config jobConfig, int numSerdes) { + Config serializers = jobConfig.subset("serializers.registry.", true); + // make sure that the serializers deserialize correctly + SerializableSerde serializableSerde = new SerializableSerde<>(); + assertEquals(numSerdes, serializers.size()); + return serializers.entrySet().stream().collect(Collectors.toMap( + e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""), + e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes())) + )); + } + + private void validateJobConfig(Config expectedConfig, JobConfig jobConfig) { + assertEquals(expectedConfig.get(JobConfig.JOB_NAME()), jobConfig.getName().get()); + assertEquals(expectedConfig.get(JobConfig.JOB_ID()), jobConfig.getJobId().get()); + assertEquals("testJobGraphJson", jobConfig.get(JobGraphConfigureGenerator.CONFIG_INTERNAL_EXECUTION_PLAN)); + assertEquals(expectedConfig.get(TaskConfig.INPUT_STREAMS()), jobConfig.get(TaskConfig.INPUT_STREAMS())); + assertEquals(expectedConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS), jobConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS)); + } + + private void validateStreamSerdeConfigure(String streamId, Config config, Map deserializedSerdes) { + Config streamConfig = config.subset(String.format("streams.%s.samza.", streamId)); + String keySerdeName = streamConfig.get("key.serde"); + String valueSerdeName = streamConfig.get("msg.serde"); + assertTrue(String.format("Serialized serdes should contain %s key serde", streamId), deserializedSerdes.containsKey(keySerdeName)); + assertTrue(String.format("Serialized %s key serde should be a StringSerde", streamId), keySerdeName.startsWith(StringSerde.class.getSimpleName())); + assertTrue(String.format("Serialized serdes should contain %s msg serde", streamId), deserializedSerdes.containsKey(valueSerdeName)); + assertTrue(String.format("Serialized %s msg serde should be a JsonSerdeV2", streamId), valueSerdeName.startsWith(JsonSerdeV2.class.getSimpleName())); + } + + private void validateTableSerdeConfigure(String tableId, Config config, Map deserializedSerdes) { + Config streamConfig = config.subset(String.format("tables.%s.", tableId)); + String keySerdeName = streamConfig.get("key.serde"); + String valueSerdeName = streamConfig.get("value.serde"); + assertTrue(String.format("Serialized serdes should contain %s key serde", tableId), deserializedSerdes.containsKey(keySerdeName)); + assertTrue(String.format("Serialized %s key serde should be a StringSerde", tableId), keySerdeName.startsWith(StringSerde.class.getSimpleName())); + assertTrue(String.format("Serialized serdes should contain %s value serde", tableId), deserializedSerdes.containsKey(valueSerdeName)); + assertTrue(String.format("Serialized %s msg serde should be a JsonSerdeV2", tableId), valueSerdeName.startsWith(JsonSerdeV2.class.getSimpleName())); + } + + private void validateIntermediateStreamConfigure(String streamId, String physicalName, Config config) { + Config intStreamConfig = config.subset(String.format("streams.%s.", streamId), true); + assertEquals("intermediate-system", intStreamConfig.get("samza.system")); + assertEquals(String.valueOf(Integer.MAX_VALUE), intStreamConfig.get("samza.priority")); + assertEquals("true", intStreamConfig.get("samza.delete.committed.messages")); + assertEquals(physicalName, intStreamConfig.get("samza.physical.name")); + assertEquals("true", intStreamConfig.get("samza.intermediate")); + assertEquals("oldest", intStreamConfig.get("samza.offset.default")); + } + + private void validateStreamConfigures(Config config, Map deserializedSerdes) { + + if (deserializedSerdes != null) { + validateStreamSerdeConfigure(input1Descriptor.getStreamId(), config, deserializedSerdes); + validateStreamSerdeConfigure(input2Descriptor.getStreamId(), config, deserializedSerdes); + validateStreamSerdeConfigure(outputDescriptor.getStreamId(), config, deserializedSerdes); + validateStreamSerdeConfigure(intermediateInputDescriptor.getStreamId(), config, deserializedSerdes); + } + + // generated stream config for intermediate stream + validateIntermediateStreamConfigure(intermediateInputDescriptor.getStreamId(), + intermediateInputDescriptor.getPhysicalName().get(), config); + } + + private void validateJoinStoreConfigures(MapConfig mapConfig, Map deserializedSerdes) { String leftJoinStoreKeySerde = mapConfig.get("stores.jobName-jobId-join-j1-L.key.serde"); String leftJoinStoreMsgSerde = mapConfig.get("stores.jobName-jobId-join-j1-L.msg.serde"); assertTrue("Serialized serdes should contain left join store key serde", @@ -250,53 +543,19 @@ private void validateJoinStoreSerdeConfigures(MapConfig mapConfig, Map deserializedSerdes) { - String input1KeySerde = config.get("streams.input1.samza.key.serde"); - String input1MsgSerde = config.get("streams.input1.samza.msg.serde"); - assertTrue("Serialized serdes should contain input1 key serde", - deserializedSerdes.containsKey(input1KeySerde)); - assertTrue("Serialized input1 key serde should be a StringSerde", - input1KeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain input1 msg serde", - deserializedSerdes.containsKey(input1MsgSerde)); - assertTrue("Serialized input1 msg serde should be a JsonSerdeV2", - input1MsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String input2KeySerde = config.get("streams.input2.samza.key.serde"); - String input2MsgSerde = config.get("streams.input2.samza.msg.serde"); - assertTrue("Serialized serdes should contain input2 key serde", - deserializedSerdes.containsKey(input2KeySerde)); - assertTrue("Serialized input2 key serde should be a StringSerde", - input2KeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain input2 msg serde", - deserializedSerdes.containsKey(input2MsgSerde)); - assertTrue("Serialized input2 msg serde should be a JsonSerdeV2", - input2MsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String outputKeySerde = config.get("streams.output.samza.key.serde"); - String outputMsgSerde = config.get("streams.output.samza.msg.serde"); - assertTrue("Serialized serdes should contain output key serde", - deserializedSerdes.containsKey(outputKeySerde)); - assertTrue("Serialized output key serde should be a StringSerde", - outputKeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain output msg serde", - deserializedSerdes.containsKey(outputMsgSerde)); - assertTrue("Serialized output msg serde should be a JsonSerdeV2", - outputMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); - - String partitionByKeySerde = config.get("streams.jobName-jobId-partition_by-p1.samza.key.serde"); - String partitionByMsgSerde = config.get("streams.jobName-jobId-partition_by-p1.samza.msg.serde"); - assertTrue("Serialized serdes should contain intermediate stream key serde", - deserializedSerdes.containsKey(partitionByKeySerde)); - assertTrue("Serialized intermediate stream key serde should be a StringSerde", - partitionByKeySerde.startsWith(StringSerde.class.getSimpleName())); - assertTrue("Serialized serdes should contain intermediate stream msg serde", - deserializedSerdes.containsKey(partitionByMsgSerde)); - assertTrue( - "Serialized intermediate stream msg serde should be a JsonSerdeV2", - partitionByMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName())); + private void validateJoinStoreConfigure(Config joinStoreConfig, String changelogName) { + assertEquals("org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory", joinStoreConfig.get("factory")); + assertEquals(changelogName, joinStoreConfig.get("changelog")); + assertEquals("delete", joinStoreConfig.get("changelog.kafka.cleanup.policy")); + assertEquals("3600000", joinStoreConfig.get("changelog.kafka.retention.ms")); + assertEquals("3600000", joinStoreConfig.get("rocksdb.ttl.ms")); } private TaskApplication getTaskApplication() { @@ -336,4 +595,63 @@ private StreamApplication getRepartitionOnlyStreamApplication() { input.partitionBy(KV::getKey, KV::getValue, "p1"); }; } + + private StreamApplication getBroadcastOnlyStreamApplication(Serde serde) { + return appDesc -> { + MessageStream> input = appDesc.getInputStream(input1Descriptor); + if (serde != null) { + input.broadcast(serde, "b1"); + } else { + input.broadcast("b1"); + } + }; + } + + private static class MockTableProvider implements TableProvider { + private final Map configMap; + + MockTableProvider(Map configMap) { + this.configMap = configMap; + } + + @Override + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + + } + + @Override + public Table getTable() { + return null; + } + + @Override + public Map generateConfig(Config jobConfig, Map generatedConfig) { + return configMap; + } + + @Override + public void close() { + + } + } + + public static class MockTableProviderFactory implements TableProviderFactory { + + @Override + public TableProvider getTableProvider(TableSpec tableSpec) { + Map configMap = new HashMap<>(); + configMap.put("mock.table.provider.config", "mock.config.value"); + return new MockTableProvider(configMap); + } + } + + public static class MockConfigRewriter implements ConfigRewriter { + + @Override + public Config rewrite(String name, Config config) { + Map configMap = new HashMap<>(config); + configMap.putAll(config.subset(String.format("job.config.rewriter.%s.", name))); + return new MapConfig(configMap); + } + } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index 9f666452e2..36d45a0749 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -20,9 +20,20 @@ package org.apache.samza.execution; import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.application.TaskApplicationDescriptorImpl; +import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -40,10 +51,15 @@ import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.StringSerde; +import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemAdmins; +import org.apache.samza.task.IdentityStreamTask; import org.apache.samza.testUtils.StreamTestUtils; import org.codehaus.jackson.map.ObjectMapper; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import static org.apache.samza.execution.TestExecutionPlanner.*; @@ -55,15 +71,71 @@ * Unit test for {@link JobGraphJsonGenerator} */ public class TestJobGraphJsonGenerator { + private ApplicationDescriptorImpl mockAppDesc; + private Config mockConfig; + private JobNode mockJobNode; + private StreamSpec input1Spec; + private StreamSpec input2Spec; + private StreamSpec outputSpec; + private StreamSpec repartitionSpec; + private KVSerde defaultSerde; + private GenericSystemDescriptor inputSystemDescriptor; + private GenericSystemDescriptor outputSystemDescriptor; + private GenericSystemDescriptor intermediateSystemDescriptor; + private GenericInputDescriptor> input1Descriptor; + private GenericInputDescriptor> input2Descriptor; + private GenericInputDescriptor> intermediateInputDescriptor; + private GenericOutputDescriptor> outputDescriptor; + private GenericOutputDescriptor> intermediateOutputDescriptor; - public class PageViewEvent { - String getCountry() { - return ""; - } + @Before + public void setUp() { + input1Spec = new StreamSpec("input1", "input1", "input-system"); + input2Spec = new StreamSpec("input2", "input2", "input-system"); + outputSpec = new StreamSpec("output", "output", "output-system"); + repartitionSpec = + new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); + + + defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); + inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); + outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); + intermediateSystemDescriptor = new GenericSystemDescriptor("intermediate-system", "mockSystemFactoryClassName"); + input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); + input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); + outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); + intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("partition_by-p1"); + intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("partition_by-p1"); + + Map configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + mockConfig = spy(new MapConfig(configs)); + + mockJobNode = mock(JobNode.class); + StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, mockConfig); + StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); + StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); + StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); + List inputEdges = new ArrayList<>(); + inputEdges.add(input1Edge); + inputEdges.add(input2Edge); + inputEdges.add(repartitionEdge); + List outputEdges = new ArrayList<>(); + outputEdges.add(outputEdge); + outputEdges.add(repartitionEdge); + when(mockJobNode.getInEdges()).thenReturn(inputEdges); + when(mockJobNode.getOutEdges()).thenReturn(outputEdges); + when(mockJobNode.getConfig()).thenReturn(mockConfig); + when(mockJobNode.getJobName()).thenReturn("jobName"); + when(mockJobNode.getJobId()).thenReturn("jobId"); + when(mockJobNode.getId()).thenReturn(JobNode.createId("jobName", "jobId")); } @Test - public void test() throws Exception { + public void testRepartitionedJoinStreamApplication() throws Exception { /** * the graph looks like the following. @@ -160,7 +232,7 @@ public void test() throws Exception { } @Test - public void test2() throws Exception { + public void testRepartitionedWindowStreamApplication() throws Exception { Map configMap = new HashMap<>(); configMap.put(JobConfig.JOB_NAME(), "test-app"); configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system"); @@ -225,4 +297,92 @@ public void test2() throws Exception { assertEquals(operatorGraphJson.operators.get("test-app-1-send_to-5").get("outputStreamId"), "PageViewCount"); } + + @Test + public void testTaskApplication() throws Exception { + mockAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); + JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(mockAppDesc); + JobGraph mockJobGraph = mock(JobGraph.class); + ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); + when(mockAppConfig.getAppName()).thenReturn("testTaskApp"); + when(mockAppConfig.getAppId()).thenReturn("testTaskAppId"); + when(mockJobGraph.getApplicationConfig()).thenReturn(mockAppConfig); + // compute the three disjoint sets of the JobGraph: input only, output only, and intermediate streams + Set inEdges = new HashSet<>(mockJobNode.getInEdges()); + Set outEdges = new HashSet<>(mockJobNode.getOutEdges()); + Set intermediateEdges = new HashSet<>(inEdges); + // intermediate streams are the intersection between input and output + intermediateEdges.retainAll(outEdges); + // remove all intermediate streams from input + inEdges.removeAll(intermediateEdges); + // remove all intermediate streams from output + outEdges.removeAll(intermediateEdges); + // set the return values for mockJobGraph + when(mockJobGraph.getSources()).thenReturn(inEdges); + when(mockJobGraph.getSinks()).thenReturn(outEdges); + when(mockJobGraph.getIntermediateStreamEdges()).thenReturn(intermediateEdges); + when(mockJobGraph.getJobNodes()).thenReturn(Collections.singletonList(mockJobNode)); + String graphJson = jsonGenerator.toJson(mockJobGraph); + ObjectMapper objectMapper = new ObjectMapper(); + JobGraphJsonGenerator.JobGraphJson jsonObject = objectMapper.readValue(graphJson.getBytes(), JobGraphJsonGenerator.JobGraphJson.class); + assertEquals("testTaskAppId", jsonObject.applicationId); + assertEquals("testTaskApp", jsonObject.applicationName); + Set inStreamIds = inEdges.stream().map(stream -> stream.getStreamSpec().getId()).collect(Collectors.toSet()); + assertThat(jsonObject.sourceStreams.keySet(), Matchers.containsInAnyOrder(inStreamIds.toArray())); + Set outStreamIds = outEdges.stream().map(stream -> stream.getStreamSpec().getId()).collect(Collectors.toSet()); + assertThat(jsonObject.sinkStreams.keySet(), Matchers.containsInAnyOrder(outStreamIds.toArray())); + Set intStreamIds = intermediateEdges.stream().map(stream -> stream.getStreamSpec().getId()).collect(Collectors.toSet()); + assertThat(jsonObject.intermediateStreams.keySet(), Matchers.containsInAnyOrder(intStreamIds.toArray())); + JobGraphJsonGenerator.JobNodeJson expectedNodeJson = new JobGraphJsonGenerator.JobNodeJson(); + expectedNodeJson.jobId = mockJobNode.getJobId(); + expectedNodeJson.jobName = mockJobNode.getJobName(); + assertEquals(1, jsonObject.jobs.size()); + JobGraphJsonGenerator.JobNodeJson actualNodeJson = jsonObject.jobs.get(0); + assertEquals(expectedNodeJson.jobId, actualNodeJson.jobId); + assertEquals(expectedNodeJson.jobName, actualNodeJson.jobName); + assertEquals(3, actualNodeJson.operatorGraph.inputStreams.size()); + assertEquals(2, actualNodeJson.operatorGraph.outputStreams.size()); + assertEquals(0, actualNodeJson.operatorGraph.operators.size()); + } + + @Test + public void testLegacyTaskApplication() throws Exception { + mockAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); + JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(mockAppDesc); + JobGraph mockJobGraph = mock(JobGraph.class); + ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); + when(mockAppConfig.getAppName()).thenReturn("testTaskApp"); + when(mockAppConfig.getAppId()).thenReturn("testTaskAppId"); + when(mockJobGraph.getApplicationConfig()).thenReturn(mockAppConfig); + String graphJson = jsonGenerator.toJson(mockJobGraph); + ObjectMapper objectMapper = new ObjectMapper(); + JobGraphJsonGenerator.JobGraphJson jsonObject = objectMapper.readValue(graphJson.getBytes(), JobGraphJsonGenerator.JobGraphJson.class); + assertEquals("testTaskAppId", jsonObject.applicationId); + assertEquals("testTaskApp", jsonObject.applicationName); + JobGraphJsonGenerator.JobNodeJson expectedNodeJson = new JobGraphJsonGenerator.JobNodeJson(); + expectedNodeJson.jobId = mockJobNode.getJobId(); + expectedNodeJson.jobName = mockJobNode.getJobName(); + assertEquals(0, jsonObject.jobs.size()); + } + + public class PageViewEvent { + String getCountry() { + return ""; + } + } + + private TaskApplication getLegacyTaskApplication() { + return new LegacyTaskApplication(IdentityStreamTask.class.getName()); + } + + private TaskApplication getTaskApplication() { + return appDesc -> { + appDesc.addInputStream(input1Descriptor); + appDesc.addInputStream(input2Descriptor); + appDesc.addInputStream(intermediateInputDescriptor); + appDesc.addOutputStream(intermediateOutputDescriptor); + appDesc.addOutputStream(outputDescriptor); + appDesc.setTaskFactory(() -> new IdentityStreamTask()); + }; + } } From 3a91b9a620112769a17172abb8c4a318ddeb19be Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Sun, 16 Sep 2018 15:31:58 -0700 Subject: [PATCH 30/38] SAMZA-1814: moving some logic to ApplicationDescriptorImpl to simplify the JobGraph/JobNode code --- build.gradle | 1 + .../ApplicationDescriptorImpl.java | 52 ++++ .../StreamApplicationDescriptorImpl.java | 29 +++ .../TaskApplicationDescriptorImpl.java | 29 +++ .../samza/execution/ExecutionPlanner.java | 26 +- .../IntermediateStreamPartitionPlanner.java | 13 +- .../org/apache/samza/execution/JobGraph.java | 30 +-- .../execution/JobGraphJsonGenerator.java | 60 +---- .../org/apache/samza/execution/JobNode.java | 87 +++++-- ...or.java => JobNodeConfigureGenerator.java} | 236 +++++------------- .../apache/samza/execution/JobPlanner.java | 12 +- .../samza/execution/LocalJobPlanner.java | 1 - .../samza/execution/RemoteJobPlanner.java | 1 - .../samza/execution/TestExecutionPlanner.java | 20 +- ...estIntermediateStreamPartitionPlanner.java | 16 +- .../apache/samza/execution/TestJobGraph.java | 34 +-- .../execution/TestJobGraphJsonGenerator.java | 25 +- .../apache/samza/execution/TestJobNode.java | 35 +-- ...ava => TestJobNodeConfigureGenerator.java} | 216 ++++++++-------- .../runtime/TestLocalApplicationRunner.java | 9 +- 20 files changed, 449 insertions(+), 483 deletions(-) rename samza-core/src/main/java/org/apache/samza/execution/{JobGraphConfigureGenerator.java => JobNodeConfigureGenerator.java} (57%) rename samza-core/src/test/java/org/apache/samza/execution/{TestJobGraphConfigureGenerator.java => TestJobNodeConfigureGenerator.java} (80%) diff --git a/build.gradle b/build.gradle index 7d23717f93..72b914dd3d 100644 --- a/build.gradle +++ b/build.gradle @@ -191,6 +191,7 @@ project(":samza-core_$scalaVersion") { testCompile "org.powermock:powermock-core:$powerMockVersion" testCompile "org.powermock:powermock-module-junit4:$powerMockVersion" testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" + testCompile "org.hamcrest:hamcrest-all:$hamcrestVersion" } checkstyle { diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java index 96791366f3..f1f35b18aa 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java @@ -30,8 +30,11 @@ import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; +import org.apache.samza.operators.spec.InputOperatorSpec; +import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; +import org.apache.samza.serializers.Serde; import org.apache.samza.task.TaskContext; @@ -176,4 +179,53 @@ public Optional getDefaultSystemDescriptor() { */ public abstract Set getSystemDescriptors(); + /** + * Get all the unique input streamIds in this application + * + * @return an immutable set of input streamIds + */ + public abstract Set getInputStreamIds(); + + /** + * Get all the unique output streamIds in this application + * + * @return an immutable set of output streamIds + */ + public abstract Set getOutputStreamIds(); + + /** + * Get the corresponding {@link Serde} for the input {@code inputStreamId} + * + * @param inputStreamId id of the input stream + * @return the {@link Serde} for the input stream. null if the serde is not defined or {@code inputStreamId} + * does not exist + */ + public abstract Serde getInputSerde(String inputStreamId); + + /** + * Get the corresponding {@link Serde} for the output {@code outputStreamId} + * + * @param outputStreamId of the output stream + * @return the {@link Serde} for the output stream. null if the serde is not defined or {@code outputStreamId} + * does not exist + */ + public abstract Serde getOutputSerde(String outputStreamId); + + /** + * Get the map of all {@link InputOperatorSpec}s in this applicaiton + * + * @return an immutable map from streamId to {@link InputOperatorSpec}. Default to empty map for low-level {@link TaskApplication} + */ + public Map getInputOperators() { + return Collections.EMPTY_MAP; + } + + /** + * Get the map of all {@link OutputStreamImpl}s in this application + * + * @return an immutable map from streamId to {@link OutputStreamImpl}. Default to empty map for low-level {@link TaskApplication} + */ + public Map getOutputStreams() { + return Collections.EMPTY_MAP; + } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java index ae7a45d256..ae200c2eb0 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java @@ -239,6 +239,35 @@ public Set getSystemDescriptors() { return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); } + @Override + public Set getInputStreamIds() { + return Collections.unmodifiableSet(new HashSet<>(inputOperators.keySet())); + } + + @Override + public Set getOutputStreamIds() { + return Collections.unmodifiableSet(new HashSet<>(outputStreams.keySet())); + } + + @Override + public Serde getInputSerde(String inputStreamId) { + if (!inputOperators.containsKey(inputStreamId)) { + // the corresponding inputStreamId does not exist + return null; + } + InputOperatorSpec inputOperator = inputOperators.get(inputStreamId); + return KVSerde.of(inputOperator.getKeySerde(), inputOperator.getValueSerde()); + } + + @Override + public Serde getOutputSerde(String outputStreamId) { + if (!outputStreams.containsKey(outputStreamId)) { + return null; + } + OutputStreamImpl outputStream = outputStreams.get(outputStreamId); + return KVSerde.of(outputStream.getKeySerde(), outputStream.getValueSerde()); + } + /** * Get the default {@link SystemDescriptor} in this application * diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java index 3597d7c4fc..bd931c2694 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java @@ -29,6 +29,7 @@ import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; +import org.apache.samza.serializers.Serde; import org.apache.samza.task.TaskFactory; @@ -111,6 +112,34 @@ public Set getSystemDescriptors() { return Collections.unmodifiableSet(new HashSet<>(systemDescriptors.values())); } + @Override + public Set getInputStreamIds() { + return Collections.unmodifiableSet(new HashSet<>(inputDescriptors.keySet())); + } + + @Override + public Set getOutputStreamIds() { + return Collections.unmodifiableSet(new HashSet<>(outputDescriptors.keySet())); + } + + @Override + public Serde getInputSerde(String inputStreamId) { + if (!inputDescriptors.containsKey(inputStreamId)) { + return null; + } + InputDescriptor inputDescriptor = inputDescriptors.get(inputStreamId); + return inputDescriptor.getSerde(); + } + + @Override + public Serde getOutputSerde(String outputStreamId) { + if (!outputDescriptors.containsKey(outputStreamId)) { + return null; + } + OutputDescriptor outputDescriptor = outputDescriptors.get(outputStreamId); + return outputDescriptor.getSerde(); + } + /** * Get the user-defined {@link TaskFactory} * @return the {@link TaskFactory} object diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 7cb96e0b70..8205abe35c 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -26,6 +26,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; @@ -35,7 +36,7 @@ import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.StreamConfig; -import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemStream; import org.apache.samza.table.TableSpec; @@ -46,7 +47,7 @@ /** - * The ExecutionPlanner creates the physical execution graph for the {@link OperatorSpecGraph}, and + * The ExecutionPlanner creates the physical execution graph for the {@link ApplicationDescriptorImpl}, and * the intermediate topics needed for the execution. */ // TODO: ExecutionPlanner needs to be able to generate single node JobGraph for low-level TaskApplication as well (SAMZA-1811) @@ -65,8 +66,7 @@ public ExecutionPlan plan(ApplicationDescriptorImpl appDesc, + JobGraphJsonGenerator jobJsonGenerator, JobNodeConfigureGenerator jobConfigureGenerator) { + JobGraph jobGraph = new JobGraph(config, appDesc, jobJsonGenerator, jobConfigureGenerator); StreamConfig streamConfig = new StreamConfig(config); - Set sourceStreams = getStreamSpecs(jobConfigureGenerator.getInputStreamIds(), streamConfig); - Set sinkStreams = getStreamSpecs(jobConfigureGenerator.getOutputStreamIds(), streamConfig); + Set sourceStreams = getStreamSpecs(appDesc.getInputStreamIds(), streamConfig); + Set sinkStreams = getStreamSpecs(appDesc.getOutputStreamIds(), streamConfig); Set intStreams = new HashSet<>(sourceStreams); - Set tables = new HashSet<>(jobConfigureGenerator.getTableSpecs()); + Set tables = appDesc.getTableDescriptors().stream() + .map(tableDescriptor -> ((BaseTableDescriptor) tableDescriptor).getTableSpec()).collect(Collectors.toSet()); intStreams.retainAll(sinkStreams); sourceStreams.removeAll(intStreams); sinkStreams.removeAll(intStreams); @@ -121,7 +123,7 @@ private void validateConfig() { // add tables tables.forEach(spec -> jobGraph.addTable(spec, node)); - if (!isLegacyTaskApplication) { + if (!LegacyTaskApplication.class.isAssignableFrom(appDesc.getAppClass())) { // skip the validation when input streamIds are empty. This is only possible for LegacyApplication jobGraph.validate(); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java index 0c5e818239..b8608c1aab 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java @@ -23,7 +23,6 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.Map; @@ -32,7 +31,6 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.StreamConfig; @@ -60,8 +58,7 @@ class IntermediateStreamPartitionPlanner { IntermediateStreamPartitionPlanner(Config config, ApplicationDescriptorImpl appDesc) { this.config = config; - this.inputOperators = appDesc instanceof StreamApplicationDescriptorImpl ? - ((StreamApplicationDescriptorImpl) appDesc).getInputOperators() : new HashMap<>(); + this.inputOperators = appDesc.getInputOperators(); } /** @@ -95,10 +92,10 @@ class IntermediateStreamPartitionPlanner { StreamConfig streamConfig = new StreamConfig(config); inputOperators.forEach((key, value) -> { - StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(getStreamSpec(key, streamConfig)); - // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge - findReachableJoins(value, streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, visited); - }); + StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(getStreamSpec(key, streamConfig)); + // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge + findReachableJoins(value, streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs, joinQ, visited); + }); // At this point, joinQ contains joinSpecs where at least one of the input stream edge partitions is known. while (!joinQ.isEmpty()) { diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 2d0c46f140..f8df2138d7 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -31,6 +31,8 @@ import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -59,16 +61,19 @@ private final Set tables = new HashSet<>(); private final Config config; private final JobGraphJsonGenerator jsonGenerator; - private final JobGraphConfigureGenerator configGenerator; + private final JobNodeConfigureGenerator configGenerator; + private final ApplicationDescriptorImpl appDesc; /** * The JobGraph is only constructed by the {@link ExecutionPlanner}. - * @param config Config + * @param appDesc Config */ - JobGraph(Config config, JobGraphJsonGenerator jsonGenerator, JobGraphConfigureGenerator configureGenerator) { + JobGraph(Config config, ApplicationDescriptorImpl appDesc, JobGraphJsonGenerator jsonGenerator, + JobNodeConfigureGenerator configureGenerator) { this.jsonGenerator = jsonGenerator; this.configGenerator = configureGenerator; this.config = config; + this.appDesc = appDesc; } @Override @@ -153,16 +158,11 @@ void addTable(TableSpec tableSpec, JobNode node) { * Get the {@link JobNode}. Create one if it does not exist. * @param jobName name of the job * @param jobId id of the job - * @return + * @return {@link JobNode} created with {@code jobName} and {@code jobId} */ JobNode getOrCreateJobNode(String jobName, String jobId) { String nodeId = JobNode.createId(jobName, jobId); - JobNode node = nodes.get(nodeId); - if (node == null) { - node = new JobNode(jobName, jobId, config, configGenerator); - nodes.put(nodeId, node); - } - return node; + return nodes.computeIfAbsent(nodeId, k -> new JobNode(jobName, jobId, config, appDesc, configGenerator)); } /** @@ -180,11 +180,11 @@ StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec) { * @param isIntermediate boolean flag indicating whether it's an intermediate stream * @return stream edge */ - StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec, boolean isIntermediate) { + private StreamEdge getOrCreateStreamEdge(StreamSpec streamSpec, boolean isIntermediate) { String streamId = streamSpec.getId(); StreamEdge edge = edges.get(streamId); if (edge == null) { - boolean isBroadcast = configGenerator.isBroadcastStream(streamId); + boolean isBroadcast = appDesc.getBroadcastStreams().contains(streamId); edge = new StreamEdge(streamSpec, isIntermediate, isBroadcast, config); edges.put(streamId, edge); } @@ -322,7 +322,7 @@ Set findReachable() { while (!queue.isEmpty()) { JobNode node = queue.poll(); - node.getOutEdges().stream().flatMap(edge -> edge.getTargetNodes().stream()).forEach(target -> { + node.getOutEdges().values().stream().flatMap(edge -> edge.getTargetNodes().stream()).forEach(target -> { if (!visited.contains(target)) { visited.add(target); queue.offer(target); @@ -350,7 +350,7 @@ List topologicalSort() { pnodes.forEach(node -> { String nid = node.getId(); //only count the degrees of intermediate streams - long degree = node.getInEdges().stream().filter(e -> !sources.contains(e)).count(); + long degree = node.getInEdges().values().stream().filter(e -> !sources.contains(e)).count(); indegree.put(nid, degree); if (degree == 0L) { @@ -375,7 +375,7 @@ List topologicalSort() { while (!q.isEmpty()) { JobNode node = q.poll(); sortedNodes.add(node); - node.getOutEdges().stream().flatMap(edge -> edge.getTargetNodes().stream()).forEach(n -> { + node.getOutEdges().values().stream().flatMap(edge -> edge.getTargetNodes().stream()).forEach(n -> { String nid = n.getId(); Long degree = indegree.get(nid) - 1; indegree.put(nid, degree); diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 30ef6c9caf..d746d0ae6b 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -29,11 +29,7 @@ import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.ApplicationDescriptor; -import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; -import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputOperatorSpec; @@ -120,12 +116,7 @@ static final class JobGraphJson { String applicationId; } - // input operators for the application. For low-level task applications, this is empty. - private final Map inputOperators; - - JobGraphJsonGenerator(ApplicationDescriptorImpl appDesc) { - this.inputOperators = appDesc instanceof StreamApplicationDescriptorImpl ? - ((StreamApplicationDescriptorImpl) appDesc).getInputOperators() : new HashMap<>(); + JobGraphJsonGenerator() { } /** @@ -151,7 +142,7 @@ static final class JobGraphJson { jobGraph.getTables().forEach(t -> buildTableJson(t, jobGraphJson.tables)); jobGraphJson.jobs = jobGraph.getJobNodes().stream() - .map(jobNode -> buildJobNodeJson(jobNode)) + .map(this::buildJobNodeJson) .collect(Collectors.toList()); ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -207,30 +198,6 @@ private Map operatorToMap(OperatorSpec spec) { return map; } - // get all next operators consuming from the input {@code streamId} - private Set getNextOperatorIds(String streamId) { - if (!this.inputOperators.containsKey(streamId)) { - return new HashSet<>(); - } - return this.inputOperators.get(streamId).getRegisteredOperatorSpecs().stream() - .map(op -> op.getOpId()).collect(Collectors.toSet()); - } - - /** - * Traverse the {@link OperatorSpec} graph recursively and update the operator graph JSON POJO. - * @param inputStreamId input streamId - * @param opGraph operator graph to build - */ - private void updateOperatorGraphJson(String inputStreamId, OperatorGraphJson opGraph) { - // TODO xiliu: render input operators instead of input streams - InputOperatorSpec operatorSpec = this.inputOperators.get(inputStreamId); - if (operatorSpec == null) { - // no corresponding input operator for input stream - return; - } - updateOperatorGraphJson(operatorSpec, opGraph); - } - /** * Create JSON POJO for a {@link JobNode}, including the {@link org.apache.samza.application.ApplicationDescriptorImpl} * for this job @@ -254,16 +221,16 @@ private JobNodeJson buildJobNodeJson(JobNode jobNode) { private OperatorGraphJson buildOperatorGraphJson(JobNode jobNode) { OperatorGraphJson opGraph = new OperatorGraphJson(); opGraph.inputStreams = new ArrayList<>(); - jobNode.getInEdges().forEach(inStream -> { + jobNode.getInEdges().values().forEach(inStream -> { StreamJson inputJson = new StreamJson(); opGraph.inputStreams.add(inputJson); inputJson.streamId = inStream.getStreamSpec().getId(); - inputJson.nextOperatorIds = getNextOperatorIds(inputJson.streamId); - updateOperatorGraphJson(inputJson.streamId, opGraph); + inputJson.nextOperatorIds = jobNode.getNextOperatorIds(inputJson.streamId); + updateOperatorGraphJson(jobNode.getInputOperator(inputJson.streamId), opGraph); }); opGraph.outputStreams = new ArrayList<>(); - jobNode.getOutEdges().forEach(outStream -> { + jobNode.getOutEdges().values().forEach(outStream -> { StreamJson outputJson = new StreamJson(); outputJson.streamId = outStream.getStreamSpec().getId(); opGraph.outputStreams.add(outputJson); @@ -290,15 +257,11 @@ private StreamEdgeJson buildStreamEdgeJson(StreamEdge edge, Map sourceJobs = new ArrayList<>(); - edge.getSourceNodes().forEach(jobNode -> { - sourceJobs.add(jobNode.getJobName()); - }); + edge.getSourceNodes().forEach(jobNode -> sourceJobs.add(jobNode.getJobName())); edgeJson.sourceJobs = sourceJobs; List targetJobs = new ArrayList<>(); - edge.getTargetNodes().forEach(jobNode -> { - targetJobs.add(jobNode.getJobName()); - }); + edge.getTargetNodes().forEach(jobNode -> targetJobs.add(jobNode.getJobName())); edgeJson.targetJobs = targetJobs; streamEdges.put(streamId, edgeJson); @@ -314,12 +277,7 @@ private StreamEdgeJson buildStreamEdgeJson(StreamEdge edge, Map tableSpecs) { String tableId = tableSpec.getId(); - TableSpecJson tableSpecJson = tableSpecs.get(tableId); - if (tableSpecJson == null) { - tableSpecJson = buildTableJson(tableSpec); - tableSpecs.put(tableId, tableSpecJson); - } - return tableSpecJson; + return tableSpecs.computeIfAbsent(tableId, k -> buildTableJson(tableSpec)); } /** diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index 5e64947d5c..efbed61dab 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -19,10 +19,20 @@ package org.apache.samza.execution; -import java.util.ArrayList; -import java.util.List; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; +import org.apache.samza.operators.spec.InputOperatorSpec; +import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.serializers.Serde; import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,19 +49,26 @@ public class JobNode { private final String jobId; private final String id; private final Config config; - private final JobGraphConfigureGenerator configGenerator; - private final List inEdges = new ArrayList<>(); - private final List outEdges = new ArrayList<>(); - private final List tables = new ArrayList<>(); - - JobNode(String jobName, String jobId, Config config, JobGraphConfigureGenerator configureGenerator) { + private final JobNodeConfigureGenerator configGenerator; + private final Map inEdges = new HashMap<>(); + private final Map outEdges = new HashMap<>(); + private final Map tables = new HashMap<>(); + private final ApplicationDescriptorImpl appDesc; + + JobNode(String jobName, String jobId, Config config, ApplicationDescriptorImpl appDesc, + JobNodeConfigureGenerator configureGenerator) { this.jobName = jobName; this.jobId = jobId; this.id = createId(jobName, jobId); this.config = config; + this.appDesc = appDesc; this.configGenerator = configureGenerator; } + static String createId(String jobName, String jobId) { + return String.format("%s-%s", jobName, jobId); + } + String getId() { return id; } @@ -69,26 +86,26 @@ Config getConfig() { } void addInEdge(StreamEdge in) { - inEdges.add(in); + inEdges.put(in.getStreamSpec().getId(), in); } void addOutEdge(StreamEdge out) { - outEdges.add(out); + outEdges.put(out.getStreamSpec().getId(), out); } void addTable(TableSpec tableSpec) { - tables.add(tableSpec); + tables.put(tableSpec.getId(), tableSpec); } - List getInEdges() { + Map getInEdges() { return inEdges; } - List getOutEdges() { + Map getOutEdges() { return outEdges; } - List getTables() { + Map getTables() { return tables; } @@ -101,8 +118,46 @@ JobConfig generateConfig(String executionPlanJson) { return configGenerator.generateJobConfig(this, executionPlanJson); } - static String createId(String jobName, String jobId) { - return String.format("%s-%s", jobName, jobId); + Serde getInputSerde(String streamId) { + return appDesc.getInputSerde(streamId); + } + + Serde getOutputSerde(String streamId) { + return appDesc.getOutputSerde(streamId); + } + + Collection getReachableOperators() { + Set inputOperatorsInJobNode = inEdges.values().stream().map(inEdge -> + appDesc.getInputOperators().get(inEdge.getStreamSpec().getId())).filter(Objects::nonNull).collect(Collectors.toSet()); + Set reachableOperators = new HashSet<>(); + findReachableOperators(inputOperatorsInJobNode, reachableOperators); + return reachableOperators; + } + + private void findReachableOperators(Collection inputOperatorsInJobNode, + Set reachableOperators) { + inputOperatorsInJobNode.forEach(op -> { + if (reachableOperators.contains(op)) { + return; + } + reachableOperators.add(op); + findReachableOperators(op.getRegisteredOperatorSpecs(), reachableOperators); + }); + } + + // get all next operators consuming from the input {@code streamId} + Set getNextOperatorIds(String streamId) { + if (!appDesc.getInputOperators().containsKey(streamId) || !inEdges.containsKey(streamId)) { + return new HashSet<>(); + } + return appDesc.getInputOperators().get(streamId).getRegisteredOperatorSpecs().stream() + .map(op -> op.getOpId()).collect(Collectors.toSet()); } + InputOperatorSpec getInputOperator(String inputStreamId) { + if (!inEdges.containsKey(inputStreamId)) { + return null; + } + return appDesc.getInputOperators().get(inputStreamId); + } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java similarity index 57% rename from samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java rename to samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java index 016cb30d8c..a8200bc8bc 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphConfigureGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java @@ -28,13 +28,8 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.function.Function; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.application.ApplicationDescriptor; -import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.StreamApplicationDescriptor; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; @@ -43,13 +38,8 @@ import org.apache.samza.config.StreamConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.config.TaskConfigJava; -import org.apache.samza.operators.BaseTableDescriptor; -import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; -import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; -import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; -import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.spec.StatefulOperatorSpec; import org.apache.samza.operators.spec.StoreDescriptor; import org.apache.samza.operators.spec.WindowOperatorSpec; @@ -68,28 +58,14 @@ /** * This class provides methods to generate configuration for a {@link JobNode} */ -public class JobGraphConfigureGenerator { +/* package private */ class JobNodeConfigureGenerator { - private static final Logger LOG = LoggerFactory.getLogger(JobGraphConfigureGenerator.class); - - private final StreamSerdeConfigureGenerator streamSerdeConfigureGenerator; - private final Set broadcastStreamIds; - private final Set inputStreamIds; - private final Set outputStreamIds; - private final Set tableSpecs; + private static final Logger LOG = LoggerFactory.getLogger(JobNodeConfigureGenerator.class); static final String CONFIG_JOB_PREFIX = "jobs.%s."; static final String CONFIG_INTERNAL_EXECUTION_PLAN = "samza.internal.execution.plan"; - JobGraphConfigureGenerator(ApplicationDescriptorImpl appDesc) { - this.streamSerdeConfigureGenerator = new StreamSerdeConfigureGenerator(appDesc); - this.broadcastStreamIds = appDesc.getBroadcastStreams(); - this.inputStreamIds = appDesc instanceof StreamApplicationDescriptor ? - ((StreamApplicationDescriptorImpl) appDesc).getInputOperators().keySet() : appDesc.getInputDescriptors().keySet(); - this.outputStreamIds = appDesc instanceof StreamApplicationDescriptor ? - ((StreamApplicationDescriptorImpl) appDesc).getOutputStreams().keySet() : appDesc.getOutputDescriptors().keySet(); - this.tableSpecs = appDesc.getTableDescriptors().stream().map(tableDesc -> ((BaseTableDescriptor) tableDesc).getTableSpec()) - .collect(Collectors.toSet()); + JobNodeConfigureGenerator() { } static Config mergeJobConfig(Config originalConfig, Config generatedConfig) { @@ -99,31 +75,15 @@ static Config mergeJobConfig(Config originalConfig, Config generatedConfig) { String.format(CONFIG_JOB_PREFIX, jobId)))); } - boolean isBroadcastStream(String streamId) { - return this.broadcastStreamIds.contains(streamId); - } - - Set getInputStreamIds() { - return inputStreamIds; - } - - Set getOutputStreamIds() { - return outputStreamIds; - } - - Set getTableSpecs() { - return tableSpecs; - } - JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { Map configs = new HashMap<>(); // set up job name and job ID configs.put(JobConfig.JOB_NAME(), jobNode.getJobName()); configs.put(JobConfig.JOB_ID(), jobNode.getJobId()); - List inEdges = jobNode.getInEdges(); - List outEdges = jobNode.getOutEdges(); - Collection reachableOperators = getReachableOperators(jobNode); + Map inEdges = jobNode.getInEdges(); + Map outEdges = jobNode.getOutEdges(); + Collection reachableOperators = jobNode.getReachableOperators(); boolean hasWindowOrJoin = reachableOperators.stream().anyMatch(op -> op.getOpCode() == OperatorSpec.OpCode.WINDOW || op.getOpCode() == OperatorSpec.OpCode.JOIN); List stores = getStoreDescriptors(reachableOperators); @@ -132,7 +92,7 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { // check all inputs to the node for broadcast and input streams final Set inputs = new HashSet<>(); final Set broadcasts = new HashSet<>(); - for (StreamEdge inEdge : inEdges) { + for (StreamEdge inEdge : inEdges.values()) { String formattedSystemStream = inEdge.getName(); if (inEdge.isBroadcast()) { broadcasts.add(formattedSystemStream + "#0"); @@ -157,10 +117,10 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { configs.put(CONFIG_INTERNAL_EXECUTION_PLAN, executionPlanJson); // write intermediate input/output streams to configs - inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); + inEdges.values().stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); // write serialized serde instances and stream /store serdes to configs - configureSerdes(configs, inEdges, outEdges, stores); + configureSerdes(configs, inEdges, outEdges, stores, jobNode); // generate table configuration and potential side input configuration configureTables(configs, config, jobNode.getTables(), inputs); @@ -245,22 +205,23 @@ private List getStoreDescriptors(Collection reach .collect(Collectors.toList()); } - private void configureTables(Map configs, Config config, List tables, Set inputs) { - configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), tables)); + private void configureTables(Map configs, Config config, Map tables, Set inputs) { + configs.putAll(TableConfigGenerator.generateConfigsForTableSpecs(new MapConfig(configs), + tables.values().stream().collect(Collectors.toList()))); // Add side inputs to the inputs and mark the stream as bootstrap - tables.forEach(tableSpec -> { - List sideInputs = tableSpec.getSideInputs(); - if (sideInputs != null && !sideInputs.isEmpty()) { - sideInputs.stream() - .map(sideInput -> StreamUtil.getSystemStreamFromNameOrId(config, sideInput)) - .forEach(systemStream -> { - inputs.add(StreamUtil.getNameFromSystemStream(systemStream)); - configs.put(String.format(StreamConfig.STREAM_PREFIX() + StreamConfig.BOOTSTRAP(), - systemStream.getSystem(), systemStream.getStream()), "true"); - }); - } - }); + tables.values().forEach(tableSpec -> { + List sideInputs = tableSpec.getSideInputs(); + if (sideInputs != null && !sideInputs.isEmpty()) { + sideInputs.stream() + .map(sideInput -> StreamUtil.getSystemStreamFromNameOrId(config, sideInput)) + .forEach(systemStream -> { + inputs.add(StreamUtil.getNameFromSystemStream(systemStream)); + configs.put(String.format(StreamConfig.STREAM_PREFIX() + StreamConfig.BOOTSTRAP(), + systemStream.getSystem(), systemStream.getStream()), "true"); + }); + } + }); } /** @@ -276,19 +237,15 @@ private void configureTables(Map configs, Config config, List configs, List inEdges, List outEdges, - List stores) { + private void configureSerdes(Map configs, Map inEdges, Map outEdges, + List stores, JobNode jobNode) { // collect all key and msg serde instances for streams Map streamKeySerdes = new HashMap<>(); Map streamMsgSerdes = new HashMap<>(); - inEdges.forEach(edge -> { - String streamId = edge.getStreamSpec().getId(); - streamSerdeConfigureGenerator.addSerde(sid -> streamSerdeConfigureGenerator.getInputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); - }); - outEdges.forEach(edge -> { - String streamId = edge.getStreamSpec().getId(); - streamSerdeConfigureGenerator.addSerde(sid -> streamSerdeConfigureGenerator.getOutputSerde(sid), streamId, streamKeySerdes, streamMsgSerdes); - }); + inEdges.keySet().forEach(streamId -> + addSerde(jobNode.getInputSerde(streamId), streamId, streamKeySerdes, streamMsgSerdes)); + outEdges.keySet().forEach(streamId -> + addSerde(jobNode.getOutputSerde(streamId), streamId, streamKeySerdes, streamMsgSerdes)); Map storeKeySerdes = new HashMap<>(); Map storeMsgSerdes = new HashMap<>(); @@ -306,35 +263,35 @@ private void configureSerdes(Map configs, List inEdg Base64.Encoder base64Encoder = Base64.getEncoder(); Map serdeUUIDs = new HashMap<>(); serdes.forEach(serde -> { - String serdeName = serdeUUIDs.computeIfAbsent(serde, - s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); - configs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), - base64Encoder.encodeToString(serializableSerde.toBytes(serde))); - }); + String serdeName = serdeUUIDs.computeIfAbsent(serde, + s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); + configs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), + base64Encoder.encodeToString(serializableSerde.toBytes(serde))); + }); // set key and msg serdes for streams to the serde names generated above streamKeySerdes.forEach((streamId, serde) -> { - String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); - String keySerdeConfigKey = streamIdPrefix + StreamConfig.KEY_SERDE(); - configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); - }); + String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); + String keySerdeConfigKey = streamIdPrefix + StreamConfig.KEY_SERDE(); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); streamMsgSerdes.forEach((streamId, serde) -> { - String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); - String valueSerdeConfigKey = streamIdPrefix + StreamConfig.MSG_SERDE(); - configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); - }); + String streamIdPrefix = String.format(StreamConfig.STREAM_ID_PREFIX(), streamId); + String valueSerdeConfigKey = streamIdPrefix + StreamConfig.MSG_SERDE(); + configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); + }); // set key and msg serdes for stores to the serde names generated above storeKeySerdes.forEach((storeName, serde) -> { - String keySerdeConfigKey = String.format(StorageConfig.KEY_SERDE(), storeName); - configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); - }); + String keySerdeConfigKey = String.format(StorageConfig.KEY_SERDE(), storeName); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); storeMsgSerdes.forEach((storeName, serde) -> { - String msgSerdeConfigKey = String.format(StorageConfig.MSG_SERDE(), storeName); - configs.put(msgSerdeConfigKey, serdeUUIDs.get(serde)); - }); + String msgSerdeConfigKey = String.format(StorageConfig.MSG_SERDE(), storeName); + configs.put(msgSerdeConfigKey, serdeUUIDs.get(serde)); + }); } /** @@ -364,91 +321,18 @@ private long computeTriggerInterval(Collection reachableOperators) return MathUtil.gcd(candidateTimerIntervals); } - Collection getReachableOperators(JobNode jobNode) { - // Filter out window operators, and obtain a list of their triggering interval values - Set inputOperatorsInJobNode = jobNode.getInEdges().stream() - .filter(streamEdge -> streamSerdeConfigureGenerator.inputOperators.containsKey(streamEdge.getStreamSpec().getId())) - .map(streamEdge -> streamSerdeConfigureGenerator.inputOperators.get(streamEdge.getStreamSpec().getId())) - .collect(Collectors.toSet()); - Set reachableOperators = new HashSet<>(); - findReachableOperators(inputOperatorsInJobNode, reachableOperators); - return reachableOperators; - } - - private void findReachableOperators(Collection inputOperatorsInJobNode, Set reachableOperators) { - inputOperatorsInJobNode.forEach(op -> { - if (reachableOperators.contains(op)) { - return; - } - reachableOperators.add(op); - findReachableOperators(op.getRegisteredOperatorSpecs(), reachableOperators); - }); - } - - private class StreamSerdeConfigureGenerator { - private final Map inputDescriptors; - private final Map inputOperators; - private final Map outputDescriptors; - private final Map outputStreams; - - private StreamSerdeConfigureGenerator(ApplicationDescriptorImpl appDesc) { - this.inputDescriptors = appDesc.getInputDescriptors(); - this.outputDescriptors = appDesc.getOutputDescriptors(); - if (appDesc instanceof StreamApplicationDescriptorImpl) { - StreamApplicationDescriptorImpl streamAppDesc = (StreamApplicationDescriptorImpl) appDesc; - this.inputOperators = streamAppDesc.getInputOperators(); - this.outputStreams = streamAppDesc.getOutputStreams(); - } else { - this.inputOperators = new HashMap<>(); - this.outputStreams = new HashMap<>(); - } - } - - private Serde getInputSerde(String streamId) { - InputDescriptor inputDescriptor = inputDescriptors.get(streamId); - if (inputDescriptor != null) { - return inputDescriptor.getSerde(); - } - - // for high-level applications, the intermediate streams don't have the input descriptor yet - InputOperatorSpec inputOp = inputOperators.get(streamId); - if (inputOp == null) { - LOG.warn("Input stream {} don't have any corresponding InputDescriptor or InputOperatorSpec.", streamId); - return null; - } - return KVSerde.of(inputOp.getKeySerde(), inputOp.getValueSerde()); - } - - private Serde getOutputSerde(String streamId) { - OutputDescriptor outputDescriptor = outputDescriptors.get(streamId); - if (outputDescriptor != null) { - return outputDescriptor.getSerde(); - } - - // for high-level applications, the intermediate streams don't have the input descriptor yet - OutputStreamImpl outputStream = outputStreams.get(streamId); - if (outputStream == null) { - LOG.warn("Output stream {} don't have any corresponding OutputDescriptor or OutputStreamImpl.", streamId); - return null; - } - return KVSerde.of(outputStream.getKeySerde(), outputStream.getValueSerde()); - } - - private void addSerde(Function serdeFinder, String streamId, Map keySerdeMap, - Map msgSerdeMap) { - Serde serde = serdeFinder.apply(streamId); - if (serde != null) { - if (serde instanceof KVSerde) { - KVSerde kvSerde = (KVSerde) serde; - if (kvSerde.getKeySerde() != null) { - keySerdeMap.put(streamId, ((KVSerde) serde).getKeySerde()); - } - if (kvSerde.getValueSerde() != null) { - msgSerdeMap.put(streamId, ((KVSerde) serde).getValueSerde()); - } - } else { - msgSerdeMap.put(streamId, serde); + private void addSerde(Serde serde, String streamId, Map keySerdeMap, Map msgSerdeMap) { + if (serde != null) { + if (serde instanceof KVSerde) { + KVSerde kvSerde = (KVSerde) serde; + if (kvSerde.getKeySerde() != null) { + keySerdeMap.put(streamId, ((KVSerde) serde).getKeySerde()); } + if (kvSerde.getValueSerde() != null) { + msgSerdeMap.put(streamId, ((KVSerde) serde).getValueSerde()); + } + } else { + msgSerdeMap.put(streamId, serde); } } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index 9d47428b57..faf29b2d2a 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -20,25 +20,20 @@ import java.io.File; import java.io.PrintWriter; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; -import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.StreamApplicationDescriptorImpl; -import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.ShellCommandConfig; import org.apache.samza.config.StreamConfig; -import org.apache.samza.operators.OperatorSpecGraph; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,9 +77,8 @@ ExecutionPlan getExecutionPlan(String runId) { } StreamConfig streamConfig = new StreamConfig(config); - JobGraphConfigureGenerator jobGraphConfigureGenerator = new JobGraphConfigureGenerator(appDesc); - Set inputStreams = jobGraphConfigureGenerator.getInputStreamIds(); - inputStreams.removeAll(jobGraphConfigureGenerator.getOutputStreamIds()); + Set inputStreams = new HashSet<>(appDesc.getInputStreamIds()); + inputStreams.removeAll(appDesc.getOutputStreamIds()); ApplicationConfig.ApplicationMode mode = inputStreams.stream().allMatch(streamConfig::getIsBounded) ? ApplicationConfig.ApplicationMode.BATCH : ApplicationConfig.ApplicationMode.STREAM; cfg.put(ApplicationConfig.APP_MODE, mode.name()); @@ -99,7 +93,7 @@ ExecutionPlan getExecutionPlan(String runId) { // create the physical execution plan and merge with overrides. This works for a single-stage job now // TODO: This should all be consolidated with ExecutionPlanner after fixing SAMZA-1811 - Config mergedConfig = JobGraphConfigureGenerator.mergeJobConfig(config, new MapConfig(cfg)); + Config mergedConfig = JobNodeConfigureGenerator.mergeJobConfig(config, new MapConfig(cfg)); // creating the StreamManager to get all input/output streams' metadata for planning StreamManager streamManager = buildAndStartStreamManager(mergedConfig); try { diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java index b360ea1633..55583b86fc 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -25,7 +25,6 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; diff --git a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java index 1c65ac6082..0a6ce12c29 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java @@ -23,7 +23,6 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; 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 82ae0c8a57..01c56b4f76 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 @@ -293,8 +293,8 @@ public void testCreateProcessorGraph() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), - new JobGraphConfigureGenerator(graphSpec), false); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), + new JobNodeConfigureGenerator()); assertTrue(jobGraph.getSources().size() == 3); assertTrue(jobGraph.getSinks().size() == 2); assertTrue(jobGraph.getIntermediateStreams().size() == 2); // two streams generated by partitionBy @@ -304,8 +304,8 @@ public void testCreateProcessorGraph() { public void testFetchExistingStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), - new JobGraphConfigureGenerator(graphSpec), false); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), + new JobNodeConfigureGenerator()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); assertTrue(jobGraph.getOrCreateStreamEdge(input1Spec).getPartitionCount() == 64); @@ -323,8 +323,8 @@ public void testFetchExistingStreamPartitions() { public void testCalculateJoinInputPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), new JobGraphJsonGenerator(graphSpec), - new JobGraphConfigureGenerator(graphSpec), false); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), + new JobNodeConfigureGenerator()); ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); @@ -514,8 +514,8 @@ public void testCreateJobGraphForTaskApplication() { systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - JobGraph jobGraph = planner.createJobGraph(config, new JobGraphJsonGenerator(taskAppDesc), - new JobGraphConfigureGenerator(taskAppDesc), false); + JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc, new JobGraphJsonGenerator(), + new JobNodeConfigureGenerator()); assertEquals(1, jobGraph.getJobNodes().size()); assertTrue(jobGraph.getSources().stream().map(edge -> edge.getName()) .filter(streamId -> inputDescriptors.containsKey(streamId)).collect(Collectors.toList()).isEmpty()); @@ -545,8 +545,8 @@ public void testCreateJobGraphForLegacyTaskApplication() { systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - JobGraph jobGraph = planner.createJobGraph(config, new JobGraphJsonGenerator(taskAppDesc), - new JobGraphConfigureGenerator(taskAppDesc), true); + JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc, new JobGraphJsonGenerator(), + new JobNodeConfigureGenerator()); assertEquals(1, jobGraph.getJobNodes().size()); JobNode jobNode = jobGraph.getJobNodes().get(0); assertEquals("test-app", jobNode.getJobName()); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java index 61fc09b159..b23bc3e98f 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java @@ -19,9 +19,7 @@ package org.apache.samza.execution; import java.time.Duration; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; @@ -108,13 +106,13 @@ public void setUp() { StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); - List inputEdges = new ArrayList<>(); - inputEdges.add(input1Edge); - inputEdges.add(input2Edge); - inputEdges.add(repartitionEdge); - List outputEdges = new ArrayList<>(); - outputEdges.add(outputEdge); - outputEdges.add(repartitionEdge); + Map inputEdges = new HashMap<>(); + inputEdges.put(input1Descriptor.getStreamId(), input1Edge); + inputEdges.put(input2Descriptor.getStreamId(), input2Edge); + inputEdges.put(repartitionSpec.getId(), repartitionEdge); + Map outputEdges = new HashMap<>(); + outputEdges.put(outputDescriptor.getStreamId(), outputEdge); + outputEdges.put(repartitionSpec.getId(), repartitionEdge); when(mockJobNode.getInEdges()).thenReturn(inputEdges); when(mockJobNode.getOutEdges()).thenReturn(outputEdges); when(mockJobNode.getConfig()).thenReturn(mockConfig); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java index e6067adcc7..9afd8e4c1d 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java @@ -19,12 +19,11 @@ package org.apache.samza.execution; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.samza.operators.OperatorSpecGraph; +import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.system.StreamSpec; import org.junit.Before; import org.junit.Test; @@ -32,7 +31,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TestJobGraph { @@ -61,9 +59,8 @@ private StreamSpec genStream() { * 2 9 10 */ private void createGraph1() { -// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); -// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph1 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); + StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); + graph1 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); JobNode n2 = graph1.getOrCreateJobNode("2", "1"); JobNode n3 = graph1.getOrCreateJobNode("3", "1"); @@ -96,9 +93,8 @@ private void createGraph1() { * |<---6 <--| <> */ private void createGraph2() { -// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); -// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph2 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); + StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); + graph2 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); JobNode n1 = graph2.getOrCreateJobNode("1", "1"); JobNode n2 = graph2.getOrCreateJobNode("2", "1"); @@ -125,9 +121,8 @@ private void createGraph2() { * 1<->1 -> 2<->2 */ private void createGraph3() { -// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); -// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph3 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); + StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); + graph3 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); JobNode n1 = graph3.getOrCreateJobNode("1", "1"); JobNode n2 = graph3.getOrCreateJobNode("2", "1"); @@ -143,9 +138,8 @@ private void createGraph3() { * 1<->1 */ private void createGraph4() { -// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); -// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - graph4 = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); + StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); + graph4 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); JobNode n1 = graph4.getOrCreateJobNode("1", "1"); @@ -163,9 +157,8 @@ public void setup() { @Test public void testAddSource() { -// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); -// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - JobGraph graph = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); + StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); + JobGraph graph = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); /** * s1 -> 1 @@ -206,9 +199,8 @@ public void testAddSink() { * 2 -> s2 * 2 -> s3 */ -// OperatorSpecGraph specGraph = mock(OperatorSpecGraph.class); -// when(specGraph.getBroadcastStreams()).thenReturn(Collections.emptySet()); - JobGraph graph = new JobGraph(null, mock(JobGraphJsonGenerator.class), mock(JobGraphConfigureGenerator.class)); + StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); + JobGraph graph = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); JobNode n1 = graph.getOrCreateJobNode("1", "1"); JobNode n2 = graph.getOrCreateJobNode("2", "1"); StreamSpec s1 = genStream(); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index 36d45a0749..0e8217440d 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -20,11 +20,9 @@ package org.apache.samza.execution; import java.time.Duration; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -58,7 +56,6 @@ import org.apache.samza.testUtils.StreamTestUtils; import org.codehaus.jackson.map.ObjectMapper; import org.hamcrest.Matchers; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -119,13 +116,13 @@ public void setUp() { StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); - List inputEdges = new ArrayList<>(); - inputEdges.add(input1Edge); - inputEdges.add(input2Edge); - inputEdges.add(repartitionEdge); - List outputEdges = new ArrayList<>(); - outputEdges.add(outputEdge); - outputEdges.add(repartitionEdge); + Map inputEdges = new HashMap<>(); + inputEdges.put(input1Descriptor.getStreamId(), input1Edge); + inputEdges.put(input2Descriptor.getStreamId(), input2Edge); + inputEdges.put(repartitionSpec.getId(), repartitionEdge); + Map outputEdges = new HashMap<>(); + outputEdges.put(outputDescriptor.getStreamId(), outputEdge); + outputEdges.put(repartitionSpec.getId(), repartitionEdge); when(mockJobNode.getInEdges()).thenReturn(inputEdges); when(mockJobNode.getOutEdges()).thenReturn(outputEdges); when(mockJobNode.getConfig()).thenReturn(mockConfig); @@ -301,15 +298,15 @@ public void testRepartitionedWindowStreamApplication() throws Exception { @Test public void testTaskApplication() throws Exception { mockAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); - JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(mockAppDesc); + JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); JobGraph mockJobGraph = mock(JobGraph.class); ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); when(mockAppConfig.getAppName()).thenReturn("testTaskApp"); when(mockAppConfig.getAppId()).thenReturn("testTaskAppId"); when(mockJobGraph.getApplicationConfig()).thenReturn(mockAppConfig); // compute the three disjoint sets of the JobGraph: input only, output only, and intermediate streams - Set inEdges = new HashSet<>(mockJobNode.getInEdges()); - Set outEdges = new HashSet<>(mockJobNode.getOutEdges()); + Set inEdges = new HashSet<>(mockJobNode.getInEdges().values()); + Set outEdges = new HashSet<>(mockJobNode.getOutEdges().values()); Set intermediateEdges = new HashSet<>(inEdges); // intermediate streams are the intersection between input and output intermediateEdges.retainAll(outEdges); @@ -348,7 +345,7 @@ public void testTaskApplication() throws Exception { @Test public void testLegacyTaskApplication() throws Exception { mockAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); - JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(mockAppDesc); + JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); JobGraph mockJobGraph = mock(JobGraph.class); ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); when(mockAppConfig.getAppName()).thenReturn("testTaskApp"); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java index dbaa72ab66..f464ed393f 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java @@ -19,38 +19,9 @@ package org.apache.samza.execution; -import java.time.Duration; -import java.util.Base64; -import java.util.HashMap; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.samza.application.StreamApplicationDescriptorImpl; -import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; -import org.apache.samza.config.SerializerConfig; -import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.descriptors.GenericInputDescriptor; -import org.apache.samza.operators.descriptors.GenericOutputDescriptor; -import org.apache.samza.operators.descriptors.GenericSystemDescriptor; -import org.apache.samza.operators.functions.JoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.serializers.JsonSerdeV2; -import org.apache.samza.serializers.KVSerde; -import org.apache.samza.serializers.Serde; -import org.apache.samza.serializers.SerializableSerde; -import org.apache.samza.serializers.StringSerde; -import org.apache.samza.system.StreamSpec; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; - +/** + * Unit tests for {@link JobNode} + */ public class TestJobNode { } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java similarity index 80% rename from samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java rename to samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java index 8b7e26614a..620153bc49 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphConfigureGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java @@ -22,12 +22,14 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Base64; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.StreamApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; @@ -68,15 +70,16 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; /** - * Unit test for {@link JobGraphConfigureGenerator} + * Unit test for {@link JobNodeConfigureGenerator} */ -public class TestJobGraphConfigureGenerator { +public class TestJobNodeConfigureGenerator { private StreamApplicationDescriptorImpl mockStreamAppDesc; private Config mockConfig; @@ -104,7 +107,7 @@ public void setUp() { outputSpec = new StreamSpec("output", "output", "output-system"); repartitionSpec = new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); - broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "broadcast-b1", "intermediate-system"); + broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "jobName-jobId-broadcast-b1", "intermediate-system"); defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); @@ -123,34 +126,31 @@ public void setUp() { Map configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), "jobName"); configs.put(JobConfig.JOB_ID(), "jobId"); + configs.putAll(input1Descriptor.toConfig()); + configs.putAll(input2Descriptor.toConfig()); + configs.putAll(intermediateInputDescriptor.toConfig()); + configs.putAll(broadcastInputDesriptor.toConfig()); + configs.putAll(outputDescriptor.toConfig()); + configs.putAll(intermediateOutputDescriptor.toConfig()); + configs.putAll(inputSystemDescriptor.toConfig()); + configs.putAll(outputSystemDescriptor.toConfig()); + configs.putAll(intermediateSystemDescriptor.toConfig()); mockConfig = spy(new MapConfig(configs)); mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + configureJobNode(mockStreamAppDesc); + } - mockJobNode = mock(JobNode.class); - StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, mockConfig); - StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); - StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); - StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); - List inputEdges = new ArrayList<>(); - inputEdges.add(input1Edge); - inputEdges.add(input2Edge); - inputEdges.add(repartitionEdge); - List outputEdges = new ArrayList<>(); - outputEdges.add(outputEdge); - outputEdges.add(repartitionEdge); - when(mockJobNode.getInEdges()).thenReturn(inputEdges); - when(mockJobNode.getOutEdges()).thenReturn(outputEdges); - when(mockJobNode.getConfig()).thenReturn(mockConfig); - when(mockJobNode.getJobName()).thenReturn("jobName"); - when(mockJobNode.getJobId()).thenReturn("jobId"); - when(mockJobNode.getId()).thenReturn(JobNode.createId("jobName", "jobId")); + private void configureJobNode(ApplicationDescriptorImpl mockStreamAppDesc) { + JobGraph jobGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, + mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + mockJobNode = spy(jobGraph.getJobNodes().get(0)); } @Test public void testConfigureSerdesWithRepartitionJoinApplication() { // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -167,18 +167,19 @@ public void testConfigureSerdesWithRepartitionJoinApplication() { public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { // set the application to RepartitionOnlyStreamApplication mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionOnlyStreamApplication(), mockConfig); + configureJobNode(mockStreamAppDesc); // add the stream edges to the node - StreamEdge reparStreamEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); - List inputEdges = new ArrayList<>(); - inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); - inputEdges.add(reparStreamEdge); - List outputEdges = new ArrayList<>(); - outputEdges.add(reparStreamEdge); - when(mockJobNode.getInEdges()).thenReturn(inputEdges); - when(mockJobNode.getOutEdges()).thenReturn(outputEdges); +// StreamEdge reparStreamEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); +// Map inputEdges = new HashMap<>(); +// inputEdges.put(input1Spec.getId(), new StreamEdge(input1Spec, false, false, mockConfig)); +// inputEdges.put(repartitionSpec.getId(), reparStreamEdge); +// Map outputEdges = new HashMap<>(); +// outputEdges.put(repartitionSpec.getId(), reparStreamEdge); +// when(mockJobNode.getInEdges()).thenReturn(inputEdges); +// when(mockJobNode.getOutEdges()).thenReturn(outputEdges); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -200,8 +201,9 @@ public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { public void testGenerateJobConfigWithTaskApplication() { // set the application to TaskApplication, which still wire up all input/output/intermediate streams TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); + configureJobNode(taskAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -214,19 +216,22 @@ public void testGenerateJobConfigWithTaskApplication() { @Test public void testGenerateJobConfigWithLegacyTaskApplication() { // set the application to LegacyTaskApplication, which only has configuration and no descriptors - Map originConfig = new HashMap<>(); - originConfig.put(JobConfig.JOB_NAME(), "jobName1"); - originConfig.put(JobConfig.JOB_ID(), "jobId1"); +// Map originConfig = new HashMap<>(); +// originConfig.put(JobConfig.JOB_NAME(), "jobName1"); +// originConfig.put(JobConfig.JOB_ID(), "jobId1"); TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); + configureJobNode(taskAppDesc); + Map originConfig = new HashMap<>(mockConfig); + //clear the JobNode's stream edges since legacy task application does not wire up any input/output streams - when(mockJobNode.getInEdges()).thenReturn(new ArrayList<>()); - when(mockJobNode.getOutEdges()).thenReturn(new ArrayList<>()); - when(mockJobNode.getJobName()).thenReturn("jobName1"); - when(mockJobNode.getJobId()).thenReturn("jobId1"); - when(mockJobNode.getConfig()).thenReturn(new MapConfig(originConfig)); +// when(mockJobNode.getInEdges()).thenReturn(Collections.EMPTY_MAP); +// when(mockJobNode.getOutEdges()).thenReturn(Collections.EMPTY_MAP); +// when(mockJobNode.getJobName()).thenReturn("jobName1"); +// when(mockJobNode.getJobId()).thenReturn("jobId1"); +// when(mockJobNode.getConfig()).thenReturn(new MapConfig(originConfig)); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); // jobConfig should be exactly the same as original config Map generatedConfig = new HashMap<>(jobConfig); @@ -237,49 +242,49 @@ public void testGenerateJobConfigWithLegacyTaskApplication() { public void testBroadcastStreamApplication() { // set the application to BroadcastStreamApplication mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(defaultSerde), mockConfig); + configureJobNode(mockStreamAppDesc); // add the stream edges to the node - StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); - List inputEdges = new ArrayList<>(); - inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); - inputEdges.add(broadcastStreamEdge); - List outputEdges = new ArrayList<>(); - outputEdges.add(broadcastStreamEdge); - when(mockJobNode.getInEdges()).thenReturn(inputEdges); - when(mockJobNode.getOutEdges()).thenReturn(outputEdges); +// StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); +// Map inputEdges = new HashMap<>(); +// inputEdges.put(input1Spec.getId(), new StreamEdge(input1Spec, false, false, mockConfig)); +// inputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); +// Map outputEdges = new HashMap<>(); +// outputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); +// when(mockJobNode.getInEdges()).thenReturn(inputEdges); +// when(mockJobNode.getOutEdges()).thenReturn(outputEdges); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); - Config expectedJobConfig = getExpectedJobConfig(mockConfig, inputEdges); + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); validateStreamSerdeConfigure(broadcastInputDesriptor.getStreamId(), jobConfig, deserializedSerdes); - validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), - broadcastStreamEdge.getStreamSpec().getPhysicalName(), jobConfig); + validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastSpec.getPhysicalName(), jobConfig); } @Test public void testBroadcastStreamApplicationWithoutSerde() { // set the application to BroadcastStreamApplication withoutSerde mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(null), mockConfig); + configureJobNode(mockStreamAppDesc); // add the stream edges to the node - StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); - List inputEdges = new ArrayList<>(); - inputEdges.add(new StreamEdge(input1Spec, false, false, mockConfig)); - inputEdges.add(broadcastStreamEdge); - List outputEdges = new ArrayList<>(); - outputEdges.add(broadcastStreamEdge); - when(mockJobNode.getInEdges()).thenReturn(inputEdges); - when(mockJobNode.getOutEdges()).thenReturn(outputEdges); +// StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); +// Map inputEdges = new HashMap<>(); +// inputEdges.put(input1Spec.getId(), new StreamEdge(input1Spec, false, false, mockConfig)); +// inputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); +// Map outputEdges = new HashMap<>(); +// outputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); +// when(mockJobNode.getInEdges()).thenReturn(inputEdges); +// when(mockJobNode.getOutEdges()).thenReturn(outputEdges); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); - Config expectedJobConfig = getExpectedJobConfig(mockConfig, inputEdges); + Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); - validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), - broadcastStreamEdge.getStreamSpec().getPhysicalName(), jobConfig); + validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastSpec.getPhysicalName(), jobConfig); String keySerde = jobConfig.get(String.format("streams.%s.samza.key.serde", broadcastInputDesriptor.getStreamId())); String msgSerde = jobConfig.get(String.format("streams.%s.samza.msg.serde", broadcastInputDesriptor.getStreamId())); @@ -305,13 +310,8 @@ public void testStreamApplicationWithTableAndSideInput() { when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); // add side input and terminate at table in the appplication mockStreamAppDesc.getInputStream(sideInput1).sendTo(mockStreamAppDesc.getTable(mockTableDescriptor)); - // add table and input edge to the node - List tables = new ArrayList<>(); - tables.add(mockTableSpec); - List inEdges = new ArrayList<>(mockJobNode.getInEdges()); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", inputSystemDescriptor.getSystemName()), false, false, mockConfig); - inEdges.add(sideInputEdge); // need to put the sideInput related stream configuration to the original config // TODO: this is confusing since part of the system and stream related configuration is generated outside the JobGraphConfigureGenerator // It would be nice if all system and stream related configuration is generated in one place and only intermediate stream @@ -319,12 +319,17 @@ public void testStreamApplicationWithTableAndSideInput() { Map configs = new HashMap<>(mockConfig); configs.putAll(sideInputEdge.generateConfig()); mockConfig = spy(new MapConfig(configs)); - when(mockJobNode.getConfig()).thenReturn(mockConfig); - when(mockJobNode.getInEdges()).thenReturn(inEdges); - when(mockJobNode.getTables()).thenReturn(tables); + configureJobNode(mockStreamAppDesc); + // add table and input edge to the node +// Map tables = new HashMap<>(); +// tables.put(mockTableDescriptor.getTableId(), mockTableSpec); +// Map inEdges = new HashMap<>(mockJobNode.getInEdges()); +// when(mockJobNode.getConfig()).thenReturn(mockConfig); +// when(mockJobNode.getInEdges()).thenReturn(inEdges); +// when(mockJobNode.getTables()).thenReturn(tables); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); @@ -347,12 +352,12 @@ public void testTaskApplicationWithTableAndSideInput() { when(mockTableDescriptor.getTableId()).thenReturn("testTable"); when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); // add table and input edge to the node - List tables = new ArrayList<>(); - tables.add(mockTableSpec); - List inEdges = new ArrayList<>(mockJobNode.getInEdges()); +// Map tables = new HashMap<>(); +// tables.put(mockTableDescriptor.getTableId(), mockTableSpec); +// Map inEdges = new HashMap<>(mockJobNode.getInEdges()); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", inputSystemDescriptor.getSystemName()), false, false, mockConfig); - inEdges.add(sideInputEdge); +// inEdges.put(sideInput1.getStreamId(), sideInputEdge); // need to put the sideInput related stream configuration to the original config // TODO: this is confusing since part of the system and stream related configuration is generated outside the JobGraphConfigureGenerator // It would be nice if all system and stream related configuration is generated in one place and only intermediate stream @@ -360,16 +365,19 @@ public void testTaskApplicationWithTableAndSideInput() { Map configs = new HashMap<>(mockConfig); configs.putAll(sideInputEdge.generateConfig()); mockConfig = spy(new MapConfig(configs)); - when(mockJobNode.getConfig()).thenReturn(mockConfig); - when(mockJobNode.getInEdges()).thenReturn(inEdges); - when(mockJobNode.getTables()).thenReturn(tables); +// when(mockJobNode.getConfig()).thenReturn(mockConfig); +// when(mockJobNode.getInEdges()).thenReturn(inEdges); +// when(mockJobNode.getTables()).thenReturn(tables); // set the application to TaskApplication, which still wire up all input/output/intermediate streams TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); // add table to the task application taskAppDesc.addTable(mockTableDescriptor); + taskAppDesc.addInputStream(inputSystemDescriptor.getInputDescriptor("sideInput1", defaultSerde)); + configureJobNode(taskAppDesc); + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(taskAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -387,7 +395,7 @@ public void testTaskInputsRemovedFromOriginalConfig() { mockConfig = spy(new MapConfig(configs)); when(mockJobNode.getConfig()).thenReturn(mockConfig); - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); @@ -397,12 +405,12 @@ public void testTaskInputsRemovedFromOriginalConfig() { public void testOverrideConfigs() { Map configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); - String overrideCfgKey = String.format(JobGraphConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; + String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; configs.put(overrideCfgKey, "customized-system"); mockConfig = spy(new MapConfig(configs)); when(mockJobNode.getConfig()).thenReturn(mockConfig); - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); @@ -410,10 +418,10 @@ public void testOverrideConfigs() { } @Test - public void testConfigureRewriter(){ + public void testConfigureRewriter() { Map configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); - String overrideCfgKey = String.format(JobGraphConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; + String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; configs.put(overrideCfgKey, "customized-system"); configs.put(String.format(JobConfig.CONFIG_REWRITER_CLASS(), "mock"), MockConfigRewriter.class.getName()); configs.put(JobConfig.CONFIG_REWRITERS(), "mock"); @@ -421,7 +429,7 @@ public void testConfigureRewriter(){ mockConfig = spy(new MapConfig(configs)); when(mockJobNode.getConfig()).thenReturn(mockConfig); - JobGraphConfigureGenerator configureGenerator = new JobGraphConfigureGenerator(mockStreamAppDesc); + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); @@ -438,11 +446,11 @@ private void validateTableConfigure(JobConfig jobConfig, Map dese validateTableSerdeConfigure(tableDescriptor.getTableId(), jobConfig, deserializedSerdes); } - private Config getExpectedJobConfig(Config originConfig, List inputEdges) { + private Config getExpectedJobConfig(Config originConfig, Map inputEdges) { Map configMap = new HashMap<>(originConfig); Set inputs = new HashSet<>(); Set broadcasts = new HashSet<>(); - for (StreamEdge inputEdge : inputEdges) { + for (StreamEdge inputEdge : inputEdges.values()) { if (inputEdge.isBroadcast()) { broadcasts.add(inputEdge.getName() + "#0"); } else { @@ -458,7 +466,7 @@ private Config getExpectedJobConfig(Config originConfig, List inputE return new MapConfig(configMap); } - private Map validateAndGetDeserializedSerdes(Config jobConfig, int numSerdes) { + private Map validateAndGetDeserializedSerdes(Config jobConfig, int numSerdes) { Config serializers = jobConfig.subset("serializers.registry.", true); // make sure that the serializers deserialize correctly SerializableSerde serializableSerde = new SerializableSerde<>(); @@ -472,7 +480,7 @@ private Map validateAndGetDeserializedSerdes(Config jobConfig, int private void validateJobConfig(Config expectedConfig, JobConfig jobConfig) { assertEquals(expectedConfig.get(JobConfig.JOB_NAME()), jobConfig.getName().get()); assertEquals(expectedConfig.get(JobConfig.JOB_ID()), jobConfig.getJobId().get()); - assertEquals("testJobGraphJson", jobConfig.get(JobGraphConfigureGenerator.CONFIG_INTERNAL_EXECUTION_PLAN)); + assertEquals("testJobGraphJson", jobConfig.get(JobNodeConfigureGenerator.CONFIG_INTERNAL_EXECUTION_PLAN)); assertEquals(expectedConfig.get(TaskConfig.INPUT_STREAMS()), jobConfig.get(TaskConfig.INPUT_STREAMS())); assertEquals(expectedConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS), jobConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS)); } @@ -575,18 +583,18 @@ private TaskApplication getLegacyTaskApplication() { private StreamApplication getRepartitionJoinStreamApplication() { return appDesc -> { - MessageStream> input1 = appDesc.getInputStream(input1Descriptor); - MessageStream> input2 = appDesc.getInputStream(input2Descriptor); - OutputStream> output = appDesc.getOutputStream(outputDescriptor); - JoinFunction> mockJoinFn = mock(JoinFunction.class); - input1 - .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") - .map(kv -> kv.value) - .join(input2.map(kv -> kv.value), mockJoinFn, - new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), - Duration.ofHours(1), "j1") - .sendTo(output); - }; + MessageStream> input1 = appDesc.getInputStream(input1Descriptor); + MessageStream> input2 = appDesc.getInputStream(input2Descriptor); + OutputStream> output = appDesc.getOutputStream(outputDescriptor); + JoinFunction> mockJoinFn = mock(JoinFunction.class); + input1 + .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") + .map(kv -> kv.value) + .join(input2.map(kv -> kv.value), mockJoinFn, + new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), + Duration.ofHours(1), "j1") + .sendTo(output); + }; } private StreamApplication getRepartitionOnlyStreamApplication() { diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java index 19ee74f9a7..fd0ddf859d 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java @@ -25,10 +25,10 @@ import java.util.Map; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.ApplicationDescriptorUtil; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.TaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -37,7 +37,6 @@ import org.apache.samza.processor.StreamProcessor; import org.apache.samza.execution.LocalJobPlanner; import org.apache.samza.task.IdentityStreamTask; -import org.apache.samza.task.StreamTaskFactory; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -73,8 +72,9 @@ public void testRunStreamTask() final Map cfgs = new HashMap<>(); cfgs.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName()); cfgs.put(JobConfig.JOB_NAME(), "test-task-job"); + cfgs.put(JobConfig.JOB_ID(), "jobId"); config = new MapConfig(cfgs); - mockApp = (TaskApplication) appDesc -> appDesc.setTaskFactory((StreamTaskFactory) () -> new IdentityStreamTask()); + mockApp = new LegacyTaskApplication(IdentityStreamTask.class.getName()); prepareTest(); StreamProcessor sp = mock(StreamProcessor.class); @@ -186,7 +186,8 @@ public void testWaitForFinishTimesout() { } private void prepareTest() { - ApplicationDescriptorImpl appDesc = ApplicationDescriptorUtil.getAppDescriptor(mockApp, config); + ApplicationDescriptorImpl appDesc = + ApplicationDescriptorUtil.getAppDescriptor(mockApp, config); localPlanner = spy(new LocalJobPlanner(appDesc)); runner = spy(new LocalApplicationRunner(appDesc, localPlanner)); } From dae98cebe1f30f423afaeb042862c9a9efb83a87 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Mon, 17 Sep 2018 01:09:29 -0700 Subject: [PATCH 31/38] SAMZA-1814: consolidate the configure generation between high and low-level API applications --- .../samza/application/ApplicationUtil.java | 4 + .../samza/execution/ExecutionPlanner.java | 3 +- .../execution/JobGraphJsonGenerator.java | 4 + .../org/apache/samza/execution/JobNode.java | 6 ++ .../execution/JobNodeConfigureGenerator.java | 13 +-- .../samza/execution/LocalJobPlanner.java | 7 +- .../samza/execution/RemoteJobPlanner.java | 7 +- .../samza/runtime/LocalContainerRunner.java | 2 +- .../samza/zk/ZkJobCoordinatorFactory.java | 5 +- .../org/apache/samza/config/JobConfig.scala | 2 +- .../samza/container/SamzaContainer.scala | 2 +- .../MetricsSnapshotReporterFactory.scala | 1 - .../samza/util/CoordinatorStreamUtil.scala | 2 +- .../samza/execution/TestExecutionPlanner.java | 25 +++++- ...estIntermediateStreamPartitionPlanner.java | 83 +++++++++++-------- .../TestJobNodeConfigureGenerator.java | 75 +++-------------- .../runtime/TestRemoteApplicationRunner.java | 2 +- .../samza/system/hdfs/HdfsSystemFactory.scala | 2 +- .../kafka/KafkaCheckpointManagerFactory.scala | 2 +- .../org/apache/samza/util/KafkaUtil.scala | 2 +- .../samza/test/framework/TestRunner.java | 62 +++++++------- .../system/CollectionStreamSystemSpec.java | 14 ++-- .../validation/YarnJobValidationTool.java | 2 +- .../org/apache/samza/job/yarn/YarnJob.scala | 4 +- 24 files changed, 159 insertions(+), 172 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java index b39ad3c377..eae6be36f6 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java @@ -60,4 +60,8 @@ public static SamzaApplication fromConfig(Config config) { return new LegacyTaskApplication(taskClassOption.get()); } + public static boolean isLegacyTaskApplication(ApplicationDescriptorImpl appDesc) { + return LegacyTaskApplication.class.isAssignableFrom(appDesc.getAppClass()); + } + } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 8205abe35c..e9ed10ea39 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -30,6 +30,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.ApplicationUtil; import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ClusterManagerConfig; @@ -123,7 +124,7 @@ JobGraph createJobGraph(Config config, ApplicationDescriptorImpl jobGraph.addTable(spec, node)); - if (!LegacyTaskApplication.class.isAssignableFrom(appDesc.getAppClass())) { + if (!ApplicationUtil.isLegacyTaskApplication(appDesc)) { // skip the validation when input streamIds are empty. This is only possible for LegacyApplication jobGraph.validate(); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index d746d0ae6b..27bb87bb4f 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -152,6 +152,10 @@ static final class JobGraphJson { } private void updateOperatorGraphJson(OperatorSpec operatorSpec, OperatorGraphJson opGraph) { + if (operatorSpec == null) { + // task application may not have any defined OperatorSpec + return; + } if (operatorSpec.getOpCode() != OperatorSpec.OpCode.INPUT) { opGraph.operators.put(operatorSpec.getOpId(), operatorToMap(operatorSpec)); } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index efbed61dab..a81088bfb6 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -28,6 +28,8 @@ import java.util.stream.Collectors; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.ApplicationUtil; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.spec.InputOperatorSpec; @@ -160,4 +162,8 @@ InputOperatorSpec getInputOperator(String inputStreamId) { } return appDesc.getInputOperators().get(inputStreamId); } + + boolean isLegacyTaskApplication() { + return ApplicationUtil.isLegacyTaskApplication(appDesc); + } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java index a8200bc8bc..11cb8354b3 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java @@ -70,7 +70,7 @@ static Config mergeJobConfig(Config originalConfig, Config generatedConfig) { JobConfig jobConfig = new JobConfig(originalConfig); - String jobId = JobNode.createId(jobConfig.getName().get(), jobConfig.getJobId().get()); + String jobId = JobNode.createId(jobConfig.getName().get(), jobConfig.getJobId()); return new JobConfig(Util.rewriteConfig(extractScopedConfig(originalConfig, generatedConfig, String.format(CONFIG_JOB_PREFIX, jobId)))); } @@ -160,10 +160,11 @@ private JobConfig applyConfigureRewritersAndOverrides(Map config // Disallow user specified job inputs/outputs. This info comes strictly from the user application. Map allowedConfigs = new HashMap<>(config); - if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { - LOG.warn("Specifying task inputs in configuration is not allowed with Fluent API. " - + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); - allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); + if (!jobNode.isLegacyTaskApplication()) { + if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { + LOG.warn("Specifying task inputs in configuration is not allowed with Fluent API. " + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); + allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); + } } LOG.debug("Job {} has allowed configs {}", jobNode.getId(), allowedConfigs); @@ -305,7 +306,7 @@ private long computeTriggerInterval(Collection reachableOperators) // Filter out the join operators, and obtain a list of their ttl values List joinTtlIntervals = reachableOperators.stream() - .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.JOIN) + .filter(spec -> spec instanceof JoinOperatorSpec) .map(spec -> ((JoinOperatorSpec) spec).getTtlMs()) .collect(Collectors.toList()); diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java index 55583b86fc..e8700a1aab 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -62,7 +62,8 @@ public List prepareJobs() throws Exception { LOG.info("Execution Plan: \n" + executionPlanJson); String planId = String.valueOf(executionPlanJson.hashCode()); - if (plan.getJobConfigs().isEmpty()) { + List jobConfigs = plan.getJobConfigs(); + if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs in the plan."); } @@ -70,7 +71,7 @@ public List prepareJobs() throws Exception { // TODO: System generated intermediate streams should have robust naming scheme. See SAMZA-1391 // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig // to be used for the whole application - JobConfig jobConfig = plan.getJobConfigs().get(0); + JobConfig jobConfig = jobConfigs.get(0); StreamManager streamManager = null; try { // create the StreamManager to create intermediate streams in the plan @@ -81,7 +82,7 @@ public List prepareJobs() throws Exception { streamManager.stop(); } } - return plan.getJobConfigs(); + return jobConfigs; } /** diff --git a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java index 0a6ce12c29..599aaeed35 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java @@ -57,14 +57,15 @@ public List prepareJobs() throws Exception { ExecutionPlan plan = getExecutionPlan(runId); writePlanJsonFile(plan.getPlanAsJson()); - if (plan.getJobConfigs().isEmpty()) { + List jobConfigs = plan.getJobConfigs(); + if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs in the plan."); } // 2. create the necessary streams // TODO: this works for single-job applications. For multi-job applications, ExecutionPlan should return an AppConfig // to be used for the whole application - JobConfig jobConfig = plan.getJobConfigs().get(0); + JobConfig jobConfig = jobConfigs.get(0); StreamManager streamManager = null; try { // create the StreamManager to create intermediate streams in the plan @@ -78,7 +79,7 @@ public List prepareJobs() throws Exception { streamManager.stop(); } } - return plan.getJobConfigs(); + return jobConfigs; } private Config getConfigFromPrevRun() { diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java index 98864d2eeb..b9bb1f6c0a 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java @@ -75,7 +75,7 @@ public static void main(String[] args) throws Exception { throw new SamzaException("can not find the job name"); } String jobName = jobConfig.getName().get(); - String jobId = jobConfig.getJobId().getOrElse(ScalaJavaUtil.defaultValue("1")); + String jobId = jobConfig.getJobId(); MDC.put("containerName", "samza-container-" + containerId); MDC.put("jobName", jobName); MDC.put("jobId", jobId); diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java index 6888df0fd6..978cedbbb6 100644 --- a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java +++ b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java @@ -36,7 +36,6 @@ public class ZkJobCoordinatorFactory implements JobCoordinatorFactory { private static final Logger LOG = LoggerFactory.getLogger(ZkJobCoordinatorFactory.class); private static final String JOB_COORDINATOR_ZK_PATH_FORMAT = "%s/%s-%s-coordinationData"; - private static final String DEFAULT_JOB_ID = "1"; private static final String DEFAULT_JOB_NAME = "defaultJob"; /** @@ -67,9 +66,7 @@ public static String getJobCoordinationZkPath(Config config) { String jobName = jobConfig.getName().isDefined() ? jobConfig.getName().get() : DEFAULT_JOB_NAME; - String jobId = jobConfig.getJobId().isDefined() - ? jobConfig.getJobId().get() - : DEFAULT_JOB_ID; + String jobId = jobConfig.getJobId(); return String.format(JOB_COORDINATOR_ZK_PATH_FORMAT, appId, jobName, jobId); } diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index ddcaa5ea31..4f19adec24 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -163,7 +163,7 @@ class JobConfig(config: Config) extends ScalaMapConfig(config) with Logging { def getStreamJobFactoryClass = getOption(JobConfig.STREAM_JOB_FACTORY_CLASS) - def getJobId = getOption(JobConfig.JOB_ID) + def getJobId = getOption(JobConfig.JOB_ID).getOrElse("1") def failOnCheckpointValidation = { getBoolean(JobConfig.JOB_FAIL_CHECKPOINT_VALIDATION, true) } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 68de4a630c..71587a0584 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -101,7 +101,7 @@ object SamzaContainer extends Logging { if(System.getenv(ShellCommandConfig.ENV_LOGGED_STORE_BASE_DIR) != null) { val jobNameAndId = ( config.getName.getOrElse(throw new ConfigException("Missing required config: job.name")), - config.getJobId.getOrElse("1") + config.getJobId ) loggedStorageBaseDir = new File(System.getenv(ShellCommandConfig.ENV_LOGGED_STORE_BASE_DIR) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala index d1e655412d..8a9c021cfa 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala @@ -44,7 +44,6 @@ class MetricsSnapshotReporterFactory extends MetricsReporterFactory with Logging val jobId = config .getJobId - .getOrElse(1.toString) val taskClass = config .getTaskClass diff --git a/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala b/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala index cd74716d18..bfb2271fc6 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala @@ -89,6 +89,6 @@ object CoordinatorStreamUtil { */ private def getJobNameAndId(config: Config) = { (config.getName.getOrElse(throw new ConfigException("Missing required config: job.name")), - config.getJobId.getOrElse("1")) + config.getJobId) } } 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 01c56b4f76..85c8c2b9d4 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 @@ -30,6 +30,9 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.samza.Partition; +import org.apache.samza.application.ApplicationDescriptor; +import org.apache.samza.application.LegacyTaskApplication; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; @@ -60,8 +63,12 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestExecutionPlanner { @@ -503,10 +510,13 @@ public void testCreateJobGraphForTaskApplication() { broadcastStreams.add(intermediateBroadcast); when(taskAppDesc.getInputDescriptors()).thenReturn(inputDescriptors); + when(taskAppDesc.getInputStreamIds()).thenReturn(inputDescriptors.keySet()); when(taskAppDesc.getOutputDescriptors()).thenReturn(outputDescriptors); - when(taskAppDesc.getTableDescriptors()).thenReturn(tableDescriptors); + when(taskAppDesc.getOutputStreamIds()).thenReturn(outputDescriptors.keySet()); + when(taskAppDesc.getTableDescriptors()).thenReturn(Collections.emptySet()); when(taskAppDesc.getSystemDescriptors()).thenReturn(systemDescriptors); when(taskAppDesc.getBroadcastStreams()).thenReturn(broadcastStreams); + doReturn(MockTaskApplication.class).when(taskAppDesc).getAppClass(); Map systemStreamConfigs = new HashMap<>(); inputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); @@ -538,6 +548,7 @@ public void testCreateJobGraphForLegacyTaskApplication() { when(taskAppDesc.getTableDescriptors()).thenReturn(new HashSet<>()); when(taskAppDesc.getSystemDescriptors()).thenReturn(new HashSet<>()); when(taskAppDesc.getBroadcastStreams()).thenReturn(new HashSet<>()); + doReturn(LegacyTaskApplication.class).when(taskAppDesc).getAppClass(); Map systemStreamConfigs = new HashMap<>(); inputDescriptors.forEach((key, value) -> systemStreamConfigs.putAll(value.toConfig())); @@ -556,4 +567,12 @@ public void testCreateJobGraphForLegacyTaskApplication() { assertEquals(0, jobNode.getTables().size()); assertEquals(config, jobNode.getConfig()); } + + public static class MockTaskApplication implements SamzaApplication { + + @Override + public void describe(ApplicationDescriptor appDesc) { + + } + } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java index b23bc3e98f..80da42a4b5 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java @@ -33,7 +33,6 @@ import org.apache.samza.operators.descriptors.GenericOutputDescriptor; import org.apache.samza.operators.descriptors.GenericSystemDescriptor; import org.apache.samza.operators.functions.JoinFunction; -import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; @@ -41,10 +40,9 @@ import org.junit.Before; import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; - /** * Unit tests for {@link IntermediateStreamPartitionPlanner} @@ -80,7 +78,6 @@ public void setUp() { new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "broadcast-b1", "intermediate-system"); - defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); @@ -97,46 +94,60 @@ public void setUp() { Map configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), "jobName"); configs.put(JobConfig.JOB_ID(), "jobId"); + configs.putAll(input1Descriptor.toConfig()); + configs.putAll(input2Descriptor.toConfig()); + configs.putAll(outputDescriptor.toConfig()); + configs.putAll(inputSystemDescriptor.toConfig()); + configs.putAll(outputSystemDescriptor.toConfig()); + configs.putAll(intermediateSystemDescriptor.toConfig()); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystemDescriptor.getSystemName()); mockConfig = spy(new MapConfig(configs)); mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + } - mockJobNode = mock(JobNode.class); - StreamEdge input1Edge = new StreamEdge(input1Spec, false, false, mockConfig); - StreamEdge input2Edge = new StreamEdge(input2Spec, false, false, mockConfig); - StreamEdge outputEdge = new StreamEdge(outputSpec, false, false, mockConfig); - StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); - Map inputEdges = new HashMap<>(); - inputEdges.put(input1Descriptor.getStreamId(), input1Edge); - inputEdges.put(input2Descriptor.getStreamId(), input2Edge); - inputEdges.put(repartitionSpec.getId(), repartitionEdge); - Map outputEdges = new HashMap<>(); - outputEdges.put(outputDescriptor.getStreamId(), outputEdge); - outputEdges.put(repartitionSpec.getId(), repartitionEdge); - when(mockJobNode.getInEdges()).thenReturn(inputEdges); - when(mockJobNode.getOutEdges()).thenReturn(outputEdges); - when(mockJobNode.getConfig()).thenReturn(mockConfig); - when(mockJobNode.getJobName()).thenReturn("jobName"); - when(mockJobNode.getJobId()).thenReturn("jobId"); - when(mockJobNode.getId()).thenReturn(JobNode.createId("jobName", "jobId")); - - mockGraph = mock(JobGraph.class); + @Test + public void testCalculateRepartitionJoinTopicPartitions() { + IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mockConfig, mockStreamAppDesc); + JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, + mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + // set the input stream partitions + mockGraph.getSources().forEach(inEdge -> { + if (inEdge.getStreamSpec().getId().equals(input1Descriptor.getStreamId())) { + inEdge.setPartitionCount(6); + } else if (inEdge.getStreamSpec().getId().equals(input2Descriptor.getStreamId())) { + inEdge.setPartitionCount(5); + } + }); + partitionPlanner.calculatePartitions(mockGraph); + assertEquals(1, mockGraph.getIntermediateStreamEdges().size()); + assertEquals(5, mockGraph.getIntermediateStreamEdges().stream() + .filter(inEdge -> inEdge.getStreamSpec().getId().equals(intermediateInputDescriptor.getStreamId())) + .findFirst().get().getPartitionCount()); } @Test - public void testConstructor() { - StreamApplicationDescriptorImpl mockAppDesc = spy(new StreamApplicationDescriptorImpl(appDesc -> { }, - mock(Config.class))); - InputOperatorSpec inputOp1 = mock(InputOperatorSpec.class); - InputOperatorSpec inputOp2 = mock(InputOperatorSpec.class); - Map inputOpMaps = new HashMap<>(); - inputOpMaps.put("input-op1", inputOp1); - inputOpMaps.put("input-op2", inputOp2); - when(mockAppDesc.getInputOperators()).thenReturn(inputOpMaps); - IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mock(Config.class), - mockAppDesc); - JobGraph mockGraph = mock(JobGraph.class); + public void testCalculateRepartitionIntermediateTopicPartitions() { + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionOnlyStreamApplication(), mockConfig); + IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mockConfig, mockStreamAppDesc); + JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, + mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + // set the input stream partitions + mockGraph.getSources().forEach(inEdge -> inEdge.setPartitionCount(7)); partitionPlanner.calculatePartitions(mockGraph); + assertEquals(1, mockGraph.getIntermediateStreamEdges().size()); + assertEquals(7, mockGraph.getIntermediateStreamEdges().stream() + .filter(inEdge -> inEdge.getStreamSpec().getId().equals(intermediateInputDescriptor.getStreamId())) + .findFirst().get().getPartitionCount()); + } + + private StreamApplication getRepartitionOnlyStreamApplication() { + return appDesc -> { + MessageStream> input1 = appDesc.getInputStream(input1Descriptor); + OutputStream> output = appDesc.getOutputStream(outputDescriptor); + JoinFunction> mockJoinFn = mock(JoinFunction.class); + input1.partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1").sendTo(output); + }; } private StreamApplication getRepartitionJoinStreamApplication() { diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java index 620153bc49..5e1a62ec36 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java @@ -22,7 +22,6 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Base64; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -70,7 +69,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -106,7 +104,7 @@ public void setUp() { input2Spec = new StreamSpec("input2", "input2", "input-system"); outputSpec = new StreamSpec("output", "output", "output-system"); repartitionSpec = - new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); + new StreamSpec("jobName-jobId-partition_by-p1", "jobName-jobId-partition_by-p1", "intermediate-system"); broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "jobName-jobId-broadcast-b1", "intermediate-system"); @@ -118,23 +116,22 @@ public void setUp() { input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("partition_by-p1"); + .withPhysicalName("jobName-jobId-partition_by-p1"); intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("partition_by-p1"); - broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde); + .withPhysicalName("jobName-jobId-partition_by-p1"); + broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde) + .withPhysicalName("jobName-jobId-broadcast-b1"); Map configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), "jobName"); configs.put(JobConfig.JOB_ID(), "jobId"); configs.putAll(input1Descriptor.toConfig()); configs.putAll(input2Descriptor.toConfig()); - configs.putAll(intermediateInputDescriptor.toConfig()); - configs.putAll(broadcastInputDesriptor.toConfig()); configs.putAll(outputDescriptor.toConfig()); - configs.putAll(intermediateOutputDescriptor.toConfig()); configs.putAll(inputSystemDescriptor.toConfig()); configs.putAll(outputSystemDescriptor.toConfig()); configs.putAll(intermediateSystemDescriptor.toConfig()); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystemDescriptor.getSystemName()); mockConfig = spy(new MapConfig(configs)); mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); @@ -168,15 +165,6 @@ public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { // set the application to RepartitionOnlyStreamApplication mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionOnlyStreamApplication(), mockConfig); configureJobNode(mockStreamAppDesc); - // add the stream edges to the node -// StreamEdge reparStreamEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); -// Map inputEdges = new HashMap<>(); -// inputEdges.put(input1Spec.getId(), new StreamEdge(input1Spec, false, false, mockConfig)); -// inputEdges.put(repartitionSpec.getId(), reparStreamEdge); -// Map outputEdges = new HashMap<>(); -// outputEdges.put(repartitionSpec.getId(), reparStreamEdge); -// when(mockJobNode.getInEdges()).thenReturn(inputEdges); -// when(mockJobNode.getOutEdges()).thenReturn(outputEdges); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); @@ -215,21 +203,10 @@ public void testGenerateJobConfigWithTaskApplication() { @Test public void testGenerateJobConfigWithLegacyTaskApplication() { - // set the application to LegacyTaskApplication, which only has configuration and no descriptors -// Map originConfig = new HashMap<>(); -// originConfig.put(JobConfig.JOB_NAME(), "jobName1"); -// originConfig.put(JobConfig.JOB_ID(), "jobId1"); TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); configureJobNode(taskAppDesc); Map originConfig = new HashMap<>(mockConfig); - //clear the JobNode's stream edges since legacy task application does not wire up any input/output streams -// when(mockJobNode.getInEdges()).thenReturn(Collections.EMPTY_MAP); -// when(mockJobNode.getOutEdges()).thenReturn(Collections.EMPTY_MAP); -// when(mockJobNode.getJobName()).thenReturn("jobName1"); -// when(mockJobNode.getJobId()).thenReturn("jobId1"); -// when(mockJobNode.getConfig()).thenReturn(new MapConfig(originConfig)); - // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); @@ -243,15 +220,6 @@ public void testBroadcastStreamApplication() { // set the application to BroadcastStreamApplication mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(defaultSerde), mockConfig); configureJobNode(mockStreamAppDesc); - // add the stream edges to the node -// StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); -// Map inputEdges = new HashMap<>(); -// inputEdges.put(input1Spec.getId(), new StreamEdge(input1Spec, false, false, mockConfig)); -// inputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); -// Map outputEdges = new HashMap<>(); -// outputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); -// when(mockJobNode.getInEdges()).thenReturn(inputEdges); -// when(mockJobNode.getOutEdges()).thenReturn(outputEdges); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); @@ -268,15 +236,6 @@ public void testBroadcastStreamApplicationWithoutSerde() { // set the application to BroadcastStreamApplication withoutSerde mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(null), mockConfig); configureJobNode(mockStreamAppDesc); - // add the stream edges to the node -// StreamEdge broadcastStreamEdge = new StreamEdge(broadcastSpec, true, true, mockConfig); -// Map inputEdges = new HashMap<>(); -// inputEdges.put(input1Spec.getId(), new StreamEdge(input1Spec, false, false, mockConfig)); -// inputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); -// Map outputEdges = new HashMap<>(); -// outputEdges.put(broadcastSpec.getId(), broadcastStreamEdge); -// when(mockJobNode.getInEdges()).thenReturn(inputEdges); -// when(mockJobNode.getOutEdges()).thenReturn(outputEdges); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); @@ -320,13 +279,6 @@ public void testStreamApplicationWithTableAndSideInput() { configs.putAll(sideInputEdge.generateConfig()); mockConfig = spy(new MapConfig(configs)); configureJobNode(mockStreamAppDesc); - // add table and input edge to the node -// Map tables = new HashMap<>(); -// tables.put(mockTableDescriptor.getTableId(), mockTableSpec); -// Map inEdges = new HashMap<>(mockJobNode.getInEdges()); -// when(mockJobNode.getConfig()).thenReturn(mockConfig); -// when(mockJobNode.getInEdges()).thenReturn(inEdges); -// when(mockJobNode.getTables()).thenReturn(tables); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); @@ -351,13 +303,8 @@ public void testTaskApplicationWithTableAndSideInput() { when(mockTableSpec.getSideInputs()).thenReturn(sideInputs); when(mockTableDescriptor.getTableId()).thenReturn("testTable"); when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); - // add table and input edge to the node -// Map tables = new HashMap<>(); -// tables.put(mockTableDescriptor.getTableId(), mockTableSpec); -// Map inEdges = new HashMap<>(mockJobNode.getInEdges()); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", inputSystemDescriptor.getSystemName()), false, false, mockConfig); -// inEdges.put(sideInput1.getStreamId(), sideInputEdge); // need to put the sideInput related stream configuration to the original config // TODO: this is confusing since part of the system and stream related configuration is generated outside the JobGraphConfigureGenerator // It would be nice if all system and stream related configuration is generated in one place and only intermediate stream @@ -365,9 +312,6 @@ public void testTaskApplicationWithTableAndSideInput() { Map configs = new HashMap<>(mockConfig); configs.putAll(sideInputEdge.generateConfig()); mockConfig = spy(new MapConfig(configs)); -// when(mockJobNode.getConfig()).thenReturn(mockConfig); -// when(mockJobNode.getInEdges()).thenReturn(inEdges); -// when(mockJobNode.getTables()).thenReturn(tables); // set the application to TaskApplication, which still wire up all input/output/intermediate streams TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); @@ -479,7 +423,7 @@ private Map validateAndGetDeserializedSerdes(Config jobConfig, in private void validateJobConfig(Config expectedConfig, JobConfig jobConfig) { assertEquals(expectedConfig.get(JobConfig.JOB_NAME()), jobConfig.getName().get()); - assertEquals(expectedConfig.get(JobConfig.JOB_ID()), jobConfig.getJobId().get()); + assertEquals(expectedConfig.get(JobConfig.JOB_ID()), jobConfig.getJobId()); assertEquals("testJobGraphJson", jobConfig.get(JobNodeConfigureGenerator.CONFIG_INTERNAL_EXECUTION_PLAN)); assertEquals(expectedConfig.get(TaskConfig.INPUT_STREAMS()), jobConfig.get(TaskConfig.INPUT_STREAMS())); assertEquals(expectedConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS), jobConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS)); @@ -525,8 +469,9 @@ private void validateStreamConfigures(Config config, Map deserial } // generated stream config for intermediate stream - validateIntermediateStreamConfigure(intermediateInputDescriptor.getStreamId(), - intermediateInputDescriptor.getPhysicalName().get(), config); + String physicalName = intermediateInputDescriptor.getPhysicalName().isPresent() ? + intermediateInputDescriptor.getPhysicalName().get() : null; + validateIntermediateStreamConfigure(intermediateInputDescriptor.getStreamId(), physicalName, config); } private void validateJoinStoreConfigures(MapConfig mapConfig, Map deserializedSerdes) { diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java index ae525fb84f..702cbfbb1b 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java @@ -124,7 +124,7 @@ public ApplicationStatus waitForStatus(ApplicationStatus status, long timeoutMs) @Override public ApplicationStatus getStatus() { - String jobId = c.getJobId().get(); + String jobId = c.getJobId(); switch (jobId) { case "newJob": return ApplicationStatus.New; diff --git a/samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemFactory.scala b/samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemFactory.scala index 05d717a78d..3f5f11c647 100644 --- a/samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemFactory.scala +++ b/samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemFactory.scala @@ -35,7 +35,7 @@ class HdfsSystemFactory extends SystemFactory with Logging { def getProducer(systemName: String, config: Config, registry: MetricsRegistry) = { val jobConfig = new JobConfig(config) val jobName = jobConfig.getName.getOrElse(throw new ConfigException("Missing job name.")) - val jobId = jobConfig.getJobId.getOrElse("1") + val jobId = jobConfig.getJobId val clientId = getClientId("samza-producer", jobName, jobId) val metrics = new HdfsSystemProducerMetrics(systemName, registry) diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala index 8d4098f50f..2999800c4b 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala @@ -32,7 +32,7 @@ class KafkaCheckpointManagerFactory extends CheckpointManagerFactory with Loggin def getCheckpointManager(config: Config, registry: MetricsRegistry): CheckpointManager = { val jobName = config.getName.getOrElse(throw new SamzaException("Missing job name in configs")) - val jobId = config.getJobId.getOrElse("1") + val jobId = config.getJobId val kafkaConfig = new KafkaConfig(config) val checkpointSystemName = kafkaConfig.getCheckpointSystem.getOrElse( diff --git a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala index 601ffa25b9..2d09301926 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala @@ -40,7 +40,7 @@ object KafkaUtil extends Logging { def getClientId(id: String, config: Config): String = getClientId( id, config.getName.getOrElse(throw new ConfigException("Missing job name.")), - config.getJobId.getOrElse("1")) + config.getJobId) def getClientId(id: String, jobName: String, jobId: String): String = "%s-%s-%s" format diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index 477d5b890e..deefea6b3d 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -31,9 +31,9 @@ import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.samza.SamzaException; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.StreamApplication; -import org.apache.samza.application.TaskApplication; import org.apache.samza.config.Config; import org.apache.samza.config.InMemorySystemConfig; import org.apache.samza.config.JobConfig; @@ -58,10 +58,7 @@ import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.inmemory.InMemorySystemFactory; import org.apache.samza.task.AsyncStreamTask; -import org.apache.samza.task.AsyncStreamTaskFactory; import org.apache.samza.task.StreamTask; -import org.apache.samza.task.StreamTaskFactory; -import org.apache.samza.task.TaskFactory; import org.apache.samza.test.framework.stream.CollectionStream; import org.apache.samza.test.framework.system.CollectionStreamSystemSpec; import org.junit.Assert; @@ -92,8 +89,9 @@ public enum Mode { private Map configs; private Map systems; - private Class taskClass; - private StreamApplication app; +// private LegacyTaskApplication taskApp; +// private StreamApplication app; + private SamzaApplication app; private String testId; private SystemFactory factory; @@ -124,7 +122,7 @@ private TestRunner(Class taskClass) { this(); Preconditions.checkNotNull(taskClass); configs.put(TaskConfig.TASK_CLASS(), taskClass.getName()); - this.taskClass = taskClass; + this.app = new LegacyTaskApplication(taskClass.getName()); } /** @@ -143,7 +141,8 @@ private TestRunner(StreamApplication app) { */ private void registerSystem(String systemName) { if (!systems.containsKey(systemName)) { - systems.put(systemName, CollectionStreamSystemSpec.create(systemName, JOB_NAME)); + systems.put(systemName, CollectionStreamSystemSpec.create(systemName, + String.format("%s-%s", JOB_NAME, configs.getOrDefault(JobConfig.JOB_ID(), "1")))); configs.putAll(systems.get(systemName).getSystemConfigs()); } } @@ -287,11 +286,10 @@ public TestRunner addOutputStream(CollectionStream stream) { * @throws SamzaException if Samza job fails with exception and returns UnsuccessfulFinish as the statuscode */ public void run(Duration timeout) { - Preconditions.checkState((app == null && taskClass != null) || (app != null && taskClass == null), + Preconditions.checkState(app != null, "TestRunner should run for Low Level Task api or High Level Application Api"); Preconditions.checkState(!timeout.isZero() || !timeout.isNegative(), "Timeouts should be positive"); - SamzaApplication testApp = app == null ? (TaskApplication) appDesc -> appDesc.setTaskFactory(createTaskFactory()) : app; - final LocalApplicationRunner runner = new LocalApplicationRunner(testApp, new MapConfig(configs)); + final LocalApplicationRunner runner = new LocalApplicationRunner(app, new MapConfig(configs)); runner.run(); boolean timedOut = !runner.waitForFinish(timeout); Assert.assertFalse("Timed out waiting for application to finish", timedOut); @@ -366,25 +364,25 @@ public static Map> consumeStream(CollectionStream stream, D entry -> entry.getValue().stream().map(e -> (T) e.getMessage()).collect(Collectors.toList()))); } - private TaskFactory createTaskFactory() { - if (StreamTask.class.isAssignableFrom(taskClass)) { - return (StreamTaskFactory) () -> { - try { - return (StreamTask) taskClass.newInstance(); - } catch (InstantiationException | IllegalAccessException e) { - throw new SamzaException(String.format("Failed to instantiate StreamTask class %s", taskClass.getName()), e); - } - }; - } else if (AsyncStreamTask.class.isAssignableFrom(taskClass)) { - return (AsyncStreamTaskFactory) () -> { - try { - return (AsyncStreamTask) taskClass.newInstance(); - } catch (InstantiationException | IllegalAccessException e) { - throw new SamzaException(String.format("Failed to instantiate AsyncStreamTask class %s", taskClass.getName()), e); - } - }; - } - throw new SamzaException(String.format("Not supported task.class %s. task.class has to implement either StreamTask " - + "or AsyncStreamTask", taskClass.getName())); - } +// private TaskFactory createTaskFactory() { +// if (StreamTask.class.isAssignableFrom(taskClass)) { +// return (StreamTaskFactory) () -> { +// try { +// return (StreamTask) taskClass.newInstance(); +// } catch (InstantiationException | IllegalAccessException e) { +// throw new SamzaException(String.format("Failed to instantiate StreamTask class %s", taskClass.getName()), e); +// } +// }; +// } else if (AsyncStreamTask.class.isAssignableFrom(taskClass)) { +// return (AsyncStreamTaskFactory) () -> { +// try { +// return (AsyncStreamTask) taskClass.newInstance(); +// } catch (InstantiationException | IllegalAccessException e) { +// throw new SamzaException(String.format("Failed to instantiate AsyncStreamTask class %s", taskClass.getName()), e); +// } +// }; +// } +// throw new SamzaException(String.format("Not supported task.class %s. task.class has to implement either StreamTask " +// + "or AsyncStreamTask", taskClass.getName())); +// } } diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java index 5658f61f94..7fc5eb8683 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java @@ -58,13 +58,13 @@ public class CollectionStreamSystemSpec { *

    * @param systemName represents unique name of the system */ - private CollectionStreamSystemSpec(String systemName, String jobName) { + private CollectionStreamSystemSpec(String systemName, String jobNameAndId) { this.systemName = systemName; systemConfigs = new HashMap(); systemConfigs.put(String.format(SYSTEM_FACTORY, systemName), InMemorySystemFactory.class.getName()); - systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_FACTORY, jobName, systemName), InMemorySystemFactory.class.getName()); + systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_FACTORY, jobNameAndId, systemName), InMemorySystemFactory.class.getName()); systemConfigs.put(String.format(SYSTEM_OFFSET, systemName), "oldest"); - systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_OFFSET, jobName, systemName), "oldest"); + systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_OFFSET, jobNameAndId, systemName), "oldest"); } public String getSystemName() { @@ -78,13 +78,13 @@ public Map getSystemConfigs() { /** * Creates a {@link CollectionStreamSystemSpec} with name {@code systemName} * @param systemName represents name of the {@link CollectionStreamSystemSpec} - * @param jobName name of the job + * @param jobNameAndId name of the job * @return an instance of {@link CollectionStreamSystemSpec} */ - public static CollectionStreamSystemSpec create(String systemName, String jobName) { + public static CollectionStreamSystemSpec create(String systemName, String jobNameAndId) { Preconditions.checkState(StringUtils.isNotBlank(systemName)); - Preconditions.checkState(StringUtils.isNotBlank(jobName)); - return new CollectionStreamSystemSpec(systemName, jobName); + Preconditions.checkState(StringUtils.isNotBlank(jobNameAndId)); + return new CollectionStreamSystemSpec(systemName, jobNameAndId); } } diff --git a/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java b/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java index 0b405f0704..c5eb6bebbc 100644 --- a/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java +++ b/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java @@ -76,7 +76,7 @@ public YarnJobValidationTool(JobConfig config, YarnClient client, MetricsValidat this.config = config; this.client = client; String name = this.config.getName().get(); - String jobId = this.config.getJobId().nonEmpty()? this.config.getJobId().get() : "1"; + String jobId = this.config.getJobId(); this.jobName = name + "_" + jobId; this.validator = validator; } diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala index d3354489aa..1d72a88184 100644 --- a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala +++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala @@ -67,7 +67,7 @@ class YarnJob(config: Config, hadoopConfig: Configuration) extends StreamJob { } envMapWithJavaHome }), - Some("%s_%s" format(config.getName.get, config.getJobId.getOrElse(1))) + Some("%s_%s" format(config.getName.get, config.getJobId)) ) } catch { case e: Throwable => @@ -169,7 +169,7 @@ class YarnJob(config: Config, hadoopConfig: Configuration) extends StreamJob { // Get by name config.getName match { case Some(jobName) => - val applicationName = "%s_%s" format(jobName, config.getJobId.getOrElse(1)) + val applicationName = "%s_%s" format(jobName, config.getJobId) logger.info("Fetching status from YARN for application name %s" format applicationName) val applicationIds = client.getActiveApplicationIds(applicationName) From 8797cdd4fee94c4e56973dbb9c455484d2385437 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Mon, 17 Sep 2018 01:57:23 -0700 Subject: [PATCH 32/38] SAMZA-1814: merge with master --- .travis.yml | 4 +- .../samza/config/ClusterManagerConfig.java | 1 - .../samza/execution/ExecutionPlanner.java | 86 +++++++++------- .../IntermediateStreamPartitionPlanner.java | 4 +- .../org/apache/samza/execution/JobGraph.java | 82 ++++++++-------- .../execution/JobGraphJsonGenerator.java | 4 +- .../org/apache/samza/execution/JobNode.java | 1 - .../samza/processor/StreamProcessor.java | 2 +- .../org/apache/samza/table/TableManager.java | 34 ++++--- .../org/apache/samza/config/JobConfig.scala | 1 + .../samza/execution/TestExecutionPlanner.java | 98 +++++++++++-------- ...estIntermediateStreamPartitionPlanner.java | 24 +---- .../apache/samza/execution/TestJobGraph.java | 38 +++---- .../execution/TestJobGraphJsonGenerator.java | 4 +- .../apache/samza/execution/TestJobNode.java | 27 ----- .../apache/samza/table/TestTableManager.java | 11 ++- .../samza/system/hdfs/HdfsSystemAdmin.java | 12 +++ samza-shell/src/main/bash/run-class.sh | 12 ++- .../samza/test/framework/TestRunner.java | 3 +- .../TestZkLocalApplicationRunner.java | 6 +- .../table/TestLocalTableWithSideInputs.java | 6 +- .../job/yarn/YarnClusterResourceManager.java | 41 +++++--- .../yarn/TestYarnClusterResourceManager.java | 81 +++++++++++++++ 23 files changed, 339 insertions(+), 243 deletions(-) delete mode 100644 samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java create mode 100644 samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnClusterResourceManager.java diff --git a/.travis.yml b/.travis.yml index 2a3ae0cc92..601ceaca83 100644 --- a/.travis.yml +++ b/.travis.yml @@ -30,8 +30,8 @@ jdk: - oraclejdk8 script: - ## travis_wait increases build idle-wait time from 10 minutes to 20 minutes. - - travis_wait 20 ./gradlew clean build + ## travis_wait increases build idle-wait time from 10 minutes to 30 minutes. + - travis_wait 30 ./gradlew clean build - type sonar-scanner &>/dev/null; if [ $? -eq 0 ]; then sonar-scanner; else echo "Not running sonar"; fi before_cache: diff --git a/samza-core/src/main/java/org/apache/samza/config/ClusterManagerConfig.java b/samza-core/src/main/java/org/apache/samza/config/ClusterManagerConfig.java index c847088d66..cb5d5c07c7 100644 --- a/samza-core/src/main/java/org/apache/samza/config/ClusterManagerConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/ClusterManagerConfig.java @@ -56,7 +56,6 @@ public class ClusterManagerConfig extends MapConfig { */ public static final String HOST_AFFINITY_ENABLED = "yarn.samza.host-affinity.enabled"; public static final String CLUSTER_MANAGER_HOST_AFFINITY_ENABLED = "job.host-affinity.enabled"; - private static final boolean DEFAULT_HOST_AFFINITY_ENABLED = false; /** * Number of CPU cores to request from the cluster manager per container diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index e9ed10ea39..34d11c0eaf 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -21,6 +21,7 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; +import com.google.common.collect.Sets; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -31,7 +32,6 @@ import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.ApplicationUtil; -import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; @@ -39,7 +39,6 @@ import org.apache.samza.config.StreamConfig; import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.system.StreamSpec; -import org.apache.samza.system.SystemStream; import org.apache.samza.table.TableSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,11 +55,13 @@ public class ExecutionPlanner { private static final Logger log = LoggerFactory.getLogger(ExecutionPlanner.class); private final Config config; + private final StreamConfig streamConfig; private final StreamManager streamManager; public ExecutionPlanner(Config config, StreamManager streamManager) { this.config = config; this.streamManager = streamManager; + this.streamConfig = new StreamConfig(config); } public ExecutionPlan plan(ApplicationDescriptorImpl appDesc) { @@ -70,7 +71,7 @@ public ExecutionPlan plan(ApplicationDescriptorImpl sourceStreams = getStreamSpecs(appDesc.getInputStreamIds(), streamConfig); Set sinkStreams = getStreamSpecs(appDesc.getOutputStreamIds(), streamConfig); - Set intStreams = new HashSet<>(sourceStreams); + Set intermediateStreams = Sets.intersection(sourceStreams, sinkStreams); + Set inputStreams = Sets.difference(sourceStreams, intermediateStreams); + Set outputStreams = Sets.difference(sinkStreams, intermediateStreams); + Set tables = appDesc.getTableDescriptors().stream() .map(tableDescriptor -> ((BaseTableDescriptor) tableDescriptor).getTableSpec()).collect(Collectors.toSet()); - intStreams.retainAll(sinkStreams); - sourceStreams.removeAll(intStreams); - sinkStreams.removeAll(intStreams); // For this phase, we have a single job node for the whole dag String jobName = config.get(JobConfig.JOB_NAME()); String jobId = config.get(JobConfig.JOB_ID(), "1"); JobNode node = jobGraph.getOrCreateJobNode(jobName, jobId); - // add sources - sourceStreams.forEach(spec -> jobGraph.addSource(spec, node)); + // Add input streams + inputStreams.forEach(spec -> jobGraph.addInputStream(spec, node)); - // add sinks - sinkStreams.forEach(spec -> jobGraph.addSink(spec, node)); + // Add output streams + outputStreams.forEach(spec -> jobGraph.addOutputStream(spec, node)); - // add intermediate streams - intStreams.forEach(spec -> jobGraph.addIntermediateStream(spec, node, node)); + // Add intermediate streams + intermediateStreams.forEach(spec -> jobGraph.addIntermediateStream(spec, node, node)); - // add tables + // Add tables tables.forEach(spec -> jobGraph.addTable(spec, node)); if (!ApplicationUtil.isLegacyTaskApplication(appDesc)) { @@ -137,30 +138,41 @@ JobGraph createJobGraph(Config config, ApplicationDescriptorImpl existingStreams = new HashSet<>(); - existingStreams.addAll(jobGraph.getSources()); - existingStreams.addAll(jobGraph.getSinks()); + existingStreams.addAll(jobGraph.getInputStreams()); + existingStreams.addAll(jobGraph.getOutputStreams()); + // System to StreamEdges Multimap systemToStreamEdges = HashMultimap.create(); - // group the StreamEdge(s) based on the system name - existingStreams.forEach(streamEdge -> { - SystemStream systemStream = streamEdge.getSystemStream(); - systemToStreamEdges.put(systemStream.getSystem(), streamEdge); - }); - for (Map.Entry> entry : systemToStreamEdges.asMap().entrySet()) { - String systemName = entry.getKey(); - Collection streamEdges = entry.getValue(); + + // Group StreamEdges by system + for (StreamEdge streamEdge : existingStreams) { + String system = streamEdge.getSystemStream().getSystem(); + systemToStreamEdges.put(system, streamEdge); + } + + // Fetch partition count for every set of StreamEdges belonging to a particular system. + for (String system : systemToStreamEdges.keySet()) { + Collection streamEdges = systemToStreamEdges.get(system); + + // Map every stream to its corresponding StreamEdge so we can retrieve a StreamEdge given its stream. Map streamToStreamEdge = new HashMap<>(); - // create the stream name to StreamEdge mapping for this system - streamEdges.forEach(streamEdge -> streamToStreamEdge.put(streamEdge.getSystemStream().getStream(), streamEdge)); - // retrieve the partition counts for the streams in this system - Map streamToPartitionCount = streamManager.getStreamPartitionCounts(systemName, streamToStreamEdge.keySet()); - // set the partitions of a stream to its StreamEdge - streamToPartitionCount.forEach((stream, partitionCount) -> { - streamToStreamEdge.get(stream).setPartitionCount(partitionCount); - log.info("Partition count is {} for stream {}", partitionCount, stream); - }); + for (StreamEdge streamEdge : streamEdges) { + streamToStreamEdge.put(streamEdge.getSystemStream().getStream(), streamEdge); + } + + // Retrieve partition count for every set of streams. + Set streams = streamToStreamEdge.keySet(); + Map streamToPartitionCount = streamManager.getStreamPartitionCounts(system, streams); + + // Retrieve StreamEdge corresponding to every stream and set partition count on it. + for (Map.Entry entry : streamToPartitionCount.entrySet()) { + String stream = entry.getKey(); + Integer partitionCount = entry.getValue(); + streamToStreamEdge.get(stream).setPartitionCount(partitionCount); + log.info("Fetched partition count value {} for stream {}", partitionCount, stream); + } } } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java index b8608c1aab..e47f219939 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java @@ -175,8 +175,8 @@ private void calculateIntStreamPartitions(JobGraph jobGraph) { // partition = MAX(MAX(Input topic partitions), MAX(Output topic partitions)) // partition will be further bounded by MAX_INFERRED_PARTITIONS. // This is important when running in hadoop where an HDFS input can have lots of files (partitions). - int maxInPartitions = maxPartition(jobGraph.getSources()); - int maxOutPartitions = maxPartition(jobGraph.getSinks()); + int maxInPartitions = maxPartition(jobGraph.getInputStreams()); + int maxOutPartitions = maxPartition(jobGraph.getOutputStreams()); partitions = Math.max(maxInPartitions, maxOutPartitions); if (partitions > MAX_INFERRED_PARTITIONS) { diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index f8df2138d7..2290644510 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -55,8 +55,8 @@ private final Map nodes = new HashMap<>(); private final Map edges = new HashMap<>(); - private final Set sources = new HashSet<>(); - private final Set sinks = new HashSet<>(); + private final Set inputStreams = new HashSet<>(); + private final Set outputStreams = new HashSet<>(); private final Set intermediateStreams = new HashSet<>(); private final Set tables = new HashSet<>(); private final Config config; @@ -112,26 +112,26 @@ public ApplicationConfig getApplicationConfig() { /** * Add a source stream to a {@link JobNode} - * @param input source stream - * @param node the job node that consumes from the source + * @param streamSpec input stream + * @param node the job node that consumes from the streamSpec */ - void addSource(StreamSpec input, JobNode node) { - StreamEdge edge = getOrCreateStreamEdge(input); + void addInputStream(StreamSpec streamSpec, JobNode node) { + StreamEdge edge = getOrCreateStreamEdge(streamSpec); edge.addTargetNode(node); node.addInEdge(edge); - sources.add(edge); + inputStreams.add(edge); } /** - * Add a sink stream to a {@link JobNode} - * @param output sink stream - * @param node the job node that outputs to the sink + * Add an output stream to a {@link JobNode} + * @param streamSpec output stream + * @param node the job node that outputs to the output stream */ - void addSink(StreamSpec output, JobNode node) { - StreamEdge edge = getOrCreateStreamEdge(output); + void addOutputStream(StreamSpec streamSpec, JobNode node) { + StreamEdge edge = getOrCreateStreamEdge(streamSpec); edge.addSourceNode(node); node.addOutEdge(edge); - sinks.add(edge); + outputStreams.add(edge); } /** @@ -201,19 +201,19 @@ List getJobNodes() { } /** - * Returns the source streams in the graph + * Returns the input streams in the graph * @return unmodifiable set of {@link StreamEdge} */ - Set getSources() { - return Collections.unmodifiableSet(sources); + Set getInputStreams() { + return Collections.unmodifiableSet(inputStreams); } /** - * Return the sink streams in the graph + * Return the output streams in the graph * @return unmodifiable set of {@link StreamEdge} */ - Set getSinks() { - return Collections.unmodifiableSet(sinks); + Set getOutputStreams() { + return Collections.unmodifiableSet(outputStreams); } /** @@ -233,22 +233,22 @@ Set getIntermediateStreamEdges() { } /** - * Validate the graph has the correct topology, meaning the sources are coming from external streams, - * sinks are going to external streams, and the nodes are connected with intermediate streams. - * Also validate all the nodes are reachable from the sources. + * Validate the graph has the correct topology, meaning the input streams are coming from external streams, + * output streams are going to external streams, and the nodes are connected with intermediate streams. + * Also validate all the nodes are reachable from the input streams. */ void validate() { - validateSources(); - validateSinks(); + validateInputStreams(); + validateOutputStreams(); validateInternalStreams(); validateReachability(); } /** - * Validate the sources should have indegree being 0 and outdegree greater than 0 + * Validate the input streams should have indegree being 0 and outdegree greater than 0 */ - private void validateSources() { - sources.forEach(edge -> { + private void validateInputStreams() { + inputStreams.forEach(edge -> { if (!edge.getSourceNodes().isEmpty()) { throw new IllegalArgumentException( String.format("Source stream %s should not have producers.", edge.getName())); @@ -261,10 +261,10 @@ private void validateSources() { } /** - * Validate the sinks should have outdegree being 0 and indegree greater than 0 + * Validate the output streams should have outdegree being 0 and indegree greater than 0 */ - private void validateSinks() { - sinks.forEach(edge -> { + private void validateOutputStreams() { + outputStreams.forEach(edge -> { if (!edge.getTargetNodes().isEmpty()) { throw new IllegalArgumentException( String.format("Sink stream %s should not have consumers", edge.getName())); @@ -281,8 +281,8 @@ private void validateSinks() { */ private void validateInternalStreams() { Set internalEdges = new HashSet<>(edges.values()); - internalEdges.removeAll(sources); - internalEdges.removeAll(sinks); + internalEdges.removeAll(inputStreams); + internalEdges.removeAll(outputStreams); internalEdges.forEach(edge -> { if (edge.getSourceNodes().isEmpty() || edge.getTargetNodes().isEmpty()) { @@ -293,10 +293,10 @@ private void validateInternalStreams() { } /** - * Validate all nodes are reachable by sources. + * Validate all nodes are reachable by input streams. */ private void validateReachability() { - // validate all nodes are reachable from the sources + // validate all nodes are reachable from the input streams final Set reachable = findReachable(); if (reachable.size() != nodes.size()) { Set unreachable = new HashSet<>(nodes.values()); @@ -314,8 +314,8 @@ Set findReachable() { Queue queue = new ArrayDeque<>(); Set visited = new HashSet<>(); - sources.forEach(source -> { - List next = source.getTargetNodes(); + inputStreams.forEach(input -> { + List next = input.getTargetNodes(); queue.addAll(next); visited.addAll(next); }); @@ -350,11 +350,11 @@ List topologicalSort() { pnodes.forEach(node -> { String nid = node.getId(); //only count the degrees of intermediate streams - long degree = node.getInEdges().values().stream().filter(e -> !sources.contains(e)).count(); + long degree = node.getInEdges().values().stream().filter(e -> !inputStreams.contains(e)).count(); indegree.put(nid, degree); if (degree == 0L) { - // start from the nodes that has no intermediate input streams, so it only consumes from sources + // start from the nodes that has no intermediate input streams, so it only consumes from input streams q.add(node); visited.add(node); } @@ -407,9 +407,9 @@ List topologicalSort() { q.add(minNode); visited.add(minNode); } else { - // all the remaining nodes should be reachable from sources - // start from sources again to find the next node that hasn't been visited - JobNode nextNode = sources.stream().flatMap(source -> source.getTargetNodes().stream()) + // all the remaining nodes should be reachable from input streams + // start from input streams again to find the next node that hasn't been visited + JobNode nextNode = inputStreams.stream().flatMap(input -> input.getTargetNodes().stream()) .filter(node -> !visited.contains(node)) .findAny().get(); q.add(nextNode); diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 27bb87bb4f..8f7c3e34dc 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -136,8 +136,8 @@ static final class JobGraphJson { jobGraphJson.sinkStreams = new HashMap<>(); jobGraphJson.intermediateStreams = new HashMap<>(); jobGraphJson.tables = new HashMap<>(); - jobGraph.getSources().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.sourceStreams)); - jobGraph.getSinks().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.sinkStreams)); + jobGraph.getInputStreams().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.sourceStreams)); + jobGraph.getOutputStreams().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.sinkStreams)); jobGraph.getIntermediateStreamEdges().forEach(e -> buildStreamEdgeJson(e, jobGraphJson.intermediateStreams)); jobGraph.getTables().forEach(t -> buildTableJson(t, jobGraphJson.tables)); diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index a81088bfb6..6cc2c95f1c 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -29,7 +29,6 @@ import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; import org.apache.samza.application.ApplicationUtil; -import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.operators.spec.InputOperatorSpec; 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 4ef9f9c03b..791021604d 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 @@ -428,7 +428,7 @@ public void afterStop() { public void afterFailure(Throwable t) { containerShutdownLatch.countDown(); synchronized (lock) { - LOGGER.error(String.format("Container: %s failed with an exception. Stopping the stream processor: %s. Original exception:", container, processorId), containerException); + LOGGER.error(String.format("Container: %s failed with an exception. Stopping the stream processor: %s. Original exception:", container, processorId), t); state = State.STOPPING; containerException = t; jobCoordinator.stop(); diff --git a/samza-core/src/main/java/org/apache/samza/table/TableManager.java b/samza-core/src/main/java/org/apache/samza/table/TableManager.java index 186b4a8379..ae724149c0 100644 --- a/samza-core/src/main/java/org/apache/samza/table/TableManager.java +++ b/samza-core/src/main/java/org/apache/samza/table/TableManager.java @@ -37,12 +37,14 @@ /** * A {@link TableManager} manages tables within a Samza task. For each table, it maintains - * the {@link TableSpec} and the {@link TableProvider}. It is used at execution for - * {@link org.apache.samza.container.TaskInstance} to retrieve table instances for - * read/write operations. + * the {@link TableSpec}, the {@link TableProvider} and the {@link Table} instance. + * It is used at execution for {@link org.apache.samza.container.TaskInstance} to retrieve + * table instances for read/write operations. * * A {@link TableManager} is constructed from job configuration, the {@link TableSpec} - * and {@link TableProvider} are constructed by processing the job configuration. + * and {@link TableProvider} are constructed by processing the job configuration + * during initialization. The {@link Table} is constructed when {@link #getTable(String)} + * is called and cached. * * After a {@link TableManager} is constructed, local tables are associated with * local store instances created during {@link org.apache.samza.container.SamzaContainer} @@ -51,19 +53,19 @@ * Method {@link TableManager#getTable(String)} will throw {@link IllegalStateException}, * if it's called before initialization. * - * For store backed tables, the list of stores must be injected into the constructor. */ public class TableManager { static public class TableCtx { private TableSpec tableSpec; private TableProvider tableProvider; + private Table table; } private final Logger logger = LoggerFactory.getLogger(TableManager.class.getName()); // tableId -> TableCtx - private final Map tables = new HashMap<>(); + private final Map tableContexts = new HashMap<>(); private boolean initialized; @@ -100,7 +102,7 @@ public TableManager(Config config, Map> serdes) { */ public void init(SamzaContainerContext containerContext, TaskContext taskContext) { Preconditions.checkNotNull(containerContext, "null container context."); - tables.values().forEach(ctx -> ctx.tableProvider.init(containerContext, taskContext)); + tableContexts.values().forEach(ctx -> ctx.tableProvider.init(containerContext, taskContext)); initialized = true; } @@ -109,7 +111,7 @@ public void init(SamzaContainerContext containerContext, TaskContext taskContext * @param tableSpec the table spec */ private void addTable(TableSpec tableSpec) { - if (tables.containsKey(tableSpec.getId())) { + if (tableContexts.containsKey(tableSpec.getId())) { throw new SamzaException("Table " + tableSpec.getId() + " already exists"); } TableCtx ctx = new TableCtx(); @@ -117,14 +119,14 @@ private void addTable(TableSpec tableSpec) { Util.getObj(tableSpec.getTableProviderFactoryClassName(), TableProviderFactory.class); ctx.tableProvider = tableProviderFactory.getTableProvider(tableSpec); ctx.tableSpec = tableSpec; - tables.put(tableSpec.getId(), ctx); + tableContexts.put(tableSpec.getId(), ctx); } /** * Shutdown the table manager, internally it shuts down all tables */ public void close() { - tables.values().forEach(ctx -> ctx.tableProvider.close()); + tableContexts.values().forEach(ctx -> ctx.tableProvider.close()); } /** @@ -133,10 +135,14 @@ public void close() { * @return table instance */ public Table getTable(String tableId) { - if (!initialized) { - throw new IllegalStateException("TableManager has not been initialized."); + Preconditions.checkState(initialized, "TableManager has not been initialized."); + + TableCtx ctx = tableContexts.get(tableId); + Preconditions.checkNotNull(ctx, "Unknown tableId " + tableId); + + if (ctx.table == null) { + ctx.table = ctx.tableProvider.getTable(); } - Preconditions.checkArgument(tables.containsKey(tableId), "Unknown tableId=" + tableId); - return tables.get(tableId).tableProvider.getTable(); + return ctx.table; } } diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index 4f19adec24..c73cf14a92 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -39,6 +39,7 @@ object JobConfig { */ val CONFIG_REWRITERS = "job.config.rewriters" // streaming.job_config_rewriters val CONFIG_REWRITER_CLASS = "job.config.rewriter.%s.class" // streaming.job_config_rewriter_class - regex, system, config + val CONFIG_JOB_PREFIX = "jobs.%s." val JOB_NAME = "job.name" // streaming.job_name val JOB_ID = "job.id" // streaming.job_id val SAMZA_FWK_PATH = "samza.fwk.path" 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 85c8c2b9d4..07fa281bf0 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 @@ -30,6 +30,7 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.samza.Partition; +import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.SamzaApplication; @@ -91,7 +92,6 @@ public class TestExecutionPlanner { private GenericInputDescriptor> input2Descriptor; private StreamSpec input3Spec; private GenericInputDescriptor> input3Descriptor; - private StreamSpec input4Spec; private GenericInputDescriptor> input4Descriptor; private StreamSpec output1Spec; private GenericOutputDescriptor> output1Descriptor; @@ -189,44 +189,49 @@ private StreamApplicationDescriptorImpl createStreamGraphWithJoin() { private StreamApplicationDescriptorImpl createStreamGraphWithJoinAndWindow() { return new StreamApplicationDescriptorImpl(appDesc -> { - MessageStream> messageStream1 = - appDesc.getInputStream(input1Descriptor) - .map(m -> m); + MessageStream> messageStream1 = appDesc.getInputStream(input1Descriptor).map(m -> m); MessageStream> messageStream2 = - appDesc.getInputStream(input2Descriptor) - .partitionBy(m -> m.key, m -> m.value, "p1") - .filter(m -> true); + appDesc.getInputStream(input2Descriptor).partitionBy(m -> m.key, m -> m.value, "p1").filter(m -> true); MessageStream> messageStream3 = - appDesc.getInputStream(input3Descriptor) - .filter(m -> true) - .partitionBy(m -> m.key, m -> m.value, "p2") - .map(m -> m); + appDesc.getInputStream(input3Descriptor).filter(m -> true).partitionBy(m -> m.key, m -> m.value, "p2").map(m -> m); OutputStream> output1 = appDesc.getOutputStream(output1Descriptor); OutputStream> output2 = appDesc.getOutputStream(output2Descriptor); messageStream1.map(m -> m) - .filter(m->true) - .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), mock(Serde.class), mock(Serde.class)), "w1"); + .filter(m -> true) + .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), mock(Serde.class), mock(Serde.class)), "w1"); messageStream2.map(m -> m) - .filter(m->true) - .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), mock(Serde.class), mock(Serde.class)), "w2"); + .filter(m -> true) + .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), mock(Serde.class), mock(Serde.class)), "w2"); + + messageStream1.join(messageStream2, (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(1600), "j1").sendTo(output1); + messageStream3.join(messageStream2, (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(100), "j2").sendTo(output2); + messageStream3.join(messageStream2, (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(252), "j3").sendTo(output2); + }, config); + } + + private StreamApplicationDescriptorImpl createStreamGraphWithInvalidJoin() { + /** + * input1 (64) -- + * | + * join -> output1 (8) + * | + * input3 (32) -- + */ + return new StreamApplicationDescriptorImpl(appDesc -> { + MessageStream> messageStream1 = appDesc.getInputStream(input1Descriptor); + MessageStream> messageStream3 = appDesc.getInputStream(input3Descriptor); + OutputStream> output1 = appDesc.getOutputStream(output1Descriptor); messageStream1 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(1600), "j1") - .sendTo(output1); - messageStream3 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(100), "j2") - .sendTo(output2); - messageStream3 - .join(messageStream2, - (JoinFunction, KV, KV>) mock(JoinFunction.class), - mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(252), "j3") - .sendTo(output2); + .join(messageStream3, + (JoinFunction, KV, KV>) mock(JoinFunction.class), + mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1") + .sendTo(output1); }, config); } @@ -246,7 +251,6 @@ public void setup() { input1Spec = new StreamSpec("input1", "input1", "system1"); input2Spec = new StreamSpec("input2", "input2", "system2"); input3Spec = new StreamSpec("input3", "input3", "system2"); - input4Spec = new StreamSpec("input4", "input4", "system1"); output1Spec = new StreamSpec("output1", "output1", "system1"); output2Spec = new StreamSpec("output2", "output2", "system2"); @@ -302,8 +306,8 @@ public void testCreateProcessorGraph() { JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), new JobNodeConfigureGenerator()); - assertTrue(jobGraph.getSources().size() == 3); - assertTrue(jobGraph.getSinks().size() == 2); + assertTrue(jobGraph.getInputStreams().size() == 3); + assertTrue(jobGraph.getOutputStreams().size() == 2); assertTrue(jobGraph.getIntermediateStreams().size() == 2); // two streams generated by partitionBy } @@ -314,7 +318,7 @@ public void testFetchExistingStreamPartitions() { JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), new JobNodeConfigureGenerator()); - ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); + ExecutionPlanner.fetchInputAndOutputStreamPartitions(jobGraph, streamManager); assertTrue(jobGraph.getOrCreateStreamEdge(input1Spec).getPartitionCount() == 64); assertTrue(jobGraph.getOrCreateStreamEdge(input2Spec).getPartitionCount() == 16); assertTrue(jobGraph.getOrCreateStreamEdge(input3Spec).getPartitionCount() == 32); @@ -333,7 +337,7 @@ public void testCalculateJoinInputPartitions() { JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), new JobNodeConfigureGenerator()); - ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager); + ExecutionPlanner.fetchInputAndOutputStreamPartitions(jobGraph, streamManager); new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); // the partitions should be the same as input1 @@ -342,6 +346,15 @@ public void testCalculateJoinInputPartitions() { }); } +// @Test(expected = SamzaException.class) + @Test + public void testRejectsInvalidJoin() { + ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); + StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithInvalidJoin(); + + planner.plan(graphSpec); + } + @Test public void testDefaultPartitions() { Map map = new HashMap<>(config); @@ -359,7 +372,7 @@ public void testDefaultPartitions() { } @Test - public void testTriggerIntervalForJoins() throws Exception { + public void testTriggerIntervalForJoins() { Map map = new HashMap<>(config); map.put(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), String.valueOf(DEFAULT_PARTITIONS)); Config cfg = new MapConfig(map); @@ -374,7 +387,7 @@ public void testTriggerIntervalForJoins() throws Exception { } @Test - public void testTriggerIntervalForWindowsAndJoins() throws Exception { + public void testTriggerIntervalForWindowsAndJoins() { Map map = new HashMap<>(config); map.put(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), String.valueOf(DEFAULT_PARTITIONS)); Config cfg = new MapConfig(map); @@ -390,7 +403,7 @@ public void testTriggerIntervalForWindowsAndJoins() throws Exception { } @Test - public void testTriggerIntervalWithInvalidWindowMs() throws Exception { + public void testTriggerIntervalWithInvalidWindowMs() { Map map = new HashMap<>(config); map.put(TaskConfig.WINDOW_MS(), "-1"); map.put(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), String.valueOf(DEFAULT_PARTITIONS)); @@ -406,9 +419,8 @@ public void testTriggerIntervalWithInvalidWindowMs() throws Exception { assertEquals("4", jobConfigs.get(0).get(TaskConfig.WINDOW_MS())); } - @Test - public void testTriggerIntervalForStatelessOperators() throws Exception { + public void testTriggerIntervalForStatelessOperators() { Map map = new HashMap<>(config); map.put(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), String.valueOf(DEFAULT_PARTITIONS)); Config cfg = new MapConfig(map); @@ -422,7 +434,7 @@ public void testTriggerIntervalForStatelessOperators() throws Exception { } @Test - public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { + public void testTriggerIntervalWhenWindowMsIsConfigured() { Map map = new HashMap<>(config); map.put(TaskConfig.WINDOW_MS(), "2000"); map.put(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), String.valueOf(DEFAULT_PARTITIONS)); @@ -437,7 +449,7 @@ public void testTriggerIntervalWhenWindowMsIsConfigured() throws Exception { } @Test - public void testCalculateIntStreamPartitions() throws Exception { + public void testCalculateIntStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); JobGraph jobGraph = (JobGraph) planner.plan(graphSpec); @@ -527,10 +539,10 @@ public void testCreateJobGraphForTaskApplication() { JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc, new JobGraphJsonGenerator(), new JobNodeConfigureGenerator()); assertEquals(1, jobGraph.getJobNodes().size()); - assertTrue(jobGraph.getSources().stream().map(edge -> edge.getName()) + assertTrue(jobGraph.getInputStreams().stream().map(edge -> edge.getName()) .filter(streamId -> inputDescriptors.containsKey(streamId)).collect(Collectors.toList()).isEmpty()); Set intermediateStreams = new HashSet<>(inputDescriptors.keySet()); - jobGraph.getSources().forEach(edge -> { + jobGraph.getInputStreams().forEach(edge -> { if (intermediateStreams.contains(edge.getStreamSpec().getId())) { intermediateStreams.remove(edge.getStreamSpec().getId()); } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java index 80da42a4b5..5b37d624b4 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java @@ -36,7 +36,6 @@ import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.StringSerde; -import org.apache.samza.system.StreamSpec; import org.junit.Before; import org.junit.Test; @@ -49,15 +48,8 @@ */ public class TestIntermediateStreamPartitionPlanner { - private JobGraph mockGraph; private StreamApplicationDescriptorImpl mockStreamAppDesc; private Config mockConfig; - private JobNode mockJobNode; - private StreamSpec input1Spec; - private StreamSpec input2Spec; - private StreamSpec outputSpec; - private StreamSpec repartitionSpec; - private StreamSpec broadcastSpec; private KVSerde defaultSerde; private GenericSystemDescriptor inputSystemDescriptor; private GenericSystemDescriptor outputSystemDescriptor; @@ -65,19 +57,10 @@ public class TestIntermediateStreamPartitionPlanner { private GenericInputDescriptor> input1Descriptor; private GenericInputDescriptor> input2Descriptor; private GenericInputDescriptor> intermediateInputDescriptor; - private GenericInputDescriptor> broadcastInputDesriptor; private GenericOutputDescriptor> outputDescriptor; - private GenericOutputDescriptor> intermediateOutputDescriptor; @Before public void setUp() { - input1Spec = new StreamSpec("input1", "input1", "input-system"); - input2Spec = new StreamSpec("input2", "input2", "input-system"); - outputSpec = new StreamSpec("output", "output", "output-system"); - repartitionSpec = - new StreamSpec("jobName-jobId-partition_by-p1", "partition_by-p1", "intermediate-system"); - broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "broadcast-b1", "intermediate-system"); - defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); @@ -87,9 +70,6 @@ public void setUp() { outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) .withPhysicalName("partition_by-p1"); - intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("partition_by-p1"); - broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde); Map configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), "jobName"); @@ -112,7 +92,7 @@ public void testCalculateRepartitionJoinTopicPartitions() { JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); // set the input stream partitions - mockGraph.getSources().forEach(inEdge -> { + mockGraph.getInputStreams().forEach(inEdge -> { if (inEdge.getStreamSpec().getId().equals(input1Descriptor.getStreamId())) { inEdge.setPartitionCount(6); } else if (inEdge.getStreamSpec().getId().equals(input2Descriptor.getStreamId())) { @@ -133,7 +113,7 @@ public void testCalculateRepartitionIntermediateTopicPartitions() { JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); // set the input stream partitions - mockGraph.getSources().forEach(inEdge -> inEdge.setPartitionCount(7)); + mockGraph.getInputStreams().forEach(inEdge -> inEdge.setPartitionCount(7)); partitionPlanner.calculatePartitions(mockGraph); assertEquals(1, mockGraph.getIntermediateStreamEdges().size()); assertEquals(7, mockGraph.getIntermediateStreamEdges().stream() diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java index 9afd8e4c1d..5e45f81021 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java @@ -71,9 +71,9 @@ private void createGraph1() { JobNode n10 = graph1.getOrCreateJobNode("10", "1"); JobNode n11 = graph1.getOrCreateJobNode("11", "1"); - graph1.addSource(genStream(), n5); - graph1.addSource(genStream(), n7); - graph1.addSource(genStream(), n3); + graph1.addInputStream(genStream(), n5); + graph1.addInputStream(genStream(), n7); + graph1.addInputStream(genStream(), n3); graph1.addIntermediateStream(genStream(), n5, n11); graph1.addIntermediateStream(genStream(), n7, n11); graph1.addIntermediateStream(genStream(), n7, n8); @@ -82,9 +82,9 @@ private void createGraph1() { graph1.addIntermediateStream(genStream(), n11, n9); graph1.addIntermediateStream(genStream(), n8, n9); graph1.addIntermediateStream(genStream(), n11, n10); - graph1.addSink(genStream(), n2); - graph1.addSink(genStream(), n9); - graph1.addSink(genStream(), n10); + graph1.addOutputStream(genStream(), n2); + graph1.addOutputStream(genStream(), n9); + graph1.addOutputStream(genStream(), n10); } /** @@ -104,7 +104,7 @@ private void createGraph2() { JobNode n6 = graph2.getOrCreateJobNode("6", "1"); JobNode n7 = graph2.getOrCreateJobNode("7", "1"); - graph2.addSource(genStream(), n1); + graph2.addInputStream(genStream(), n1); graph2.addIntermediateStream(genStream(), n1, n2); graph2.addIntermediateStream(genStream(), n2, n3); graph2.addIntermediateStream(genStream(), n3, n4); @@ -113,7 +113,7 @@ private void createGraph2() { graph2.addIntermediateStream(genStream(), n6, n2); graph2.addIntermediateStream(genStream(), n5, n5); graph2.addIntermediateStream(genStream(), n5, n7); - graph2.addSink(genStream(), n7); + graph2.addOutputStream(genStream(), n7); } /** @@ -127,7 +127,7 @@ private void createGraph3() { JobNode n1 = graph3.getOrCreateJobNode("1", "1"); JobNode n2 = graph3.getOrCreateJobNode("2", "1"); - graph3.addSource(genStream(), n1); + graph3.addInputStream(genStream(), n1); graph3.addIntermediateStream(genStream(), n1, n1); graph3.addIntermediateStream(genStream(), n1, n2); graph3.addIntermediateStream(genStream(), n2, n2); @@ -143,7 +143,7 @@ private void createGraph4() { JobNode n1 = graph4.getOrCreateJobNode("1", "1"); - graph4.addSource(genStream(), n1); + graph4.addInputStream(genStream(), n1); graph4.addIntermediateStream(genStream(), n1, n1); } @@ -173,12 +173,12 @@ public void testAddSource() { StreamSpec s1 = genStream(); StreamSpec s2 = genStream(); StreamSpec s3 = genStream(); - graph.addSource(s1, n1); - graph.addSource(s2, n1); - graph.addSource(s3, n2); - graph.addSource(s3, n3); + graph.addInputStream(s1, n1); + graph.addInputStream(s2, n1); + graph.addInputStream(s3, n2); + graph.addInputStream(s3, n3); - assertTrue(graph.getSources().size() == 3); + assertTrue(graph.getInputStreams().size() == 3); assertTrue(graph.getOrCreateJobNode("1", "1").getInEdges().size() == 2); assertTrue(graph.getOrCreateJobNode("2", "1").getInEdges().size() == 1); @@ -206,11 +206,11 @@ public void testAddSink() { StreamSpec s1 = genStream(); StreamSpec s2 = genStream(); StreamSpec s3 = genStream(); - graph.addSink(s1, n1); - graph.addSink(s2, n2); - graph.addSink(s3, n2); + graph.addOutputStream(s1, n1); + graph.addOutputStream(s2, n2); + graph.addOutputStream(s3, n2); - assertTrue(graph.getSinks().size() == 3); + assertTrue(graph.getOutputStreams().size() == 3); assertTrue(graph.getOrCreateJobNode("1", "1").getOutEdges().size() == 1); assertTrue(graph.getOrCreateJobNode("2", "1").getOutEdges().size() == 2); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index 0e8217440d..e962db8bcb 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -315,8 +315,8 @@ public void testTaskApplication() throws Exception { // remove all intermediate streams from output outEdges.removeAll(intermediateEdges); // set the return values for mockJobGraph - when(mockJobGraph.getSources()).thenReturn(inEdges); - when(mockJobGraph.getSinks()).thenReturn(outEdges); + when(mockJobGraph.getInputStreams()).thenReturn(inEdges); + when(mockJobGraph.getOutputStreams()).thenReturn(outEdges); when(mockJobGraph.getIntermediateStreamEdges()).thenReturn(intermediateEdges); when(mockJobGraph.getJobNodes()).thenReturn(Collections.singletonList(mockJobNode)); String graphJson = jsonGenerator.toJson(mockJobGraph); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java deleted file mode 100644 index f464ed393f..0000000000 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.samza.execution; - -/** - * Unit tests for {@link JobNode} - */ -public class TestJobNode { - -} diff --git a/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java b/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java index 24178d0479..42f05c0bad 100644 --- a/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java +++ b/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java @@ -124,11 +124,14 @@ private void doTestInit(Map map) { TableManager tableManager = new TableManager(new MapConfig(map), serdeMap); tableManager.init(mock(SamzaContainerContext.class), mock(TaskContext.class)); - Table table = tableManager.getTable(TABLE_ID); - verify(DummyTableProviderFactory.tableProvider, times(1)).init(anyObject(), anyObject()); - Assert.assertEquals(DummyTableProviderFactory.table, table); + for (int i = 0; i < 2; i++) { + Table table = tableManager.getTable(TABLE_ID); + verify(DummyTableProviderFactory.tableProvider, times(1)).init(anyObject(), anyObject()); + verify(DummyTableProviderFactory.tableProvider, times(1)).getTable(); + Assert.assertEquals(DummyTableProviderFactory.table, table); + } - Map ctxMap = getFieldValue(tableManager, "tables"); + Map ctxMap = getFieldValue(tableManager, "tableContexts"); TableManager.TableCtx ctx = ctxMap.get(TABLE_ID); TableSpec tableSpec = getFieldValue(ctx, "tableSpec"); diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java index 28a1bac00a..0d50f26b2e 100644 --- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java +++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java @@ -37,6 +37,7 @@ import org.apache.samza.Partition; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; +import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; @@ -221,6 +222,17 @@ public Integer offsetComparator(String offset1, String offset2) { if (StringUtils.isBlank(offset1) || StringUtils.isBlank(offset2)) { return null; } + /* + * Properly handle END_OF_STREAM offset here. If both are END_OF_STREAM, + * then they are equal. Otherwise END_OF_STREAM is always greater than any + * other offsets. + */ + if (offset1.equals(IncomingMessageEnvelope.END_OF_STREAM_OFFSET)) { + return offset2.equals(IncomingMessageEnvelope.END_OF_STREAM_OFFSET) ? 0 : 1; + } + if (offset2.equals(IncomingMessageEnvelope.END_OF_STREAM_OFFSET)) { + return -1; + } int fileIndex1 = MultiFileHdfsReader.getCurFileIndex(offset1); int fileIndex2 = MultiFileHdfsReader.getCurFileIndex(offset2); if (fileIndex1 == fileIndex2) { diff --git a/samza-shell/src/main/bash/run-class.sh b/samza-shell/src/main/bash/run-class.sh index ac5f0ff87b..ee9eb31b61 100755 --- a/samza-shell/src/main/bash/run-class.sh +++ b/samza-shell/src/main/bash/run-class.sh @@ -44,10 +44,12 @@ DEFAULT_LOG4J_FILE=$base_dir/lib/log4j.xml BASE_LIB_DIR="$base_dir/lib" # JOB_LIB_DIR will be set for yarn container in ContainerUtil.java # for others we set it to home_dir/lib -JOB_LIB_DIR="${JOB_LIB_DIR:-$home_dir/lib}" +JOB_LIB_DIR="${JOB_LIB_DIR:-$base_dir/lib}" export JOB_LIB_DIR=$JOB_LIB_DIR +echo JOB_LIB_DIR=$JOB_LIB_DIR +echo BASE_LIB_DIR=$BASE_LIB_DIR if [ -d "$JOB_LIB_DIR" ] && [ "$JOB_LIB_DIR" != "$BASE_LIB_DIR" ]; then # build a common classpath # this class path will contain all the jars from the framework and the job's libs. @@ -68,11 +70,15 @@ if [ -d "$JOB_LIB_DIR" ] && [ "$JOB_LIB_DIR" != "$BASE_LIB_DIR" ]; then echo all_jars=$all_jars echo generated combined CLASSPATH=$CLASSPATH else - #default behavior - for file in $BASE_LIB_DIR/*.[jw]ar; + # default behaviour + # Wildcarding only includes *.jar and *.JAR files in classpath + CLASSPATH=$CLASSPATH:"$BASE_LIB_DIR/*"; + # We handle .war separately + for file in $BASE_LIB_DIR/*.war; do CLASSPATH=$CLASSPATH:$file done + echo generated from BASE_LIB_DIR CLASSPATH=$CLASSPATH fi if [ -z "$JAVA_HOME" ]; then diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index deefea6b3d..e6e604a493 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -190,7 +190,8 @@ public TestRunner addConfigs(Config config) { public TestRunner addOverrideConfig(String key, String value) { Preconditions.checkNotNull(key); Preconditions.checkNotNull(value); - configs.put(key, value); + String configKeyPrefix = String.format(JobConfig.CONFIG_JOB_PREFIX(), JOB_NAME); + configs.put(String.format("%s%s", configKeyPrefix, key), value); return this; } diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 3b2d08a48e..b249d4d287 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -87,9 +87,9 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne private static final String TEST_TASK_GROUPER_FACTORY = "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"; private static final String TEST_JOB_COORDINATOR_FACTORY = "org.apache.samza.zk.ZkJobCoordinatorFactory"; private static final String TEST_SYSTEM_FACTORY = "org.apache.samza.system.kafka.KafkaSystemFactory"; - private static final String TASK_SHUTDOWN_MS = "2000"; - private static final String JOB_DEBOUNCE_TIME_MS = "2000"; - private static final String BARRIER_TIMEOUT_MS = "2000"; + private static final String TASK_SHUTDOWN_MS = "10000"; + private static final String JOB_DEBOUNCE_TIME_MS = "10000"; + private static final String BARRIER_TIMEOUT_MS = "10000"; private static final String[] PROCESSOR_IDS = new String[] {"0000000000", "0000000001", "0000000002"}; private String inputKafkaTopic; 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 0d9df8b66f..5c067ad0a8 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 @@ -32,6 +32,7 @@ import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StreamConfig; +import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.operators.KV; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.serializers.IntegerSerde; @@ -66,8 +67,7 @@ public void testJoinWithSideInputsTable() { Arrays.asList(TestTableData.generateProfiles(10))); } - // @Test - // TODO: re-enable after fixing the coordinator stream issue in SAMZA-1786 + @Test public void testJoinWithDurableSideInputTable() { runTest( "durable-side-input", @@ -98,6 +98,7 @@ private void runTest(String systemName, StreamApplication app, List pa .addInputStream(profileStream) .addOutputStream(outputStream) .addConfigs(new MapConfig(configs)) + .addOverrideConfig(ClusterManagerConfig.CLUSTER_MANAGER_HOST_AFFINITY_ENABLED, Boolean.FALSE.toString()) .run(Duration.ofMillis(100000)); try { @@ -116,7 +117,6 @@ private void runTest(String systemName, StreamApplication app, List pa assertEquals("Mismatch between the expected and actual join count", results.size(), expectedEnrichedPageviews.size()); assertTrue("Pageview profile join did not succeed for all inputs", successfulJoin); - } catch (InterruptedException e) { e.printStackTrace(); } diff --git a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java index 6f175ea88b..53b61d9fbf 100644 --- a/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java +++ b/samza-yarn/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java @@ -30,11 +30,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.records.*; import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.client.api.async.NMClientAsync; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.Records; @@ -114,17 +112,30 @@ public class YarnClusterResourceManager extends ClusterResourceManager implement private final ConcurrentHashMap allocatedResources = new ConcurrentHashMap<>(); private final ConcurrentHashMap requestsMap = new ConcurrentHashMap<>(); - private final ConcurrentHashMap containersPendingStartup = new ConcurrentHashMap<>(); - private final SamzaAppMasterMetrics metrics; - final AtomicBoolean started = new AtomicBoolean(false); + private final AtomicBoolean started = new AtomicBoolean(false); private final Object lock = new Object(); private final NMClientAsync nmClientAsync; private static final Logger log = LoggerFactory.getLogger(YarnClusterResourceManager.class); private final Config config; + YarnClusterResourceManager(AMRMClientAsync amClientAsync, NMClientAsync nmClientAsync, Callback callback, + YarnAppState yarnAppState, SamzaYarnAppMasterLifecycle lifecycle, SamzaYarnAppMasterService service, + SamzaAppMasterMetrics metrics, YarnConfiguration yarnConfiguration, Config config) { + super(callback); + this.yarnConfiguration = yarnConfiguration; + this.metrics = metrics; + this.yarnConfig = new YarnConfig(config); + this.config = config; + this.amClient = amClientAsync; + this.state = yarnAppState; + this.lifecycle = lifecycle; + this.service = service; + this.nmClientAsync = nmClientAsync; + } + /** * Creates an YarnClusterResourceManager from config, a jobModelReader and a callback. * @param config to instantiate the container manager with @@ -513,18 +524,20 @@ public void onContainerStopped(ContainerId containerId) { } @Override - public void onStartContainerError(ContainerId containerId, Throwable t) { - log.error(String.format("Container: %s could not start.", containerId), t); + public void onStartContainerError(ContainerId yarnContainerId, Throwable t) { + log.error(String.format("Yarn Container: %s could not start.", yarnContainerId), t); - Container container = containersPendingStartup.remove(containerId); + String samzaContainerId = getPendingSamzaContainerId(yarnContainerId); - if (container != null) { - SamzaResource resource = new SamzaResource(container.getResource().getVirtualCores(), - container.getResource().getMemory(), container.getNodeId().getHost(), containerId.toString()); - log.info("Invoking failure callback for container: {}", containerId); + if (samzaContainerId != null) { + YarnContainer container = state.pendingYarnContainers.remove(samzaContainerId); + log.info("Failed Yarn Container: {} had Samza ContainerId: {} ", yarnContainerId, samzaContainerId); + SamzaResource resource = new SamzaResource(container.resource().getVirtualCores(), + container.resource().getMemory(), container.nodeId().getHost(), yarnContainerId.toString()); + log.info("Invoking failure callback for container: {}", yarnContainerId); clusterManagerCallback.onStreamProcessorLaunchFailure(resource, new SamzaContainerLaunchException(t)); } else { - log.info("Got an invalid notification for container: {}", containerId); + log.info("Got an invalid notification for container: {}", yarnContainerId); } } @@ -680,7 +693,6 @@ private void printContainerEnvironmentVariables(String samzaContainerId, Map Date: Mon, 17 Sep 2018 02:15:57 -0700 Subject: [PATCH 33/38] SAMZA-1814: merge with master --- .../java/org/apache/samza/execution/ExecutionPlanner.java | 6 ++---- .../org/apache/samza/execution/TestExecutionPlanner.java | 3 +-- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index 34d11c0eaf..f8208f6f08 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -73,10 +73,8 @@ public ExecutionPlan plan(ApplicationDescriptorImpl Date: Mon, 17 Sep 2018 02:41:29 -0700 Subject: [PATCH 34/38] SAMZA-1814: consolidate configuration generation in ExecutionPlanner between high and low-level API applications --- .../samza/test/framework/TestRunner.java | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index e6e604a493..3326db94bf 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -89,8 +89,6 @@ public enum Mode { private Map configs; private Map systems; -// private LegacyTaskApplication taskApp; -// private StreamApplication app; private SamzaApplication app; private String testId; private SystemFactory factory; @@ -364,26 +362,4 @@ public static Map> consumeStream(CollectionStream stream, D .collect(Collectors.toMap(entry -> entry.getKey().getPartition().getPartitionId(), entry -> entry.getValue().stream().map(e -> (T) e.getMessage()).collect(Collectors.toList()))); } - -// private TaskFactory createTaskFactory() { -// if (StreamTask.class.isAssignableFrom(taskClass)) { -// return (StreamTaskFactory) () -> { -// try { -// return (StreamTask) taskClass.newInstance(); -// } catch (InstantiationException | IllegalAccessException e) { -// throw new SamzaException(String.format("Failed to instantiate StreamTask class %s", taskClass.getName()), e); -// } -// }; -// } else if (AsyncStreamTask.class.isAssignableFrom(taskClass)) { -// return (AsyncStreamTaskFactory) () -> { -// try { -// return (AsyncStreamTask) taskClass.newInstance(); -// } catch (InstantiationException | IllegalAccessException e) { -// throw new SamzaException(String.format("Failed to instantiate AsyncStreamTask class %s", taskClass.getName()), e); -// } -// }; -// } -// throw new SamzaException(String.format("Not supported task.class %s. task.class has to implement either StreamTask " -// + "or AsyncStreamTask", taskClass.getName())); -// } } From 2c856c5f5b5b6cdcec8a9dff357238c6bb936d9b Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Mon, 17 Sep 2018 02:58:42 -0700 Subject: [PATCH 35/38] SAMZA-1814: consolidate configuration generation for high and low-level APIs --- .../java/org/apache/samza/execution/JobGraph.java | 6 +++++- .../framework/system/CollectionStreamSystemSpec.java | 11 ++++++----- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 2290644510..089d8bd4f7 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -66,7 +66,11 @@ /** * The JobGraph is only constructed by the {@link ExecutionPlanner}. - * @param appDesc Config + * + * @param config configuration for the application + * @param appDesc {@link ApplicationDescriptorImpl} describing the application + * @param jsonGenerator {@link JobGraphJsonGenerator} to create Json graph for the application + * @param configureGenerator configuration generator to create configuration for each {@link JobNode} */ JobGraph(Config config, ApplicationDescriptorImpl appDesc, JobGraphJsonGenerator jsonGenerator, JobNodeConfigureGenerator configureGenerator) { diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java index 7fc5eb8683..c764607763 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java @@ -33,13 +33,13 @@ * Following system level configs are set by default *

      *
    1. "systems.%s.default.stream.samza.offset.default" = "oldest"
    2. - *
    3. "jobs.job-name.systems.%s.default.stream.samza.offset.default" = "oldest"
    4. + *
    5. "jobs.job-name-and-id.systems.%s.default.stream.samza.offset.default" = "oldest"
    6. *
    7. "systems.%s.samza.factory" = {@link InMemorySystemFactory}
    8. - *
    9. "jobs.job-name.systems.%s.samza.factory" = {@link InMemorySystemFactory}
    10. + *
    11. "jobs.job-name-and-id.systems.%s.samza.factory" = {@link InMemorySystemFactory}
    12. *
    * The "systems.*" configs are required since the planner uses the system to get metadata about streams during - * planning. The "jobs.job-name.systems.*" configs are required since configs generated from user provided - * system/stream descriptors override configs originally supplied to the planner. Configs in the "jobs.job-name.*" + * planning. The "jobs.job-name-and-id.systems.*" configs are required since configs generated from user provided + * system/stream descriptors override configs originally supplied to the planner. Configs in the "jobs.job-name-and-id.*" * scope have the highest precedence. */ public class CollectionStreamSystemSpec { @@ -57,6 +57,7 @@ public class CollectionStreamSystemSpec { * is used for testing purpose. System uses {@link InMemorySystemFactory} to initialize in memory streams. *

    * @param systemName represents unique name of the system + * @param jobNameAndId the job name and ID */ private CollectionStreamSystemSpec(String systemName, String jobNameAndId) { this.systemName = systemName; @@ -78,7 +79,7 @@ public Map getSystemConfigs() { /** * Creates a {@link CollectionStreamSystemSpec} with name {@code systemName} * @param systemName represents name of the {@link CollectionStreamSystemSpec} - * @param jobNameAndId name of the job + * @param jobNameAndId the job name and ID * @return an instance of {@link CollectionStreamSystemSpec} */ public static CollectionStreamSystemSpec create(String systemName, String jobNameAndId) { From 0db5068dd4f0e5c0b681d2984abe03f5d3daa19b Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Tue, 18 Sep 2018 00:25:52 -0700 Subject: [PATCH 36/38] SAMZA-1814: fix merge issue and consolidated some test classes --- .../execution/ExecutionPlannerTestBase.java | 157 +++++++++++++++++ ...estIntermediateStreamPartitionPlanner.java | 84 +-------- .../execution/TestJobGraphJsonGenerator.java | 29 --- .../TestJobNodeConfigureGenerator.java | 165 ++++-------------- .../samza/test/framework/TestRunner.java | 9 +- 5 files changed, 196 insertions(+), 248 deletions(-) create mode 100644 samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java diff --git a/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java b/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java new file mode 100644 index 0000000000..bdec3754bc --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java @@ -0,0 +1,157 @@ +/* + * 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.execution; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.application.ApplicationDescriptorImpl; +import org.apache.samza.application.LegacyTaskApplication; +import org.apache.samza.application.StreamApplication; +import org.apache.samza.application.StreamApplicationDescriptorImpl; +import org.apache.samza.application.TaskApplication; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; +import org.apache.samza.operators.KV; +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.OutputStream; +import org.apache.samza.operators.descriptors.GenericInputDescriptor; +import org.apache.samza.operators.descriptors.GenericOutputDescriptor; +import org.apache.samza.operators.descriptors.GenericSystemDescriptor; +import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.serializers.JsonSerdeV2; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.task.IdentityStreamTask; +import org.junit.Before; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + + +/** + * Unit test base class to set up commonly used test application and configuration. + */ +class ExecutionPlannerTestBase { + protected StreamApplicationDescriptorImpl mockStreamAppDesc; + protected Config mockConfig; + protected JobNode mockJobNode; + protected KVSerde defaultSerde; + protected GenericSystemDescriptor inputSystemDescriptor; + protected GenericSystemDescriptor outputSystemDescriptor; + protected GenericSystemDescriptor intermediateSystemDescriptor; + protected GenericInputDescriptor> input1Descriptor; + protected GenericInputDescriptor> input2Descriptor; + protected GenericInputDescriptor> intermediateInputDescriptor; + protected GenericInputDescriptor> broadcastInputDesriptor; + protected GenericOutputDescriptor> outputDescriptor; + protected GenericOutputDescriptor> intermediateOutputDescriptor; + + @Before + public void setUp() { + defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); + inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); + outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); + intermediateSystemDescriptor = new GenericSystemDescriptor("intermediate-system", "mockSystemFactoryClassName"); + input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); + input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); + outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); + intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("jobName-jobId-partition_by-p1"); + intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) + .withPhysicalName("jobName-jobId-partition_by-p1"); + broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde) + .withPhysicalName("jobName-jobId-broadcast-b1"); + + Map configs = new HashMap<>(); + configs.put(JobConfig.JOB_NAME(), "jobName"); + configs.put(JobConfig.JOB_ID(), "jobId"); + configs.putAll(input1Descriptor.toConfig()); + configs.putAll(input2Descriptor.toConfig()); + configs.putAll(outputDescriptor.toConfig()); + configs.putAll(inputSystemDescriptor.toConfig()); + configs.putAll(outputSystemDescriptor.toConfig()); + configs.putAll(intermediateSystemDescriptor.toConfig()); + configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystemDescriptor.getSystemName()); + mockConfig = spy(new MapConfig(configs)); + + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + } + + String getJobNameAndId() { + return "jobName-jobId"; + } + + void configureJobNode(ApplicationDescriptorImpl mockStreamAppDesc) { + JobGraph jobGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, + mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + mockJobNode = spy(jobGraph.getJobNodes().get(0)); + } + + StreamApplication getRepartitionOnlyStreamApplication() { + return appDesc -> { + MessageStream> input1 = appDesc.getInputStream(input1Descriptor); + input1.partitionBy(KV::getKey, KV::getValue, "p1"); + }; + } + + StreamApplication getRepartitionJoinStreamApplication() { + return appDesc -> { + MessageStream> input1 = appDesc.getInputStream(input1Descriptor); + MessageStream> input2 = appDesc.getInputStream(input2Descriptor); + OutputStream> output = appDesc.getOutputStream(outputDescriptor); + JoinFunction> mockJoinFn = mock(JoinFunction.class); + input1 + .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") + .map(kv -> kv.value) + .join(input2.map(kv -> kv.value), mockJoinFn, + new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), + Duration.ofHours(1), "j1") + .sendTo(output); + }; + } + + TaskApplication getTaskApplication() { + return appDesc -> { + appDesc.addInputStream(input1Descriptor); + appDesc.addInputStream(input2Descriptor); + appDesc.addInputStream(intermediateInputDescriptor); + appDesc.addOutputStream(intermediateOutputDescriptor); + appDesc.addOutputStream(outputDescriptor); + appDesc.setTaskFactory(() -> new IdentityStreamTask()); + }; + } + + TaskApplication getLegacyTaskApplication() { + return new LegacyTaskApplication(IdentityStreamTask.class.getName()); + } + + StreamApplication getBroadcastOnlyStreamApplication(Serde serde) { + return appDesc -> { + MessageStream> input = appDesc.getInputStream(input1Descriptor); + if (serde != null) { + input.broadcast(serde, "b1"); + } else { + input.broadcast("b1"); + } + }; + } +} diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java index 5b37d624b4..94107aaac0 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java @@ -18,76 +18,20 @@ */ package org.apache.samza.execution; -import java.time.Duration; -import java.util.HashMap; -import java.util.Map; -import org.apache.samza.application.StreamApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; -import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; -import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; -import org.apache.samza.operators.descriptors.GenericInputDescriptor; -import org.apache.samza.operators.descriptors.GenericOutputDescriptor; -import org.apache.samza.operators.descriptors.GenericSystemDescriptor; -import org.apache.samza.operators.functions.JoinFunction; -import org.apache.samza.serializers.JsonSerdeV2; -import org.apache.samza.serializers.KVSerde; -import org.apache.samza.serializers.StringSerde; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; /** * Unit tests for {@link IntermediateStreamPartitionPlanner} */ -public class TestIntermediateStreamPartitionPlanner { - - private StreamApplicationDescriptorImpl mockStreamAppDesc; - private Config mockConfig; - private KVSerde defaultSerde; - private GenericSystemDescriptor inputSystemDescriptor; - private GenericSystemDescriptor outputSystemDescriptor; - private GenericSystemDescriptor intermediateSystemDescriptor; - private GenericInputDescriptor> input1Descriptor; - private GenericInputDescriptor> input2Descriptor; - private GenericInputDescriptor> intermediateInputDescriptor; - private GenericOutputDescriptor> outputDescriptor; - - @Before - public void setUp() { - defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); - inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); - outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); - intermediateSystemDescriptor = new GenericSystemDescriptor("intermediate-system", "mockSystemFactoryClassName"); - input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); - input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); - outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); - intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("partition_by-p1"); - - Map configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), "jobName"); - configs.put(JobConfig.JOB_ID(), "jobId"); - configs.putAll(input1Descriptor.toConfig()); - configs.putAll(input2Descriptor.toConfig()); - configs.putAll(outputDescriptor.toConfig()); - configs.putAll(inputSystemDescriptor.toConfig()); - configs.putAll(outputSystemDescriptor.toConfig()); - configs.putAll(intermediateSystemDescriptor.toConfig()); - configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystemDescriptor.getSystemName()); - mockConfig = spy(new MapConfig(configs)); - - mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); - } +public class TestIntermediateStreamPartitionPlanner extends ExecutionPlannerTestBase { @Test public void testCalculateRepartitionJoinTopicPartitions() { + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mockConfig, mockStreamAppDesc); JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); @@ -121,28 +65,4 @@ public void testCalculateRepartitionIntermediateTopicPartitions() { .findFirst().get().getPartitionCount()); } - private StreamApplication getRepartitionOnlyStreamApplication() { - return appDesc -> { - MessageStream> input1 = appDesc.getInputStream(input1Descriptor); - OutputStream> output = appDesc.getOutputStream(outputDescriptor); - JoinFunction> mockJoinFn = mock(JoinFunction.class); - input1.partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1").sendTo(output); - }; - } - - private StreamApplication getRepartitionJoinStreamApplication() { - return appDesc -> { - MessageStream> input1 = appDesc.getInputStream(input1Descriptor); - MessageStream> input2 = appDesc.getInputStream(input2Descriptor); - OutputStream> output = appDesc.getOutputStream(outputDescriptor); - JoinFunction> mockJoinFn = mock(JoinFunction.class); - input1 - .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") - .map(kv -> kv.value) - .join(input2.map(kv -> kv.value), mockJoinFn, - new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), - Duration.ofHours(1), "j1") - .sendTo(output); - }; - } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index e962db8bcb..69ac9ccc09 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -26,11 +26,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -52,7 +48,6 @@ import org.apache.samza.system.StreamSpec; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemAdmins; -import org.apache.samza.task.IdentityStreamTask; import org.apache.samza.testUtils.StreamTestUtils; import org.codehaus.jackson.map.ObjectMapper; import org.hamcrest.Matchers; @@ -68,7 +63,6 @@ * Unit test for {@link JobGraphJsonGenerator} */ public class TestJobGraphJsonGenerator { - private ApplicationDescriptorImpl mockAppDesc; private Config mockConfig; private JobNode mockJobNode; private StreamSpec input1Spec; @@ -81,9 +75,7 @@ public class TestJobGraphJsonGenerator { private GenericSystemDescriptor intermediateSystemDescriptor; private GenericInputDescriptor> input1Descriptor; private GenericInputDescriptor> input2Descriptor; - private GenericInputDescriptor> intermediateInputDescriptor; private GenericOutputDescriptor> outputDescriptor; - private GenericOutputDescriptor> intermediateOutputDescriptor; @Before public void setUp() { @@ -101,10 +93,6 @@ public void setUp() { input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); - intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("partition_by-p1"); - intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("partition_by-p1"); Map configs = new HashMap<>(); configs.put(JobConfig.JOB_NAME(), "jobName"); @@ -297,7 +285,6 @@ public void testRepartitionedWindowStreamApplication() throws Exception { @Test public void testTaskApplication() throws Exception { - mockAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); JobGraph mockJobGraph = mock(JobGraph.class); ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); @@ -344,7 +331,6 @@ public void testTaskApplication() throws Exception { @Test public void testLegacyTaskApplication() throws Exception { - mockAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator(); JobGraph mockJobGraph = mock(JobGraph.class); ApplicationConfig mockAppConfig = mock(ApplicationConfig.class); @@ -367,19 +353,4 @@ String getCountry() { return ""; } } - - private TaskApplication getLegacyTaskApplication() { - return new LegacyTaskApplication(IdentityStreamTask.class.getName()); - } - - private TaskApplication getTaskApplication() { - return appDesc -> { - appDesc.addInputStream(input1Descriptor); - appDesc.addInputStream(input2Descriptor); - appDesc.addInputStream(intermediateInputDescriptor); - appDesc.addOutputStream(intermediateOutputDescriptor); - appDesc.addOutputStream(outputDescriptor); - appDesc.setTaskFactory(() -> new IdentityStreamTask()); - }; - } } diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java index 5e1a62ec36..0da77331b8 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java @@ -19,7 +19,6 @@ package org.apache.samza.execution; import com.google.common.base.Joiner; -import java.time.Duration; import java.util.ArrayList; import java.util.Base64; import java.util.HashMap; @@ -28,11 +27,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.LegacyTaskApplication; -import org.apache.samza.application.StreamApplication; import org.apache.samza.application.StreamApplicationDescriptorImpl; -import org.apache.samza.application.TaskApplication; import org.apache.samza.application.TaskApplicationDescriptorImpl; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigRewriter; @@ -44,13 +39,8 @@ import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.KV; -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.OutputStream; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.GenericInputDescriptor; -import org.apache.samza.operators.descriptors.GenericOutputDescriptor; -import org.apache.samza.operators.descriptors.GenericSystemDescriptor; -import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; import org.apache.samza.serializers.JsonSerdeV2; import org.apache.samza.serializers.KVSerde; @@ -62,9 +52,7 @@ import org.apache.samza.table.TableProvider; import org.apache.samza.table.TableProviderFactory; import org.apache.samza.table.TableSpec; -import org.apache.samza.task.IdentityStreamTask; import org.apache.samza.task.TaskContext; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -77,75 +65,12 @@ /** * Unit test for {@link JobNodeConfigureGenerator} */ -public class TestJobNodeConfigureGenerator { - - private StreamApplicationDescriptorImpl mockStreamAppDesc; - private Config mockConfig; - private JobNode mockJobNode; - private StreamSpec input1Spec; - private StreamSpec input2Spec; - private StreamSpec outputSpec; - private StreamSpec repartitionSpec; - private StreamSpec broadcastSpec; - private KVSerde defaultSerde; - private GenericSystemDescriptor inputSystemDescriptor; - private GenericSystemDescriptor outputSystemDescriptor; - private GenericSystemDescriptor intermediateSystemDescriptor; - private GenericInputDescriptor> input1Descriptor; - private GenericInputDescriptor> input2Descriptor; - private GenericInputDescriptor> intermediateInputDescriptor; - private GenericInputDescriptor> broadcastInputDesriptor; - private GenericOutputDescriptor> outputDescriptor; - private GenericOutputDescriptor> intermediateOutputDescriptor; - - @Before - public void setUp() { - input1Spec = new StreamSpec("input1", "input1", "input-system"); - input2Spec = new StreamSpec("input2", "input2", "input-system"); - outputSpec = new StreamSpec("output", "output", "output-system"); - repartitionSpec = - new StreamSpec("jobName-jobId-partition_by-p1", "jobName-jobId-partition_by-p1", "intermediate-system"); - broadcastSpec = new StreamSpec("jobName-jobId-broadcast-b1", "jobName-jobId-broadcast-b1", "intermediate-system"); - - - defaultSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>()); - inputSystemDescriptor = new GenericSystemDescriptor("input-system", "mockSystemFactoryClassName"); - outputSystemDescriptor = new GenericSystemDescriptor("output-system", "mockSystemFactoryClassName"); - intermediateSystemDescriptor = new GenericSystemDescriptor("intermediate-system", "mockSystemFactoryClassName"); - input1Descriptor = inputSystemDescriptor.getInputDescriptor("input1", defaultSerde); - input2Descriptor = inputSystemDescriptor.getInputDescriptor("input2", defaultSerde); - outputDescriptor = outputSystemDescriptor.getOutputDescriptor("output", defaultSerde); - intermediateInputDescriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("jobName-jobId-partition_by-p1"); - intermediateOutputDescriptor = intermediateSystemDescriptor.getOutputDescriptor("jobName-jobId-partition_by-p1", defaultSerde) - .withPhysicalName("jobName-jobId-partition_by-p1"); - broadcastInputDesriptor = intermediateSystemDescriptor.getInputDescriptor("jobName-jobId-broadcast-b1", defaultSerde) - .withPhysicalName("jobName-jobId-broadcast-b1"); - - Map configs = new HashMap<>(); - configs.put(JobConfig.JOB_NAME(), "jobName"); - configs.put(JobConfig.JOB_ID(), "jobId"); - configs.putAll(input1Descriptor.toConfig()); - configs.putAll(input2Descriptor.toConfig()); - configs.putAll(outputDescriptor.toConfig()); - configs.putAll(inputSystemDescriptor.toConfig()); - configs.putAll(outputSystemDescriptor.toConfig()); - configs.putAll(intermediateSystemDescriptor.toConfig()); - configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), intermediateSystemDescriptor.getSystemName()); - mockConfig = spy(new MapConfig(configs)); - - mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); - configureJobNode(mockStreamAppDesc); - } - - private void configureJobNode(ApplicationDescriptorImpl mockStreamAppDesc) { - JobGraph jobGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, - mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); - mockJobNode = spy(jobGraph.getJobNodes().get(0)); - } +public class TestJobNodeConfigureGenerator extends ExecutionPlannerTestBase { @Test public void testConfigureSerdesWithRepartitionJoinApplication() { + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); @@ -228,7 +153,7 @@ public void testBroadcastStreamApplication() { validateJobConfig(expectedJobConfig, jobConfig); Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); validateStreamSerdeConfigure(broadcastInputDesriptor.getStreamId(), jobConfig, deserializedSerdes); - validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastSpec.getPhysicalName(), jobConfig); + validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastInputDesriptor.getPhysicalName().get(), jobConfig); } @Test @@ -243,7 +168,7 @@ public void testBroadcastStreamApplicationWithoutSerde() { Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); - validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastSpec.getPhysicalName(), jobConfig); + validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastInputDesriptor.getPhysicalName().get(), jobConfig); String keySerde = jobConfig.get(String.format("streams.%s.samza.key.serde", broadcastInputDesriptor.getStreamId())); String msgSerde = jobConfig.get(String.format("streams.%s.samza.msg.serde", broadcastInputDesriptor.getStreamId())); @@ -255,6 +180,7 @@ public void testBroadcastStreamApplicationWithoutSerde() { @Test public void testStreamApplicationWithTableAndSideInput() { + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); // add table to the RepartitionJoinStreamApplication GenericInputDescriptor> sideInput1 = inputSystemDescriptor.getInputDescriptor("sideInput1", defaultSerde); BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class); @@ -337,7 +263,9 @@ public void testTaskInputsRemovedFromOriginalConfig() { Map configs = new HashMap<>(mockConfig); configs.put(TaskConfig.INPUT_STREAMS(), "not.allowed1,not.allowed2"); mockConfig = spy(new MapConfig(configs)); - when(mockJobNode.getConfig()).thenReturn(mockConfig); + + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(defaultSerde), mockConfig); + configureJobNode(mockStreamAppDesc); JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); @@ -345,14 +273,31 @@ public void testTaskInputsRemovedFromOriginalConfig() { validateJobConfig(expectedConfig, jobConfig); } + @Test + public void testTaskInputsRetainedForLegacyTaskApplication() { + Map originConfig = new HashMap<>(mockConfig); + originConfig.put(TaskConfig.INPUT_STREAMS(), "must.retain1,must.retain2"); + mockConfig = new MapConfig(originConfig); + TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getLegacyTaskApplication(), mockConfig); + configureJobNode(taskAppDesc); + + // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode + JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); + // jobConfig should be exactly the same as original config + Map generatedConfig = new HashMap<>(jobConfig); + assertEquals(originConfig, generatedConfig); + } + @Test public void testOverrideConfigs() { Map configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); - String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; + String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, getJobNameAndId()) + streamCfgToOverride; configs.put(overrideCfgKey, "customized-system"); mockConfig = spy(new MapConfig(configs)); - when(mockJobNode.getConfig()).thenReturn(mockConfig); + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + configureJobNode(mockStreamAppDesc); JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); @@ -365,13 +310,14 @@ public void testOverrideConfigs() { public void testConfigureRewriter() { Map configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); - String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, mockJobNode.getId()) + streamCfgToOverride; + String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, getJobNameAndId()) + streamCfgToOverride; configs.put(overrideCfgKey, "customized-system"); configs.put(String.format(JobConfig.CONFIG_REWRITER_CLASS(), "mock"), MockConfigRewriter.class.getName()); configs.put(JobConfig.CONFIG_REWRITERS(), "mock"); configs.put(String.format("job.config.rewriter.mock.%s", streamCfgToOverride), "rewritten-system"); mockConfig = spy(new MapConfig(configs)); - when(mockJobNode.getConfig()).thenReturn(mockConfig); + mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); + configureJobNode(mockStreamAppDesc); JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); @@ -511,55 +457,6 @@ private void validateJoinStoreConfigure(Config joinStoreConfig, String changelog assertEquals("3600000", joinStoreConfig.get("rocksdb.ttl.ms")); } - private TaskApplication getTaskApplication() { - return appDesc -> { - appDesc.addInputStream(input1Descriptor); - appDesc.addInputStream(input2Descriptor); - appDesc.addInputStream(intermediateInputDescriptor); - appDesc.addOutputStream(intermediateOutputDescriptor); - appDesc.addOutputStream(outputDescriptor); - appDesc.setTaskFactory(() -> new IdentityStreamTask()); - }; - } - - private TaskApplication getLegacyTaskApplication() { - return new LegacyTaskApplication(IdentityStreamTask.class.getName()); - } - - private StreamApplication getRepartitionJoinStreamApplication() { - return appDesc -> { - MessageStream> input1 = appDesc.getInputStream(input1Descriptor); - MessageStream> input2 = appDesc.getInputStream(input2Descriptor); - OutputStream> output = appDesc.getOutputStream(outputDescriptor); - JoinFunction> mockJoinFn = mock(JoinFunction.class); - input1 - .partitionBy(KV::getKey, KV::getValue, defaultSerde, "p1") - .map(kv -> kv.value) - .join(input2.map(kv -> kv.value), mockJoinFn, - new StringSerde(), new JsonSerdeV2<>(Object.class), new JsonSerdeV2<>(Object.class), - Duration.ofHours(1), "j1") - .sendTo(output); - }; - } - - private StreamApplication getRepartitionOnlyStreamApplication() { - return appDesc -> { - MessageStream> input = appDesc.getInputStream(input1Descriptor); - input.partitionBy(KV::getKey, KV::getValue, "p1"); - }; - } - - private StreamApplication getBroadcastOnlyStreamApplication(Serde serde) { - return appDesc -> { - MessageStream> input = appDesc.getInputStream(input1Descriptor); - if (serde != null) { - input.broadcast(serde, "b1"); - } else { - input.broadcast("b1"); - } - }; - } - private static class MockTableProvider implements TableProvider { private final Map configMap; diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index 3326db94bf..8de4ea83f1 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -139,8 +139,7 @@ private TestRunner(StreamApplication app) { */ private void registerSystem(String systemName) { if (!systems.containsKey(systemName)) { - systems.put(systemName, CollectionStreamSystemSpec.create(systemName, - String.format("%s-%s", JOB_NAME, configs.getOrDefault(JobConfig.JOB_ID(), "1")))); + systems.put(systemName, CollectionStreamSystemSpec.create(systemName, getJobNameAndId())); configs.putAll(systems.get(systemName).getSystemConfigs()); } } @@ -188,7 +187,7 @@ public TestRunner addConfigs(Config config) { public TestRunner addOverrideConfig(String key, String value) { Preconditions.checkNotNull(key); Preconditions.checkNotNull(value); - String configKeyPrefix = String.format(JobConfig.CONFIG_JOB_PREFIX(), JOB_NAME); + String configKeyPrefix = String.format(JobConfig.CONFIG_JOB_PREFIX(), getJobNameAndId()); configs.put(String.format("%s%s", configKeyPrefix, key), value); return this; } @@ -221,6 +220,10 @@ public TestRunner addInputStream(CollectionStream stream) { return this; } + private String getJobNameAndId() { + return String.format("%s-%s", JOB_NAME, configs.getOrDefault(JobConfig.JOB_ID(), "1")); + } + /** * Creates an in memory stream with {@link InMemorySystemFactory} and initializes the metadata for the stream. * Initializes each partition of that stream with messages from {@code stream.getInitPartitions} From b66b9fa9d857b7d6279f2551ad483ce8f1d86054 Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Thu, 20 Sep 2018 17:00:45 -0700 Subject: [PATCH 37/38] SAMZA-1814: moving serde generation to a single top-level configuration generation, not embedded in table. Address review comments --- .../ApplicationDescriptorImpl.java | 116 ++++++++---- .../samza/application/ApplicationUtil.java | 4 - .../StreamApplicationDescriptorImpl.java | 63 ++----- .../TaskApplicationDescriptorImpl.java | 23 +-- .../samza/execution/ExecutionPlanner.java | 21 +-- ...er.java => IntermediateStreamManager.java} | 9 +- .../org/apache/samza/execution/JobGraph.java | 21 +-- .../execution/JobGraphJsonGenerator.java | 3 - .../org/apache/samza/execution/JobNode.java | 62 ++++--- ...ava => JobNodeConfigurationGenerator.java} | 167 ++++++++++-------- .../apache/samza/execution/JobPlanner.java | 4 +- .../samza/execution/LocalJobPlanner.java | 9 +- .../samza/execution/RemoteJobPlanner.java | 8 +- .../samza/operators/BaseTableDescriptor.java | 9 + .../samza/operators/OperatorSpecGraph.java | 7 - .../samza/table/TableConfigGenerator.java | 48 ----- .../org/apache/samza/config/JobConfig.scala | 2 +- .../TestStreamApplicationDescriptorImpl.java | 3 +- .../TestTaskApplicationDescriptorImpl.java | 8 +- .../execution/ExecutionPlannerTestBase.java | 4 +- .../samza/execution/TestExecutionPlanner.java | 31 ++-- ...ava => TestIntermediateStreamManager.java} | 16 +- .../apache/samza/execution/TestJobGraph.java | 12 +- .../execution/TestJobGraphJsonGenerator.java | 2 +- ...=> TestJobNodeConfigurationGenerator.java} | 40 +++-- .../operators/TestOperatorSpecGraph.java | 1 - .../operators/spec/OperatorSpecTestUtils.java | 1 - .../kv/BaseLocalStoreBackedTableProvider.java | 17 +- .../samza/test/framework/TestRunner.java | 2 +- .../system/CollectionStreamSystemSpec.java | 6 +- .../table/TestTableDescriptorsProvider.java | 6 - 31 files changed, 352 insertions(+), 373 deletions(-) rename samza-core/src/main/java/org/apache/samza/execution/{IntermediateStreamPartitionPlanner.java => IntermediateStreamManager.java} (96%) rename samza-core/src/main/java/org/apache/samza/execution/{JobNodeConfigureGenerator.java => JobNodeConfigurationGenerator.java} (77%) rename samza-core/src/test/java/org/apache/samza/execution/{TestIntermediateStreamPartitionPlanner.java => TestIntermediateStreamManager.java} (79%) rename samza-core/src/test/java/org/apache/samza/execution/{TestJobNodeConfigureGenerator.java => TestJobNodeConfigurationGenerator.java} (93%) diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java index f1f35b18aa..b58d5a567a 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationDescriptorImpl.java @@ -19,6 +19,7 @@ package org.apache.samza.application; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; @@ -26,16 +27,20 @@ import org.apache.samza.config.Config; import org.apache.samza.metrics.MetricsReporterFactory; import org.apache.samza.operators.ContextManager; +import org.apache.samza.operators.KV; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.operators.spec.InputOperatorSpec; -import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.runtime.ProcessorLifecycleListener; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.task.TaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -49,10 +54,15 @@ */ public abstract class ApplicationDescriptorImpl implements ApplicationDescriptor { + private static final Logger LOGGER = LoggerFactory.getLogger(ApplicationDescriptorImpl.class); - final Config config; private final Class appClass; private final Map reporterFactories = new LinkedHashMap<>(); + // serdes used by input/output/intermediate streams, keyed by streamId + private final Map> streamSerdes = new HashMap<>(); + // serdes used by tables, keyed by tableId + private final Map> tableSerdes = new HashMap<>(); + final Config config; // Default to no-op functions in ContextManager // TODO: this should be replaced by shared context factory defined in SAMZA-1714 @@ -144,6 +154,35 @@ public Optional getDefaultSystemDescriptor() { return Optional.empty(); } + /** + * Get the corresponding {@link KVSerde} for the input {@code inputStreamId} + * + * @param streamId id of the stream + * @return the {@link KVSerde} for the stream. null if the serde is not defined or {@code streamId} does not exist + */ + public KV getStreamSerdes(String streamId) { + return streamSerdes.get(streamId); + } + + /** + * Get the corresponding {@link KVSerde} for the input {@code inputStreamId} + * + * @param tableId id of the table + * @return the {@link KVSerde} for the stream. null if the serde is not defined or {@code streamId} does not exist + */ + public KV getTableSerdes(String tableId) { + return tableSerdes.get(tableId); + } + + /** + * Get the map of all {@link InputOperatorSpec}s in this applicaiton + * + * @return an immutable map from streamId to {@link InputOperatorSpec}. Default to empty map for low-level {@link TaskApplication} + */ + public Map getInputOperators() { + return Collections.EMPTY_MAP; + } + /** * Get all the {@link InputDescriptor}s to this application * @@ -193,39 +232,52 @@ public Optional getDefaultSystemDescriptor() { */ public abstract Set getOutputStreamIds(); - /** - * Get the corresponding {@link Serde} for the input {@code inputStreamId} - * - * @param inputStreamId id of the input stream - * @return the {@link Serde} for the input stream. null if the serde is not defined or {@code inputStreamId} - * does not exist - */ - public abstract Serde getInputSerde(String inputStreamId); + KV getOrCreateStreamSerdes(String streamId, Serde serde) { + Serde keySerde, valueSerde; - /** - * Get the corresponding {@link Serde} for the output {@code outputStreamId} - * - * @param outputStreamId of the output stream - * @return the {@link Serde} for the output stream. null if the serde is not defined or {@code outputStreamId} - * does not exist - */ - public abstract Serde getOutputSerde(String outputStreamId); + KV currentSerdePair = streamSerdes.get(streamId); - /** - * Get the map of all {@link InputOperatorSpec}s in this applicaiton - * - * @return an immutable map from streamId to {@link InputOperatorSpec}. Default to empty map for low-level {@link TaskApplication} - */ - public Map getInputOperators() { - return Collections.EMPTY_MAP; + if (serde instanceof KVSerde) { + keySerde = ((KVSerde) serde).getKeySerde(); + valueSerde = ((KVSerde) serde).getValueSerde(); + } else { + keySerde = new NoOpSerde(); + valueSerde = serde; + } + + if (currentSerdePair == null) { + if (keySerde instanceof NoOpSerde) { + LOGGER.info("Using NoOpSerde as the key serde for stream " + streamId + + ". Keys will not be (de)serialized"); + } + if (valueSerde instanceof NoOpSerde) { + LOGGER.info("Using NoOpSerde as the value serde for stream " + streamId + + ". Values will not be (de)serialized"); + } + streamSerdes.put(streamId, KV.of(keySerde, valueSerde)); + } else if (!currentSerdePair.getKey().equals(keySerde) || !currentSerdePair.getValue().equals(valueSerde)) { + throw new IllegalArgumentException(String.format("Serde for stream %s is already defined. Cannot change it to " + + "different serdes.", streamId)); + } + return streamSerdes.get(streamId); } - /** - * Get the map of all {@link OutputStreamImpl}s in this application - * - * @return an immutable map from streamId to {@link OutputStreamImpl}. Default to empty map for low-level {@link TaskApplication} - */ - public Map getOutputStreams() { - return Collections.EMPTY_MAP; + KV getOrCreateTableSerdes(String tableId, KVSerde kvSerde) { + Serde keySerde, valueSerde; + keySerde = kvSerde.getKeySerde(); + valueSerde = kvSerde.getValueSerde(); + + if (!tableSerdes.containsKey(tableId)) { + tableSerdes.put(tableId, KV.of(keySerde, valueSerde)); + return tableSerdes.get(tableId); + } + + KV currentSerdePair = tableSerdes.get(tableId); + if (!currentSerdePair.getKey().equals(keySerde) || !currentSerdePair.getValue().equals(valueSerde)) { + throw new IllegalArgumentException(String.format("Serde for table %s is already defined. Cannot change it to " + + "different serdes.", tableId)); + } + return streamSerdes.get(tableId); } + } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java b/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java index eae6be36f6..b39ad3c377 100644 --- a/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java +++ b/samza-core/src/main/java/org/apache/samza/application/ApplicationUtil.java @@ -60,8 +60,4 @@ public static SamzaApplication fromConfig(Config config) { return new LegacyTaskApplication(taskClassOption.get()); } - public static boolean isLegacyTaskApplication(ApplicationDescriptorImpl appDesc) { - return LegacyTaskApplication.class.isAssignableFrom(appDesc.getAppClass()); - } - } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java index ae200c2eb0..4e0f5a18f2 100644 --- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java @@ -50,7 +50,6 @@ import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.operators.stream.IntermediateMessageStreamImpl; import org.apache.samza.serializers.KVSerde; -import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.table.Table; import org.apache.samza.table.TableSpec; @@ -76,7 +75,7 @@ public class StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl inputOperators = new LinkedHashMap<>(); private final Map outputStreams = new LinkedHashMap<>(); - private final Map tables = new LinkedHashMap<>(); + private final Map tables = new LinkedHashMap<>(); private final Set operatorIds = new HashSet<>(); private Optional defaultSystemDescriptorOptional = Optional.empty(); @@ -123,7 +122,7 @@ public MessageStream getInputStream(InputDescriptor inputDescriptor "getInputStream must not be called multiple times with the same streamId: " + streamId); Serde serde = inputDescriptor.getSerde(); - KV kvSerdes = getKVSerdes(streamId, serde); + KV kvSerdes = getOrCreateStreamSerdes(streamId, serde); if (outputStreams.containsKey(streamId)) { OutputStreamImpl outputStream = outputStreams.get(streamId); Serde keySerde = outputStream.getKeySerde(); @@ -154,7 +153,7 @@ public OutputStream getOutputStream(OutputDescriptor outputDescript "getOutputStream must not be called multiple times with the same streamId: " + streamId); Serde serde = outputDescriptor.getSerde(); - KV kvSerdes = getKVSerdes(streamId, serde); + KV kvSerdes = getOrCreateStreamSerdes(streamId, serde); if (inputOperators.containsKey(streamId)) { InputOperatorSpec inputOperatorSpec = inputOperators.get(streamId); Serde keySerde = inputOperatorSpec.getKeySerde(); @@ -178,13 +177,15 @@ public Table> getTable(TableDescriptor tableDescriptor) String.format("add table descriptors multiple times with the same tableId: %s", tableDescriptor.getTableId())); tableDescriptors.put(tableDescriptor.getTableId(), tableDescriptor); - TableSpec tableSpec = ((BaseTableDescriptor) tableDescriptor).getTableSpec(); - if (tables.containsKey(tableSpec)) { + BaseTableDescriptor baseTableDescriptor = (BaseTableDescriptor) tableDescriptor; + TableSpec tableSpec = baseTableDescriptor.getTableSpec(); + if (tables.containsKey(tableSpec.getId())) { throw new IllegalStateException( String.format("getTable() invoked multiple times with the same tableId: %s", tableId)); } - tables.put(tableSpec, new TableImpl(tableSpec)); - return tables.get(tableSpec); + tables.put(tableSpec.getId(), new TableImpl(tableSpec)); + getOrCreateTableSerdes(tableSpec.getId(), baseTableDescriptor.getSerde()); + return tables.get(tableSpec.getId()); } /** @@ -249,25 +250,6 @@ public Set getOutputStreamIds() { return Collections.unmodifiableSet(new HashSet<>(outputStreams.keySet())); } - @Override - public Serde getInputSerde(String inputStreamId) { - if (!inputOperators.containsKey(inputStreamId)) { - // the corresponding inputStreamId does not exist - return null; - } - InputOperatorSpec inputOperator = inputOperators.get(inputStreamId); - return KVSerde.of(inputOperator.getKeySerde(), inputOperator.getValueSerde()); - } - - @Override - public Serde getOutputSerde(String outputStreamId) { - if (!outputStreams.containsKey(outputStreamId)) { - return null; - } - OutputStreamImpl outputStream = outputStreams.get(outputStreamId); - return KVSerde.of(outputStream.getKeySerde(), outputStream.getValueSerde()); - } - /** * Get the default {@link SystemDescriptor} in this application * @@ -327,7 +309,7 @@ public Map getOutputStreams() { return Collections.unmodifiableMap(outputStreams); } - public Map getTables() { + public Map getTables() { return Collections.unmodifiableMap(tables); } @@ -363,7 +345,7 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI kvSerdes = new KV<>(null, null); // and that key and msg serdes are provided for job.default.system in configs } else { isKeyed = serde instanceof KVSerde; - kvSerdes = getKVSerdes(streamId, serde); + kvSerdes = getOrCreateStreamSerdes(streamId, serde); } InputTransformer transformer = (InputTransformer) getDefaultSystemDescriptor() @@ -377,29 +359,6 @@ public IntermediateMessageStreamImpl getIntermediateStream(String streamI return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamId), outputStreams.get(streamId)); } - private KV getKVSerdes(String streamId, Serde serde) { - Serde keySerde, valueSerde; - - if (serde instanceof KVSerde) { - keySerde = ((KVSerde) serde).getKeySerde(); - valueSerde = ((KVSerde) serde).getValueSerde(); - } else { - keySerde = new NoOpSerde(); - valueSerde = serde; - } - - if (keySerde instanceof NoOpSerde) { - LOGGER.info("Using NoOpSerde as the key serde for stream " + streamId + - ". Keys will not be (de)serialized"); - } - if (valueSerde instanceof NoOpSerde) { - LOGGER.info("Using NoOpSerde as the value serde for stream " + streamId + - ". Values will not be (de)serialized"); - } - - return KV.of(keySerde, valueSerde); - } - // check uniqueness of the {@code systemDescriptor} and add if it is unique private void addSystemDescriptor(SystemDescriptor systemDescriptor) { Preconditions.checkState(!systemDescriptors.containsKey(systemDescriptor.getSystemName()) diff --git a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java index bd931c2694..d140a907a0 100644 --- a/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/application/TaskApplicationDescriptorImpl.java @@ -25,11 +25,11 @@ import java.util.Map; import java.util.Set; import org.apache.samza.config.Config; +import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; -import org.apache.samza.serializers.Serde; import org.apache.samza.task.TaskFactory; @@ -66,6 +66,7 @@ public void addInputStream(InputDescriptor inputDescriptor) { // TODO: SAMZA-1841: need to add to the broadcast streams if inputDescriptor is for a broadcast stream Preconditions.checkState(!inputDescriptors.containsKey(inputDescriptor.getStreamId()), String.format("add input descriptors multiple times with the same streamId: %s", inputDescriptor.getStreamId())); + getOrCreateStreamSerdes(inputDescriptor.getStreamId(), inputDescriptor.getSerde()); inputDescriptors.put(inputDescriptor.getStreamId(), inputDescriptor); addSystemDescriptor(inputDescriptor.getSystemDescriptor()); } @@ -74,6 +75,7 @@ public void addInputStream(InputDescriptor inputDescriptor) { public void addOutputStream(OutputDescriptor outputDescriptor) { Preconditions.checkState(!outputDescriptors.containsKey(outputDescriptor.getStreamId()), String.format("add output descriptors multiple times with the same streamId: %s", outputDescriptor.getStreamId())); + getOrCreateStreamSerdes(outputDescriptor.getStreamId(), outputDescriptor.getSerde()); outputDescriptors.put(outputDescriptor.getStreamId(), outputDescriptor); addSystemDescriptor(outputDescriptor.getSystemDescriptor()); } @@ -82,6 +84,7 @@ public void addOutputStream(OutputDescriptor outputDescriptor) { public void addTable(TableDescriptor tableDescriptor) { Preconditions.checkState(!tableDescriptors.containsKey(tableDescriptor.getTableId()), String.format("add table descriptors multiple times with the same tableId: %s", tableDescriptor.getTableId())); + getOrCreateTableSerdes(tableDescriptor.getTableId(), ((BaseTableDescriptor) tableDescriptor).getSerde()); tableDescriptors.put(tableDescriptor.getTableId(), tableDescriptor); } @@ -122,24 +125,6 @@ public Set getOutputStreamIds() { return Collections.unmodifiableSet(new HashSet<>(outputDescriptors.keySet())); } - @Override - public Serde getInputSerde(String inputStreamId) { - if (!inputDescriptors.containsKey(inputStreamId)) { - return null; - } - InputDescriptor inputDescriptor = inputDescriptors.get(inputStreamId); - return inputDescriptor.getSerde(); - } - - @Override - public Serde getOutputSerde(String outputStreamId) { - if (!outputDescriptors.containsKey(outputStreamId)) { - return null; - } - OutputDescriptor outputDescriptor = outputDescriptors.get(outputStreamId); - return outputDescriptor.getSerde(); - } - /** * Get the user-defined {@link TaskFactory} * @return the {@link TaskFactory} object diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java index f8208f6f08..438d827795 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java @@ -31,7 +31,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.ApplicationUtil; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; @@ -55,26 +55,24 @@ public class ExecutionPlanner { private static final Logger log = LoggerFactory.getLogger(ExecutionPlanner.class); private final Config config; - private final StreamConfig streamConfig; private final StreamManager streamManager; public ExecutionPlanner(Config config, StreamManager streamManager) { this.config = config; this.streamManager = streamManager; - this.streamConfig = new StreamConfig(config); } public ExecutionPlan plan(ApplicationDescriptorImpl appDesc) { validateConfig(); // create physical job graph based on stream graph - JobGraph jobGraph = createJobGraph(config, appDesc, new JobGraphJsonGenerator(), new JobNodeConfigureGenerator()); + JobGraph jobGraph = createJobGraph(config, appDesc); // fetch the external streams partition info - fetchInputAndOutputStreamPartitions(jobGraph, streamManager); + setInputAndOutputStreamPartitionCount(jobGraph, streamManager); // figure out the partitions for internal streams - new IntermediateStreamPartitionPlanner(config, appDesc).calculatePartitions(jobGraph); + new IntermediateStreamManager(config, appDesc).calculatePartitions(jobGraph); return jobGraph; } @@ -93,9 +91,8 @@ private void validateConfig() { * Create the physical graph from {@link ApplicationDescriptorImpl} */ /* package private */ - JobGraph createJobGraph(Config config, ApplicationDescriptorImpl appDesc, - JobGraphJsonGenerator jobJsonGenerator, JobNodeConfigureGenerator jobConfigureGenerator) { - JobGraph jobGraph = new JobGraph(config, appDesc, jobJsonGenerator, jobConfigureGenerator); + JobGraph createJobGraph(Config config, ApplicationDescriptorImpl appDesc) { + JobGraph jobGraph = new JobGraph(config, appDesc); StreamConfig streamConfig = new StreamConfig(config); Set sourceStreams = getStreamSpecs(appDesc.getInputStreamIds(), streamConfig); Set sinkStreams = getStreamSpecs(appDesc.getOutputStreamIds(), streamConfig); @@ -123,8 +120,8 @@ JobGraph createJobGraph(Config config, ApplicationDescriptorImpl jobGraph.addTable(spec, node)); - if (!ApplicationUtil.isLegacyTaskApplication(appDesc)) { - // skip the validation when input streamIds are empty. This is only possible for LegacyApplication + if (!LegacyTaskApplication.class.isAssignableFrom(appDesc.getAppClass())) { + // skip the validation when input streamIds are empty. This is only possible for LegacyTaskApplication jobGraph.validate(); } @@ -136,7 +133,7 @@ JobGraph createJobGraph(Config config, ApplicationDescriptorImpl existingStreams = new HashSet<>(); existingStreams.addAll(jobGraph.getInputStreams()); existingStreams.addAll(jobGraph.getOutputStreams()); diff --git a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamManager.java similarity index 96% rename from samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java rename to samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamManager.java index e47f219939..b98aa8a41d 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamPartitionPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/IntermediateStreamManager.java @@ -44,11 +44,11 @@ /** - * {@link IntermediateStreamPartitionPlanner} calculates intermediate stream partitions based on the high-level application graph. + * {@link IntermediateStreamManager} calculates intermediate stream partitions based on the high-level application graph. */ -class IntermediateStreamPartitionPlanner { +class IntermediateStreamManager { - private static final Logger log = LoggerFactory.getLogger(IntermediateStreamPartitionPlanner.class); + private static final Logger log = LoggerFactory.getLogger(IntermediateStreamManager.class); private final Config config; private final Map inputOperators; @@ -56,7 +56,7 @@ class IntermediateStreamPartitionPlanner { @VisibleForTesting static final int MAX_INFERRED_PARTITIONS = 256; - IntermediateStreamPartitionPlanner(Config config, ApplicationDescriptorImpl appDesc) { + IntermediateStreamManager(Config config, ApplicationDescriptorImpl appDesc) { this.config = config; this.inputOperators = appDesc.getInputOperators(); } @@ -79,6 +79,7 @@ class IntermediateStreamPartitionPlanner { /** * Calculate the partitions for the input streams of join operators */ + @VisibleForTesting /* package private */ void calculateJoinInputPartitions(JobGraph jobGraph) { // mapping from a source stream to all join specs reachable from it Multimap joinSpecToStreamEdges = HashMultimap.create(); diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java index 089d8bd4f7..97bd51869b 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java @@ -61,7 +61,7 @@ private final Set tables = new HashSet<>(); private final Config config; private final JobGraphJsonGenerator jsonGenerator; - private final JobNodeConfigureGenerator configGenerator; + private final JobNodeConfigurationGenerator configGenerator; private final ApplicationDescriptorImpl appDesc; /** @@ -69,15 +69,12 @@ * * @param config configuration for the application * @param appDesc {@link ApplicationDescriptorImpl} describing the application - * @param jsonGenerator {@link JobGraphJsonGenerator} to create Json graph for the application - * @param configureGenerator configuration generator to create configuration for each {@link JobNode} */ - JobGraph(Config config, ApplicationDescriptorImpl appDesc, JobGraphJsonGenerator jsonGenerator, - JobNodeConfigureGenerator configureGenerator) { - this.jsonGenerator = jsonGenerator; - this.configGenerator = configureGenerator; + JobGraph(Config config, ApplicationDescriptorImpl appDesc) { this.config = config; this.appDesc = appDesc; + this.jsonGenerator = new JobGraphJsonGenerator(); + this.configGenerator = new JobNodeConfigurationGenerator(); } @Override @@ -165,7 +162,7 @@ void addTable(TableSpec tableSpec, JobNode node) { * @return {@link JobNode} created with {@code jobName} and {@code jobId} */ JobNode getOrCreateJobNode(String jobName, String jobId) { - String nodeId = JobNode.createId(jobName, jobId); + String nodeId = JobNode.createJobNameAndId(jobName, jobId); return nodes.computeIfAbsent(nodeId, k -> new JobNode(jobName, jobId, config, appDesc, configGenerator)); } @@ -306,7 +303,7 @@ private void validateReachability() { Set unreachable = new HashSet<>(nodes.values()); unreachable.removeAll(reachable); throw new IllegalArgumentException(String.format("Jobs %s cannot be reached from Sources.", - String.join(", ", unreachable.stream().map(JobNode::getId).collect(Collectors.toList())))); + String.join(", ", unreachable.stream().map(JobNode::getJobNameAndId).collect(Collectors.toList())))); } } @@ -352,7 +349,7 @@ List topologicalSort() { Map indegree = new HashMap<>(); Set visited = new HashSet<>(); pnodes.forEach(node -> { - String nid = node.getId(); + String nid = node.getJobNameAndId(); //only count the degrees of intermediate streams long degree = node.getInEdges().values().stream().filter(e -> !inputStreams.contains(e)).count(); indegree.put(nid, degree); @@ -380,7 +377,7 @@ List topologicalSort() { JobNode node = q.poll(); sortedNodes.add(node); node.getOutEdges().values().stream().flatMap(edge -> edge.getTargetNodes().stream()).forEach(n -> { - String nid = n.getId(); + String nid = n.getJobNameAndId(); Long degree = indegree.get(nid) - 1; indegree.put(nid, degree); if (degree == 0L && !visited.contains(n)) { @@ -401,7 +398,7 @@ List topologicalSort() { long min = Long.MAX_VALUE; JobNode minNode = null; for (JobNode node : reachable) { - Long degree = indegree.get(node.getId()); + Long degree = indegree.get(node.getJobNameAndId()); if (degree < min) { min = degree; minNode = node; diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java index 8f7c3e34dc..18705e4f41 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java @@ -116,9 +116,6 @@ static final class JobGraphJson { String applicationId; } - JobGraphJsonGenerator() { - } - /** * Returns the JSON representation of a {@link JobGraph} * @param jobGraph {@link JobGraph} diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java index 6cc2c95f1c..af556f5e1d 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java @@ -28,9 +28,10 @@ import java.util.stream.Collectors; import org.apache.samza.application.ApplicationDescriptor; import org.apache.samza.application.ApplicationDescriptorImpl; -import org.apache.samza.application.ApplicationUtil; +import org.apache.samza.application.LegacyTaskApplication; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; +import org.apache.samza.operators.KV; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.serializers.Serde; @@ -48,30 +49,32 @@ public class JobNode { private final String jobName; private final String jobId; - private final String id; + private final String jobNameAndId; private final Config config; - private final JobNodeConfigureGenerator configGenerator; + private final JobNodeConfigurationGenerator configGenerator; + // The following maps (i.e. inEdges and outEdges) uses the streamId as the key private final Map inEdges = new HashMap<>(); private final Map outEdges = new HashMap<>(); + // Similarly, tables uses tableId as the key private final Map tables = new HashMap<>(); private final ApplicationDescriptorImpl appDesc; JobNode(String jobName, String jobId, Config config, ApplicationDescriptorImpl appDesc, - JobNodeConfigureGenerator configureGenerator) { + JobNodeConfigurationGenerator configureGenerator) { this.jobName = jobName; this.jobId = jobId; - this.id = createId(jobName, jobId); + this.jobNameAndId = createJobNameAndId(jobName, jobId); this.config = config; this.appDesc = appDesc; this.configGenerator = configureGenerator; } - static String createId(String jobName, String jobId) { + static String createJobNameAndId(String jobName, String jobId) { return String.format("%s-%s", jobName, jobId); } - String getId() { - return id; + String getJobNameAndId() { + return jobNameAndId; } String getJobName() { @@ -119,12 +122,18 @@ JobConfig generateConfig(String executionPlanJson) { return configGenerator.generateJobConfig(this, executionPlanJson); } - Serde getInputSerde(String streamId) { - return appDesc.getInputSerde(streamId); + KV getInputSerdes(String streamId) { + if (!inEdges.containsKey(streamId)) { + return null; + } + return appDesc.getStreamSerdes(streamId); } - Serde getOutputSerde(String streamId) { - return appDesc.getOutputSerde(streamId); + KV getOutputSerde(String streamId) { + if (!outEdges.containsKey(streamId)) { + return null; + } + return appDesc.getStreamSerdes(streamId); } Collection getReachableOperators() { @@ -135,17 +144,6 @@ Collection getReachableOperators() { return reachableOperators; } - private void findReachableOperators(Collection inputOperatorsInJobNode, - Set reachableOperators) { - inputOperatorsInJobNode.forEach(op -> { - if (reachableOperators.contains(op)) { - return; - } - reachableOperators.add(op); - findReachableOperators(op.getRegisteredOperatorSpecs(), reachableOperators); - }); - } - // get all next operators consuming from the input {@code streamId} Set getNextOperatorIds(String streamId) { if (!appDesc.getInputOperators().containsKey(streamId) || !inEdges.containsKey(streamId)) { @@ -163,6 +161,22 @@ InputOperatorSpec getInputOperator(String inputStreamId) { } boolean isLegacyTaskApplication() { - return ApplicationUtil.isLegacyTaskApplication(appDesc); + return LegacyTaskApplication.class.isAssignableFrom(appDesc.getAppClass()); + } + + KV getTableSerdes(String tableId) { + //TODO: SAMZA-1893: should test whether the table is used in the current JobNode + return appDesc.getTableSerdes(tableId); + } + + private void findReachableOperators(Collection inputOperatorsInJobNode, + Set reachableOperators) { + inputOperatorsInJobNode.forEach(op -> { + if (reachableOperators.contains(op)) { + return; + } + reachableOperators.add(op); + findReachableOperators(op.getRegisteredOperatorSpecs(), reachableOperators); + }); } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigurationGenerator.java similarity index 77% rename from samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java rename to samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigurationGenerator.java index 11cb8354b3..676d28ebf1 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigureGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobNodeConfigurationGenerator.java @@ -31,6 +31,7 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.samza.config.Config; +import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.SerializerConfig; @@ -38,12 +39,13 @@ import org.apache.samza.config.StreamConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.config.TaskConfigJava; +import org.apache.samza.operators.KV; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.StatefulOperatorSpec; import org.apache.samza.operators.spec.StoreDescriptor; import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.NoOpSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerializableSerde; import org.apache.samza.table.TableConfigGenerator; @@ -58,21 +60,17 @@ /** * This class provides methods to generate configuration for a {@link JobNode} */ -/* package private */ class JobNodeConfigureGenerator { +/* package private */ class JobNodeConfigurationGenerator { - private static final Logger LOG = LoggerFactory.getLogger(JobNodeConfigureGenerator.class); + private static final Logger LOG = LoggerFactory.getLogger(JobNodeConfigurationGenerator.class); - static final String CONFIG_JOB_PREFIX = "jobs.%s."; static final String CONFIG_INTERNAL_EXECUTION_PLAN = "samza.internal.execution.plan"; - JobNodeConfigureGenerator() { - } - - static Config mergeJobConfig(Config originalConfig, Config generatedConfig) { + static JobConfig mergeJobConfig(Config originalConfig, Config generatedConfig) { JobConfig jobConfig = new JobConfig(originalConfig); - String jobId = JobNode.createId(jobConfig.getName().get(), jobConfig.getJobId()); + String jobNameAndId = JobNode.createJobNameAndId(jobConfig.getName().get(), jobConfig.getJobId()); return new JobConfig(Util.rewriteConfig(extractScopedConfig(originalConfig, generatedConfig, - String.format(CONFIG_JOB_PREFIX, jobId)))); + String.format(JobConfig.CONFIG_OVERRIDE_JOBS_PREFIX(), jobNameAndId)))); } JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { @@ -84,9 +82,8 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { Map inEdges = jobNode.getInEdges(); Map outEdges = jobNode.getOutEdges(); Collection reachableOperators = jobNode.getReachableOperators(); - boolean hasWindowOrJoin = reachableOperators.stream().anyMatch(op -> op.getOpCode() == OperatorSpec.OpCode.WINDOW - || op.getOpCode() == OperatorSpec.OpCode.JOIN); List stores = getStoreDescriptors(reachableOperators); + Map reachableTables = getReachableTables(reachableOperators, jobNode); Config config = jobNode.getConfig(); // check all inputs to the node for broadcast and input streams @@ -101,14 +98,10 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { } } - if (!broadcasts.isEmpty()) { - configureBroadcastInputs(configs, config, broadcasts); - } + configureBroadcastInputs(configs, config, broadcasts); // compute window and join operator intervals in this node - if (hasWindowOrJoin) { - configureWindowAndJoinInterval(configs, config, reachableOperators); - } + configureWindowInterval(configs, config, reachableOperators); // set store configuration for stateful operators. stores.forEach(sd -> configs.putAll(sd.getStorageConfigs())); @@ -119,33 +112,48 @@ JobConfig generateJobConfig(JobNode jobNode, String executionPlanJson) { // write intermediate input/output streams to configs inEdges.values().stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig())); - // write serialized serde instances and stream /store serdes to configs - configureSerdes(configs, inEdges, outEdges, stores, jobNode); + // write serialized serde instances and stream, store, and table serdes to configs + // serde configuration generation has to happen before table configuration, since the serde configuration + // is required when generating configurations for some TableProvider (i.e. local store backed tables) + configureSerdes(configs, inEdges, outEdges, stores, reachableTables.keySet(), jobNode); // generate table configuration and potential side input configuration - configureTables(configs, config, jobNode.getTables(), inputs); + configureTables(configs, config, reachableTables, inputs); // finalize the task.inputs configuration configs.put(TaskConfig.INPUT_STREAMS(), Joiner.on(',').join(inputs)); - LOG.info("Job {} has generated configs {}", jobNode.getId(), configs); + LOG.info("Job {} has generated configs {}", jobNode.getJobNameAndId(), configs); // apply configure rewriters and user configure overrides return applyConfigureRewritersAndOverrides(configs, config, jobNode); } - private void configureBroadcastInputs(Map configs, Config config, Set broadcasts) { - // TODO: remove this once we support defining broadcast input stream in high-level + private Map getReachableTables(Collection reachableOperators, JobNode jobNode) { + // TODO: Fix this in SAMZA-1893. For now, returning all tables for single-job execution plan + return jobNode.getTables(); + } + + private void configureBroadcastInputs(Map configs, Config config, Set broadcastStreams) { + // TODO: SAMZA-1841: remove this once we support defining broadcast input stream in high-level // task.broadcast.input should be generated by the planner in the future. + if (broadcastStreams.isEmpty()) { + return; + } final String taskBroadcasts = config.get(TaskConfigJava.BROADCAST_INPUT_STREAMS); if (StringUtils.isNoneEmpty(taskBroadcasts)) { - broadcasts.add(taskBroadcasts); + broadcastStreams.add(taskBroadcasts); } - configs.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcasts)); + configs.put(TaskConfigJava.BROADCAST_INPUT_STREAMS, Joiner.on(',').join(broadcastStreams)); } - private void configureWindowAndJoinInterval(Map configs, Config config, + private void configureWindowInterval(Map configs, Config config, Collection reachableOperators) { + if (!reachableOperators.stream().anyMatch(op -> op.getOpCode() == OperatorSpec.OpCode.WINDOW + || op.getOpCode() == OperatorSpec.OpCode.JOIN)) { + return; + } + // set triggering interval if a window or join is defined. Only applies to high-level applications if ("-1".equals(config.get(TaskConfig.WINDOW_MS(), "-1"))) { long triggerInterval = computeTriggerInterval(reachableOperators); @@ -155,21 +163,46 @@ private void configureWindowAndJoinInterval(Map configs, Config } } - private JobConfig applyConfigureRewritersAndOverrides(Map configs, Config config, JobNode jobNode) { - String configPrefix = String.format(CONFIG_JOB_PREFIX, jobNode.getId()); + /** + * Computes the triggering interval to use during the execution of this {@link JobNode} + */ + private long computeTriggerInterval(Collection reachableOperators) { + List windowTimerIntervals = reachableOperators.stream() + .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.WINDOW) + .map(spec -> ((WindowOperatorSpec) spec).getDefaultTriggerMs()) + .collect(Collectors.toList()); + // Filter out the join operators, and obtain a list of their ttl values + List joinTtlIntervals = reachableOperators.stream() + .filter(spec -> spec instanceof JoinOperatorSpec) + .map(spec -> ((JoinOperatorSpec) spec).getTtlMs()) + .collect(Collectors.toList()); + + // Combine both the above lists + List candidateTimerIntervals = new ArrayList<>(joinTtlIntervals); + candidateTimerIntervals.addAll(windowTimerIntervals); + + if (candidateTimerIntervals.isEmpty()) { + return -1; + } + + // Compute the gcd of the resultant list + return MathUtil.gcd(candidateTimerIntervals); + } + + private JobConfig applyConfigureRewritersAndOverrides(Map configs, Config config, JobNode jobNode) { // Disallow user specified job inputs/outputs. This info comes strictly from the user application. Map allowedConfigs = new HashMap<>(config); if (!jobNode.isLegacyTaskApplication()) { if (allowedConfigs.containsKey(TaskConfig.INPUT_STREAMS())) { - LOG.warn("Specifying task inputs in configuration is not allowed with Fluent API. " + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); + LOG.warn("Specifying task inputs in configuration is not allowed for SamzaApplication. " + + "Ignoring configured value for " + TaskConfig.INPUT_STREAMS()); allowedConfigs.remove(TaskConfig.INPUT_STREAMS()); } } - LOG.debug("Job {} has allowed configs {}", jobNode.getId(), allowedConfigs); - return new JobConfig(Util.rewriteConfig( - extractScopedConfig(new MapConfig(allowedConfigs), new MapConfig(configs), configPrefix))); + LOG.debug("Job {} has allowed configs {}", jobNode.getJobNameAndId(), allowedConfigs); + return mergeJobConfig(new MapConfig(allowedConfigs), new MapConfig(configs)); } /** @@ -239,14 +272,14 @@ private void configureTables(Map configs, Config config, Map configs, Map inEdges, Map outEdges, - List stores, JobNode jobNode) { + List stores, Collection tables, JobNode jobNode) { // collect all key and msg serde instances for streams Map streamKeySerdes = new HashMap<>(); Map streamMsgSerdes = new HashMap<>(); inEdges.keySet().forEach(streamId -> - addSerde(jobNode.getInputSerde(streamId), streamId, streamKeySerdes, streamMsgSerdes)); + addSerdes(jobNode.getInputSerdes(streamId), streamId, streamKeySerdes, streamMsgSerdes)); outEdges.keySet().forEach(streamId -> - addSerde(jobNode.getOutputSerde(streamId), streamId, streamKeySerdes, streamMsgSerdes)); + addSerdes(jobNode.getOutputSerde(streamId), streamId, streamKeySerdes, streamMsgSerdes)); Map storeKeySerdes = new HashMap<>(); Map storeMsgSerdes = new HashMap<>(); @@ -255,11 +288,19 @@ private void configureSerdes(Map configs, Map tableKeySerdes = new HashMap<>(); + Map tableMsgSerdes = new HashMap<>(); + tables.forEach(tableId -> { + addSerdes(jobNode.getTableSerdes(tableId), tableId, tableKeySerdes, tableMsgSerdes); + }); + // for each unique stream or store serde instance, generate a unique name and serialize to config HashSet serdes = new HashSet<>(streamKeySerdes.values()); serdes.addAll(streamMsgSerdes.values()); serdes.addAll(storeKeySerdes.values()); serdes.addAll(storeMsgSerdes.values()); + serdes.addAll(tableKeySerdes.values()); + serdes.addAll(tableMsgSerdes.values()); SerializableSerde serializableSerde = new SerializableSerde<>(); Base64.Encoder base64Encoder = Base64.getEncoder(); Map serdeUUIDs = new HashMap<>(); @@ -293,47 +334,27 @@ private void configureSerdes(Map configs, Map reachableOperators) { - List windowTimerIntervals = reachableOperators.stream() - .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.WINDOW) - .map(spec -> ((WindowOperatorSpec) spec).getDefaultTriggerMs()) - .collect(Collectors.toList()); - - // Filter out the join operators, and obtain a list of their ttl values - List joinTtlIntervals = reachableOperators.stream() - .filter(spec -> spec instanceof JoinOperatorSpec) - .map(spec -> ((JoinOperatorSpec) spec).getTtlMs()) - .collect(Collectors.toList()); - // Combine both the above lists - List candidateTimerIntervals = new ArrayList<>(joinTtlIntervals); - candidateTimerIntervals.addAll(windowTimerIntervals); - - if (candidateTimerIntervals.isEmpty()) { - return -1; - } + // set key and msg serdes for stores to the serde names generated above + tableKeySerdes.forEach((tableId, serde) -> { + String keySerdeConfigKey = String.format(JavaTableConfig.TABLE_KEY_SERDE, tableId); + configs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); + }); - // Compute the gcd of the resultant list - return MathUtil.gcd(candidateTimerIntervals); + tableMsgSerdes.forEach((tableId, serde) -> { + String valueSerdeConfigKey = String.format(JavaTableConfig.TABLE_VALUE_SERDE, tableId); + configs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); + }); } - private void addSerde(Serde serde, String streamId, Map keySerdeMap, Map msgSerdeMap) { - if (serde != null) { - if (serde instanceof KVSerde) { - KVSerde kvSerde = (KVSerde) serde; - if (kvSerde.getKeySerde() != null) { - keySerdeMap.put(streamId, ((KVSerde) serde).getKeySerde()); - } - if (kvSerde.getValueSerde() != null) { - msgSerdeMap.put(streamId, ((KVSerde) serde).getValueSerde()); - } - } else { - msgSerdeMap.put(streamId, serde); + private void addSerdes(KV serdes, String streamId, Map keySerdeMap, + Map msgSerdeMap) { + if (serdes != null) { + if (serdes.getKey() != null && !(serdes.getKey() instanceof NoOpSerde)) { + keySerdeMap.put(streamId, serdes.getKey()); + } + if (serdes.getValue() != null && !(serdes.getValue() instanceof NoOpSerde)) { + msgSerdeMap.put(streamId, serdes.getValue()); } } } diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java index faf29b2d2a..abbec18c68 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/JobPlanner.java @@ -55,7 +55,7 @@ public abstract class JobPlanner { this.config = descriptor.getConfig(); } - public abstract List prepareJobs() throws Exception; + public abstract List prepareJobs(); StreamManager buildAndStartStreamManager(Config config) { StreamManager streamManager = new StreamManager(config); @@ -93,7 +93,7 @@ ExecutionPlan getExecutionPlan(String runId) { // create the physical execution plan and merge with overrides. This works for a single-stage job now // TODO: This should all be consolidated with ExecutionPlanner after fixing SAMZA-1811 - Config mergedConfig = JobNodeConfigureGenerator.mergeJobConfig(config, new MapConfig(cfg)); + Config mergedConfig = JobNodeConfigurationGenerator.mergeJobConfig(config, new MapConfig(cfg)); // creating the StreamManager to get all input/output streams' metadata for planning StreamManager streamManager = buildAndStartStreamManager(mergedConfig); try { diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java index e8700a1aab..86aca0fa39 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -52,12 +52,17 @@ public LocalJobPlanner(ApplicationDescriptorImpl prepareJobs() throws Exception { + public List prepareJobs() { // for high-level DAG, generating the plan and job configs // 1. initialize and plan ExecutionPlan plan = getExecutionPlan(); - String executionPlanJson = plan.getPlanAsJson(); + String executionPlanJson = ""; + try { + executionPlanJson = plan.getPlanAsJson(); + } catch (Exception e) { + throw new SamzaException("Failed to create plan JSON.", e); + } writePlanJsonFile(executionPlanJson); LOG.info("Execution Plan: \n" + executionPlanJson); String planId = String.valueOf(executionPlanJson.hashCode()); diff --git a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java index 599aaeed35..54f86d5989 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/RemoteJobPlanner.java @@ -46,7 +46,7 @@ public RemoteJobPlanner(ApplicationDescriptorImpl prepareJobs() throws Exception { + public List prepareJobs() { // for high-level DAG, generate the plan and job configs // TODO: run.id needs to be set for standalone: SAMZA-1531 // run.id is based on current system time with the most significant bits in UUID (8 digits) to avoid collision @@ -55,7 +55,11 @@ public List prepareJobs() throws Exception { // 1. initialize and plan ExecutionPlan plan = getExecutionPlan(runId); - writePlanJsonFile(plan.getPlanAsJson()); + try { + writePlanJsonFile(plan.getPlanAsJson()); + } catch (Exception e) { + throw new SamzaException("Failed to create plan JSON.", e); + } List jobConfigs = plan.getJobConfigs(); if (jobConfigs.isEmpty()) { diff --git a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java index f81f3b8717..c31cecf7f9 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java +++ b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java @@ -71,6 +71,15 @@ public String getTableId() { return tableId; } + /** + * Get the serde assigned to this {@link TableDescriptor} + * + * @return {@link KVSerde} used by this table + */ + public KVSerde getSerde() { + return serde; + } + /** * Generate config for {@link TableSpec}; this method is used internally. * @param tableSpecConfig configuration for the {@link TableSpec} diff --git a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java index b75b1e8538..5329fd7557 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java @@ -30,7 +30,6 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputStreamImpl; import org.apache.samza.serializers.SerializableSerde; -import org.apache.samza.table.TableSpec; /** @@ -45,7 +44,6 @@ public class OperatorSpecGraph implements Serializable { private final Map inputOperators; private final Map outputStreams; private final Set broadcastStreams; - private final Map tables; private final Set allOpSpecs; private final boolean hasWindowOrJoins; @@ -57,7 +55,6 @@ public OperatorSpecGraph(StreamApplicationDescriptorImpl streamAppDesc) { this.inputOperators = streamAppDesc.getInputOperators(); this.outputStreams = streamAppDesc.getOutputStreams(); this.broadcastStreams = streamAppDesc.getBroadcastStreams(); - this.tables = streamAppDesc.getTables(); this.allOpSpecs = Collections.unmodifiableSet(this.findAllOperatorSpecs()); this.hasWindowOrJoins = checkWindowOrJoins(); this.serializedOpSpecGraph = opSpecGraphSerde.toBytes(this); @@ -75,10 +72,6 @@ public Set getBroadcastStreams() { return broadcastStreams; } - public Map getTables() { - return tables; - } - /** * Get all {@link OperatorSpec}s available in this {@link StreamApplicationDescriptorImpl} * diff --git a/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java b/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java index 085131c6cd..03be758f10 100644 --- a/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java +++ b/samza-core/src/main/java/org/apache/samza/table/TableConfigGenerator.java @@ -20,22 +20,16 @@ package org.apache.samza.table; import java.util.ArrayList; -import java.util.Base64; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.UUID; import org.apache.samza.config.Config; import org.apache.samza.config.JavaTableConfig; -import org.apache.samza.config.SerializerConfig; import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.TableImpl; -import org.apache.samza.serializers.Serde; -import org.apache.samza.serializers.SerializableSerde; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,8 +60,6 @@ static public Map generateConfigsForTableDescs(Config config, Li static public Map generateConfigsForTableSpecs(Config config, List tableSpecs) { Map tableConfigs = new HashMap<>(); - tableConfigs.putAll(generateTableKVSerdeConfigs(tableSpecs)); - tableSpecs.forEach(tableSpec -> { // Add table provider factory config tableConfigs.put(String.format(JavaTableConfig.TABLE_PROVIDER_FACTORY, tableSpec.getId()), @@ -103,44 +95,4 @@ static public List getTableSpecs(List tableDescs) { }); return new ArrayList<>(tableSpecs.keySet()); } - - static private Map generateTableKVSerdeConfigs(List tableSpecs) { - Map serdeConfigs = new HashMap<>(); - - // Collect key and msg serde instances for all the tables - Map tableKeySerdes = new HashMap<>(); - Map tableValueSerdes = new HashMap<>(); - HashSet serdes = new HashSet<>(); - - tableSpecs.forEach(tableSpec -> { - tableKeySerdes.put(tableSpec.getId(), tableSpec.getSerde().getKeySerde()); - tableValueSerdes.put(tableSpec.getId(), tableSpec.getSerde().getValueSerde()); - }); - serdes.addAll(tableKeySerdes.values()); - serdes.addAll(tableValueSerdes.values()); - - // Generate serde names - SerializableSerde serializableSerde = new SerializableSerde<>(); - Base64.Encoder base64Encoder = Base64.getEncoder(); - Map serdeUUIDs = new HashMap<>(); - serdes.forEach(serde -> { - String serdeName = serdeUUIDs.computeIfAbsent(serde, - s -> serde.getClass().getSimpleName() + "-" + UUID.randomUUID().toString()); - serdeConfigs.putIfAbsent(String.format(SerializerConfig.SERDE_SERIALIZED_INSTANCE(), serdeName), - base64Encoder.encodeToString(serializableSerde.toBytes(serde))); - }); - - // Set key and msg serdes for tables to the serde names generated above - tableKeySerdes.forEach((tableId, serde) -> { - String keySerdeConfigKey = String.format(JavaTableConfig.TABLE_KEY_SERDE, tableId); - serdeConfigs.put(keySerdeConfigKey, serdeUUIDs.get(serde)); - }); - - tableValueSerdes.forEach((tableId, serde) -> { - String valueSerdeConfigKey = String.format(JavaTableConfig.TABLE_VALUE_SERDE, tableId); - serdeConfigs.put(valueSerdeConfigKey, serdeUUIDs.get(serde)); - }); - - return serdeConfigs; - } } diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index c73cf14a92..d7b71b5195 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -39,7 +39,7 @@ object JobConfig { */ val CONFIG_REWRITERS = "job.config.rewriters" // streaming.job_config_rewriters val CONFIG_REWRITER_CLASS = "job.config.rewriter.%s.class" // streaming.job_config_rewriter_class - regex, system, config - val CONFIG_JOB_PREFIX = "jobs.%s." + val CONFIG_OVERRIDE_JOBS_PREFIX = "jobs.%s." val JOB_NAME = "job.name" // streaming.job_name val JOB_ID = "job.id" // streaming.job_id val SAMZA_FWK_PATH = "samza.fwk.path" diff --git a/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java index db85e3335e..1fe602308d 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestStreamApplicationDescriptorImpl.java @@ -522,10 +522,11 @@ public void testGetTable() throws Exception { TableSpec testTableSpec = new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()); when(mockTableDescriptor.getTableSpec()).thenReturn(testTableSpec); when(mockTableDescriptor.getTableId()).thenReturn(testTableSpec.getId()); + when(mockTableDescriptor.getSerde()).thenReturn(testTableSpec.getSerde()); StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { appDesc.getTable(mockTableDescriptor); }, mockConfig); - assertNotNull(streamAppDesc.getTables().get(testTableSpec)); + assertNotNull(streamAppDesc.getTables().get(testTableSpec.getId())); } @Test diff --git a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java index 9418c1f056..abe5ce1249 100644 --- a/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/application/TestTaskApplicationDescriptorImpl.java @@ -23,12 +23,14 @@ import java.util.List; import java.util.Set; import org.apache.samza.config.Config; +import org.apache.samza.operators.BaseTableDescriptor; import org.apache.samza.operators.ContextManager; import org.apache.samza.operators.TableDescriptor; import org.apache.samza.operators.descriptors.base.stream.InputDescriptor; import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor; import org.apache.samza.operators.descriptors.base.system.SystemDescriptor; import org.apache.samza.runtime.ProcessorLifecycleListenerFactory; +import org.apache.samza.serializers.KVSerde; import org.apache.samza.task.TaskFactory; import org.junit.Before; import org.junit.Test; @@ -64,10 +66,12 @@ public class TestTaskApplicationDescriptorImpl { this.add(mock2); } }; private Set mockTables = new HashSet() { { - TableDescriptor mock1 = mock(TableDescriptor.class); - TableDescriptor mock2 = mock(TableDescriptor.class); + BaseTableDescriptor mock1 = mock(BaseTableDescriptor.class); + BaseTableDescriptor mock2 = mock(BaseTableDescriptor.class); when(mock1.getTableId()).thenReturn("test-table1"); when(mock2.getTableId()).thenReturn("test-table2"); + when(mock1.getSerde()).thenReturn(mock(KVSerde.class)); + when(mock2.getSerde()).thenReturn(mock(KVSerde.class)); this.add(mock1); this.add(mock2); } }; diff --git a/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java b/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java index bdec3754bc..f507c70827 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java +++ b/samza-core/src/test/java/org/apache/samza/execution/ExecutionPlannerTestBase.java @@ -101,8 +101,8 @@ String getJobNameAndId() { } void configureJobNode(ApplicationDescriptorImpl mockStreamAppDesc) { - JobGraph jobGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, - mockStreamAppDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + JobGraph jobGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)) + .createJobGraph(mockConfig, mockStreamAppDesc); mockJobNode = spy(jobGraph.getJobNodes().get(0)); } 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 3dd97af544..944b193f44 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 @@ -285,7 +285,7 @@ public void setup() { Map system1Map = new HashMap<>(); system1Map.put("input1", 64); system1Map.put("output1", 8); - system1Map.put("input4", IntermediateStreamPartitionPlanner.MAX_INFERRED_PARTITIONS * 2); + system1Map.put("input4", IntermediateStreamManager.MAX_INFERRED_PARTITIONS * 2); Map system2Map = new HashMap<>(); system2Map.put("input2", 16); system2Map.put("input3", 32); @@ -304,8 +304,7 @@ public void testCreateProcessorGraph() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), - new JobNodeConfigureGenerator()); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec); assertTrue(jobGraph.getInputStreams().size() == 3); assertTrue(jobGraph.getOutputStreams().size() == 2); assertTrue(jobGraph.getIntermediateStreams().size() == 2); // two streams generated by partitionBy @@ -315,10 +314,9 @@ public void testCreateProcessorGraph() { public void testFetchExistingStreamPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), - new JobNodeConfigureGenerator()); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec); - ExecutionPlanner.fetchInputAndOutputStreamPartitions(jobGraph, streamManager); + ExecutionPlanner.setInputAndOutputStreamPartitionCount(jobGraph, streamManager); assertTrue(jobGraph.getOrCreateStreamEdge(input1Spec).getPartitionCount() == 64); assertTrue(jobGraph.getOrCreateStreamEdge(input2Spec).getPartitionCount() == 16); assertTrue(jobGraph.getOrCreateStreamEdge(input3Spec).getPartitionCount() == 32); @@ -334,11 +332,10 @@ public void testFetchExistingStreamPartitions() { public void testCalculateJoinInputPartitions() { ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = createStreamGraphWithJoin(); - JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec, new JobGraphJsonGenerator(), - new JobNodeConfigureGenerator()); + JobGraph jobGraph = planner.createJobGraph(graphSpec.getConfig(), graphSpec); - ExecutionPlanner.fetchInputAndOutputStreamPartitions(jobGraph, streamManager); - new IntermediateStreamPartitionPlanner(config, graphSpec).calculateJoinInputPartitions(jobGraph); + ExecutionPlanner.setInputAndOutputStreamPartitionCount(jobGraph, streamManager); + new IntermediateStreamManager(config, graphSpec).calculateJoinInputPartitions(jobGraph); // the partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { @@ -472,15 +469,15 @@ public void testMaxPartition() { edge.setPartitionCount(16); edges.add(edge); - assertEquals(32, IntermediateStreamPartitionPlanner.maxPartition(edges)); + assertEquals(32, IntermediateStreamManager.maxPartition(edges)); edges = Collections.emptyList(); - assertEquals(StreamEdge.PARTITIONS_UNKNOWN, IntermediateStreamPartitionPlanner.maxPartition(edges)); + assertEquals(StreamEdge.PARTITIONS_UNKNOWN, IntermediateStreamManager.maxPartition(edges)); } @Test public void testMaxPartitionLimit() throws Exception { - int partitionLimit = IntermediateStreamPartitionPlanner.MAX_INFERRED_PARTITIONS; + int partitionLimit = IntermediateStreamManager.MAX_INFERRED_PARTITIONS; ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); StreamApplicationDescriptorImpl graphSpec = new StreamApplicationDescriptorImpl(appDesc -> { @@ -535,8 +532,7 @@ public void testCreateJobGraphForTaskApplication() { systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc, new JobGraphJsonGenerator(), - new JobNodeConfigureGenerator()); + JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc); assertEquals(1, jobGraph.getJobNodes().size()); assertTrue(jobGraph.getInputStreams().stream().map(edge -> edge.getName()) .filter(streamId -> inputDescriptors.containsKey(streamId)).collect(Collectors.toList()).isEmpty()); @@ -567,12 +563,11 @@ public void testCreateJobGraphForLegacyTaskApplication() { systemDescriptors.forEach(sd -> systemStreamConfigs.putAll(sd.toConfig())); ExecutionPlanner planner = new ExecutionPlanner(config, streamManager); - JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc, new JobGraphJsonGenerator(), - new JobNodeConfigureGenerator()); + JobGraph jobGraph = planner.createJobGraph(config, taskAppDesc); assertEquals(1, jobGraph.getJobNodes().size()); JobNode jobNode = jobGraph.getJobNodes().get(0); assertEquals("test-app", jobNode.getJobName()); - assertEquals("test-app-1", jobNode.getId()); + assertEquals("test-app-1", jobNode.getJobNameAndId()); assertEquals(0, jobNode.getInEdges().size()); assertEquals(0, jobNode.getOutEdges().size()); assertEquals(0, jobNode.getTables().size()); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamManager.java similarity index 79% rename from samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java rename to samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamManager.java index 94107aaac0..bc1570976e 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamPartitionPlanner.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestIntermediateStreamManager.java @@ -25,16 +25,16 @@ import static org.mockito.Mockito.mock; /** - * Unit tests for {@link IntermediateStreamPartitionPlanner} + * Unit tests for {@link IntermediateStreamManager} */ -public class TestIntermediateStreamPartitionPlanner extends ExecutionPlannerTestBase { +public class TestIntermediateStreamManager extends ExecutionPlannerTestBase { @Test public void testCalculateRepartitionJoinTopicPartitions() { mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); - IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mockConfig, mockStreamAppDesc); - JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, - mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + IntermediateStreamManager partitionPlanner = new IntermediateStreamManager(mockConfig, mockStreamAppDesc); + JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)) + .createJobGraph(mockConfig, mockStreamAppDesc); // set the input stream partitions mockGraph.getInputStreams().forEach(inEdge -> { if (inEdge.getStreamSpec().getId().equals(input1Descriptor.getStreamId())) { @@ -53,9 +53,9 @@ public void testCalculateRepartitionJoinTopicPartitions() { @Test public void testCalculateRepartitionIntermediateTopicPartitions() { mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionOnlyStreamApplication(), mockConfig); - IntermediateStreamPartitionPlanner partitionPlanner = new IntermediateStreamPartitionPlanner(mockConfig, mockStreamAppDesc); - JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)).createJobGraph(mockConfig, mockStreamAppDesc, - mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + IntermediateStreamManager partitionPlanner = new IntermediateStreamManager(mockConfig, mockStreamAppDesc); + JobGraph mockGraph = new ExecutionPlanner(mockConfig, mock(StreamManager.class)) + .createJobGraph(mockConfig, mockStreamAppDesc); // set the input stream partitions mockGraph.getInputStreams().forEach(inEdge -> inEdge.setPartitionCount(7)); partitionPlanner.calculatePartitions(mockGraph); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java index 5e45f81021..4de0485a8a 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java @@ -60,7 +60,7 @@ private StreamSpec genStream() { */ private void createGraph1() { StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); - graph1 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + graph1 = new JobGraph(null, appDesc); JobNode n2 = graph1.getOrCreateJobNode("2", "1"); JobNode n3 = graph1.getOrCreateJobNode("3", "1"); @@ -94,7 +94,7 @@ private void createGraph1() { */ private void createGraph2() { StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); - graph2 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + graph2 = new JobGraph(null, appDesc); JobNode n1 = graph2.getOrCreateJobNode("1", "1"); JobNode n2 = graph2.getOrCreateJobNode("2", "1"); @@ -122,7 +122,7 @@ private void createGraph2() { */ private void createGraph3() { StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); - graph3 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + graph3 = new JobGraph(null, appDesc); JobNode n1 = graph3.getOrCreateJobNode("1", "1"); JobNode n2 = graph3.getOrCreateJobNode("2", "1"); @@ -139,7 +139,7 @@ private void createGraph3() { */ private void createGraph4() { StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); - graph4 = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + graph4 = new JobGraph(null, appDesc); JobNode n1 = graph4.getOrCreateJobNode("1", "1"); @@ -158,7 +158,7 @@ public void setup() { @Test public void testAddSource() { StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); - JobGraph graph = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + JobGraph graph = new JobGraph(null, appDesc); /** * s1 -> 1 @@ -200,7 +200,7 @@ public void testAddSink() { * 2 -> s3 */ StreamApplicationDescriptorImpl appDesc = mock(StreamApplicationDescriptorImpl.class); - JobGraph graph = new JobGraph(null, appDesc, mock(JobGraphJsonGenerator.class), mock(JobNodeConfigureGenerator.class)); + JobGraph graph = new JobGraph(null, appDesc); JobNode n1 = graph.getOrCreateJobNode("1", "1"); JobNode n2 = graph.getOrCreateJobNode("2", "1"); StreamSpec s1 = genStream(); diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java index 69ac9ccc09..c207118f96 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java @@ -116,7 +116,7 @@ public void setUp() { when(mockJobNode.getConfig()).thenReturn(mockConfig); when(mockJobNode.getJobName()).thenReturn("jobName"); when(mockJobNode.getJobId()).thenReturn("jobId"); - when(mockJobNode.getId()).thenReturn(JobNode.createId("jobName", "jobId")); + when(mockJobNode.getJobNameAndId()).thenReturn(JobNode.createJobNameAndId("jobName", "jobId")); } @Test diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java similarity index 93% rename from samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java rename to samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java index 0da77331b8..f351c4472b 100644 --- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigureGenerator.java +++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNodeConfigurationGenerator.java @@ -63,16 +63,16 @@ /** - * Unit test for {@link JobNodeConfigureGenerator} + * Unit test for {@link JobNodeConfigurationGenerator} */ -public class TestJobNodeConfigureGenerator extends ExecutionPlannerTestBase { +public class TestJobNodeConfigurationGenerator extends ExecutionPlannerTestBase { @Test public void testConfigureSerdesWithRepartitionJoinApplication() { mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -92,7 +92,7 @@ public void testConfigureSerdesForRepartitionWithNoDefaultSystem() { configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -116,7 +116,7 @@ public void testGenerateJobConfigWithTaskApplication() { TaskApplicationDescriptorImpl taskAppDesc = new TaskApplicationDescriptorImpl(getTaskApplication(), mockConfig); configureJobNode(taskAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results @@ -133,7 +133,7 @@ public void testGenerateJobConfigWithLegacyTaskApplication() { Map originConfig = new HashMap<>(mockConfig); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); // jobConfig should be exactly the same as original config Map generatedConfig = new HashMap<>(jobConfig); @@ -147,7 +147,7 @@ public void testBroadcastStreamApplication() { configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); @@ -163,7 +163,7 @@ public void testBroadcastStreamApplicationWithoutSerde() { configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); @@ -193,6 +193,7 @@ public void testStreamApplicationWithTableAndSideInput() { when(mockTableSpec.getSideInputs()).thenReturn(sideInputs); when(mockTableDescriptor.getTableId()).thenReturn("testTable"); when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); + when(mockTableDescriptor.getSerde()).thenReturn(defaultSerde); // add side input and terminate at table in the appplication mockStreamAppDesc.getInputStream(sideInput1).sendTo(mockStreamAppDesc.getTable(mockTableDescriptor)); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", @@ -207,11 +208,11 @@ public void testStreamApplicationWithTableAndSideInput() { configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); - Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 7); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 5); validateTableConfigure(jobConfig, deserializedSerdes, mockTableDescriptor); } @@ -229,6 +230,7 @@ public void testTaskApplicationWithTableAndSideInput() { when(mockTableSpec.getSideInputs()).thenReturn(sideInputs); when(mockTableDescriptor.getTableId()).thenReturn("testTable"); when(mockTableDescriptor.getTableSpec()).thenReturn(mockTableSpec); + when(mockTableDescriptor.getSerde()).thenReturn(defaultSerde); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", inputSystemDescriptor.getSystemName()), false, false, mockConfig); // need to put the sideInput related stream configuration to the original config @@ -247,13 +249,13 @@ public void testTaskApplicationWithTableAndSideInput() { configureJobNode(taskAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); // Verify the results Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); - Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 4); + Map deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); validateStreamConfigures(jobConfig, deserializedSerdes); validateTableConfigure(jobConfig, deserializedSerdes, mockTableDescriptor); } @@ -267,7 +269,7 @@ public void testTaskInputsRemovedFromOriginalConfig() { mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(defaultSerde), mockConfig); configureJobNode(mockStreamAppDesc); - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); @@ -282,7 +284,7 @@ public void testTaskInputsRetainedForLegacyTaskApplication() { configureJobNode(taskAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, ""); // jobConfig should be exactly the same as original config Map generatedConfig = new HashMap<>(jobConfig); @@ -293,13 +295,13 @@ public void testTaskInputsRetainedForLegacyTaskApplication() { public void testOverrideConfigs() { Map configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); - String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, getJobNameAndId()) + streamCfgToOverride; + String overrideCfgKey = String.format(JobConfig.CONFIG_OVERRIDE_JOBS_PREFIX(), getJobNameAndId()) + streamCfgToOverride; configs.put(overrideCfgKey, "customized-system"); mockConfig = spy(new MapConfig(configs)); mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); configureJobNode(mockStreamAppDesc); - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); @@ -310,7 +312,7 @@ public void testOverrideConfigs() { public void testConfigureRewriter() { Map configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); - String overrideCfgKey = String.format(JobNodeConfigureGenerator.CONFIG_JOB_PREFIX, getJobNameAndId()) + streamCfgToOverride; + String overrideCfgKey = String.format(JobConfig.CONFIG_OVERRIDE_JOBS_PREFIX(), getJobNameAndId()) + streamCfgToOverride; configs.put(overrideCfgKey, "customized-system"); configs.put(String.format(JobConfig.CONFIG_REWRITER_CLASS(), "mock"), MockConfigRewriter.class.getName()); configs.put(JobConfig.CONFIG_REWRITERS(), "mock"); @@ -319,7 +321,7 @@ public void testConfigureRewriter() { mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); configureJobNode(mockStreamAppDesc); - JobNodeConfigureGenerator configureGenerator = new JobNodeConfigureGenerator(); + JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); @@ -370,7 +372,7 @@ private Map validateAndGetDeserializedSerdes(Config jobConfig, in private void validateJobConfig(Config expectedConfig, JobConfig jobConfig) { assertEquals(expectedConfig.get(JobConfig.JOB_NAME()), jobConfig.getName().get()); assertEquals(expectedConfig.get(JobConfig.JOB_ID()), jobConfig.getJobId()); - assertEquals("testJobGraphJson", jobConfig.get(JobNodeConfigureGenerator.CONFIG_INTERNAL_EXECUTION_PLAN)); + assertEquals("testJobGraphJson", jobConfig.get(JobNodeConfigurationGenerator.CONFIG_INTERNAL_EXECUTION_PLAN)); assertEquals(expectedConfig.get(TaskConfig.INPUT_STREAMS()), jobConfig.get(TaskConfig.INPUT_STREAMS())); assertEquals(expectedConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS), jobConfig.get(TaskConfigJava.BROADCAST_INPUT_STREAMS)); } diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java index a5b15b8b1e..57ae6d87c6 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java @@ -117,7 +117,6 @@ public void testConstructor() { OperatorSpecGraph specGraph = new OperatorSpecGraph(mockAppDesc); assertEquals(specGraph.getInputOperators(), inputOpSpecMap); assertEquals(specGraph.getOutputStreams(), outputStrmMap); - assertTrue(specGraph.getTables().isEmpty()); assertTrue(!specGraph.hasWindowOrJoins()); assertEquals(specGraph.getAllOperatorSpecs(), this.allOpSpecs); } diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java b/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java index 7704a5b034..a34fdc386e 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java +++ b/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java @@ -53,7 +53,6 @@ enum TestEnum { public static void assertClonedGraph(OperatorSpecGraph originalGraph, OperatorSpecGraph clonedGraph) { assertClonedInputs(originalGraph.getInputOperators(), clonedGraph.getInputOperators()); assertClonedOutputs(originalGraph.getOutputStreams(), clonedGraph.getOutputStreams()); - assertClonedTables(originalGraph.getTables(), clonedGraph.getTables()); assertAllOperators(originalGraph.getAllOperatorSpecs(), clonedGraph.getAllOperatorSpecs()); } diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java index 07f4f55882..823190581f 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java @@ -89,10 +89,15 @@ protected Map generateCommonStoreConfig(Config jobConfig, Map storeConfig = new HashMap<>(); - // We assume the configuration for serde are already generated for this table, - // so we simply carry them over to store configuration. - // - JavaTableConfig tableConfig = new JavaTableConfig(new MapConfig(generatedConfig)); + // serde configurations for tables are generated at top level by JobNodeConfigurationGenerator and are included + // in the global jobConfig. generatedConfig has all table specific configuration generated from TableSpec, such + // as TableProviderFactory, sideInputs, etc. + // Merge the global jobConfig and generatedConfig to get full access to configuration needed to create local + // store configuration + Map mergedConfigMap = new HashMap<>(jobConfig); + mergedConfigMap.putAll(generatedConfig); + JobConfig mergedJobConfig = new JobConfig(new MapConfig(mergedConfigMap)); + JavaTableConfig tableConfig = new JavaTableConfig(mergedJobConfig); String keySerde = tableConfig.getKeySerde(tableSpec.getId()); storeConfig.put(String.format(StorageConfig.KEY_SERDE(), tableSpec.getId()), keySerde); @@ -116,9 +121,7 @@ protected Map generateCommonStoreConfig(Config jobConfig, Map(); systemConfigs.put(String.format(SYSTEM_FACTORY, systemName), InMemorySystemFactory.class.getName()); - systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_FACTORY, jobNameAndId, systemName), InMemorySystemFactory.class.getName()); + systemConfigs.put(String.format(JobConfig.CONFIG_OVERRIDE_JOBS_PREFIX() + SYSTEM_FACTORY, jobNameAndId, systemName), InMemorySystemFactory.class.getName()); systemConfigs.put(String.format(SYSTEM_OFFSET, systemName), "oldest"); - systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_OFFSET, jobNameAndId, systemName), "oldest"); + systemConfigs.put(String.format(JobConfig.CONFIG_OVERRIDE_JOBS_PREFIX() + SYSTEM_OFFSET, jobNameAndId, systemName), "oldest"); } public String getSystemName() { diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java b/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java index 41b6509fc9..e1e62409ca 100644 --- a/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java +++ b/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java @@ -94,8 +94,6 @@ public void testWithTableDescriptorsProviderClass() throws Exception { Assert.assertEquals(storageConfig.getStoreNames().get(0), localTableId); Assert.assertEquals(storageConfig.getStorageFactoryClassName(localTableId), RocksDbKeyValueStorageEngineFactory.class.getName()); - Assert.assertTrue(storageConfig.getStorageKeySerde(localTableId).startsWith("StringSerde")); - Assert.assertTrue(storageConfig.getStorageMsgSerde(localTableId).startsWith("StringSerde")); Config storeConfig = resultConfig.subset("stores." + localTableId + ".", true); Assert.assertEquals(4, storeConfig.size()); Assert.assertEquals(4096, storeConfig.getInt("rocksdb.block.size.bytes")); @@ -105,10 +103,6 @@ public void testWithTableDescriptorsProviderClass() throws Exception { RocksDbTableProviderFactory.class.getName()); Assert.assertEquals(tableConfig.getTableProviderFactory(remoteTableId), RemoteTableProviderFactory.class.getName()); - Assert.assertTrue(tableConfig.getKeySerde(localTableId).startsWith("StringSerde")); - Assert.assertTrue(tableConfig.getValueSerde(localTableId).startsWith("StringSerde")); - Assert.assertTrue(tableConfig.getKeySerde(remoteTableId).startsWith("StringSerde")); - Assert.assertTrue(tableConfig.getValueSerde(remoteTableId).startsWith("LongSerde")); Assert.assertEquals(tableConfig.getTableProviderFactory(localTableId), RocksDbTableProviderFactory.class.getName()); Assert.assertEquals(tableConfig.getTableProviderFactory(remoteTableId), RemoteTableProviderFactory.class.getName()); } From f8c8108ace46bb474a62421ed4590a801eba086a Mon Sep 17 00:00:00 2001 From: "Yi Pan (Data Infrastructure)" Date: Fri, 21 Sep 2018 00:28:08 -0700 Subject: [PATCH 38/38] SAMZA-1814: Fix merging errors. --- .../org/apache/samza/test/framework/TestRunner.java | 13 ++++++++++++- .../framework/system/InMemorySystemDescriptor.java | 10 +--------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java index b5eabaccc7..6d6613c98a 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java @@ -154,6 +154,17 @@ public TestRunner addConfigs(Map config) { return this; } + /** + * Only adds a config from {@code config} to samza job {@code configs} if they dont exist in it. + * @param config configs for the application + * @return this {@link TestRunner} + */ + public TestRunner addConfigs(Map config, String configPrefix) { + Preconditions.checkNotNull(config); + config.forEach((key, value) -> this.configs.putIfAbsent(String.format("%s%s", configPrefix, key), value)); + return this; + } + /** * Adds a config to {@code configs} if its not already present. Overrides a config value for which key is already * exisiting in {@code configs} @@ -344,7 +355,7 @@ private void initializeInMemoryInputStream(InMemoryInputDesc InMemorySystemDescriptor imsd = (InMemorySystemDescriptor) descriptor.getSystemDescriptor(); imsd.withInMemoryScope(this.inMemoryScope); addConfigs(descriptor.toConfig()); - addConfigs(descriptor.getSystemDescriptor().toConfig()); + addConfigs(descriptor.getSystemDescriptor().toConfig(), String.format(JobConfig.CONFIG_OVERRIDE_JOBS_PREFIX(), getJobNameAndId())); StreamSpec spec = new StreamSpec(descriptor.getStreamId(), streamName, systemName, partitonData.size()); SystemFactory factory = new InMemorySystemFactory(); Config config = new MapConfig(descriptor.toConfig(), descriptor.getSystemDescriptor().toConfig()); diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java index 92b23ef96f..e6e423f5af 100644 --- a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java +++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java @@ -29,7 +29,6 @@ import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.inmemory.InMemorySystemFactory; import org.apache.samza.config.JavaSystemConfig; -import org.apache.samza.test.framework.TestRunner; /** @@ -60,9 +59,6 @@ public class InMemorySystemDescriptor extends SystemDescriptor * **/ - private static final String CONFIG_OVERRIDE_PREFIX = "jobs.%s."; - private static final String DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY = "systems.%s.default.stream.samza.offset.default"; - private String inMemoryScope; /** @@ -106,11 +102,7 @@ public InMemorySystemDescriptor withInMemoryScope(String inMemoryScope) { public Map toConfig() { HashMap configs = new HashMap<>(super.toConfig()); configs.put(InMemorySystemConfig.INMEMORY_SCOPE, this.inMemoryScope); - configs.put(String.format(CONFIG_OVERRIDE_PREFIX + JavaSystemConfig.SYSTEM_FACTORY_FORMAT, TestRunner.JOB_NAME, getSystemName()), - FACTORY_CLASS_NAME); - configs.put( - String.format(CONFIG_OVERRIDE_PREFIX + DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY, TestRunner.JOB_NAME, - getSystemName()), SystemStreamMetadata.OffsetType.OLDEST.toString()); + configs.put(String.format(JavaSystemConfig.SYSTEM_FACTORY_FORMAT, getSystemName()), FACTORY_CLASS_NAME); return configs; }